From 7d72e8bef78abbeddb96dda20cd93937c0983f4a Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 16 Apr 2015 18:47:59 +0900 Subject: [PATCH 01/37] TAJO-1553 --- .../engine/planner/global/ExecutionBlock.java | 4 ++ .../rewriter/GlobalPlanTestRuleProvider.java | 2 + .../rewriter/rules/BroadcastJoinRule.java | 54 +++++++++++++++++++ 3 files changed, 60 insertions(+) create mode 100644 tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java index aecb36438b..06062da0c1 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java @@ -130,6 +130,10 @@ public void removeBroadcastTable(String tableName) { enforcer.removeBroadcast(tableName); } + public boolean hasBroadcastTable() { + return broadcasted.size() > 0; + } + public boolean isBroadcastTable(String tableName) { return broadcasted.contains(tableName); } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/GlobalPlanTestRuleProvider.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/GlobalPlanTestRuleProvider.java index dc91577214..f6868307ff 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/GlobalPlanTestRuleProvider.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/GlobalPlanTestRuleProvider.java @@ -20,6 +20,7 @@ import com.google.common.collect.Lists; import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.engine.planner.global.rewriter.rules.BroadcastJoinRule; import org.apache.tajo.engine.planner.global.rewriter.rules.GlobalPlanEqualityTester; import java.util.Collection; @@ -39,6 +40,7 @@ public GlobalPlanTestRuleProvider(TajoConf conf) { public Collection> getRules() { List> injectedRules = Lists.newArrayList(super.getRules()); injectedRules.add(GlobalPlanEqualityTester.class); + injectedRules.add(BroadcastJoinRule.class); return injectedRules; } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java new file mode 100644 index 0000000000..758ade978d --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java @@ -0,0 +1,54 @@ +package org.apache.tajo.engine.planner.global.rewriter.rules; + +import org.apache.tajo.engine.planner.global.ExecutionBlock; +import org.apache.tajo.engine.planner.global.MasterPlan; +import org.apache.tajo.engine.planner.global.rewriter.GlobalPlanRewriteRule; +import org.apache.tajo.plan.LogicalPlan; +import org.apache.tajo.plan.logical.NodeType; + +public class BroadcastJoinRule implements GlobalPlanRewriteRule { + + @Override + public String getName() { + return "BroadcastJoinRule"; + } + + @Override + public boolean isEligible(MasterPlan plan) { + for (LogicalPlan.QueryBlock block : plan.getLogicalPlan().getQueryBlocks()) { + if (block.hasNode(NodeType.JOIN)) { + return true; + } + } + return false; + } + + @Override + public MasterPlan rewrite(MasterPlan plan) { + rewrite(plan, plan.getTerminalBlock()); + return plan; + } + + private void rewrite(MasterPlan plan, ExecutionBlock current) { + if (plan.isLeaf(current)) { + // compute input size + } else { + for (ExecutionBlock child : plan.getChilds(current)) { + rewrite(plan, child); + } + if (current.hasJoin()) { + for (ExecutionBlock child : plan.getChilds(current)) { + if (child.hasBroadcastTable()) { + merge(plan, child, current); + } + } + } + } + } + + private ExecutionBlock merge(MasterPlan plan, ExecutionBlock child, ExecutionBlock parent) { + + + return parent; + } +} From 62f8ec79508cbc34273546d0e4103cd2d36348d4 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 16 Apr 2015 21:21:33 +0900 Subject: [PATCH 02/37] TAJO-1553 --- .../engine/planner/global/rewriter/rules/BroadcastJoinRule.java | 1 - 1 file changed, 1 deletion(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java index 758ade978d..6a9aebb045 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java @@ -47,7 +47,6 @@ private void rewrite(MasterPlan plan, ExecutionBlock current) { } private ExecutionBlock merge(MasterPlan plan, ExecutionBlock child, ExecutionBlock parent) { - return parent; } From a47a6025b753e2c2e4c8eb5b486657fd2b2c8d2f Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 17 Apr 2015 15:22:51 +0900 Subject: [PATCH 03/37] TAJO-1553 --- .../engine/planner/global/ExecutionBlock.java | 19 ++- .../engine/planner/global/GlobalPlanner.java | 9 +- .../engine/planner/global/MasterPlan.java | 7 + .../rewriter/GlobalPlanRewriteEngine.java | 5 +- .../rewriter/GlobalPlanRewriteRule.java | 9 +- .../rewriter/rules/BroadcastJoinRule.java | 147 ++++++++++++++++-- .../rules/GlobalPlanEqualityTester.java | 3 +- .../NonForwardQueryResultSystemScanner.java | 2 +- .../tajo/master/exec/QueryExecutor.java | 2 +- .../tajo/querymaster/QueryMasterTask.java | 2 +- .../tajo/querymaster/Repartitioner.java | 6 +- .../planner/global/TestBroadcastJoinPlan.java | 28 ++-- .../tajo/master/TestExecutionBlockCursor.java | 2 +- .../apache/tajo/master/TestGlobalPlanner.java | 2 +- .../tajo/querymaster/TestKillQuery.java | 2 +- 15 files changed, 197 insertions(+), 48 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java index 06062da0c1..090696bc17 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java @@ -120,22 +120,31 @@ public boolean hasUnion() { return hasUnionPlan; } - public void addBroadcastTable(String tableName) { + public void addBroadcastRelation(String tableName) { broadcasted.add(tableName); enforcer.addBroadcast(tableName); } - public void removeBroadcastTable(String tableName) { + public void removeBroadcastRelation(String tableName) { broadcasted.remove(tableName); enforcer.removeBroadcast(tableName); } - public boolean hasBroadcastTable() { + public boolean isBroadcastRelation(String tableName) { + return broadcasted.contains(tableName); + } + + public boolean hasBroadcastRelation() { return broadcasted.size() > 0; } - public boolean isBroadcastTable(String tableName) { - return broadcasted.contains(tableName); + /** + * Check this execution block is broadcastable. + * This execution block is broadcastable only when its every input is broadcastable. + * @return true if broadcastable + */ + public boolean isBroadcastable() { + return broadcasted.size() == scanlist.size(); } public Collection getBroadcastTables() { diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java index cd35d96ea2..816ec8beb6 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java @@ -40,6 +40,7 @@ import org.apache.tajo.engine.planner.global.builder.DistinctGroupbyBuilder; import org.apache.tajo.engine.planner.global.rewriter.GlobalPlanRewriteEngine; import org.apache.tajo.engine.planner.global.rewriter.GlobalPlanRewriteRuleProvider; +import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.exception.InternalException; import org.apache.tajo.plan.LogicalPlan; import org.apache.tajo.plan.PlanningException; @@ -119,7 +120,7 @@ public Map getExecBlockMap() { /** * Builds a master plan from the given logical plan. */ - public void build(MasterPlan masterPlan) throws IOException, PlanningException { + public void build(QueryContext queryContext, MasterPlan masterPlan) throws IOException, PlanningException { DistributedPlannerVisitor planner = new DistributedPlannerVisitor(); GlobalPlanContext globalPlanContext = new GlobalPlanContext(); @@ -169,7 +170,7 @@ public void build(MasterPlan masterPlan) throws IOException, PlanningException { masterPlan.setTerminal(terminalBlock); LOG.info("\n" + masterPlan.toString()); - rewriteEngine.rewrite(masterPlan); + rewriteEngine.rewrite(queryContext, masterPlan); } private static void setFinalOutputChannel(DataChannel outputChannel, Schema outputSchema) { @@ -382,7 +383,7 @@ private ExecutionBlock buildJoinPlan(GlobalPlanContext context, JoinNode joinNod } for (ScanNode eachBroadcastTarget: broadcastTargetScanNodes) { - currentBlock.addBroadcastTable(eachBroadcastTarget.getCanonicalName()); + currentBlock.addBroadcastRelation(eachBroadcastTarget.getCanonicalName()); context.execBlockMap.remove(eachBroadcastTarget.getPID()); } @@ -1344,7 +1345,7 @@ public LogicalNode visitJoin(GlobalPlanContext context, LogicalPlan plan, Logica if (leftChild.getType() == NodeType.JOIN && checkIfCanBeOneOfBroadcastJoin(node.getRightChild())) { ScanNode scanNode = node.getRightChild(); - if (leftChildBlock.isBroadcastTable(scanNode.getCanonicalName())) { + if (leftChildBlock.isBroadcastRelation(scanNode.getCanonicalName())) { context.execBlockMap.put(node.getPID(), leftChildBlock); return node; } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/MasterPlan.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/MasterPlan.java index 3cca4f2943..2399e8c6e2 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/MasterPlan.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/MasterPlan.java @@ -108,6 +108,13 @@ public ExecutionBlock getExecBlock(ExecutionBlockId execBlockId) { return execBlockMap.get(execBlockId); } + public void removeExecBlock(ExecutionBlockId execBlockId) throws IllegalStateException { + if (getIncomingChannels(execBlockId).size() > 0 || getOutgoingChannels(execBlockId).size() > 0) { + throw new IllegalStateException("Cannot remove execution blocks because some other execution blocks are connected"); + } + execBlockMap.remove(execBlockId); + } + public void addConnect(DataChannel dataChannel) { execBlockGraph.addEdge(dataChannel.getSrcId(), dataChannel.getTargetId(), dataChannel); } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/GlobalPlanRewriteEngine.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/GlobalPlanRewriteEngine.java index c01ed0eb04..cc1f3c6be0 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/GlobalPlanRewriteEngine.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/GlobalPlanRewriteEngine.java @@ -20,6 +20,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.tajo.OverridableConf; import org.apache.tajo.engine.planner.global.MasterPlan; import org.apache.tajo.plan.PlanningException; import org.apache.tajo.util.ReflectionUtil; @@ -67,11 +68,11 @@ public void addRewriteRule(GlobalPlanRewriteRule rule) { * @param plan The plan to be rewritten with all query rewrite rule. * @return The rewritten plan. */ - public MasterPlan rewrite(MasterPlan plan) throws PlanningException { + public MasterPlan rewrite(OverridableConf queryContext, MasterPlan plan) throws PlanningException { GlobalPlanRewriteRule rule; for (Map.Entry rewriteRule : rewriteRules.entrySet()) { rule = rewriteRule.getValue(); - if (rule.isEligible(plan)) { + if (rule.isEligible(queryContext, plan)) { plan = rule.rewrite(plan); if (LOG.isDebugEnabled()) { LOG.debug("The rule \"" + rule.getName() + " \" rewrites the query."); diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/GlobalPlanRewriteRule.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/GlobalPlanRewriteRule.java index 4a37207a85..f30160fdcc 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/GlobalPlanRewriteRule.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/GlobalPlanRewriteRule.java @@ -18,7 +18,9 @@ package org.apache.tajo.engine.planner.global.rewriter; +import org.apache.tajo.OverridableConf; import org.apache.tajo.engine.planner.global.MasterPlan; +import org.apache.tajo.plan.PlanningException; /** * A rewrite rule for global plans @@ -29,15 +31,16 @@ public interface GlobalPlanRewriteRule { * Return rule name * @return Rule name */ - public abstract String getName(); + String getName(); /** * Check if this rule should be applied. * + * @param queryContext Query context * @param plan Global Plan * @return */ - public abstract boolean isEligible(MasterPlan plan); + boolean isEligible(OverridableConf queryContext, MasterPlan plan); /** * Rewrite a global plan @@ -45,5 +48,5 @@ public interface GlobalPlanRewriteRule { * @param plan Global Plan * @return */ - public abstract MasterPlan rewrite(MasterPlan plan); + MasterPlan rewrite(MasterPlan plan) throws PlanningException; } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java index 6a9aebb045..1af4f9ea56 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java @@ -1,12 +1,36 @@ +/** + * 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.global.rewriter.rules; +import org.apache.tajo.OverridableConf; +import org.apache.tajo.SessionVars; import org.apache.tajo.engine.planner.global.ExecutionBlock; import org.apache.tajo.engine.planner.global.MasterPlan; import org.apache.tajo.engine.planner.global.rewriter.GlobalPlanRewriteRule; import org.apache.tajo.plan.LogicalPlan; -import org.apache.tajo.plan.logical.NodeType; +import org.apache.tajo.plan.PlanningException; +import org.apache.tajo.plan.logical.*; +import org.apache.tajo.plan.util.PlannerUtil; public class BroadcastJoinRule implements GlobalPlanRewriteRule { + private long broadcastTableSizeThreshold; + private ParentFinder parentFinder; @Override public String getName() { @@ -14,31 +38,43 @@ public String getName() { } @Override - public boolean isEligible(MasterPlan plan) { - for (LogicalPlan.QueryBlock block : plan.getLogicalPlan().getQueryBlocks()) { - if (block.hasNode(NodeType.JOIN)) { - return true; + public boolean isEligible(OverridableConf queryContext, MasterPlan plan) { + if (queryContext.getBool(SessionVars.TEST_BROADCAST_JOIN_ENABLED)) { + for (LogicalPlan.QueryBlock block : plan.getLogicalPlan().getQueryBlocks()) { + if (block.hasNode(NodeType.JOIN)) { + broadcastTableSizeThreshold = queryContext.getLong(SessionVars.BROADCAST_TABLE_SIZE_LIMIT); + if (broadcastTableSizeThreshold > 0) { + parentFinder = new ParentFinder(); + return true; + } + } } } return false; } @Override - public MasterPlan rewrite(MasterPlan plan) { + public MasterPlan rewrite(MasterPlan plan) throws PlanningException{ rewrite(plan, plan.getTerminalBlock()); return plan; } - private void rewrite(MasterPlan plan, ExecutionBlock current) { + private void rewrite(MasterPlan plan, ExecutionBlock current) throws PlanningException { if (plan.isLeaf(current)) { - // compute input size + // in leaf execution blocks, find input tables which size is less than the predefined threshold. + for (ScanNode scanNode : current.getScanNodes()) { + if (getTableVolume(scanNode) > broadcastTableSizeThreshold) { + current.addBroadcastRelation(scanNode.getTableName()); + } + } } else { + // in intermediate execution blocks, merge broadcastable children's plan with the current plan. for (ExecutionBlock child : plan.getChilds(current)) { rewrite(plan, child); } if (current.hasJoin()) { for (ExecutionBlock child : plan.getChilds(current)) { - if (child.hasBroadcastTable()) { + if (child.isBroadcastable()) { merge(plan, child, current); } } @@ -46,8 +82,99 @@ private void rewrite(MasterPlan plan, ExecutionBlock current) { } } - private ExecutionBlock merge(MasterPlan plan, ExecutionBlock child, ExecutionBlock parent) { + /** + * Merge child execution blocks. + * + * @param plan master plan + * @param child child block + * @param parent parent block + * @return + */ + private ExecutionBlock merge(MasterPlan plan, ExecutionBlock child, ExecutionBlock parent) throws PlanningException { + ScanNode scanForChild = null; + for (ScanNode scanNode : parent.getScanNodes()) { + if (scanNode.getTableName().equals(child.getId().toString())) { + scanForChild = scanNode; + } + } + if (scanForChild == null) { + throw new PlanningException("Cannot find any scan nodes for " + child.getId() + " in " + parent.getId()); + } + + parentFinder.find(scanForChild); + LogicalNode parentOfScanForChild = parentFinder.found; + if (parentOfScanForChild == null) { + throw new PlanningException("Cannot find the parent of " + scanForChild.getCanonicalName()); + } + + StoreTableNode storeForParent = PlannerUtil.findTopNode(child.getPlan(), NodeType.STORE); + if (storeForParent == null) { + throw new PlanningException("Cannot find any store nodes for " + parent.getId() + " in " + child.getId()); + } + + if (parentOfScanForChild instanceof UnaryNode) { + ((UnaryNode) parentOfScanForChild).setChild(storeForParent.getChild()); + } else if (parentOfScanForChild instanceof BinaryNode) { + BinaryNode binary = (BinaryNode) parentOfScanForChild; + if (binary.getLeftChild().equals(scanForChild)) { + binary.setLeftChild(storeForParent.getChild()); + } else if (binary.getRightChild().equals(scanForChild)) { + binary.setRightChild(storeForParent.getChild()); + } else { + throw new PlanningException(scanForChild.getPID() + " is not a child of " + parentOfScanForChild.getPID()); + } + } else { + throw new PlanningException(parentOfScanForChild + " seems to not have any children"); + } + + plan.disconnect(child, parent); + plan.removeExecBlock(child.getId()); return parent; } + + /** + * Get a volume of a table of a partitioned table + * @param scanNode ScanNode corresponding to a table + * @return table volume (bytes) + */ + private static long getTableVolume(ScanNode scanNode) { + long scanBytes = scanNode.getTableDesc().getStats().getNumBytes(); + if (scanNode.getType() == NodeType.PARTITIONS_SCAN) { + PartitionedTableScanNode pScanNode = (PartitionedTableScanNode)scanNode; + if (pScanNode.getInputPaths() == null || pScanNode.getInputPaths().length == 0) { + scanBytes = 0L; + } + } + + return scanBytes; + } + + private static class ParentFinder implements LogicalNodeVisitor { + private LogicalNode target; + private LogicalNode found; + + public void set(LogicalNode child) { + this.target = child; + this.found = null; + } + + public void find(LogicalNode childOfTarget) { + this.visit(childOfTarget); + } + + @Override + public void visit(LogicalNode node) { + for (int i = 0; i < node.childNum(); i++) { + if (node.getChild(i).equals(target)) { + found = node; + break; + } else { + if (found == null) { + visit(node.getChild(i)); + } + } + } + } + } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanEqualityTester.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanEqualityTester.java index e2fd47f944..ddb456a335 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanEqualityTester.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanEqualityTester.java @@ -18,6 +18,7 @@ package org.apache.tajo.engine.planner.global.rewriter.rules; +import org.apache.tajo.OverridableConf; import org.apache.tajo.engine.planner.global.ExecutionBlock; import org.apache.tajo.engine.planner.global.ExecutionBlockCursor; import org.apache.tajo.engine.planner.global.MasterPlan; @@ -38,7 +39,7 @@ public String getName() { } @Override - public boolean isEligible(MasterPlan plan) { + public boolean isEligible(OverridableConf queryContext, MasterPlan plan) { return true; } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultSystemScanner.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultSystemScanner.java index 958c252304..73bb83c54c 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultSystemScanner.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultSystemScanner.java @@ -94,7 +94,7 @@ public void init() throws IOException { MasterPlan masterPlan = new MasterPlan(queryId, queryContext, logicalPlan); GlobalPlanner globalPlanner = new GlobalPlanner(masterContext.getConf(), masterContext.getCatalog()); try { - globalPlanner.build(masterPlan); + globalPlanner.build(queryContext, masterPlan); } catch (PlanningException e) { throw new RuntimeException(e); } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java index 2eb3c5f752..258a622f82 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java @@ -465,7 +465,7 @@ public static MasterPlan compileMasterPlan(LogicalPlan plan, QueryContext contex } MasterPlan masterPlan = new MasterPlan(QueryIdFactory.NULL_QUERY_ID, context, plan); - planner.build(masterPlan); + planner.build(context, masterPlan); return masterPlan; } diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java index 465fa8494f..8622d1b23a 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java @@ -363,7 +363,7 @@ public synchronized void startQuery() { } } MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); - queryMasterContext.getGlobalPlanner().build(masterPlan); + queryMasterContext.getGlobalPlanner().build(queryContext, masterPlan); query = new Query(queryTaskContext, queryId, querySubmitTime, "", queryTaskContext.getEventHandler(), masterPlan); diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java index 8e9e343c0c..eec7441afa 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java @@ -194,7 +194,7 @@ public static void scheduleFragmentsForJoinQuery(TaskSchedulerContext schedulerC // Assigning either fragments or fetch urls to query units boolean isAllBroadcastTable = true; for (int i = 0; i < scans.length; i++) { - if (!execBlock.isBroadcastTable(scans[i].getCanonicalName())) { + if (!execBlock.isBroadcastRelation(scans[i].getCanonicalName())) { isAllBroadcastTable = false; break; } @@ -218,7 +218,7 @@ public static void scheduleFragmentsForJoinQuery(TaskSchedulerContext schedulerC } int baseScanIdx = maxStatsScanIdx; scans[baseScanIdx].setBroadcastTable(false); - execBlock.removeBroadcastTable(scans[baseScanIdx].getCanonicalName()); + execBlock.removeBroadcastRelation(scans[baseScanIdx].getCanonicalName()); LOG.info(String.format("[Distributed Join Strategy] : Broadcast Join with all tables, base_table=%s, base_volume=%d", scans[baseScanIdx].getCanonicalName(), stats[baseScanIdx])); scheduleLeafTasksWithBroadcastTable(schedulerContext, stage, baseScanIdx, fragments); @@ -239,7 +239,7 @@ public static void scheduleFragmentsForJoinQuery(TaskSchedulerContext schedulerC nonLeafScanNamesBuilder.append(namePrefix).append(scans[i].getCanonicalName()); namePrefix = ","; } - if (execBlock.isBroadcastTable(scans[i].getCanonicalName())) { + if (execBlock.isBroadcastRelation(scans[i].getCanonicalName())) { broadcastIndexList.add(i); } else { // finding largest table. diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/global/TestBroadcastJoinPlan.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/global/TestBroadcastJoinPlan.java index eb92bcfe8f..6d25d6e137 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/global/TestBroadcastJoinPlan.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/global/TestBroadcastJoinPlan.java @@ -190,7 +190,7 @@ public final void testBroadcastJoin() throws IOException, PlanningException { QueryContext queryContext = new QueryContext(conf); MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); - globalPlanner.build(masterPlan); + globalPlanner.build(queryContext, masterPlan); /* |-eb_1395714781593_0000_000005 (TERMINAL) @@ -251,7 +251,7 @@ public final void testBroadcastJoinAllSmallTables() throws IOException, Planning QueryContext queryContext = new QueryContext(conf); MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); - globalPlanner.build(masterPlan); + globalPlanner.build(queryContext, masterPlan); /* |-eb_1402500846700_0000_000005 @@ -315,7 +315,7 @@ public final void testNotBroadcastJoinTwoLargeTable() throws IOException, Planni QueryContext queryContext = new QueryContext(conf); MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); - globalPlanner.build(masterPlan); + globalPlanner.build(queryContext, masterPlan); ExecutionBlockCursor ebCursor = new ExecutionBlockCursor(masterPlan); while (ebCursor.hasNext()) { @@ -343,7 +343,7 @@ public final void testTwoBroadcastJoin() throws IOException, PlanningException { QueryContext queryContext = new QueryContext(conf); MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); - globalPlanner.build(masterPlan); + globalPlanner.build(queryContext, masterPlan); /* |-eb_1395736346625_0000_000009 @@ -393,7 +393,7 @@ public final void testNotBroadcastJoinSubquery() throws IOException, PlanningExc QueryContext queryContext = new QueryContext(conf); MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); - globalPlanner.build(masterPlan); + globalPlanner.build(queryContext, masterPlan); /* |-eb_1395749810370_0000_000007 @@ -434,7 +434,7 @@ public final void testBroadcastJoinSubquery() throws IOException, PlanningExcept QueryContext queryContext = new QueryContext(conf); MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); - globalPlanner.build(masterPlan); + globalPlanner.build(queryContext, masterPlan); /* |-eb_1395794091662_0000_000007 @@ -490,7 +490,7 @@ public final void testLeftOuterJoinCase1() throws IOException, PlanningException QueryContext queryContext = new QueryContext(conf); MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); - globalPlanner.build(masterPlan); + globalPlanner.build(queryContext, masterPlan); // ((((default.small1 ⟕ default.small2) ⟕ default.small3) ⟕ default.large1) ⟕ default.large2) /* @@ -544,7 +544,7 @@ public final void testLeftOuterJoinCase2() throws IOException, PlanningException QueryContext queryContext = new QueryContext(conf); MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); - globalPlanner.build(masterPlan); + globalPlanner.build(queryContext, masterPlan); // ((((default.large1 ⟕ default.large2) ⟕ default.small1) ⟕ default.small2) ⟕ default.small3) /* @@ -627,7 +627,7 @@ public final void testLeftOuterJoinCase3() throws IOException, PlanningException QueryContext queryContext = new QueryContext(conf); MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); - globalPlanner.build(masterPlan); + globalPlanner.build(queryContext, masterPlan); //(((((default.large1 ⟕ default.large2) ⟕ default.small1) ⟕ default.large3) ⟕ default.small2) ⟕ default.small3) /* @@ -710,7 +710,7 @@ public final void testLeftOuterJoinCase4() throws IOException, PlanningException QueryContext queryContext = new QueryContext(conf); MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); - globalPlanner.build(masterPlan); + globalPlanner.build(queryContext, masterPlan); /* |-eb_1406022971444_0000_000005 @@ -769,7 +769,7 @@ public final void testLeftOuterJoinCase5() throws IOException, PlanningException QueryContext queryContext = new QueryContext(conf); MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); - globalPlanner.build(masterPlan); + globalPlanner.build(queryContext, masterPlan); //(((default.small1 ⟕ default.small2) ⟕ default.large1) ⟕ default.small3) /* @@ -822,7 +822,7 @@ public final void testLeftOuterJoinCase6() throws IOException, PlanningException QueryContext queryContext = new QueryContext(conf); MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); - globalPlanner.build(masterPlan); + globalPlanner.build(queryContext, masterPlan); // ((((default.small1 ⟕ default.small2) ⟕ default.large1) ⟕ default.large2) ⟕ default.small3) @@ -914,7 +914,7 @@ public final void testInnerLeftOuterJoinCase1() throws IOException, PlanningExce QueryContext queryContext = new QueryContext(conf); MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); - globalPlanner.build(masterPlan); + globalPlanner.build(queryContext, masterPlan); // (((default.small1 ⋈θ default.small2) ⟕ default.large1) ⟕ default.small3) /* @@ -979,7 +979,7 @@ public final void testBroadcastCasebyCase1() throws IOException, PlanningExcepti QueryContext queryContext = new QueryContext(conf); MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); - globalPlanner.build(masterPlan); + globalPlanner.build(queryContext, masterPlan); // (((default.large1 ⋈θ default.small1) ⟕ default.large2) ⟕ default.small2) /* diff --git a/tajo-core/src/test/java/org/apache/tajo/master/TestExecutionBlockCursor.java b/tajo-core/src/test/java/org/apache/tajo/master/TestExecutionBlockCursor.java index 712243bf61..d6ce164108 100644 --- a/tajo-core/src/test/java/org/apache/tajo/master/TestExecutionBlockCursor.java +++ b/tajo-core/src/test/java/org/apache/tajo/master/TestExecutionBlockCursor.java @@ -110,7 +110,7 @@ public void testNextBlock() throws Exception { optimizer.optimize(logicalPlan); QueryContext queryContext = new QueryContext(conf); MasterPlan plan = new MasterPlan(LocalTajoTestingUtility.newQueryId(), queryContext, logicalPlan); - planner.build(plan); + planner.build(queryContext, plan); ExecutionBlockCursor cursor = new ExecutionBlockCursor(plan); diff --git a/tajo-core/src/test/java/org/apache/tajo/master/TestGlobalPlanner.java b/tajo-core/src/test/java/org/apache/tajo/master/TestGlobalPlanner.java index 45c94a37e6..f84bc6bb9e 100644 --- a/tajo-core/src/test/java/org/apache/tajo/master/TestGlobalPlanner.java +++ b/tajo-core/src/test/java/org/apache/tajo/master/TestGlobalPlanner.java @@ -130,7 +130,7 @@ private MasterPlan buildPlan(String sql) throws PlanningException, IOException { LogicalPlan plan = planner.createPlan(context, expr); optimizer.optimize(context, plan); MasterPlan masterPlan = new MasterPlan(LocalTajoTestingUtility.newQueryId(), context, plan); - globalPlanner.build(masterPlan); + globalPlanner.build(context, masterPlan); return masterPlan; } diff --git a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java index 09be700dbc..fe46292c96 100644 --- a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java @@ -108,7 +108,7 @@ public final void testKillQueryFromInitState() throws Exception { QueryContext queryContext = new QueryContext(conf); MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); - globalPlanner.build(masterPlan); + globalPlanner.build(queryContext, masterPlan); CountDownLatch barrier = new CountDownLatch(1); MockAsyncDispatch dispatch = new MockAsyncDispatch(barrier, StageEventType.SQ_INIT); From 5feaeac20bc38ba3846f0da2e8267c2c164c0ada Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 20 Apr 2015 14:59:55 +0900 Subject: [PATCH 04/37] TAJO-1553 --- .../engine/planner/global/GlobalPlanner.java | 265 +++++++++--------- .../engine/planner/global/MasterPlan.java | 9 +- .../rewriter/GlobalPlanTestRuleProvider.java | 2 +- .../rewriter/rules/BroadcastJoinRule.java | 48 +++- 4 files changed, 177 insertions(+), 147 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java index 816ec8beb6..89ba8e795d 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java @@ -133,18 +133,18 @@ public void build(QueryContext queryContext, MasterPlan masterPlan) throws IOExc LogicalNode inputPlan = PlannerUtil.clone(masterPlan.getLogicalPlan(), masterPlan.getLogicalPlan().getRootBlock().getRoot()); - boolean broadcastEnabled = masterPlan.getContext().getBool(SessionVars.TEST_BROADCAST_JOIN_ENABLED); - if (broadcastEnabled) { - // pre-visit the master plan in order to find tables to be broadcasted - // this visiting does not make any execution block and change plan. - BroadcastJoinMarkCandidateVisitor markCandidateVisitor = new BroadcastJoinMarkCandidateVisitor(); - markCandidateVisitor.visit(globalPlanContext, - masterPlan.getLogicalPlan(), masterPlan.getLogicalPlan().getRootBlock(), inputPlan, new Stack()); - - BroadcastJoinPlanVisitor broadcastJoinPlanVisitor = new BroadcastJoinPlanVisitor(); - broadcastJoinPlanVisitor.visit(globalPlanContext, - masterPlan.getLogicalPlan(), masterPlan.getLogicalPlan().getRootBlock(), inputPlan, new Stack()); - } +// boolean broadcastEnabled = masterPlan.getContext().getBool(SessionVars.TEST_BROADCAST_JOIN_ENABLED); +// if (broadcastEnabled) { +// // pre-visit the master plan in order to find tables to be broadcasted +// // this visiting does not make any execution block and change plan. +// BroadcastJoinMarkCandidateVisitor markCandidateVisitor = new BroadcastJoinMarkCandidateVisitor(); +// markCandidateVisitor.visit(globalPlanContext, +// masterPlan.getLogicalPlan(), masterPlan.getLogicalPlan().getRootBlock(), inputPlan, new Stack()); +// +// BroadcastJoinPlanVisitor broadcastJoinPlanVisitor = new BroadcastJoinPlanVisitor(); +// broadcastJoinPlanVisitor.visit(globalPlanContext, +// masterPlan.getLogicalPlan(), masterPlan.getLogicalPlan().getRootBlock(), inputPlan, new Stack()); +// } // create a distributed execution plan by visiting each logical node. // Its output is a graph, where each vertex is an execution block, and each edge is a data channel. @@ -168,9 +168,10 @@ public void build(QueryContext queryContext, MasterPlan masterPlan) throws IOExc } masterPlan.setTerminal(terminalBlock); - LOG.info("\n" + masterPlan.toString()); + LOG.info("\nNon-optimized master plan" + masterPlan.toString()); rewriteEngine.rewrite(queryContext, masterPlan); + LOG.info("\nOptimized master plan\n" + masterPlan.toString()); } private static void setFinalOutputChannel(DataChannel outputChannel, Schema outputSchema) { @@ -282,114 +283,114 @@ private ExecutionBlock buildJoinPlan(GlobalPlanContext context, JoinNode joinNod MasterPlan masterPlan = context.plan; ExecutionBlock currentBlock; - boolean broadcastEnabled = context.getPlan().getContext().getBool(SessionVars.TEST_BROADCAST_JOIN_ENABLED); - long broadcastTableSizeLimit = context.getPlan().getContext().getLong(SessionVars.BROADCAST_TABLE_SIZE_LIMIT); +// boolean broadcastEnabled = context.getPlan().getContext().getBool(SessionVars.TEST_BROADCAST_JOIN_ENABLED); +// long broadcastTableSizeLimit = context.getPlan().getContext().getLong(SessionVars.BROADCAST_TABLE_SIZE_LIMIT); // to check when the tajo.dist-query.join.broadcast.auto property is true - if (broadcastEnabled && joinNode.isCandidateBroadcast()) { - LogicalNode leftNode = joinNode.getLeftChild(); - LogicalNode rightNode = joinNode.getRightChild(); - - List broadcastTargetScanNodes = new ArrayList(); - int numLargeTables = 0; - boolean leftBroadcast = false; - boolean rightBroadcast = false; - - // TODO - in the the current implementation, a broadcast join on a bush join tree is not supported yet. - // - // Join - // / \ - // Join Join - // / \ / \ - // Scan Scan Scan Scan - - - // Checking Left Side of Join - if (ScanNode.isScanNode(leftNode)) { - ScanNode scanNode = (ScanNode)leftNode; - if (joinNode.getJoinType() == JoinType.LEFT_OUTER || getTableVolume(scanNode) >= broadcastTableSizeLimit) { - numLargeTables++; - } else { - leftBroadcast = true; - broadcastTargetScanNodes.add(scanNode); - LOG.info("JoinNode's left table " + scanNode.getCanonicalName() + " (" - + getTableVolume(scanNode) + ") is marked a broadcasted table"); - } - } - - // Checking Right Side OF Join - if (ScanNode.isScanNode(rightNode)) { - ScanNode scanNode = (ScanNode)rightNode; - if (joinNode.getJoinType() == JoinType.RIGHT_OUTER || getTableVolume(scanNode) >= broadcastTableSizeLimit) { - numLargeTables++; - } else { - rightBroadcast = true; - broadcastTargetScanNodes.add(scanNode); - LOG.info("JoinNode's right table " + scanNode.getCanonicalName() + " (" - + getTableVolume(scanNode) + ") is marked a broadcasted table"); - } - } - - JoinNode blockJoinNode = null; - if (!leftBroadcast && !rightBroadcast) { - // In the case of large, large, small, small - // all small tables broadcast to right large table - numLargeTables = 1; - } - for(LogicalNode eachNode: joinNode.getBroadcastCandidateTargets()) { - if (eachNode.getPID() == joinNode.getPID()) { - continue; - } - if (numLargeTables >= 2) { - break; - } - JoinNode broadcastJoinNode = (JoinNode)eachNode; - ScanNode scanNode = broadcastJoinNode.getRightChild(); - if (getTableVolume(scanNode) < broadcastTableSizeLimit) { - broadcastTargetScanNodes.add(scanNode); - blockJoinNode = broadcastJoinNode; - LOG.info("The table " + scanNode.getCanonicalName() + " (" - + getTableVolume(scanNode) + ") is marked a broadcasted table"); - } else { - numLargeTables++; - if (numLargeTables < 2) { - blockJoinNode = broadcastJoinNode; - } - } - } - - if (!broadcastTargetScanNodes.isEmpty()) { - // make new execution block - currentBlock = masterPlan.newExecutionBlock(); - - if (!leftBroadcast && !rightBroadcast) { - DataChannel leftChannel = createDataChannelFromJoin(leftBlock, rightBlock, currentBlock, joinNode, true); - ScanNode leftScan = buildInputExecutor(masterPlan.getLogicalPlan(), leftChannel); - joinNode.setLeftChild(leftScan); - masterPlan.addConnect(leftChannel); - - DataChannel rightChannel = createDataChannelFromJoin(leftBlock, rightBlock, currentBlock, joinNode, false); - ScanNode rightScan = buildInputExecutor(masterPlan.getLogicalPlan(), rightChannel); - joinNode.setRightChild(rightScan); - masterPlan.addConnect(rightChannel); - } - - if (blockJoinNode != null) { - LOG.info("Set execution's plan with join " + blockJoinNode + " for broadcast join"); - // set current execution block's plan with last broadcast join node - currentBlock.setPlan(blockJoinNode); - } else { - currentBlock.setPlan(joinNode); - } - - for (ScanNode eachBroadcastTarget: broadcastTargetScanNodes) { - currentBlock.addBroadcastRelation(eachBroadcastTarget.getCanonicalName()); - context.execBlockMap.remove(eachBroadcastTarget.getPID()); - } - - return currentBlock; - } - } +// if (broadcastEnabled && joinNode.isCandidateBroadcast()) { +// LogicalNode leftNode = joinNode.getLeftChild(); +// LogicalNode rightNode = joinNode.getRightChild(); +// +// List broadcastTargetScanNodes = new ArrayList(); +// int numLargeTables = 0; +// boolean leftBroadcast = false; +// boolean rightBroadcast = false; +// +// // TODO - in the the current implementation, a broadcast join on a bush join tree is not supported yet. +// // +// // Join +// // / \ +// // Join Join +// // / \ / \ +// // Scan Scan Scan Scan +// +// +// // Checking Left Side of Join +// if (ScanNode.isScanNode(leftNode)) { +// ScanNode scanNode = (ScanNode)leftNode; +// if (joinNode.getJoinType() == JoinType.LEFT_OUTER || getTableVolume(scanNode) >= broadcastTableSizeLimit) { +// numLargeTables++; +// } else { +// leftBroadcast = true; +// broadcastTargetScanNodes.add(scanNode); +// LOG.info("JoinNode's left table " + scanNode.getCanonicalName() + " (" +// + getTableVolume(scanNode) + ") is marked a broadcasted table"); +// } +// } +// +// // Checking Right Side OF Join +// if (ScanNode.isScanNode(rightNode)) { +// ScanNode scanNode = (ScanNode)rightNode; +// if (joinNode.getJoinType() == JoinType.RIGHT_OUTER || getTableVolume(scanNode) >= broadcastTableSizeLimit) { +// numLargeTables++; +// } else { +// rightBroadcast = true; +// broadcastTargetScanNodes.add(scanNode); +// LOG.info("JoinNode's right table " + scanNode.getCanonicalName() + " (" +// + getTableVolume(scanNode) + ") is marked a broadcasted table"); +// } +// } +// +// JoinNode blockJoinNode = null; +// if (!leftBroadcast && !rightBroadcast) { +// // In the case of large, large, small, small +// // all small tables broadcast to right large table +// numLargeTables = 1; +// } +// for(LogicalNode eachNode: joinNode.getBroadcastCandidateTargets()) { +// if (eachNode.getPID() == joinNode.getPID()) { +// continue; +// } +// if (numLargeTables >= 2) { +// break; +// } +// JoinNode broadcastJoinNode = (JoinNode)eachNode; +// ScanNode scanNode = broadcastJoinNode.getRightChild(); +// if (getTableVolume(scanNode) < broadcastTableSizeLimit) { +// broadcastTargetScanNodes.add(scanNode); +// blockJoinNode = broadcastJoinNode; +// LOG.info("The table " + scanNode.getCanonicalName() + " (" +// + getTableVolume(scanNode) + ") is marked a broadcasted table"); +// } else { +// numLargeTables++; +// if (numLargeTables < 2) { +// blockJoinNode = broadcastJoinNode; +// } +// } +// } +// +// if (!broadcastTargetScanNodes.isEmpty()) { +// // make new execution block +// currentBlock = masterPlan.newExecutionBlock(); +// +// if (!leftBroadcast && !rightBroadcast) { +// DataChannel leftChannel = createDataChannelFromJoin(leftBlock, rightBlock, currentBlock, joinNode, true); +// ScanNode leftScan = buildInputExecutor(masterPlan.getLogicalPlan(), leftChannel); +// joinNode.setLeftChild(leftScan); +// masterPlan.addConnect(leftChannel); +// +// DataChannel rightChannel = createDataChannelFromJoin(leftBlock, rightBlock, currentBlock, joinNode, false); +// ScanNode rightScan = buildInputExecutor(masterPlan.getLogicalPlan(), rightChannel); +// joinNode.setRightChild(rightScan); +// masterPlan.addConnect(rightChannel); +// } +// +// if (blockJoinNode != null) { +// LOG.info("Set execution's plan with join " + blockJoinNode + " for broadcast join"); +// // set current execution block's plan with last broadcast join node +// currentBlock.setPlan(blockJoinNode); +// } else { +// currentBlock.setPlan(joinNode); +// } +// +// for (ScanNode eachBroadcastTarget: broadcastTargetScanNodes) { +// currentBlock.addBroadcastRelation(eachBroadcastTarget.getCanonicalName()); +// context.execBlockMap.remove(eachBroadcastTarget.getPID()); +// } +// +// return currentBlock; +// } +// } LogicalNode leftNode = joinNode.getLeftChild(); LogicalNode rightNode = joinNode.getRightChild(); @@ -1343,19 +1344,19 @@ public LogicalNode visitJoin(GlobalPlanContext context, LogicalPlan plan, Logica LogicalNode leftChild = visit(context, plan, block, node.getLeftChild(), stack); ExecutionBlock leftChildBlock = context.execBlockMap.get(leftChild.getPID()); - if (leftChild.getType() == NodeType.JOIN && checkIfCanBeOneOfBroadcastJoin(node.getRightChild())) { - ScanNode scanNode = node.getRightChild(); - if (leftChildBlock.isBroadcastRelation(scanNode.getCanonicalName())) { - context.execBlockMap.put(node.getPID(), leftChildBlock); - return node; - } - - // if left execution block's plan is replaced with parent node(join node) - if (leftChildBlock.getPlan().getPID() == node.getPID()) { - context.execBlockMap.put(node.getPID(), leftChildBlock); - return node; - } - } +// if (leftChild.getType() == NodeType.JOIN && checkIfCanBeOneOfBroadcastJoin(node.getRightChild())) { +// ScanNode scanNode = node.getRightChild(); +// if (leftChildBlock.isBroadcastRelation(scanNode.getCanonicalName())) { +// context.execBlockMap.put(node.getPID(), leftChildBlock); +// return node; +// } +// +// // if left execution block's plan is replaced with parent node(join node) +// if (leftChildBlock.getPlan().getPID() == node.getPID()) { +// context.execBlockMap.put(node.getPID(), leftChildBlock); +// return node; +// } +// } LogicalNode rightChild = visit(context, plan, block, node.getRightChild(), stack); ExecutionBlock rightChildBlock = context.execBlockMap.get(rightChild.getPID()); diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/MasterPlan.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/MasterPlan.java index 2399e8c6e2..97241955d5 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/MasterPlan.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/MasterPlan.java @@ -109,9 +109,16 @@ public ExecutionBlock getExecBlock(ExecutionBlockId execBlockId) { } public void removeExecBlock(ExecutionBlockId execBlockId) throws IllegalStateException { - if (getIncomingChannels(execBlockId).size() > 0 || getOutgoingChannels(execBlockId).size() > 0) { + List channels = getIncomingChannels(execBlockId); + if (channels != null && channels.size() > 0) { throw new IllegalStateException("Cannot remove execution blocks because some other execution blocks are connected"); } + + channels = getOutgoingChannels(execBlockId); + if (channels != null && channels.size() > 0) { + throw new IllegalStateException("Cannot remove execution blocks because some other execution blocks are connected"); + } + execBlockMap.remove(execBlockId); } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/GlobalPlanTestRuleProvider.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/GlobalPlanTestRuleProvider.java index f6868307ff..6fa487b712 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/GlobalPlanTestRuleProvider.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/GlobalPlanTestRuleProvider.java @@ -39,8 +39,8 @@ public GlobalPlanTestRuleProvider(TajoConf conf) { @Override public Collection> getRules() { List> injectedRules = Lists.newArrayList(super.getRules()); - injectedRules.add(GlobalPlanEqualityTester.class); injectedRules.add(BroadcastJoinRule.class); + injectedRules.add(GlobalPlanEqualityTester.class); return injectedRules; } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java index 1af4f9ea56..e419b82544 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java @@ -20,13 +20,15 @@ import org.apache.tajo.OverridableConf; import org.apache.tajo.SessionVars; +import org.apache.tajo.engine.planner.global.DataChannel; import org.apache.tajo.engine.planner.global.ExecutionBlock; import org.apache.tajo.engine.planner.global.MasterPlan; import org.apache.tajo.engine.planner.global.rewriter.GlobalPlanRewriteRule; import org.apache.tajo.plan.LogicalPlan; import org.apache.tajo.plan.PlanningException; import org.apache.tajo.plan.logical.*; -import org.apache.tajo.plan.util.PlannerUtil; + +import java.util.List; public class BroadcastJoinRule implements GlobalPlanRewriteRule { private long broadcastTableSizeThreshold; @@ -44,7 +46,9 @@ public boolean isEligible(OverridableConf queryContext, MasterPlan plan) { if (block.hasNode(NodeType.JOIN)) { broadcastTableSizeThreshold = queryContext.getLong(SessionVars.BROADCAST_TABLE_SIZE_LIMIT); if (broadcastTableSizeThreshold > 0) { - parentFinder = new ParentFinder(); + if (parentFinder == null) { + parentFinder = new ParentFinder(); + } return true; } } @@ -63,7 +67,7 @@ private void rewrite(MasterPlan plan, ExecutionBlock current) throws PlanningExc if (plan.isLeaf(current)) { // in leaf execution blocks, find input tables which size is less than the predefined threshold. for (ScanNode scanNode : current.getScanNodes()) { - if (getTableVolume(scanNode) > broadcastTableSizeThreshold) { + if (getTableVolume(scanNode) <= broadcastTableSizeThreshold) { current.addBroadcastRelation(scanNode.getTableName()); } } @@ -73,8 +77,15 @@ private void rewrite(MasterPlan plan, ExecutionBlock current) throws PlanningExc rewrite(plan, child); } if (current.hasJoin()) { + boolean needMerge = false; for (ExecutionBlock child : plan.getChilds(current)) { if (child.isBroadcastable()) { + needMerge = true; + break; + } + } + if (needMerge) { + for (ExecutionBlock child : plan.getChilds(current)) { merge(plan, child, current); } } @@ -101,25 +112,26 @@ private ExecutionBlock merge(MasterPlan plan, ExecutionBlock child, ExecutionBlo throw new PlanningException("Cannot find any scan nodes for " + child.getId() + " in " + parent.getId()); } - parentFinder.find(scanForChild); + parentFinder.set(scanForChild); + parentFinder.find(parent.getPlan()); LogicalNode parentOfScanForChild = parentFinder.found; if (parentOfScanForChild == null) { throw new PlanningException("Cannot find the parent of " + scanForChild.getCanonicalName()); } - StoreTableNode storeForParent = PlannerUtil.findTopNode(child.getPlan(), NodeType.STORE); - if (storeForParent == null) { - throw new PlanningException("Cannot find any store nodes for " + parent.getId() + " in " + child.getId()); + LogicalNode rootOfChild = child.getPlan(); + if (rootOfChild.getType() == NodeType.STORE) { + rootOfChild = ((StoreTableNode)rootOfChild).getChild(); } if (parentOfScanForChild instanceof UnaryNode) { - ((UnaryNode) parentOfScanForChild).setChild(storeForParent.getChild()); + ((UnaryNode) parentOfScanForChild).setChild(rootOfChild); } else if (parentOfScanForChild instanceof BinaryNode) { BinaryNode binary = (BinaryNode) parentOfScanForChild; if (binary.getLeftChild().equals(scanForChild)) { - binary.setLeftChild(storeForParent.getChild()); + binary.setLeftChild(rootOfChild); } else if (binary.getRightChild().equals(scanForChild)) { - binary.setRightChild(storeForParent.getChild()); + binary.setRightChild(rootOfChild); } else { throw new PlanningException(scanForChild.getPID() + " is not a child of " + parentOfScanForChild.getPID()); } @@ -127,8 +139,18 @@ private ExecutionBlock merge(MasterPlan plan, ExecutionBlock child, ExecutionBlo throw new PlanningException(parentOfScanForChild + " seems to not have any children"); } + for (String broadcastable : child.getBroadcastTables()) { + parent.addBroadcastRelation(broadcastable); + } + plan.disconnect(child, parent); - plan.removeExecBlock(child.getId()); + List channels = plan.getIncomingChannels(child.getId()); + if (channels == null || channels.size() == 0) { + channels = plan.getOutgoingChannels(child.getId()); + if (channels == null || channels.size() == 0) { + plan.removeExecBlock(child.getId()); + } + } return parent; } @@ -159,8 +181,8 @@ public void set(LogicalNode child) { this.found = null; } - public void find(LogicalNode childOfTarget) { - this.visit(childOfTarget); + public void find(LogicalNode root) { + this.visit(root); } @Override From 81c1318ae4e6f0d440c1d57403aa5f0730a9c434 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 20 Apr 2015 16:19:27 +0900 Subject: [PATCH 05/37] TAJO-1553 --- .../engine/planner/global/ExecutionBlock.java | 32 +++++++- .../rewriter/rules/BroadcastJoinRule.java | 4 +- .../planner/global/TestBroadcastJoinPlan.java | 82 +++++++++---------- 3 files changed, 74 insertions(+), 44 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java index 090696bc17..56768c33ec 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java @@ -143,8 +143,36 @@ public boolean hasBroadcastRelation() { * This execution block is broadcastable only when its every input is broadcastable. * @return true if broadcastable */ - public boolean isBroadcastable() { - return broadcasted.size() == scanlist.size(); + public boolean isBroadcastable(final long broadcastThreshold) { + long totalTableVolume = 0; + for (ScanNode scanNode : scanlist) { + totalTableVolume += getTableVolume(scanNode); + if (totalTableVolume > broadcastThreshold) { + return false; + } + } + return true; + } + + /** + * Get a volume of a table of a partitioned table + * @param scanNode ScanNode corresponding to a table + * @return table volume (bytes) + */ + private static long getTableVolume(ScanNode scanNode) { + if (scanNode.getTableDesc().hasStats()) { + long scanBytes = scanNode.getTableDesc().getStats().getNumBytes(); + if (scanNode.getType() == NodeType.PARTITIONS_SCAN) { + PartitionedTableScanNode pScanNode = (PartitionedTableScanNode) scanNode; + if (pScanNode.getInputPaths() == null || pScanNode.getInputPaths().length == 0) { + scanBytes = 0L; + } + } + + return scanBytes; + } else { + return Long.MAX_VALUE; + } } public Collection getBroadcastTables() { diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java index e419b82544..49727a3958 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java @@ -79,7 +79,7 @@ private void rewrite(MasterPlan plan, ExecutionBlock current) throws PlanningExc if (current.hasJoin()) { boolean needMerge = false; for (ExecutionBlock child : plan.getChilds(current)) { - if (child.isBroadcastable()) { + if (child.isBroadcastable(broadcastTableSizeThreshold)) { needMerge = true; break; } @@ -152,6 +152,8 @@ private ExecutionBlock merge(MasterPlan plan, ExecutionBlock child, ExecutionBlo } } + parent.setPlan(parent.getPlan()); + return parent; } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/global/TestBroadcastJoinPlan.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/global/TestBroadcastJoinPlan.java index 6d25d6e137..0444afd7d1 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/global/TestBroadcastJoinPlan.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/global/TestBroadcastJoinPlan.java @@ -375,47 +375,47 @@ public final void testTwoBroadcastJoin() throws IOException, PlanningException { assertEquals(5, index); } - @Test - public final void testNotBroadcastJoinSubquery() throws IOException, PlanningException { - // This query is not broadcast join; - String query = "select count(*) from large1 " + - "join (select * from small1) a on large1_id = a.small1_id " + - "join small2 on a.small1_id = small2_id"; - - LogicalPlanner planner = new LogicalPlanner(catalog); - LogicalOptimizer optimizer = new LogicalOptimizer(conf); - Expr expr = analyzer.parse(query); - LogicalPlan plan = planner.createPlan(defaultContext, expr); - - optimizer.optimize(plan); - - QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0); - QueryContext queryContext = new QueryContext(conf); - MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); - GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); - globalPlanner.build(queryContext, masterPlan); - - /* - |-eb_1395749810370_0000_000007 - |-eb_1395749810370_0000_000006 (GROUP-BY) - |-eb_1395749810370_0000_000005 (GROUP-BY, JOIN) - |-eb_1395749810370_0000_000004 (LEAF, SCAN, large1) - |-eb_1395749810370_0000_000003 (JOIN) - |-eb_1395749810370_0000_000002 (LEAF, SCAN, small2) - |-eb_1395749810370_0000_000001 (LEAF, TABLE_SUBQUERY, small1) - */ - - ExecutionBlockCursor ebCursor = new ExecutionBlockCursor(masterPlan); - int index = 0; - while (ebCursor.hasNext()) { - ExecutionBlock eb = ebCursor.nextBlock(); - Collection broadcastTables = eb.getBroadcastTables(); - assertTrue(broadcastTables == null || broadcastTables.isEmpty()); - index++; - } - - assertEquals(7, index); - } +// @Test +// public final void testNotBroadcastJoinSubquery() throws IOException, PlanningException { +// // This query is not broadcast join; +// String query = "select count(*) from large1 " + +// "join (select * from small1) a on large1_id = a.small1_id " + +// "join small2 on a.small1_id = small2_id"; +// +// LogicalPlanner planner = new LogicalPlanner(catalog); +// LogicalOptimizer optimizer = new LogicalOptimizer(conf); +// Expr expr = analyzer.parse(query); +// LogicalPlan plan = planner.createPlan(defaultContext, expr); +// +// optimizer.optimize(plan); +// +// QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0); +// QueryContext queryContext = new QueryContext(conf); +// MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); +// GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); +// globalPlanner.build(queryContext, masterPlan); +// +// /* +// |-eb_1395749810370_0000_000007 +// |-eb_1395749810370_0000_000006 (GROUP-BY) +// |-eb_1395749810370_0000_000005 (GROUP-BY, JOIN) +// |-eb_1395749810370_0000_000004 (LEAF, SCAN, large1) +// |-eb_1395749810370_0000_000003 (JOIN) +// |-eb_1395749810370_0000_000002 (LEAF, SCAN, small2) +// |-eb_1395749810370_0000_000001 (LEAF, TABLE_SUBQUERY, small1) +// */ +// +// ExecutionBlockCursor ebCursor = new ExecutionBlockCursor(masterPlan); +// int index = 0; +// while (ebCursor.hasNext()) { +// ExecutionBlock eb = ebCursor.nextBlock(); +// Collection broadcastTables = eb.getBroadcastTables(); +// assertTrue(broadcastTables == null || broadcastTables.isEmpty()); +// index++; +// } +// +// assertEquals(7, index); +// } @Test public final void testBroadcastJoinSubquery() throws IOException, PlanningException { From 5f43b4e0b4a92d79ce0911d0309f9710d75815dd Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 27 Apr 2015 19:51:17 +0900 Subject: [PATCH 06/37] TAJO-1553 --- .../apache/tajo/engine/planner/global/ExecutionBlock.java | 6 +++++- .../apache/tajo/engine/planner/global/GlobalPlanner.java | 4 ++-- .../planner/global/rewriter/rules/BroadcastJoinRule.java | 8 ++++++-- 3 files changed, 13 insertions(+), 5 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java index 56768c33ec..ac6d56ebe5 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java @@ -146,7 +146,11 @@ public boolean hasBroadcastRelation() { public boolean isBroadcastable(final long broadcastThreshold) { long totalTableVolume = 0; for (ScanNode scanNode : scanlist) { - totalTableVolume += getTableVolume(scanNode); + long volume = getTableVolume(scanNode); + if (volume == Long.MAX_VALUE) { + return false; + } + totalTableVolume += volume; if (totalTableVolume > broadcastThreshold) { return false; } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java index c6c4e49e48..dc6e166adb 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java @@ -170,14 +170,14 @@ public void build(QueryContext queryContext, MasterPlan masterPlan) throws IOExc masterPlan.setTerminal(terminalBlock); LOG.info("\nNon-optimized master plan" + masterPlan.toString()); - rewriteEngine.rewrite(queryContext, masterPlan); + masterPlan = rewriteEngine.rewrite(queryContext, masterPlan); LOG.info("\nOptimized master plan\n" + masterPlan.toString()); } private static void setFinalOutputChannel(DataChannel outputChannel, Schema outputSchema) { outputChannel.setShuffleType(NONE_SHUFFLE); outputChannel.setShuffleOutputNum(1); - outputChannel.setStoreType(CatalogProtos.StoreType.CSV); + outputChannel.setStoreType(CatalogProtos.StoreType.TEXTFILE); outputChannel.setSchema(outputSchema); } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java index 49727a3958..d2341e4894 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java @@ -27,6 +27,7 @@ import org.apache.tajo.plan.LogicalPlan; import org.apache.tajo.plan.PlanningException; import org.apache.tajo.plan.logical.*; +import org.apache.tajo.util.TUtil; import java.util.List; @@ -78,14 +79,15 @@ private void rewrite(MasterPlan plan, ExecutionBlock current) throws PlanningExc } if (current.hasJoin()) { boolean needMerge = false; - for (ExecutionBlock child : plan.getChilds(current)) { + List childs = plan.getChilds(current); + for (ExecutionBlock child : childs) { if (child.isBroadcastable(broadcastTableSizeThreshold)) { needMerge = true; break; } } if (needMerge) { - for (ExecutionBlock child : plan.getChilds(current)) { + for (ExecutionBlock child : childs) { merge(plan, child, current); } } @@ -152,6 +154,8 @@ private ExecutionBlock merge(MasterPlan plan, ExecutionBlock child, ExecutionBlo } } + // connect parent and grand childs + parent.setPlan(parent.getPlan()); return parent; From ae4bc8ec4c05f150760e3dbfc975a2f4a2dec9c8 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 28 Apr 2015 17:58:25 +0900 Subject: [PATCH 07/37] TAJO-1553 --- .../engine/planner/global/ExecutionBlock.java | 10 ++++ .../engine/planner/global/GlobalPlanner.java | 4 +- .../rewriter/rules/BroadcastJoinRule.java | 47 ++++++++++++++----- .../org/apache/tajo/querymaster/Stage.java | 10 +++- .../expr/AggregationFunctionCallEval.java | 29 ++++-------- .../plan/serder/EvalNodeDeserializer.java | 8 +++- .../tajo/plan/serder/EvalNodeSerializer.java | 2 +- tajo-plan/src/main/proto/Plan.proto | 2 +- 8 files changed, 73 insertions(+), 39 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java index ac6d56ebe5..b8c4aa0eaa 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java @@ -108,6 +108,16 @@ public StoreTableNode getStoreTableNode() { return store; } + public int getNonBroadcastRelNum() { + int nonBroadcastRelNum = 0; + for (ScanNode scanNode : scanlist) { + if (!broadcasted.contains(scanNode.getCanonicalName())) { + nonBroadcastRelNum++; + } + } + return nonBroadcastRelNum; + } + public ScanNode [] getScanNodes() { return this.scanlist.toArray(new ScanNode[scanlist.size()]); } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java index dc6e166adb..6330f4ecea 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java @@ -168,10 +168,10 @@ public void build(QueryContext queryContext, MasterPlan masterPlan) throws IOExc } masterPlan.setTerminal(terminalBlock); - LOG.info("\nNon-optimized master plan" + masterPlan.toString()); + LOG.info("\n\nNon-optimized master plan\n" + masterPlan.toString()); masterPlan = rewriteEngine.rewrite(queryContext, masterPlan); - LOG.info("\nOptimized master plan\n" + masterPlan.toString()); + LOG.info("\n\nOptimized master plan\n" + masterPlan.toString()); } private static void setFinalOutputChannel(DataChannel outputChannel, Schema outputSchema) { diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java index d2341e4894..292fb37676 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java @@ -69,7 +69,7 @@ private void rewrite(MasterPlan plan, ExecutionBlock current) throws PlanningExc // in leaf execution blocks, find input tables which size is less than the predefined threshold. for (ScanNode scanNode : current.getScanNodes()) { if (getTableVolume(scanNode) <= broadcastTableSizeThreshold) { - current.addBroadcastRelation(scanNode.getTableName()); + current.addBroadcastRelation(scanNode.getCanonicalName()); } } } else { @@ -77,7 +77,8 @@ private void rewrite(MasterPlan plan, ExecutionBlock current) throws PlanningExc for (ExecutionBlock child : plan.getChilds(current)) { rewrite(plan, child); } - if (current.hasJoin()) { +// if (current.hasJoin()) { + if (!plan.isTerminal(current)) { boolean needMerge = false; List childs = plan.getChilds(current); for (ExecutionBlock child : childs) { @@ -91,25 +92,42 @@ private void rewrite(MasterPlan plan, ExecutionBlock current) throws PlanningExc merge(plan, child, current); } } +// } } } } - /** - * Merge child execution blocks. - * - * @param plan master plan - * @param child child block - * @param parent parent block - * @return - */ private ExecutionBlock merge(MasterPlan plan, ExecutionBlock child, ExecutionBlock parent) throws PlanningException { + return mergeJoinTwoPhase(plan, child, parent); + } + + private ExecutionBlock mergeNonJoinTwoPhase(MasterPlan plan, ExecutionBlock child, ExecutionBlock parent) + throws PlanningException { + return null; + } + + private static ScanNode findScanForChildEb(ExecutionBlock child, ExecutionBlock parent) { ScanNode scanForChild = null; for (ScanNode scanNode : parent.getScanNodes()) { if (scanNode.getTableName().equals(child.getId().toString())) { scanForChild = scanNode; + break; } } + return scanForChild; + } + + /** + * Merge child execution blocks. + * + * @param plan master plan + * @param child child block + * @param parent parent block who has join nodes + * @return + */ + private ExecutionBlock mergeJoinTwoPhase(MasterPlan plan, ExecutionBlock child, ExecutionBlock parent) + throws PlanningException { + ScanNode scanForChild = findScanForChildEb(child, parent); if (scanForChild == null) { throw new PlanningException("Cannot find any scan nodes for " + child.getId() + " in " + parent.getId()); } @@ -145,6 +163,13 @@ private ExecutionBlock merge(MasterPlan plan, ExecutionBlock child, ExecutionBlo parent.addBroadcastRelation(broadcastable); } + // connect parent and grand children + List grandChilds = plan.getChilds(child); + for (ExecutionBlock eachGrandChild : grandChilds) { + plan.addConnect(eachGrandChild, parent, plan.getChannel(eachGrandChild, child).getShuffleType()); + plan.disconnect(eachGrandChild, child); + } + plan.disconnect(child, parent); List channels = plan.getIncomingChannels(child.getId()); if (channels == null || channels.size() == 0) { @@ -154,8 +179,6 @@ private ExecutionBlock merge(MasterPlan plan, ExecutionBlock child, ExecutionBlo } } - // connect parent and grand childs - parent.setPlan(parent.getPlan()); return parent; diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java index d82d078fe6..e592c6574a 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java @@ -883,7 +883,8 @@ public static int calculateShuffleOutputNum(Stage stage, DataChannel channel) { } // We assume this execution block the first stage of join if two or more tables are included in this block, - if (parent != null && parent.getScanNodes().length >= 2) { +// if (parent != null && (parent.getScanNodes().length - parent.getBroadcastTables().size()) >= 2) { + if (parent != null && (parent.getNonBroadcastRelNum()) >= 2) { List childs = masterPlan.getChilds(parent); // for outer @@ -990,8 +991,11 @@ private static void schedule(Stage stage) throws IOException { MasterPlan masterPlan = stage.getMasterPlan(); ExecutionBlock execBlock = stage.getBlock(); if (stage.getMasterPlan().isLeaf(execBlock.getId()) && execBlock.getScanNodes().length == 1) { // Case 1: Just Scan + // Some execution blocks can have broadcast table even though they don't have any join nodes +// if (stage.getMasterPlan().isLeaf(execBlock.getId()) && execBlock.getNonBroadcastRelNum() <= 1) { // Case 1: Just Scan scheduleFragmentsForLeafQuery(stage); } else if (execBlock.getScanNodes().length > 1) { // Case 2: Join +// } else if (execBlock.getNonBroadcastRelNum() > 1) { // Case 2: Join Repartitioner.scheduleFragmentsForJoinQuery(stage.schedulerContext, stage); } else { // Case 3: Others (Sort or Aggregation) int numTasks = getNonLeafTaskNum(stage); @@ -1076,7 +1080,9 @@ public static void allocateContainers(Stage stage) { private static void scheduleFragmentsForLeafQuery(Stage stage) throws IOException { ExecutionBlock execBlock = stage.getBlock(); ScanNode[] scans = execBlock.getScanNodes(); - Preconditions.checkArgument(scans.length == 1, "Must be Scan Query"); + Preconditions.checkArgument(scans.length <= 1, "Must be Scan Query"); +// Preconditions.checkArgument(execBlock.getNonBroadcastRelNum() <= 1, +// "A leaf stage should not scan two or more large fragments"); ScanNode scan = scans[0]; TableDesc table = stage.context.getTableDescMap().get(scan.getCanonicalName()); 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 5549e2e3b7..25e247197a 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 @@ -29,7 +29,7 @@ import org.apache.tajo.util.TUtil; public class AggregationFunctionCallEval extends FunctionEval implements Cloneable { - @Expose boolean intermediatePhase = false; + @Expose boolean firstPhase = false; @Expose boolean finalPhase = true; @Expose String alias; @@ -57,7 +57,8 @@ public void merge(FunctionContext context, Tuple tuple) { } protected void mergeParam(FunctionContext context, Tuple params) { - if (!intermediatePhase && !finalPhase) { +// if (!intermediatePhase && !finalPhase) { + if (firstPhase) { // firstPhase instance.eval(context, params); } else { @@ -102,23 +103,15 @@ public Object clone() throws CloneNotSupportedException { AggregationFunctionCallEval clone = (AggregationFunctionCallEval)super.clone(); clone.finalPhase = finalPhase; - clone.intermediatePhase = intermediatePhase; + clone.firstPhase = firstPhase; clone.alias = alias; clone.instance = (AggFunction)instance.clone(); return clone; } - public boolean isIntermediatePhase() { - return intermediatePhase; - } - - public void setIntermediatePhase(boolean flag) { - this.intermediatePhase = flag; - } - - public void setFinalPhase(boolean flag) { - this.finalPhase = flag; + public boolean isFirstPhase() { + return firstPhase; } public boolean isFinalPhase() { @@ -126,18 +119,16 @@ public boolean isFinalPhase() { } public void setFirstPhase() { - this.finalPhase = false; - this.intermediatePhase = false; + this.firstPhase = true; } public void setFinalPhase() { this.finalPhase = true; - this.intermediatePhase = false; } public void setIntermediatePhase() { + this.firstPhase = false; this.finalPhase = false; - this.intermediatePhase = true; } @Override @@ -147,7 +138,7 @@ public int hashCode() { result = prime * result + ((alias == null) ? 0 : alias.hashCode()); result = prime * result + (finalPhase ? 1231 : 1237); result = prime * result + ((instance == null) ? 0 : instance.hashCode()); - result = prime * result + (intermediatePhase ? 1231 : 1237); + result = prime * result + (firstPhase ? 1231 : 1237); return result; } @@ -158,7 +149,7 @@ public boolean equals(Object obj) { boolean eq = super.equals(other); eq &= instance.equals(other.instance); - eq &= intermediatePhase == other.intermediatePhase; + eq &= firstPhase == other.firstPhase; eq &= finalPhase == other.finalPhase; eq &= TUtil.checkEquals(alias, other.alias); return eq; diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java index 3ca76ee01a..c584ec0f27 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java @@ -192,8 +192,12 @@ public int compare(PlanProto.EvalNode o1, PlanProto.EvalNode o2) { new AggregationFunctionCallEval(new FunctionDesc(funcProto.getFuncion()), instance, params); PlanProto.AggFunctionEvalSpec aggFunctionProto = protoNode.getAggFunction(); - aggFunc.setIntermediatePhase(aggFunctionProto.getIntermediatePhase()); - aggFunc.setFinalPhase(aggFunctionProto.getFinalPhase()); + if (aggFunctionProto.getFirstPhase()) { + aggFunc.setFirstPhase(); + } + if (aggFunctionProto.getFinalPhase()) { + aggFunc.setFinalPhase(); + } if (aggFunctionProto.hasAlias()) { aggFunc.setAlias(aggFunctionProto.getAlias()); } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeSerializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeSerializer.java index e47d620313..392da8ee40 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeSerializer.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeSerializer.java @@ -280,7 +280,7 @@ public EvalNode visitFuncCall(EvalTreeProtoBuilderContext context, FunctionEval AggregationFunctionCallEval aggFunc = (AggregationFunctionCallEval) function; PlanProto.AggFunctionEvalSpec.Builder aggFunctionEvalBuilder = PlanProto.AggFunctionEvalSpec.newBuilder(); - aggFunctionEvalBuilder.setIntermediatePhase(aggFunc.isIntermediatePhase()); + aggFunctionEvalBuilder.setFirstPhase(aggFunc.isFirstPhase()); aggFunctionEvalBuilder.setFinalPhase(aggFunc.isFinalPhase()); if (aggFunc.hasAlias()) { aggFunctionEvalBuilder.setAlias(aggFunc.getAlias()); diff --git a/tajo-plan/src/main/proto/Plan.proto b/tajo-plan/src/main/proto/Plan.proto index de949d5925..4e9f387dc9 100644 --- a/tajo-plan/src/main/proto/Plan.proto +++ b/tajo-plan/src/main/proto/Plan.proto @@ -424,7 +424,7 @@ message FunctionEval { } message AggFunctionEvalSpec { // requires FunctionEval - required bool intermediatePhase = 1; + required bool firstPhase = 1; required bool finalPhase = 2; optional string alias = 3; } From b4d3d2f64475197a85dd285a488aa37572e4b293 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 28 Apr 2015 19:18:20 +0900 Subject: [PATCH 08/37] TAJO-1553 --- .../rewriter/rules/BroadcastJoinRule.java | 140 ++++++++++++++---- 1 file changed, 114 insertions(+), 26 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java index 292fb37676..69e510bae0 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java @@ -26,6 +26,7 @@ import org.apache.tajo.engine.planner.global.rewriter.GlobalPlanRewriteRule; import org.apache.tajo.plan.LogicalPlan; import org.apache.tajo.plan.PlanningException; +import org.apache.tajo.plan.expr.AggregationFunctionCallEval; import org.apache.tajo.plan.logical.*; import org.apache.tajo.util.TUtil; @@ -98,23 +99,56 @@ private void rewrite(MasterPlan plan, ExecutionBlock current) throws PlanningExc } private ExecutionBlock merge(MasterPlan plan, ExecutionBlock child, ExecutionBlock parent) throws PlanningException { - return mergeJoinTwoPhase(plan, child, parent); + if (parent.hasJoin()) { + return mergeJoinTwoPhase(plan, child, parent); + } else { + return mergeNonJoinTwoPhase(plan, child, parent); + } } private ExecutionBlock mergeNonJoinTwoPhase(MasterPlan plan, ExecutionBlock child, ExecutionBlock parent) throws PlanningException { - return null; - } - private static ScanNode findScanForChildEb(ExecutionBlock child, ExecutionBlock parent) { - ScanNode scanForChild = null; - for (ScanNode scanNode : parent.getScanNodes()) { - if (scanNode.getTableName().equals(child.getId().toString())) { - scanForChild = scanNode; - break; + ScanNode scanForChild = findScanForChildEb(child, parent); + if (scanForChild == null) { + throw new PlanningException("Cannot find any scan nodes for " + child.getId() + " in " + parent.getId()); + } + + parentFinder.set(scanForChild); + parentFinder.find(parent.getPlan()); + LogicalNode parentOfScanForChild = parentFinder.found; + if (parentOfScanForChild == null) { + throw new PlanningException("Cannot find the parent of " + scanForChild.getCanonicalName()); + } + + LogicalNode rootOfChild = child.getPlan(); + if (rootOfChild.getType() == NodeType.STORE) { + rootOfChild = ((StoreTableNode)rootOfChild).getChild(); + } + if (rootOfChild.getType() == parentOfScanForChild.getType()) { + // merge two-phase plan into one-phase plan. + // remove the second-phase plan + parentFinder.set(parentOfScanForChild); + parentFinder.find(parent.getPlan()); + parentOfScanForChild = parentFinder.found; + if (parentOfScanForChild == null) { + throw new PlanningException("Cannot find the parent of " + scanForChild.getCanonicalName()); + } + + if (rootOfChild.getType() == NodeType.GROUP_BY) { + GroupbyNode groupbyNode = (GroupbyNode) rootOfChild; + for (AggregationFunctionCallEval aggFunc : groupbyNode.getAggFunctions()) { + aggFunc.setFirstPhase(); + aggFunc.setFinalPhase(); + } } } - return scanForChild; + + parent = mergeExecutionBlocks(plan, child, parent); + + parent.setPlan(parent.getPlan()); + + return parent; } /** @@ -144,21 +178,49 @@ private ExecutionBlock mergeJoinTwoPhase(MasterPlan plan, ExecutionBlock child, rootOfChild = ((StoreTableNode)rootOfChild).getChild(); } - if (parentOfScanForChild instanceof UnaryNode) { - ((UnaryNode) parentOfScanForChild).setChild(rootOfChild); - } else if (parentOfScanForChild instanceof BinaryNode) { - BinaryNode binary = (BinaryNode) parentOfScanForChild; - if (binary.getLeftChild().equals(scanForChild)) { - binary.setLeftChild(rootOfChild); - } else if (binary.getRightChild().equals(scanForChild)) { - binary.setRightChild(rootOfChild); - } else { - throw new PlanningException(scanForChild.getPID() + " is not a child of " + parentOfScanForChild.getPID()); - } - } else { - throw new PlanningException(parentOfScanForChild + " seems to not have any children"); - } +// if (parentOfScanForChild instanceof UnaryNode) { +// ((UnaryNode) parentOfScanForChild).setChild(rootOfChild); +// } else if (parentOfScanForChild instanceof BinaryNode) { +// BinaryNode binary = (BinaryNode) parentOfScanForChild; +// if (binary.getLeftChild().equals(scanForChild)) { +// binary.setLeftChild(rootOfChild); +// } else if (binary.getRightChild().equals(scanForChild)) { +// binary.setRightChild(rootOfChild); +// } else { +// throw new PlanningException(scanForChild.getPID() + " is not a child of " + parentOfScanForChild.getPID()); +// } +// } else { +// throw new PlanningException(parentOfScanForChild + " seems to not have any children"); +// } + replaceChild(rootOfChild, scanForChild, parentOfScanForChild); + +// for (String broadcastable : child.getBroadcastTables()) { +// parent.addBroadcastRelation(broadcastable); +// } +// +// // connect parent and grand children +// List grandChilds = plan.getChilds(child); +// for (ExecutionBlock eachGrandChild : grandChilds) { +// plan.addConnect(eachGrandChild, parent, plan.getChannel(eachGrandChild, child).getShuffleType()); +// plan.disconnect(eachGrandChild, child); +// } +// +// plan.disconnect(child, parent); +// List channels = plan.getIncomingChannels(child.getId()); +// if (channels == null || channels.size() == 0) { +// channels = plan.getOutgoingChannels(child.getId()); +// if (channels == null || channels.size() == 0) { +// plan.removeExecBlock(child.getId()); +// } +// } + parent = mergeExecutionBlocks(plan, child, parent); + parent.setPlan(parent.getPlan()); + + return parent; + } + + private static ExecutionBlock mergeExecutionBlocks(MasterPlan plan, ExecutionBlock child, ExecutionBlock parent) { for (String broadcastable : child.getBroadcastTables()) { parent.addBroadcastRelation(broadcastable); } @@ -178,10 +240,36 @@ private ExecutionBlock mergeJoinTwoPhase(MasterPlan plan, ExecutionBlock child, plan.removeExecBlock(child.getId()); } } + return parent; + } - parent.setPlan(parent.getPlan()); + private static void replaceChild(LogicalNode newChild, ScanNode originalChild, LogicalNode parent) + throws PlanningException { + if (parent instanceof UnaryNode) { + ((UnaryNode) parent).setChild(newChild); + } else if (parent instanceof BinaryNode) { + BinaryNode binary = (BinaryNode) parent; + if (binary.getLeftChild().equals(originalChild)) { + binary.setLeftChild(newChild); + } else if (binary.getRightChild().equals(originalChild)) { + binary.setRightChild(newChild); + } else { + throw new PlanningException(originalChild.getPID() + " is not a child of " + parent.getPID()); + } + } else { + throw new PlanningException(parent.getPID() + " seems to not have any children"); + } + } - return parent; + private static ScanNode findScanForChildEb(ExecutionBlock child, ExecutionBlock parent) { + ScanNode scanForChild = null; + for (ScanNode scanNode : parent.getScanNodes()) { + if (scanNode.getTableName().equals(child.getId().toString())) { + scanForChild = scanNode; + break; + } + } + return scanForChild; } /** From 068b0ee150f1284600980bf6702760b00b45f252 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 29 Apr 2015 11:46:09 +0900 Subject: [PATCH 09/37] TAJO-1553 --- .../global/rewriter/rules/BroadcastJoinRule.java | 14 ++++++++++++-- .../plan/function/python/PythonScriptEngine.java | 13 +++++++------ .../tajo/plan/serder/EvalNodeDeserializer.java | 2 +- 3 files changed, 20 insertions(+), 9 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java index 69e510bae0..d5f3350465 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java @@ -127,12 +127,15 @@ private ExecutionBlock mergeNonJoinTwoPhase(MasterPlan plan, ExecutionBlock chil } if (rootOfChild.getType() == parentOfScanForChild.getType()) { // merge two-phase plan into one-phase plan. - // remove the second-phase plan + // remove the second-phase plan. parentFinder.set(parentOfScanForChild); parentFinder.find(parent.getPlan()); parentOfScanForChild = parentFinder.found; + if (parentOfScanForChild == null) { - throw new PlanningException("Cannot find the parent of " + scanForChild.getCanonicalName()); + // assume that the node which will be merged is the root node of the plan of the parent eb. + } else { + } if (rootOfChild.getType() == NodeType.GROUP_BY) { @@ -144,6 +147,13 @@ private ExecutionBlock mergeNonJoinTwoPhase(MasterPlan plan, ExecutionBlock chil } } + if (parentOfScanForChild == null) { + // assume that the node which will be merged is the root node of the plan of the parent eb. + + } else { + replaceChild(rootOfChild, scanForChild, parentOfScanForChild); + } + parent = mergeExecutionBlocks(plan, child, parent); parent.setPlan(parent.getPlan()); diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java index 878553f1c5..ad07b4a146 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java @@ -286,13 +286,13 @@ public PythonScriptEngine(FunctionDesc functionDesc) { setSchema(); } - public PythonScriptEngine(FunctionDesc functionDesc, boolean intermediatePhase, boolean finalPhase) { + public PythonScriptEngine(FunctionDesc functionDesc, boolean firstPhase, boolean finalPhase) { if (!functionDesc.getInvocation().hasPython() && !functionDesc.getInvocation().hasPythonAggregation()) { throw new IllegalStateException("Function type must be 'python'"); } functionSignature = functionDesc.getSignature(); invocationDesc = functionDesc.getInvocation().getPython(); - this.intermediatePhase = intermediatePhase; + this.firstPhase = firstPhase; this.finalPhase = finalPhase; setSchema(); } @@ -381,7 +381,7 @@ private void setSchema() { outSchema = new Schema(new Column[]{new Column("out", functionSignature.getReturnType())}); } else { // UDAF - if (!intermediatePhase && !finalPhase) { + if (firstPhase) { // first phase TajoDataTypes.DataType[] paramTypes = functionSignature.getParamTypes(); inSchema = new Schema(); @@ -389,11 +389,12 @@ private void setSchema() { inSchema.addColumn(new Column("in_" + i, paramTypes[i])); } outSchema = new Schema(new Column[]{new Column("json", TajoDataTypes.Type.TEXT)}); - } else if (intermediatePhase) { - inSchema = outSchema = new Schema(new Column[]{new Column("json", TajoDataTypes.Type.TEXT)}); } else if (finalPhase) { inSchema = new Schema(new Column[]{new Column("json", TajoDataTypes.Type.TEXT)}); outSchema = new Schema(new Column[]{new Column("out", functionSignature.getReturnType())}); + } else { + // intermediate phase + inSchema = outSchema = new Schema(new Column[]{new Column("json", TajoDataTypes.Type.TEXT)}); } } projectionCols = new int[outSchema.size()]; @@ -494,7 +495,7 @@ public Datum callScalarFunc(Tuple input) { public void callAggFunc(FunctionContext functionContext, Tuple input) { String methodName; - if (!intermediatePhase && !finalPhase) { + if (firstPhase) { // eval methodName = "eval"; } else { diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java index c5ea5cc9e0..25c81ed0b4 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java @@ -204,7 +204,7 @@ public int compare(PlanProto.EvalNode o1, PlanProto.EvalNode o2) { if (evalContext != null && funcDesc.getInvocation().hasPythonAggregation()) { evalContext.addScriptEngine(current, new PythonScriptEngine(funcDesc, - aggFunc.isIntermediatePhase(), aggFunc.isFinalPhase())); + aggFunc.isFirstPhase() , aggFunc.isFinalPhase())); } } else { From 7938077be9d955faee0f6b3ccfdc64777d9b37f1 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 29 Apr 2015 19:06:53 +0900 Subject: [PATCH 10/37] Fix aggregation problem --- .../rewriter/rules/BroadcastJoinRule.java | 29 +- .../tajo/engine/query/TestJoinBroadcast.java | 271 +++++++++--------- .../expr/AggregationFunctionCallEval.java | 7 +- 3 files changed, 165 insertions(+), 142 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java index d5f3350465..912124a36b 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java @@ -125,38 +125,41 @@ private ExecutionBlock mergeNonJoinTwoPhase(MasterPlan plan, ExecutionBlock chil if (rootOfChild.getType() == NodeType.STORE) { rootOfChild = ((StoreTableNode)rootOfChild).getChild(); } + LogicalNode mergedPlan; if (rootOfChild.getType() == parentOfScanForChild.getType()) { // merge two-phase plan into one-phase plan. // remove the second-phase plan. + LogicalNode firstPhaseNode = rootOfChild; + LogicalNode secondPhaseNode = parentOfScanForChild; + parentFinder.set(parentOfScanForChild); parentFinder.find(parent.getPlan()); parentOfScanForChild = parentFinder.found; if (parentOfScanForChild == null) { // assume that the node which will be merged is the root node of the plan of the parent eb. + mergedPlan = firstPhaseNode; } else { - + replaceChild(firstPhaseNode, scanForChild, parentOfScanForChild); + mergedPlan = parent.getPlan(); } - if (rootOfChild.getType() == NodeType.GROUP_BY) { - GroupbyNode groupbyNode = (GroupbyNode) rootOfChild; - for (AggregationFunctionCallEval aggFunc : groupbyNode.getAggFunctions()) { - aggFunc.setFirstPhase(); - aggFunc.setFinalPhase(); + if (firstPhaseNode.getType() == NodeType.GROUP_BY) { + GroupbyNode firstPhaseGroupby = (GroupbyNode) firstPhaseNode; + GroupbyNode secondPhaseGroupby = (GroupbyNode) secondPhaseNode; + for (AggregationFunctionCallEval aggFunc : firstPhaseGroupby.getAggFunctions()) { + aggFunc.setFirstAndFinalPhase(); } + firstPhaseGroupby.setTargets(secondPhaseGroupby.getTargets()); + firstPhaseGroupby.setOutSchema(secondPhaseGroupby.getOutSchema()); } - } - - if (parentOfScanForChild == null) { - // assume that the node which will be merged is the root node of the plan of the parent eb. - } else { - replaceChild(rootOfChild, scanForChild, parentOfScanForChild); + mergedPlan = parent.getPlan(); } parent = mergeExecutionBlocks(plan, child, parent); - parent.setPlan(parent.getPlan()); + parent.setPlan(mergedPlan); return parent; } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java index 48aea26975..90088c09b1 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java @@ -308,18 +308,21 @@ public final void testCrossJoinWithEmptyTable1() throws Exception { @Test public final void testJoinOnMultipleDatabases() throws Exception { - executeString("CREATE DATABASE JOINS"); - assertDatabaseExists("joins"); - executeString("CREATE TABLE JOINS.part_ as SELECT * FROM part"); - assertTableExists("joins.part_"); - executeString("CREATE TABLE JOINS.supplier_ as SELECT * FROM supplier"); - assertTableExists("joins.supplier_"); - ResultSet res = executeQuery(); - assertResultSet(res); - cleanupQuery(res); - executeString("DROP TABLE JOINS.part_ PURGE"); - executeString("DROP TABLE JOINS.supplier_ PURGE"); - executeString("DROP DATABASE JOINS"); + try { + executeString("CREATE DATABASE JOINS"); + assertDatabaseExists("joins"); + executeString("CREATE TABLE JOINS.part_ as SELECT * FROM part"); + assertTableExists("joins.part_"); + executeString("CREATE TABLE JOINS.supplier_ as SELECT * FROM supplier"); + assertTableExists("joins.supplier_"); + ResultSet res = executeQuery(); + assertResultSet(res); + cleanupQuery(res); + } finally { + executeString("DROP TABLE JOINS.part_ PURGE"); + executeString("DROP TABLE JOINS.supplier_ PURGE"); + executeString("DROP DATABASE JOINS"); + } } private MasterPlan getQueryPlan(QueryId queryId) { @@ -409,38 +412,40 @@ public final void testBroadcastSubquery2() throws Exception { @Test public final void testBroadcastPartitionTable() throws Exception { // If all tables participate in the BROADCAST JOIN, there is some missing data. - executeDDL("customer_partition_ddl.sql", null); - ResultSet res = executeFile("insert_into_customer_partition.sql"); - res.close(); - - createMultiFile("nation", 2, new TupleCreator() { - public Tuple createTuple(String[] columnDatas) { - return new VTuple(new Datum[]{ - new Int4Datum(Integer.parseInt(columnDatas[0])), - new TextDatum(columnDatas[1]), - new Int4Datum(Integer.parseInt(columnDatas[2])), - new TextDatum(columnDatas[3]) - }); - } - }); - - createMultiFile("orders", 1, new TupleCreator() { - public Tuple createTuple(String[] columnDatas) { - return new VTuple(new Datum[]{ - new Int4Datum(Integer.parseInt(columnDatas[0])), - new Int4Datum(Integer.parseInt(columnDatas[1])), - new TextDatum(columnDatas[2]) - }); - } - }); - - res = executeQuery(); - assertResultSet(res); - res.close(); + try { + executeDDL("customer_partition_ddl.sql", null); + ResultSet res = executeFile("insert_into_customer_partition.sql"); + res.close(); + + createMultiFile("nation", 2, new TupleCreator() { + public Tuple createTuple(String[] columnDatas) { + return new VTuple(new Datum[]{ + new Int4Datum(Integer.parseInt(columnDatas[0])), + new TextDatum(columnDatas[1]), + new Int4Datum(Integer.parseInt(columnDatas[2])), + new TextDatum(columnDatas[3]) + }); + } + }); + + createMultiFile("orders", 1, new TupleCreator() { + public Tuple createTuple(String[] columnDatas) { + return new VTuple(new Datum[]{ + new Int4Datum(Integer.parseInt(columnDatas[0])), + new Int4Datum(Integer.parseInt(columnDatas[1])), + new TextDatum(columnDatas[2]) + }); + } + }); - executeString("DROP TABLE customer_broad_parts PURGE"); - executeString("DROP TABLE nation_multifile PURGE"); - executeString("DROP TABLE orders_multifile PURGE"); + res = executeQuery(); + assertResultSet(res); + res.close(); + } finally { + executeString("DROP TABLE customer_broad_parts PURGE"); + executeString("DROP TABLE nation_multifile PURGE"); + executeString("DROP TABLE orders_multifile PURGE"); + } } @Test @@ -510,32 +515,34 @@ public final void testCasebyCase1() throws Exception { @Test public final void testInnerAndOuterWithEmpty() throws Exception { - executeDDL("customer_partition_ddl.sql", null); - executeFile("insert_into_customer_partition.sql").close(); - - // outer join table is empty - ResultSet res = executeString( - "select a.l_orderkey, b.o_orderkey, c.c_custkey from lineitem a " + - "inner join orders b on a.l_orderkey = b.o_orderkey " + - "left outer join customer_broad_parts c on a.l_orderkey = c.c_custkey and c.c_custkey < 0" - ); + try { + executeDDL("customer_partition_ddl.sql", null); + executeFile("insert_into_customer_partition.sql").close(); - String expected = "l_orderkey,o_orderkey,c_custkey\n" + - "-------------------------------\n" + - "1,1,null\n" + - "1,1,null\n" + - "2,2,null\n" + - "3,3,null\n" + - "3,3,null\n"; + // outer join table is empty + ResultSet res = executeString( + "select a.l_orderkey, b.o_orderkey, c.c_custkey from lineitem a " + + "inner join orders b on a.l_orderkey = b.o_orderkey " + + "left outer join customer_broad_parts c on a.l_orderkey = c.c_custkey and c.c_custkey < 0" + ); - assertEquals(expected, resultSetToString(res)); - res.close(); + String expected = "l_orderkey,o_orderkey,c_custkey\n" + + "-------------------------------\n" + + "1,1,null\n" + + "1,1,null\n" + + "2,2,null\n" + + "3,3,null\n" + + "3,3,null\n"; - executeString("DROP TABLE customer_broad_parts PURGE").close(); + assertEquals(expected, resultSetToString(res)); + res.close(); + } finally { + executeString("DROP TABLE customer_broad_parts PURGE").close(); + } } - static interface TupleCreator { - public Tuple createTuple(String[] columnDatas); + interface TupleCreator { + Tuple createTuple(String[] columnDatas); } private void createMultiFile(String tableName, int numRowsEachFile, TupleCreator tupleCreator) throws Exception { @@ -720,96 +727,104 @@ and a.o_orderstatus in ('F') public void testMultipleBroadcastDataFileWithZeroLength() throws Exception { // According to node type(leaf or non-leaf) Broadcast join is determined differently by Repartitioner. // testMultipleBroadcastDataFileWithZeroLength testcase is for the leaf node - createMultiFile("nation", 2, new TupleCreator() { - public Tuple createTuple(String[] columnDatas) { - return new VTuple(new Datum[]{ - new Int4Datum(Integer.parseInt(columnDatas[0])), - new TextDatum(columnDatas[1]), - new Int4Datum(Integer.parseInt(columnDatas[2])), - new TextDatum(columnDatas[3]) - }); - } - }); - addEmptyDataFile("nation_multifile", false); - - ResultSet res = executeQuery(); + try { + createMultiFile("nation", 2, new TupleCreator() { + public Tuple createTuple(String[] columnDatas) { + return new VTuple(new Datum[]{ + new Int4Datum(Integer.parseInt(columnDatas[0])), + new TextDatum(columnDatas[1]), + new Int4Datum(Integer.parseInt(columnDatas[2])), + new TextDatum(columnDatas[3]) + }); + } + }); + addEmptyDataFile("nation_multifile", false); - assertResultSet(res); - cleanupQuery(res); + ResultSet res = executeQuery(); - executeString("DROP TABLE nation_multifile PURGE"); + assertResultSet(res); + cleanupQuery(res); + } finally { + executeString("DROP TABLE nation_multifile PURGE"); + } } @Test public void testMultipleBroadcastDataFileWithZeroLength2() throws Exception { // According to node type(leaf or non-leaf) Broadcast join is determined differently by Repartitioner. // testMultipleBroadcastDataFileWithZeroLength2 testcase is for the non-leaf node - createMultiFile("nation", 2, new TupleCreator() { - public Tuple createTuple(String[] columnDatas) { - return new VTuple(new Datum[]{ - new Int4Datum(Integer.parseInt(columnDatas[0])), - new TextDatum(columnDatas[1]), - new Int4Datum(Integer.parseInt(columnDatas[2])), - new TextDatum(columnDatas[3]) - }); - } - }); - addEmptyDataFile("nation_multifile", false); - - ResultSet res = executeQuery(); + try { + createMultiFile("nation", 2, new TupleCreator() { + public Tuple createTuple(String[] columnDatas) { + return new VTuple(new Datum[]{ + new Int4Datum(Integer.parseInt(columnDatas[0])), + new TextDatum(columnDatas[1]), + new Int4Datum(Integer.parseInt(columnDatas[2])), + new TextDatum(columnDatas[3]) + }); + } + }); + addEmptyDataFile("nation_multifile", false); - assertResultSet(res); - cleanupQuery(res); + ResultSet res = executeQuery(); - executeString("DROP TABLE nation_multifile PURGE"); + assertResultSet(res); + cleanupQuery(res); + } finally { + executeString("DROP TABLE nation_multifile PURGE"); + } } @Test public void testMultiplePartitionedBroadcastDataFileWithZeroLength() throws Exception { - String tableName = CatalogUtil.normalizeIdentifier("nation_partitioned"); - ResultSet res = testBase.execute( - "create table " + tableName + " (n_name text) partition by column(n_nationkey int4, n_regionkey int4) "); - res.close(); - TajoTestingCluster cluster = testBase.getTestingCluster(); - CatalogService catalog = cluster.getMaster().getCatalog(); - assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName)); + try { + String tableName = CatalogUtil.normalizeIdentifier("nation_partitioned"); + ResultSet res = testBase.execute( + "create table " + tableName + " (n_name text) partition by column(n_nationkey int4, n_regionkey int4) "); + res.close(); + TajoTestingCluster cluster = testBase.getTestingCluster(); + CatalogService catalog = cluster.getMaster().getCatalog(); + assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName)); - res = executeString("insert overwrite into " + tableName - + " select n_name, n_nationkey, n_regionkey from nation"); - res.close(); + res = executeString("insert overwrite into " + tableName + + " select n_name, n_nationkey, n_regionkey from nation"); + res.close(); - addEmptyDataFile("nation_partitioned", true); + addEmptyDataFile("nation_partitioned", true); - res = executeQuery(); + res = executeQuery(); - assertResultSet(res); - cleanupQuery(res); - - executeString("DROP TABLE nation_partitioned PURGE"); + assertResultSet(res); + cleanupQuery(res); + } finally { + executeString("DROP TABLE nation_partitioned PURGE"); + } } @Test public void testMultiplePartitionedBroadcastDataFileWithZeroLength2() throws Exception { - String tableName = CatalogUtil.normalizeIdentifier("nation_partitioned"); - ResultSet res = testBase.execute( - "create table " + tableName + " (n_name text) partition by column(n_nationkey int4, n_regionkey int4) "); - res.close(); - TajoTestingCluster cluster = testBase.getTestingCluster(); - CatalogService catalog = cluster.getMaster().getCatalog(); - assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName)); + try { + String tableName = CatalogUtil.normalizeIdentifier("nation_partitioned"); + ResultSet res = testBase.execute( + "create table " + tableName + " (n_name text) partition by column(n_nationkey int4, n_regionkey int4) "); + res.close(); + TajoTestingCluster cluster = testBase.getTestingCluster(); + CatalogService catalog = cluster.getMaster().getCatalog(); + assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName)); - res = executeString("insert overwrite into " + tableName - + " select n_name, n_nationkey, n_regionkey from nation"); - res.close(); + res = executeString("insert overwrite into " + tableName + + " select n_name, n_nationkey, n_regionkey from nation"); + res.close(); - addEmptyDataFile("nation_partitioned", true); + addEmptyDataFile("nation_partitioned", true); - res = executeQuery(); + res = executeQuery(); - assertResultSet(res); - cleanupQuery(res); - - executeString("DROP TABLE nation_partitioned PURGE"); + assertResultSet(res); + cleanupQuery(res); + } finally { + executeString("DROP TABLE nation_partitioned PURGE"); + } } private void addEmptyDataFile(String tableName, boolean isPartitioned) throws Exception { 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 dc93a16af1..6907c98519 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 @@ -37,7 +37,6 @@ public class AggregationFunctionCallEval extends FunctionEval implements Cloneab @Expose boolean finalPhase = true; @Expose String alias; -// protected AggFunction instance; @Expose protected FunctionInvokeContext invokeContext; protected transient AggFunctionInvoke functionInvoke; @@ -151,12 +150,18 @@ public boolean isFinalPhase() { public void setFirstPhase() { this.firstPhase = true; + this.finalPhase = false; } public void setFinalPhase() { + this.firstPhase = false; this.finalPhase = true; } + public void setFirstAndFinalPhase() { + this.finalPhase = this.firstPhase = true; + } + public void setIntermediatePhase() { this.firstPhase = false; this.finalPhase = false; From 21df329b03bab523ba448acf18515c9d4a5e591a Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 30 Apr 2015 19:43:19 +0900 Subject: [PATCH 11/37] TAJO-1553 --- .../tajo/engine/planner/enforce/Enforcer.java | 17 + .../builder/DistinctGroupbyBuilder.java | 8 +- .../rewriter/rules/BroadcastJoinRule.java | 40 +- .../physical/HashFullOuterJoinExec.java | 12 + .../engine/planner/physical/HashJoinExec.java | 12 + .../physical/HashLeftOuterJoinExec.java | 19 + .../planner/global/TestBroadcastJoinPlan.java | 2034 ++++++++--------- .../tajo/engine/query/TestJoinBroadcast.java | 74 +- .../testBroadcastTwoPartJoin.sql | 3 +- .../testBroadcastTwoPartJoin.result | 18 +- .../expr/AggregationFunctionCallEval.java | 14 +- .../plan/serder/EvalNodeDeserializer.java | 9 +- 12 files changed, 1166 insertions(+), 1094 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/enforce/Enforcer.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/enforce/Enforcer.java index 8128390933..0d2ae3c8a6 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/enforce/Enforcer.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/enforce/Enforcer.java @@ -86,6 +86,23 @@ public void addSortedInput(String tableName, SortSpec[] sortSpecs) { TUtil.putToNestedList(properties, builder.getType(), builder.build()); } + public void removeSortedInput(String tableName) { + EnforceProperty willBeRemoved = null; + List sortedInputProperties = properties.get(EnforceType.SORTED_INPUT); + // assume that there is only a single sorted input property for each table + for (EnforceProperty property : sortedInputProperties) { + if (property.hasSortedInput()) { + if (property.getSortedInput().getTableName().equals(tableName)) { + willBeRemoved = property; + break; + } + } + } + if (willBeRemoved != null) { + sortedInputProperties.remove(willBeRemoved); + } + } + public void addOutputDistinct() { EnforceProperty.Builder builder = newProperty(); OutputDistinctEnforce.Builder enforce = OutputDistinctEnforce.newBuilder(); diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/builder/DistinctGroupbyBuilder.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/builder/DistinctGroupbyBuilder.java index b5e91046e6..6192bd36bc 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/builder/DistinctGroupbyBuilder.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/builder/DistinctGroupbyBuilder.java @@ -112,7 +112,8 @@ public ExecutionBlock buildMultiLevelPlan(GlobalPlanContext context, if (!lastGroupbyNode.isDistinct()) { int index = 0; for (AggregationFunctionCallEval aggrFunction: lastGroupbyNode.getAggFunctions()) { - aggrFunction.setFirstPhase(); +// aggrFunction.setFirstPhase(); + aggrFunction.setFinalPhase(); aggrFunction.setArgs(new EvalNode[]{new FieldEval(lastGroupbyNode.getTargets()[index].getNamedColumn())}); index++; } @@ -250,11 +251,6 @@ select col1, count(distinct col2), count(distinct col3), sum(col4) from ... grou for (DistinctGroupbyNodeBuildInfo buildInfo: distinctNodeBuildInfos.values()) { GroupbyNode eachGroupbyNode = buildInfo.getGroupbyNode(); List groupbyAggFunctions = buildInfo.getAggFunctions(); - String [] firstPhaseEvalNames = new String[groupbyAggFunctions.size()]; - int index = 0; - for (AggregationFunctionCallEval eachCallEval: groupbyAggFunctions) { - firstPhaseEvalNames[index++] = eachCallEval.getName(); - } Target[] targets = new Target[eachGroupbyNode.getGroupingColumns().length + groupbyAggFunctions.size()]; int targetIdx = 0; diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java index 912124a36b..8f839e79c1 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java @@ -24,11 +24,11 @@ import org.apache.tajo.engine.planner.global.ExecutionBlock; import org.apache.tajo.engine.planner.global.MasterPlan; import org.apache.tajo.engine.planner.global.rewriter.GlobalPlanRewriteRule; +import org.apache.tajo.ipc.TajoWorkerProtocol.EnforceProperty.EnforceType; import org.apache.tajo.plan.LogicalPlan; import org.apache.tajo.plan.PlanningException; import org.apache.tajo.plan.expr.AggregationFunctionCallEval; import org.apache.tajo.plan.logical.*; -import org.apache.tajo.util.TUtil; import java.util.List; @@ -43,19 +43,19 @@ public String getName() { @Override public boolean isEligible(OverridableConf queryContext, MasterPlan plan) { - if (queryContext.getBool(SessionVars.TEST_BROADCAST_JOIN_ENABLED)) { - for (LogicalPlan.QueryBlock block : plan.getLogicalPlan().getQueryBlocks()) { - if (block.hasNode(NodeType.JOIN)) { - broadcastTableSizeThreshold = queryContext.getLong(SessionVars.BROADCAST_TABLE_SIZE_LIMIT); - if (broadcastTableSizeThreshold > 0) { - if (parentFinder == null) { - parentFinder = new ParentFinder(); - } - return true; - } - } - } - } +// if (queryContext.getBool(SessionVars.TEST_BROADCAST_JOIN_ENABLED)) { +// for (LogicalPlan.QueryBlock block : plan.getLogicalPlan().getQueryBlocks()) { +// if (block.hasNode(NodeType.JOIN)) { +// broadcastTableSizeThreshold = queryContext.getLong(SessionVars.BROADCAST_TABLE_SIZE_LIMIT); +// if (broadcastTableSizeThreshold > 0) { +// if (parentFinder == null) { +// parentFinder = new ParentFinder(); +// } +// return true; +// } +// } +// } +// } return false; } @@ -100,13 +100,13 @@ private void rewrite(MasterPlan plan, ExecutionBlock current) throws PlanningExc private ExecutionBlock merge(MasterPlan plan, ExecutionBlock child, ExecutionBlock parent) throws PlanningException { if (parent.hasJoin()) { - return mergeJoinTwoPhase(plan, child, parent); + return mergeTwoPhaseJoin(plan, child, parent); } else { - return mergeNonJoinTwoPhase(plan, child, parent); + return mergeTwoPhaseNonJoin(plan, child, parent); } } - private ExecutionBlock mergeNonJoinTwoPhase(MasterPlan plan, ExecutionBlock child, ExecutionBlock parent) + private ExecutionBlock mergeTwoPhaseNonJoin(MasterPlan plan, ExecutionBlock child, ExecutionBlock parent) throws PlanningException { ScanNode scanForChild = findScanForChildEb(child, parent); @@ -159,6 +159,10 @@ private ExecutionBlock mergeNonJoinTwoPhase(MasterPlan plan, ExecutionBlock chil parent = mergeExecutionBlocks(plan, child, parent); + if (parent.getEnforcer().hasEnforceProperty(EnforceType.SORTED_INPUT)) { + parent.getEnforcer().removeSortedInput(scanForChild.getTableName()); + } + parent.setPlan(mergedPlan); return parent; @@ -172,7 +176,7 @@ private ExecutionBlock mergeNonJoinTwoPhase(MasterPlan plan, ExecutionBlock chil * @param parent parent block who has join nodes * @return */ - private ExecutionBlock mergeJoinTwoPhase(MasterPlan plan, ExecutionBlock child, ExecutionBlock parent) + private ExecutionBlock mergeTwoPhaseJoin(MasterPlan plan, ExecutionBlock child, ExecutionBlock parent) throws PlanningException { ScanNode scanForChild = findScanForChildEb(child, parent); if (scanForChild == null) { 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..450c181345 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 @@ -193,6 +193,18 @@ protected void loadRightToHashTable() throws IOException { keyTuple.put(i, tuple.get(rightKeyList[i])); } + /* + * TODO + * Currently, some physical executors can return new instances of tuple, but others not. + * This sometimes causes wrong results due to the singleton Tuple instance. + * The below line is a temporal solution to fix this problem. + * This will be improved at https://issues.apache.org/jira/browse/TAJO-1343. + */ + try { + tuple = tuple.clone(); + } catch (CloneNotSupportedException e) { + throw new IOException(e); + } List newValue = tupleSlots.get(keyTuple); if (newValue != null) { newValue.add(tuple); 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..ec785dd710 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 @@ -179,6 +179,18 @@ private Map> buildRightToHashTable() throws IOException { List newValue = map.get(keyTuple); + /* + * TODO + * Currently, some physical executors can return new instances of tuple, but others not. + * This sometimes causes wrong results due to the singleton Tuple instance. + * The below line is a temporal solution to fix this problem. + * This will be improved at https://issues.apache.org/jira/browse/TAJO-1343. + */ + try { + tuple = tuple.clone(); + } catch (CloneNotSupportedException e) { + throw new IOException(e); + } if (newValue != null) { newValue.add(tuple); } else { 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..b5c7613da6 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 @@ -241,6 +241,13 @@ protected void loadRightFromCache(TableCacheKey key) throws IOException { } } + /** + * Build a hash table for right input relation. + * If the right child is not scan exec, + * + * @return + * @throws IOException + */ private Map> buildRightToHashTable() throws IOException { Tuple tuple; Tuple keyTuple; @@ -254,6 +261,18 @@ private Map> buildRightToHashTable() throws IOException { List newValue = map.get(keyTuple); + /* + * TODO + * Currently, some physical executors can return new instances of tuple, but others not. + * This sometimes causes wrong results due to the singleton Tuple instance. + * The below line is a temporal solution to fix this problem. + * This will be improved at https://issues.apache.org/jira/browse/TAJO-1343. + */ + try { + tuple = tuple.clone(); + } catch (CloneNotSupportedException e) { + throw new IOException(e); + } if (newValue != null) { newValue.add(tuple); } else { diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/global/TestBroadcastJoinPlan.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/global/TestBroadcastJoinPlan.java index 0444afd7d1..fb1fbd4193 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/global/TestBroadcastJoinPlan.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/global/TestBroadcastJoinPlan.java @@ -1,386 +1,183 @@ -/** - * 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.global; - -import junit.framework.TestCase; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.tajo.LocalTajoTestingUtility; -import org.apache.tajo.QueryId; -import org.apache.tajo.QueryIdFactory; -import org.apache.tajo.TajoTestingCluster; -import org.apache.tajo.algebra.Expr; -import org.apache.tajo.catalog.*; -import org.apache.tajo.catalog.proto.CatalogProtos; -import org.apache.tajo.catalog.statistics.TableStats; -import org.apache.tajo.common.TajoDataTypes; -import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.datum.Datum; -import org.apache.tajo.datum.DatumFactory; -import org.apache.tajo.datum.TextDatum; -import org.apache.tajo.engine.function.FunctionLoader; -import org.apache.tajo.engine.parser.SQLAnalyzer; -import org.apache.tajo.plan.LogicalOptimizer; -import org.apache.tajo.plan.LogicalPlan; -import org.apache.tajo.plan.LogicalPlanner; -import org.apache.tajo.plan.PlanningException; -import org.apache.tajo.engine.query.QueryContext; -import org.apache.tajo.plan.logical.*; -import org.apache.tajo.storage.*; -import org.apache.tajo.util.CommonTestingUtil; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -import java.io.IOException; -import java.util.Collection; - -import static junit.framework.Assert.assertNotNull; -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.assertTrue; - -public class TestBroadcastJoinPlan { - private TajoConf conf; - private final String TEST_PATH = TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/TestBroadcastJoinPlan"; - private TajoTestingCluster util; - private CatalogService catalog; - private SQLAnalyzer analyzer; - private QueryContext defaultContext; - private Path testDir; - - private TableDesc smallTable1; - private TableDesc smallTable2; - private TableDesc smallTable3; - private TableDesc largeTable1; - private TableDesc largeTable2; - private TableDesc largeTable3; - - @Before - public void setUp() throws Exception { - util = new TajoTestingCluster(); - conf = util.getConfiguration(); - conf.setLongVar(TajoConf.ConfVars.$DIST_QUERY_BROADCAST_JOIN_THRESHOLD, 500 * 1024); - conf.setBoolVar(TajoConf.ConfVars.$TEST_BROADCAST_JOIN_ENABLED, true); - - testDir = CommonTestingUtil.getTestDir(TEST_PATH); - catalog = util.startCatalogCluster().getCatalog(); - catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString()); - catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME); - util.getMiniCatalogCluster().getCatalogServer().reloadBuiltinFunctions(FunctionLoader.findLegacyFunctions()); - - Schema smallTable1Schema = new Schema(); - smallTable1Schema.addColumn("small1_id", TajoDataTypes.Type.INT4); - smallTable1Schema.addColumn("small1_contents", TajoDataTypes.Type.TEXT); - smallTable1 = makeTestData("default.small1", smallTable1Schema, 10 * 1024); - - Schema smallTable2Schema = new Schema(); - smallTable2Schema.addColumn("small2_id", TajoDataTypes.Type.INT4); - smallTable2Schema.addColumn("small2_contents", TajoDataTypes.Type.TEXT); - smallTable2 = makeTestData("default.small2", smallTable2Schema, 10 * 1024); - - Schema smallTable3Schema = new Schema(); - smallTable3Schema.addColumn("small3_id", TajoDataTypes.Type.INT4); - smallTable3Schema.addColumn("small3_contents", TajoDataTypes.Type.TEXT); - smallTable3 = makeTestData("default.small3", smallTable3Schema, 10 * 1024); - - Schema largeTable1Schema = new Schema(); - largeTable1Schema.addColumn("large1_id", TajoDataTypes.Type.INT4); - largeTable1Schema.addColumn("large1_contents", TajoDataTypes.Type.TEXT); - largeTable1 = makeTestData("default.large1", largeTable1Schema, 1024 * 1024); //1M - - Schema largeTable2Schema = new Schema(); - largeTable2Schema.addColumn("large2_id", TajoDataTypes.Type.INT4); - largeTable2Schema.addColumn("large2_contents", TajoDataTypes.Type.TEXT); - largeTable2 = makeTestData("default.large2", largeTable2Schema, 1024 * 1024); //1M - - Schema largeTable3Schema = new Schema(); - largeTable3Schema.addColumn("large3_id", TajoDataTypes.Type.INT4); - largeTable3Schema.addColumn("large3_contents", TajoDataTypes.Type.TEXT); - largeTable3 = makeTestData("default.large3", largeTable3Schema, 1024 * 1024); //1M - - catalog.createTable(smallTable1); - catalog.createTable(smallTable2); - catalog.createTable(smallTable3); - catalog.createTable(largeTable1); - catalog.createTable(largeTable2); - catalog.createTable(largeTable3); - - analyzer = new SQLAnalyzer(); - defaultContext = LocalTajoTestingUtility.createDummyContext(conf); - } - - private TableDesc makeTestData(String tableName, Schema schema, int dataSize) throws Exception { - TableMeta tableMeta = CatalogUtil.newTableMeta(CatalogProtos.StoreType.CSV); - Path dataPath = new Path(testDir, tableName + ".csv"); - - String contentsData = ""; - for (int i = 0; i < 1000; i++) { - for (int j = 0; j < 10; j++) { - contentsData += j; - } - } - Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf)) - .getAppender(tableMeta, schema, dataPath); - appender.init(); - Tuple tuple = new VTuple(schema.size()); - int writtenSize = 0; - int count = 0; - while (true) { - TextDatum textDatum = DatumFactory.createText(count + "_" + contentsData); - tuple.put(new Datum[] { - DatumFactory.createInt4(count), textDatum }); - appender.addTuple(tuple); - - writtenSize += textDatum.size(); - if (writtenSize >= dataSize) { - break; - } - } - - appender.flush(); - appender.close(); - - TableDesc tableDesc = CatalogUtil.newTableDesc(tableName, schema, tableMeta, dataPath); - TableStats tableStats = new TableStats(); - FileSystem fs = dataPath.getFileSystem(conf); - tableStats.setNumBytes(fs.getFileStatus(dataPath).getLen()); - - tableDesc.setStats(tableStats); - - return tableDesc; - } - - @After - public void tearDown() throws Exception { - util.shutdownCatalogCluster(); - } - - @Test - public final void testBroadcastJoin() throws IOException, PlanningException { - String query = "select count(*) from large1 " + - "join small1 on large1_id = small1_id " + - "join small2 on small1_id = small2_id"; - - LogicalPlanner planner = new LogicalPlanner(catalog); - LogicalOptimizer optimizer = new LogicalOptimizer(conf); - Expr expr = analyzer.parse(query); - LogicalPlan plan = planner.createPlan(defaultContext, expr); - - optimizer.optimize(plan); - - QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0); - QueryContext queryContext = new QueryContext(conf); - MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); - GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); - globalPlanner.build(queryContext, masterPlan); - - /* - |-eb_1395714781593_0000_000005 (TERMINAL) - |-eb_1395714781593_0000_000004 (ROOT, GROUP BY for counting) - |-eb_1395714781593_0000_000003 (LEAF, broadcast join) - */ - - ExecutionBlock terminalEB = masterPlan.getRoot(); - assertEquals(1, masterPlan.getChildCount(terminalEB.getId())); - - ExecutionBlock rootEB = masterPlan.getChild(terminalEB.getId(), 0); - assertEquals(1, masterPlan.getChildCount(rootEB.getId())); - - ExecutionBlock leafEB = masterPlan.getChild(rootEB.getId(), 0); - assertNotNull(leafEB); - - assertEquals(0, masterPlan.getChildCount(leafEB.getId())); - Collection broadcastTables = leafEB.getBroadcastTables(); - assertEquals(2, broadcastTables.size()); - - assertTrue(broadcastTables.contains("default.small1")); - assertTrue(broadcastTables.contains("default.small2")); - assertTrue(!broadcastTables.contains("default.large1")); - - LogicalNode leafNode = leafEB.getPlan(); - assertEquals(NodeType.GROUP_BY, leafNode.getType()); - - LogicalNode joinNode = ((GroupbyNode)leafNode).getChild(); - assertEquals(NodeType.JOIN, joinNode.getType()); - - LogicalNode leftNode = ((JoinNode)joinNode).getLeftChild(); - LogicalNode rightNode = ((JoinNode)joinNode).getRightChild(); - - assertEquals(NodeType.JOIN, leftNode.getType()); - assertEquals(NodeType.SCAN, rightNode.getType()); - - LogicalNode lastLeftNode = ((JoinNode)leftNode).getLeftChild(); - LogicalNode lastRightNode = ((JoinNode)leftNode).getRightChild(); - - assertEquals(NodeType.SCAN, lastLeftNode.getType()); - assertEquals(NodeType.SCAN, lastRightNode.getType()); - } - - @Test - public final void testBroadcastJoinAllSmallTables() throws IOException, PlanningException { - String query = "select count(*) from small1 " + - "join small2 on small1_id = small2_id " + - "join small3 on small1_id = small3_id"; - - LogicalPlanner planner = new LogicalPlanner(catalog); - LogicalOptimizer optimizer = new LogicalOptimizer(conf); - Expr expr = analyzer.parse(query); - LogicalPlan plan = planner.createPlan(defaultContext, expr); - - optimizer.optimize(plan); - - QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0); - QueryContext queryContext = new QueryContext(conf); - MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); - GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); - globalPlanner.build(queryContext, masterPlan); - - /* - |-eb_1402500846700_0000_000005 - |-eb_1402500846700_0000_000004 - |-eb_1402500846700_0000_000003 (LEAF, broadcast join small1, small2, small3) - */ - - ExecutionBlock terminalEB = masterPlan.getRoot(); - assertEquals(1, masterPlan.getChildCount(terminalEB.getId())); - - ExecutionBlock rootEB = masterPlan.getChild(terminalEB.getId(), 0); - assertEquals(1, masterPlan.getChildCount(rootEB.getId())); - - ExecutionBlock leafEB = masterPlan.getChild(rootEB.getId(), 0); - assertNotNull(leafEB); - - assertEquals(0, masterPlan.getChildCount(leafEB.getId())); - Collection broadcastTables = leafEB.getBroadcastTables(); - assertEquals(3, broadcastTables.size()); - - assertTrue(broadcastTables.contains("default.small2")); - assertTrue(broadcastTables.contains("default.small1")); - assertTrue(broadcastTables.contains("default.small3")); - - LogicalNode leafNode = leafEB.getPlan(); - assertEquals(NodeType.GROUP_BY, leafNode.getType()); - - LogicalNode joinNode = ((GroupbyNode)leafNode).getChild(); - assertEquals(NodeType.JOIN, joinNode.getType()); - - LogicalNode leftNode = ((JoinNode)joinNode).getLeftChild(); - LogicalNode rightNode = ((JoinNode)joinNode).getRightChild(); - - assertEquals(NodeType.JOIN, leftNode.getType()); - assertEquals(NodeType.SCAN, rightNode.getType()); - assertEquals("default.small3", ((ScanNode)rightNode).getCanonicalName()); - - LogicalNode lastLeftNode = ((JoinNode)leftNode).getLeftChild(); - LogicalNode lastRightNode = ((JoinNode)leftNode).getRightChild(); - - assertEquals(NodeType.SCAN, lastLeftNode.getType()); - assertEquals(NodeType.SCAN, lastRightNode.getType()); - assertEquals("default.small1", ((ScanNode)lastLeftNode).getCanonicalName()); - assertEquals("default.small2", ((ScanNode)lastRightNode).getCanonicalName()); - } - - @Test - public final void testNotBroadcastJoinTwoLargeTable() throws IOException, PlanningException { - // This query is not broadcast join - String query = "select count(*) from large1 " + - "join large2 on large1_id = large2_id "; - - LogicalPlanner planner = new LogicalPlanner(catalog); - LogicalOptimizer optimizer = new LogicalOptimizer(conf); - Expr expr = analyzer.parse(query); - LogicalPlan plan = planner.createPlan(defaultContext, expr); - - optimizer.optimize(plan); - - QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0); - QueryContext queryContext = new QueryContext(conf); - MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); - GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); - globalPlanner.build(queryContext, masterPlan); - - ExecutionBlockCursor ebCursor = new ExecutionBlockCursor(masterPlan); - while (ebCursor.hasNext()) { - ExecutionBlock eb = ebCursor.nextBlock(); - Collection broadcastTables = eb.getBroadcastTables(); - assertTrue(broadcastTables == null || broadcastTables.isEmpty()); - } - } - - @Test - public final void testTwoBroadcastJoin() throws IOException, PlanningException { - String query = "select count(*) from large1 " + - "join small1 on large1_id = small1_id " + - "join large2 on large1_id = large2_id " + - "join small2 on large2_id = small2_id"; - - LogicalPlanner planner = new LogicalPlanner(catalog); - LogicalOptimizer optimizer = new LogicalOptimizer(conf); - Expr expr = analyzer.parse(query); - LogicalPlan plan = planner.createPlan(defaultContext, expr); - - optimizer.optimize(plan); - - QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0); - QueryContext queryContext = new QueryContext(conf); - MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); - GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); - globalPlanner.build(queryContext, masterPlan); - - /* - |-eb_1395736346625_0000_000009 - |-eb_1395736346625_0000_000008 (GROUP-BY) - |-eb_1395736346625_0000_000007 (GROUP-BY, JOIN) - |-eb_1395736346625_0000_000006 (LEAF, JOIN) - |-eb_1395736346625_0000_000003 (LEAF, JOIN) - */ - - ExecutionBlockCursor ebCursor = new ExecutionBlockCursor(masterPlan); - int index = 0; - while (ebCursor.hasNext()) { - ExecutionBlock eb = ebCursor.nextBlock(); - if(index == 0) { - Collection broadcastTables = eb.getBroadcastTables(); - assertEquals(1, broadcastTables.size()); - - assertTrue(!broadcastTables.contains("default.large1")); - assertTrue(broadcastTables.contains("default.small1")); - } else if(index == 1) { - Collection broadcastTables = eb.getBroadcastTables(); - assertEquals(1, broadcastTables.size()); - assertTrue(!broadcastTables.contains("default.large2")); - assertTrue(broadcastTables.contains("default.small2")); - } - index++; - } - - assertEquals(5, index); - } - +///** +// * 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.global; +// +//import junit.framework.TestCase; +//import org.apache.hadoop.fs.FileSystem; +//import org.apache.hadoop.fs.Path; +//import org.apache.tajo.LocalTajoTestingUtility; +//import org.apache.tajo.QueryId; +//import org.apache.tajo.QueryIdFactory; +//import org.apache.tajo.TajoTestingCluster; +//import org.apache.tajo.algebra.Expr; +//import org.apache.tajo.catalog.*; +//import org.apache.tajo.catalog.proto.CatalogProtos; +//import org.apache.tajo.catalog.statistics.TableStats; +//import org.apache.tajo.common.TajoDataTypes; +//import org.apache.tajo.conf.TajoConf; +//import org.apache.tajo.datum.Datum; +//import org.apache.tajo.datum.DatumFactory; +//import org.apache.tajo.datum.TextDatum; +//import org.apache.tajo.engine.function.FunctionLoader; +//import org.apache.tajo.engine.parser.SQLAnalyzer; +//import org.apache.tajo.plan.LogicalOptimizer; +//import org.apache.tajo.plan.LogicalPlan; +//import org.apache.tajo.plan.LogicalPlanner; +//import org.apache.tajo.plan.PlanningException; +//import org.apache.tajo.engine.query.QueryContext; +//import org.apache.tajo.plan.logical.*; +//import org.apache.tajo.storage.*; +//import org.apache.tajo.util.CommonTestingUtil; +//import org.junit.After; +//import org.junit.Before; +//import org.junit.Test; +// +//import java.io.IOException; +//import java.util.Collection; +// +//import static junit.framework.Assert.assertNotNull; +//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.assertTrue; +// +//public class TestBroadcastJoinPlan { +// private TajoConf conf; +// private final String TEST_PATH = TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/TestBroadcastJoinPlan"; +// private TajoTestingCluster util; +// private CatalogService catalog; +// private SQLAnalyzer analyzer; +// private QueryContext defaultContext; +// private Path testDir; +// +// private TableDesc smallTable1; +// private TableDesc smallTable2; +// private TableDesc smallTable3; +// private TableDesc largeTable1; +// private TableDesc largeTable2; +// private TableDesc largeTable3; +// +// @Before +// public void setUp() throws Exception { +// util = new TajoTestingCluster(); +// conf = util.getConfiguration(); +// conf.setLongVar(TajoConf.ConfVars.$DIST_QUERY_BROADCAST_JOIN_THRESHOLD, 500 * 1024); +// conf.setBoolVar(TajoConf.ConfVars.$TEST_BROADCAST_JOIN_ENABLED, true); +// +// testDir = CommonTestingUtil.getTestDir(TEST_PATH); +// catalog = util.startCatalogCluster().getCatalog(); +// catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString()); +// catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME); +// util.getMiniCatalogCluster().getCatalogServer().reloadBuiltinFunctions(FunctionLoader.findLegacyFunctions()); +// +// Schema smallTable1Schema = new Schema(); +// smallTable1Schema.addColumn("small1_id", TajoDataTypes.Type.INT4); +// smallTable1Schema.addColumn("small1_contents", TajoDataTypes.Type.TEXT); +// smallTable1 = makeTestData("default.small1", smallTable1Schema, 10 * 1024); +// +// Schema smallTable2Schema = new Schema(); +// smallTable2Schema.addColumn("small2_id", TajoDataTypes.Type.INT4); +// smallTable2Schema.addColumn("small2_contents", TajoDataTypes.Type.TEXT); +// smallTable2 = makeTestData("default.small2", smallTable2Schema, 10 * 1024); +// +// Schema smallTable3Schema = new Schema(); +// smallTable3Schema.addColumn("small3_id", TajoDataTypes.Type.INT4); +// smallTable3Schema.addColumn("small3_contents", TajoDataTypes.Type.TEXT); +// smallTable3 = makeTestData("default.small3", smallTable3Schema, 10 * 1024); +// +// Schema largeTable1Schema = new Schema(); +// largeTable1Schema.addColumn("large1_id", TajoDataTypes.Type.INT4); +// largeTable1Schema.addColumn("large1_contents", TajoDataTypes.Type.TEXT); +// largeTable1 = makeTestData("default.large1", largeTable1Schema, 1024 * 1024); //1M +// +// Schema largeTable2Schema = new Schema(); +// largeTable2Schema.addColumn("large2_id", TajoDataTypes.Type.INT4); +// largeTable2Schema.addColumn("large2_contents", TajoDataTypes.Type.TEXT); +// largeTable2 = makeTestData("default.large2", largeTable2Schema, 1024 * 1024); //1M +// +// Schema largeTable3Schema = new Schema(); +// largeTable3Schema.addColumn("large3_id", TajoDataTypes.Type.INT4); +// largeTable3Schema.addColumn("large3_contents", TajoDataTypes.Type.TEXT); +// largeTable3 = makeTestData("default.large3", largeTable3Schema, 1024 * 1024); //1M +// +// catalog.createTable(smallTable1); +// catalog.createTable(smallTable2); +// catalog.createTable(smallTable3); +// catalog.createTable(largeTable1); +// catalog.createTable(largeTable2); +// catalog.createTable(largeTable3); +// +// analyzer = new SQLAnalyzer(); +// defaultContext = LocalTajoTestingUtility.createDummyContext(conf); +// } +// +// private TableDesc makeTestData(String tableName, Schema schema, int dataSize) throws Exception { +// TableMeta tableMeta = CatalogUtil.newTableMeta(CatalogProtos.StoreType.CSV); +// Path dataPath = new Path(testDir, tableName + ".csv"); +// +// String contentsData = ""; +// for (int i = 0; i < 1000; i++) { +// for (int j = 0; j < 10; j++) { +// contentsData += j; +// } +// } +// Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf)) +// .getAppender(tableMeta, schema, dataPath); +// appender.init(); +// Tuple tuple = new VTuple(schema.size()); +// int writtenSize = 0; +// int count = 0; +// while (true) { +// TextDatum textDatum = DatumFactory.createText(count + "_" + contentsData); +// tuple.put(new Datum[] { +// DatumFactory.createInt4(count), textDatum }); +// appender.addTuple(tuple); +// +// writtenSize += textDatum.size(); +// if (writtenSize >= dataSize) { +// break; +// } +// } +// +// appender.flush(); +// appender.close(); +// +// TableDesc tableDesc = CatalogUtil.newTableDesc(tableName, schema, tableMeta, dataPath); +// TableStats tableStats = new TableStats(); +// FileSystem fs = dataPath.getFileSystem(conf); +// tableStats.setNumBytes(fs.getFileStatus(dataPath).getLen()); +// +// tableDesc.setStats(tableStats); +// +// return tableDesc; +// } +// +// @After +// public void tearDown() throws Exception { +// util.shutdownCatalogCluster(); +// } +// // @Test -// public final void testNotBroadcastJoinSubquery() throws IOException, PlanningException { -// // This query is not broadcast join; +// public final void testBroadcastJoin() throws IOException, PlanningException { // String query = "select count(*) from large1 " + -// "join (select * from small1) a on large1_id = a.small1_id " + -// "join small2 on a.small1_id = small2_id"; +// "join small1 on large1_id = small1_id " + +// "join small2 on small1_id = small2_id"; // // LogicalPlanner planner = new LogicalPlanner(catalog); // LogicalOptimizer optimizer = new LogicalOptimizer(conf); @@ -396,650 +193,853 @@ public final void testTwoBroadcastJoin() throws IOException, PlanningException { // globalPlanner.build(queryContext, masterPlan); // // /* -// |-eb_1395749810370_0000_000007 -// |-eb_1395749810370_0000_000006 (GROUP-BY) -// |-eb_1395749810370_0000_000005 (GROUP-BY, JOIN) -// |-eb_1395749810370_0000_000004 (LEAF, SCAN, large1) -// |-eb_1395749810370_0000_000003 (JOIN) -// |-eb_1395749810370_0000_000002 (LEAF, SCAN, small2) -// |-eb_1395749810370_0000_000001 (LEAF, TABLE_SUBQUERY, small1) -// */ +// |-eb_1395714781593_0000_000005 (TERMINAL) +// |-eb_1395714781593_0000_000004 (ROOT, GROUP BY for counting) +// |-eb_1395714781593_0000_000003 (LEAF, broadcast join) +// */ +// +// ExecutionBlock terminalEB = masterPlan.getRoot(); +// assertEquals(1, masterPlan.getChildCount(terminalEB.getId())); +// +// ExecutionBlock rootEB = masterPlan.getChild(terminalEB.getId(), 0); +// assertEquals(1, masterPlan.getChildCount(rootEB.getId())); +// +// ExecutionBlock leafEB = masterPlan.getChild(rootEB.getId(), 0); +// assertNotNull(leafEB); +// +// assertEquals(0, masterPlan.getChildCount(leafEB.getId())); +// Collection broadcastTables = leafEB.getBroadcastTables(); +// assertEquals(2, broadcastTables.size()); +// +// assertTrue(broadcastTables.contains("default.small1")); +// assertTrue(broadcastTables.contains("default.small2")); +// assertTrue(!broadcastTables.contains("default.large1")); +// +// LogicalNode leafNode = leafEB.getPlan(); +// assertEquals(NodeType.GROUP_BY, leafNode.getType()); +// +// LogicalNode joinNode = ((GroupbyNode)leafNode).getChild(); +// assertEquals(NodeType.JOIN, joinNode.getType()); +// +// LogicalNode leftNode = ((JoinNode)joinNode).getLeftChild(); +// LogicalNode rightNode = ((JoinNode)joinNode).getRightChild(); +// +// assertEquals(NodeType.JOIN, leftNode.getType()); +// assertEquals(NodeType.SCAN, rightNode.getType()); +// +// LogicalNode lastLeftNode = ((JoinNode)leftNode).getLeftChild(); +// LogicalNode lastRightNode = ((JoinNode)leftNode).getRightChild(); +// +// assertEquals(NodeType.SCAN, lastLeftNode.getType()); +// assertEquals(NodeType.SCAN, lastRightNode.getType()); +// } +// +// @Test +// public final void testBroadcastJoinAllSmallTables() throws IOException, PlanningException { +// String query = "select count(*) from small1 " + +// "join small2 on small1_id = small2_id " + +// "join small3 on small1_id = small3_id"; +// +// LogicalPlanner planner = new LogicalPlanner(catalog); +// LogicalOptimizer optimizer = new LogicalOptimizer(conf); +// Expr expr = analyzer.parse(query); +// LogicalPlan plan = planner.createPlan(defaultContext, expr); +// +// optimizer.optimize(plan); +// +// QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0); +// QueryContext queryContext = new QueryContext(conf); +// MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); +// GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); +// globalPlanner.build(queryContext, masterPlan); +// +// /* +// |-eb_1402500846700_0000_000005 +// |-eb_1402500846700_0000_000004 +// |-eb_1402500846700_0000_000003 (LEAF, broadcast join small1, small2, small3) +// */ +// +// ExecutionBlock terminalEB = masterPlan.getRoot(); +// assertEquals(1, masterPlan.getChildCount(terminalEB.getId())); +// +// ExecutionBlock rootEB = masterPlan.getChild(terminalEB.getId(), 0); +// assertEquals(1, masterPlan.getChildCount(rootEB.getId())); +// +// ExecutionBlock leafEB = masterPlan.getChild(rootEB.getId(), 0); +// assertNotNull(leafEB); +// +// assertEquals(0, masterPlan.getChildCount(leafEB.getId())); +// Collection broadcastTables = leafEB.getBroadcastTables(); +// assertEquals(3, broadcastTables.size()); +// +// assertTrue(broadcastTables.contains("default.small2")); +// assertTrue(broadcastTables.contains("default.small1")); +// assertTrue(broadcastTables.contains("default.small3")); +// +// LogicalNode leafNode = leafEB.getPlan(); +// assertEquals(NodeType.GROUP_BY, leafNode.getType()); +// +// LogicalNode joinNode = ((GroupbyNode)leafNode).getChild(); +// assertEquals(NodeType.JOIN, joinNode.getType()); +// +// LogicalNode leftNode = ((JoinNode)joinNode).getLeftChild(); +// LogicalNode rightNode = ((JoinNode)joinNode).getRightChild(); +// +// assertEquals(NodeType.JOIN, leftNode.getType()); +// assertEquals(NodeType.SCAN, rightNode.getType()); +// assertEquals("default.small3", ((ScanNode)rightNode).getCanonicalName()); +// +// LogicalNode lastLeftNode = ((JoinNode)leftNode).getLeftChild(); +// LogicalNode lastRightNode = ((JoinNode)leftNode).getRightChild(); +// +// assertEquals(NodeType.SCAN, lastLeftNode.getType()); +// assertEquals(NodeType.SCAN, lastRightNode.getType()); +// assertEquals("default.small1", ((ScanNode)lastLeftNode).getCanonicalName()); +// assertEquals("default.small2", ((ScanNode)lastRightNode).getCanonicalName()); +// } +// +// @Test +// public final void testNotBroadcastJoinTwoLargeTable() throws IOException, PlanningException { +// // This query is not broadcast join +// String query = "select count(*) from large1 " + +// "join large2 on large1_id = large2_id "; +// +// LogicalPlanner planner = new LogicalPlanner(catalog); +// LogicalOptimizer optimizer = new LogicalOptimizer(conf); +// Expr expr = analyzer.parse(query); +// LogicalPlan plan = planner.createPlan(defaultContext, expr); +// +// optimizer.optimize(plan); +// +// QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0); +// QueryContext queryContext = new QueryContext(conf); +// MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); +// GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); +// globalPlanner.build(queryContext, masterPlan); // // ExecutionBlockCursor ebCursor = new ExecutionBlockCursor(masterPlan); -// int index = 0; // while (ebCursor.hasNext()) { // ExecutionBlock eb = ebCursor.nextBlock(); // Collection broadcastTables = eb.getBroadcastTables(); // assertTrue(broadcastTables == null || broadcastTables.isEmpty()); +// } +// } +// +// @Test +// public final void testTwoBroadcastJoin() throws IOException, PlanningException { +// String query = "select count(*) from large1 " + +// "join small1 on large1_id = small1_id " + +// "join large2 on large1_id = large2_id " + +// "join small2 on large2_id = small2_id"; +// +// LogicalPlanner planner = new LogicalPlanner(catalog); +// LogicalOptimizer optimizer = new LogicalOptimizer(conf); +// Expr expr = analyzer.parse(query); +// LogicalPlan plan = planner.createPlan(defaultContext, expr); +// +// optimizer.optimize(plan); +// +// QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0); +// QueryContext queryContext = new QueryContext(conf); +// MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); +// GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); +// globalPlanner.build(queryContext, masterPlan); +// +// /* +// |-eb_1395736346625_0000_000009 +// |-eb_1395736346625_0000_000008 (GROUP-BY) +// |-eb_1395736346625_0000_000007 (GROUP-BY, JOIN) +// |-eb_1395736346625_0000_000006 (LEAF, JOIN) +// |-eb_1395736346625_0000_000003 (LEAF, JOIN) +// */ +// +// ExecutionBlockCursor ebCursor = new ExecutionBlockCursor(masterPlan); +// int index = 0; +// while (ebCursor.hasNext()) { +// ExecutionBlock eb = ebCursor.nextBlock(); +// if(index == 0) { +// Collection broadcastTables = eb.getBroadcastTables(); +// assertEquals(1, broadcastTables.size()); +// +// assertTrue(!broadcastTables.contains("default.large1")); +// assertTrue(broadcastTables.contains("default.small1")); +// } else if(index == 1) { +// Collection broadcastTables = eb.getBroadcastTables(); +// assertEquals(1, broadcastTables.size()); +// assertTrue(!broadcastTables.contains("default.large2")); +// assertTrue(broadcastTables.contains("default.small2")); +// } +// index++; +// } +// +// assertEquals(5, index); +// } +// +//// @Test +//// public final void testNotBroadcastJoinSubquery() throws IOException, PlanningException { +//// // This query is not broadcast join; +//// String query = "select count(*) from large1 " + +//// "join (select * from small1) a on large1_id = a.small1_id " + +//// "join small2 on a.small1_id = small2_id"; +//// +//// LogicalPlanner planner = new LogicalPlanner(catalog); +//// LogicalOptimizer optimizer = new LogicalOptimizer(conf); +//// Expr expr = analyzer.parse(query); +//// LogicalPlan plan = planner.createPlan(defaultContext, expr); +//// +//// optimizer.optimize(plan); +//// +//// QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0); +//// QueryContext queryContext = new QueryContext(conf); +//// MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); +//// GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); +//// globalPlanner.build(queryContext, masterPlan); +//// +//// /* +//// |-eb_1395749810370_0000_000007 +//// |-eb_1395749810370_0000_000006 (GROUP-BY) +//// |-eb_1395749810370_0000_000005 (GROUP-BY, JOIN) +//// |-eb_1395749810370_0000_000004 (LEAF, SCAN, large1) +//// |-eb_1395749810370_0000_000003 (JOIN) +//// |-eb_1395749810370_0000_000002 (LEAF, SCAN, small2) +//// |-eb_1395749810370_0000_000001 (LEAF, TABLE_SUBQUERY, small1) +//// */ +//// +//// ExecutionBlockCursor ebCursor = new ExecutionBlockCursor(masterPlan); +//// int index = 0; +//// while (ebCursor.hasNext()) { +//// ExecutionBlock eb = ebCursor.nextBlock(); +//// Collection broadcastTables = eb.getBroadcastTables(); +//// assertTrue(broadcastTables == null || broadcastTables.isEmpty()); +//// index++; +//// } +//// +//// assertEquals(7, index); +//// } +// +// @Test +// public final void testBroadcastJoinSubquery() throws IOException, PlanningException { +// String query = "select count(*) from large1 " + +// "join small2 on large1_id = small2_id " + +// "join (select * from small1) a on large1_id = a.small1_id"; +// +// LogicalPlanner planner = new LogicalPlanner(catalog); +// LogicalOptimizer optimizer = new LogicalOptimizer(conf); +// Expr expr = analyzer.parse(query); +// LogicalPlan plan = planner.createPlan(defaultContext, expr); +// +// optimizer.optimize(plan); +// +// QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0); +// QueryContext queryContext = new QueryContext(conf); +// MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); +// GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); +// globalPlanner.build(queryContext, masterPlan); +// +// /* +// |-eb_1395794091662_0000_000007 +// |-eb_1395794091662_0000_000006 +// |-eb_1395794091662_0000_000005 (JOIN) +// |-eb_1395794091662_0000_000004 (LEAF, SUBQUERY) +// |-eb_1395794091662_0000_000003 (LEAF, JOIN) +// */ +// +// ExecutionBlockCursor ebCursor = new ExecutionBlockCursor(masterPlan); +// int index = 0; +// while (ebCursor.hasNext()) { +// ExecutionBlock eb = ebCursor.nextBlock(); +// if(index == 0) { +// //LEAF, JOIN +// Collection broadcastTables = eb.getBroadcastTables(); +// assertEquals(1, broadcastTables.size()); +// +// assertTrue(!broadcastTables.contains("default.large1")); +// assertTrue(broadcastTables.contains("default.small2")); +// } else if(index == 1) { +// //LEAF, SUBQUERY +// Collection broadcastTables = eb.getBroadcastTables(); +// assertTrue(broadcastTables == null || broadcastTables.isEmpty()); +// } else if(index == 2) { +// //JOIN +// Collection broadcastTables = eb.getBroadcastTables(); +// assertTrue(broadcastTables == null || broadcastTables.isEmpty()); +// } +// index++; +// } +// +// assertEquals(5, index); +// } +// +// @Test +// public final void testLeftOuterJoinCase1() throws IOException, PlanningException { +// // small, small, small, large, large +// String query = "select count(*) from small1 " + +// "left outer join small2 on small1_id = small2_id " + +// "left outer join small3 on small1_id = small3_id " + +// "left outer join large1 on small1_id = large1_id " + +// "left outer join large2 on small1_id = large2_id "; +// +// LogicalPlanner planner = new LogicalPlanner(catalog); +// LogicalOptimizer optimizer = new LogicalOptimizer(conf); +// Expr expr = analyzer.parse(query); +// LogicalPlan plan = planner.createPlan(defaultContext, expr); +// +// optimizer.optimize(plan); +// +// QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0); +// QueryContext queryContext = new QueryContext(conf); +// MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); +// GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); +// globalPlanner.build(queryContext, masterPlan); +// +// // ((((default.small1 ⟕ default.small2) ⟕ default.small3) ⟕ default.large1) ⟕ default.large2) +// /* +// |-eb_1406022243130_0000_000009 +// |-eb_1406022243130_0000_000008 +// |-eb_1406022243130_0000_000007 (join) +// |-eb_1406022243130_0000_000006 (scan large2) +// |-eb_1406022243130_0000_000005 (join) +// |-eb_1406022243130_0000_000004 (scan large1) +// |-eb_1406022243130_0000_000003 (scan small1, broadcast join small2, small3) +// */ +// +// ExecutionBlockCursor ebCursor = new ExecutionBlockCursor(masterPlan); +// int index = 0; +// while (ebCursor.hasNext()) { +// ExecutionBlock eb = ebCursor.nextBlock(); +// if(index == 0) { +// Collection broadcastTables = eb.getBroadcastTables(); +// assertEquals(2, broadcastTables.size()); +// +// assertTrue(!broadcastTables.contains("default.small1")); +// assertTrue(broadcastTables.contains("default.small2")); +// assertTrue(broadcastTables.contains("default.small3")); +// } else if(index == 1 || index == 2 || index == 3) { +// Collection broadcastTables = eb.getBroadcastTables(); +// assertEquals(0, broadcastTables.size()); +// } // index++; // } // // assertEquals(7, index); // } - - @Test - public final void testBroadcastJoinSubquery() throws IOException, PlanningException { - String query = "select count(*) from large1 " + - "join small2 on large1_id = small2_id " + - "join (select * from small1) a on large1_id = a.small1_id"; - - LogicalPlanner planner = new LogicalPlanner(catalog); - LogicalOptimizer optimizer = new LogicalOptimizer(conf); - Expr expr = analyzer.parse(query); - LogicalPlan plan = planner.createPlan(defaultContext, expr); - - optimizer.optimize(plan); - - QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0); - QueryContext queryContext = new QueryContext(conf); - MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); - GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); - globalPlanner.build(queryContext, masterPlan); - - /* - |-eb_1395794091662_0000_000007 - |-eb_1395794091662_0000_000006 - |-eb_1395794091662_0000_000005 (JOIN) - |-eb_1395794091662_0000_000004 (LEAF, SUBQUERY) - |-eb_1395794091662_0000_000003 (LEAF, JOIN) - */ - - ExecutionBlockCursor ebCursor = new ExecutionBlockCursor(masterPlan); - int index = 0; - while (ebCursor.hasNext()) { - ExecutionBlock eb = ebCursor.nextBlock(); - if(index == 0) { - //LEAF, JOIN - Collection broadcastTables = eb.getBroadcastTables(); - assertEquals(1, broadcastTables.size()); - - assertTrue(!broadcastTables.contains("default.large1")); - assertTrue(broadcastTables.contains("default.small2")); - } else if(index == 1) { - //LEAF, SUBQUERY - Collection broadcastTables = eb.getBroadcastTables(); - assertTrue(broadcastTables == null || broadcastTables.isEmpty()); - } else if(index == 2) { - //JOIN - Collection broadcastTables = eb.getBroadcastTables(); - assertTrue(broadcastTables == null || broadcastTables.isEmpty()); - } - index++; - } - - assertEquals(5, index); - } - - @Test - public final void testLeftOuterJoinCase1() throws IOException, PlanningException { - // small, small, small, large, large - String query = "select count(*) from small1 " + - "left outer join small2 on small1_id = small2_id " + - "left outer join small3 on small1_id = small3_id " + - "left outer join large1 on small1_id = large1_id " + - "left outer join large2 on small1_id = large2_id "; - - LogicalPlanner planner = new LogicalPlanner(catalog); - LogicalOptimizer optimizer = new LogicalOptimizer(conf); - Expr expr = analyzer.parse(query); - LogicalPlan plan = planner.createPlan(defaultContext, expr); - - optimizer.optimize(plan); - - QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0); - QueryContext queryContext = new QueryContext(conf); - MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); - GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); - globalPlanner.build(queryContext, masterPlan); - - // ((((default.small1 ⟕ default.small2) ⟕ default.small3) ⟕ default.large1) ⟕ default.large2) - /* - |-eb_1406022243130_0000_000009 - |-eb_1406022243130_0000_000008 - |-eb_1406022243130_0000_000007 (join) - |-eb_1406022243130_0000_000006 (scan large2) - |-eb_1406022243130_0000_000005 (join) - |-eb_1406022243130_0000_000004 (scan large1) - |-eb_1406022243130_0000_000003 (scan small1, broadcast join small2, small3) - */ - - ExecutionBlockCursor ebCursor = new ExecutionBlockCursor(masterPlan); - int index = 0; - while (ebCursor.hasNext()) { - ExecutionBlock eb = ebCursor.nextBlock(); - if(index == 0) { - Collection broadcastTables = eb.getBroadcastTables(); - assertEquals(2, broadcastTables.size()); - - assertTrue(!broadcastTables.contains("default.small1")); - assertTrue(broadcastTables.contains("default.small2")); - assertTrue(broadcastTables.contains("default.small3")); - } else if(index == 1 || index == 2 || index == 3) { - Collection broadcastTables = eb.getBroadcastTables(); - assertEquals(0, broadcastTables.size()); - } - index++; - } - - assertEquals(7, index); - } - - @Test - public final void testLeftOuterJoinCase2() throws IOException, PlanningException { - // large, large, small, small, small - String query = "select count(*) from large1 " + - "left outer join large2 on large1_id = large2_id " + - "left outer join small1 on large1_id = small1_id " + - "left outer join small2 on large1_id = small2_id " + - "left outer join small3 on large1_id = small3_id "; - - LogicalPlanner planner = new LogicalPlanner(catalog); - LogicalOptimizer optimizer = new LogicalOptimizer(conf); - Expr expr = analyzer.parse(query); - LogicalPlan plan = planner.createPlan(defaultContext, expr); - - optimizer.optimize(plan); - - QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0); - QueryContext queryContext = new QueryContext(conf); - MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); - GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); - globalPlanner.build(queryContext, masterPlan); - - // ((((default.large1 ⟕ default.large2) ⟕ default.small1) ⟕ default.small2) ⟕ default.small3) - /* - |-eb_1404132555037_0000_000005 - |-eb_1404132555037_0000_000004 - |-eb_1404132555037_0000_000003 (JOIN, broadcast small1, small2, small3) - |-eb_1404132555037_0000_000002 (LEAF, Scan large2) - |-eb_1404132555037_0000_000001 (LEAF, Scan large1) - */ - - ExecutionBlockCursor ebCursor = new ExecutionBlockCursor(masterPlan); - int index = 0; - while (ebCursor.hasNext()) { - ExecutionBlock eb = ebCursor.nextBlock(); - if(index == 0) { - LogicalNode node = eb.getPlan(); - assertEquals(NodeType.SCAN, node.getType()); - assertEquals("default.large1", ((ScanNode) node).getCanonicalName()); - - assertEquals(0, eb.getBroadcastTables().size()); - } else if (index == 1) { - LogicalNode node = eb.getPlan(); - assertEquals(NodeType.SCAN, node.getType()); - assertEquals("default.large2", ((ScanNode)node).getCanonicalName()); - - assertEquals(0, eb.getBroadcastTables().size()); - } else if(index == 2) { - LogicalNode node = eb.getPlan(); - assertEquals(NodeType.GROUP_BY, node.getType()); - - JoinNode joinNode = ((GroupbyNode)node).getChild(); - JoinNode joinNode2 = joinNode.getLeftChild(); - ScanNode scanNode2 = joinNode.getRightChild(); - assertEquals("default.small3", scanNode2.getCanonicalName()); - - JoinNode joinNode3 = joinNode2.getLeftChild(); - ScanNode scanNode3 = joinNode2.getRightChild(); - assertEquals("default.small2", scanNode3.getCanonicalName()); - - JoinNode joinNode4 = joinNode3.getLeftChild(); - ScanNode scanNode4 = joinNode3.getRightChild(); - assertEquals("default.small1", scanNode4.getCanonicalName()); - - ScanNode scanNode5 = joinNode4.getLeftChild(); - ScanNode scanNode6 = joinNode4.getRightChild(); - assertTrue(scanNode5.getCanonicalName().indexOf("0000_000001") > 0); - assertTrue(scanNode6.getCanonicalName().indexOf("0000_000002") > 0); - - Collection broadcastTables = eb.getBroadcastTables(); - assertEquals(3, broadcastTables.size()); - - assertTrue(broadcastTables.contains("default.small1")); - assertTrue(broadcastTables.contains("default.small2")); - assertTrue(broadcastTables.contains("default.small3")); - } - index++; - } - - assertEquals(5, index); - } - - @Test - public final void testLeftOuterJoinCase3() throws IOException, PlanningException { - // large1, large2, small1, large3, small2, small3 - String query = "select count(*) from large1 " + - "left outer join large2 on large1_id = large2_id " + - "left outer join small1 on large2_id = small1_id " + - "left outer join large3 on large1_id = large3_id " + - "left outer join small2 on large3_id = small2_id " + - "left outer join small3 on large3_id = small3_id "; - - LogicalPlanner planner = new LogicalPlanner(catalog); - LogicalOptimizer optimizer = new LogicalOptimizer(conf); - Expr expr = analyzer.parse(query); - LogicalPlan plan = planner.createPlan(defaultContext, expr); - - optimizer.optimize(plan); - - QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0); - QueryContext queryContext = new QueryContext(conf); - MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); - GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); - globalPlanner.build(queryContext, masterPlan); - - //(((((default.large1 ⟕ default.large2) ⟕ default.small1) ⟕ default.large3) ⟕ default.small2) ⟕ default.small3) - /* - |-eb_1402634570910_0000_000007 - |-eb_1402634570910_0000_000006 (GROUP BY) - |-eb_1402634570910_0000_000005 (JOIN, broadcast small2, small3) - |-eb_1402634570910_0000_000004 (LEAF, scan large3) - |-eb_1402634570910_0000_000003 (JOIN, broadcast small1) - |-eb_1402634570910_0000_000002 (LEAF, scan large2) - |-eb_1402634570910_0000_000001 (LEAF, scan large1) - */ - ExecutionBlockCursor ebCursor = new ExecutionBlockCursor(masterPlan); - int index = 0; - while (ebCursor.hasNext()) { - ExecutionBlock eb = ebCursor.nextBlock(); - if(index == 0) { - LogicalNode node = eb.getPlan(); - assertEquals(NodeType.SCAN, node.getType()); - ScanNode scanNode = (ScanNode)node; - assertEquals("default.large1", scanNode.getCanonicalName()); - - Collection broadcastTables = eb.getBroadcastTables(); - assertEquals(0, broadcastTables.size()); - } else if (index == 1) { - LogicalNode node = eb.getPlan(); - assertEquals(NodeType.SCAN, node.getType()); - ScanNode scanNode = (ScanNode)node; - assertEquals("default.large2", scanNode.getCanonicalName()); - - Collection broadcastTables = eb.getBroadcastTables(); - assertEquals(0, broadcastTables.size()); - } else if(index == 2) { - LogicalNode node = eb.getPlan(); - assertEquals(NodeType.JOIN, node.getType()); - JoinNode joinNode = (JoinNode)node; - - ScanNode leftNode = ((JoinNode)joinNode.getLeftChild()).getLeftChild(); - ScanNode rightNode = ((JoinNode)joinNode.getLeftChild()).getRightChild(); - assertTrue(leftNode.getCanonicalName().indexOf("0000_000001") > 0); - assertTrue(rightNode.getCanonicalName().indexOf("0000_000002") > 0); - - Collection broadcastTables = eb.getBroadcastTables(); - assertEquals(1, broadcastTables.size()); - assertTrue(broadcastTables.contains("default.small1")); - } else if(index == 3) { - LogicalNode node = eb.getPlan(); - assertEquals(NodeType.SCAN, node.getType()); - ScanNode scanNode = (ScanNode)node; - assertEquals("default.large3", scanNode.getCanonicalName()); - - Collection broadcastTables = eb.getBroadcastTables(); - assertEquals(0, broadcastTables.size()); - } else if(index == 4) { - Collection broadcastTables = eb.getBroadcastTables(); - assertEquals(2, broadcastTables.size()); - assertTrue(broadcastTables.contains("default.small2")); - assertTrue(broadcastTables.contains("default.small3")); - } - index++; - } - - assertEquals(7, index); - } - - @Test - public final void testLeftOuterJoinCase4() throws IOException, PlanningException { - // small1, small2, small3 - String query = "select count(*) from small1 " + - "left outer join small2 on small1_id = small2_id " + - "left outer join small3 on small1_id = small3_id "; - - LogicalPlanner planner = new LogicalPlanner(catalog); - LogicalOptimizer optimizer = new LogicalOptimizer(conf); - Expr expr = analyzer.parse(query); - LogicalPlan plan = planner.createPlan(defaultContext, expr); - - optimizer.optimize(plan); - - QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0); - QueryContext queryContext = new QueryContext(conf); - MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); - GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); - globalPlanner.build(queryContext, masterPlan); - - /* - |-eb_1406022971444_0000_000005 - |-eb_1406022971444_0000_000004 (group by) - |-eb_1406022971444_0000_000003 (scan small1, broadcast join small2, small3) - */ - - ExecutionBlockCursor ebCursor = new ExecutionBlockCursor(masterPlan); - int index = 0; - while (ebCursor.hasNext()) { - ExecutionBlock eb = ebCursor.nextBlock(); - if(index == 0) { - GroupbyNode node = (GroupbyNode)eb.getPlan(); - JoinNode joinNode = node.getChild(); - - ScanNode scanNode = joinNode.getRightChild(); - assertEquals("default.small3", scanNode.getCanonicalName()); - - joinNode = joinNode.getLeftChild(); - scanNode = joinNode.getLeftChild(); - assertEquals("default.small1", scanNode.getCanonicalName()); - scanNode = joinNode.getRightChild(); - assertEquals("default.small2", scanNode.getCanonicalName()); - - Collection broadcastTables = eb.getBroadcastTables(); - assertEquals(2, broadcastTables.size()); - - assertTrue(broadcastTables.contains("default.small2")); - assertTrue(broadcastTables.contains("default.small3")); - } else if(index == 1) { - Collection broadcastTables = eb.getBroadcastTables(); - assertEquals(0, broadcastTables.size()); - } - index++; - } - - assertEquals(3, index); - } - - @Test - public final void testLeftOuterJoinCase5() throws IOException, PlanningException { - // small, small, large, small - String query = "select count(*) from small1 " + - "left outer join small2 on small1_id = small2_id " + - "left outer join large1 on small1_id = large1_id " + - "left outer join small3 on small1_id = small3_id " ; - - LogicalPlanner planner = new LogicalPlanner(catalog); - LogicalOptimizer optimizer = new LogicalOptimizer(conf); - Expr expr = analyzer.parse(query); - LogicalPlan plan = planner.createPlan(defaultContext, expr); - - optimizer.optimize(plan); - - QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0); - QueryContext queryContext = new QueryContext(conf); - MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); - GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); - globalPlanner.build(queryContext, masterPlan); - - //(((default.small1 ⟕ default.small2) ⟕ default.large1) ⟕ default.small3) - /* - |-eb_1406023347983_0000_000007 - |-eb_1406023347983_0000_000006 - |-eb_1406023347983_0000_000005 (join, broadcast small3) - |-eb_1406023347983_0000_000004 (scan large1) - |-eb_1406023347983_0000_000003 (scan small1, broadcast join small2) - */ - - ExecutionBlockCursor ebCursor = new ExecutionBlockCursor(masterPlan); - int index = 0; - while (ebCursor.hasNext()) { - ExecutionBlock eb = ebCursor.nextBlock(); - if(index == 0) { - Collection broadcastTables = eb.getBroadcastTables(); - assertEquals(1, broadcastTables.size()); - assertTrue(broadcastTables.contains("default.small2")); - } else if (index == 2) { - Collection broadcastTables = eb.getBroadcastTables(); - assertEquals(1, broadcastTables.size()); - assertTrue(broadcastTables.contains("default.small3")); - } else if(index == 1 || index == 3) { - Collection broadcastTables = eb.getBroadcastTables(); - assertEquals(0, broadcastTables.size()); - } - index++; - } - - assertEquals(5, index); - } - - @Test - public final void testLeftOuterJoinCase6() throws IOException, PlanningException { - // small1, small2, large1, large2, small3 - String query = "select count(*) from small1 " + - "left outer join small2 on small1_id = small2_id " + - "left outer join large1 on small1_id = large1_id " + - "left outer join large2 on small1_id = large2_id " + - "left outer join small3 on small1_id = small3_id " ; - - LogicalPlanner planner = new LogicalPlanner(catalog); - LogicalOptimizer optimizer = new LogicalOptimizer(conf); - Expr expr = analyzer.parse(query); - LogicalPlan plan = planner.createPlan(defaultContext, expr); - - optimizer.optimize(plan); - - QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0); - QueryContext queryContext = new QueryContext(conf); - MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); - GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); - globalPlanner.build(queryContext, masterPlan); - - // ((((default.small1 ⟕ default.small2) ⟕ default.large1) ⟕ default.large2) ⟕ default.small3) - - /* - |-eb_1406023537578_0000_000009 - |-eb_1406023537578_0000_000008 - |-eb_1406023537578_0000_000007 (join, broadcast small3) - |-eb_1406023537578_0000_000006 (scan large2) - |-eb_1406023537578_0000_000005 (join) - |-eb_1406023537578_0000_000004 (scan large1) - |-eb_1406023537578_0000_000003 (scan small1, broadcast join small2) - */ - ExecutionBlockCursor ebCursor = new ExecutionBlockCursor(masterPlan); - int index = 0; - while (ebCursor.hasNext()) { - ExecutionBlock eb = ebCursor.nextBlock(); - if(index == 0) { - LogicalNode node = eb.getPlan(); - assertEquals(NodeType.JOIN, node.getType()); - JoinNode joinNode = (JoinNode)node; - - ScanNode scanNode1 = joinNode.getLeftChild(); - ScanNode scanNode2 = joinNode.getRightChild(); - assertEquals("default.small1", scanNode1.getCanonicalName()); - assertEquals("default.small2", scanNode2.getCanonicalName()); - - Collection broadcastTables = eb.getBroadcastTables(); - assertEquals(1, broadcastTables.size()); - assertTrue(broadcastTables.contains("default.small2")); - } else if (index == 1) { - LogicalNode node = eb.getPlan(); - assertEquals(NodeType.SCAN, node.getType()); - ScanNode scanNode = (ScanNode) node; - assertEquals("default.large1", scanNode.getCanonicalName()); - - Collection broadcastTables = eb.getBroadcastTables(); - assertEquals(0, broadcastTables.size()); - } else if (index == 2) { - LogicalNode node = eb.getPlan(); - assertEquals(NodeType.JOIN, node.getType()); - } else if (index == 3) { - LogicalNode node = eb.getPlan(); - assertEquals(NodeType.SCAN, node.getType()); - ScanNode scanNode = (ScanNode) node; - assertEquals("default.large2", scanNode.getCanonicalName()); - - Collection broadcastTables = eb.getBroadcastTables(); - assertEquals(0, broadcastTables.size()); - } else if(index == 4) { - LogicalNode node = eb.getPlan(); - assertEquals(NodeType.GROUP_BY, node.getType()); - - JoinNode joinNode = ((GroupbyNode)node).getChild(); - - JoinNode joinNode1 = joinNode.getLeftChild(); - ScanNode scanNode1 = joinNode.getRightChild(); - assertEquals("default.small3", scanNode1.getCanonicalName()); - - ScanNode scanNode2 = joinNode1.getLeftChild(); - ScanNode scanNode3 = joinNode1.getRightChild(); - assertTrue(scanNode2.getCanonicalName().indexOf("0000_000005") > 0); - assertTrue(scanNode3.getCanonicalName().indexOf("0000_000006") > 0); - - Collection broadcastTables = eb.getBroadcastTables(); - assertEquals(1, broadcastTables.size()); - } - index++; - } - - assertEquals(7, index); - } - - @Test - public final void testInnerLeftOuterJoinCase1() throws IOException, PlanningException { - // small, small, large, small - String query = "select count(*) from small1 " + - "inner join small2 on small1_id = small2_id " + - "left outer join large1 on small1_id = large1_id " + - "left outer join small3 on small3_id = large1_id " ; - - LogicalPlanner planner = new LogicalPlanner(catalog); - LogicalOptimizer optimizer = new LogicalOptimizer(conf); - Expr expr = analyzer.parse(query); - LogicalPlan plan = planner.createPlan(defaultContext, expr); - - optimizer.optimize(plan); - - QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0); - QueryContext queryContext = new QueryContext(conf); - MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); - GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); - globalPlanner.build(queryContext, masterPlan); - - // (((default.small1 ⋈θ default.small2) ⟕ default.large1) ⟕ default.small3) - /* - |-eb_1404139312268_0000_000006 - |-eb_1404139312268_0000_000005 - |-eb_1404139312268_0000_000003 (LEAF scan large1, broadcast small1, small2, small3) - */ - - ExecutionBlockCursor ebCursor = new ExecutionBlockCursor(masterPlan); - int index = 0; - while (ebCursor.hasNext()) { - ExecutionBlock eb = ebCursor.nextBlock(); - if(index == 0) { - LogicalNode node = eb.getPlan(); - assertEquals(NodeType.GROUP_BY, node.getType()); - JoinNode joinNode = ((GroupbyNode)node).getChild(); - - JoinNode joinNode2 = joinNode.getLeftChild(); - ScanNode scanNode = joinNode.getRightChild(); - assertEquals("default.small3", scanNode.getCanonicalName()); - - JoinNode joinNode3 = joinNode2.getLeftChild(); - ScanNode scanNode2 = joinNode2.getRightChild(); - - assertEquals("default.large1", scanNode2.getCanonicalName()); - - ScanNode scanNode3 = joinNode3.getLeftChild(); - ScanNode scanNode4 = joinNode3.getRightChild(); - - assertEquals("default.small1", scanNode3.getCanonicalName()); - assertEquals("default.small2", scanNode4.getCanonicalName()); - - Collection broadcastTables = eb.getBroadcastTables(); - - assertEquals(3, broadcastTables.size()); - assertTrue(broadcastTables.contains("default.small1")); - assertTrue(broadcastTables.contains("default.small2")); - assertTrue(broadcastTables.contains("default.small3")); - } - index++; - } - - assertEquals(3, index); - } - - @Test - public final void testBroadcastCasebyCase1() throws IOException, PlanningException { - // large, small, large, small - String query = "select count(*) from large1 " + - "inner join small1 on large1_id = small1_id " + - "left outer join large2 on large1_id = large2_id " + - "left outer join small2 on large1_id = small2_id " ; - - LogicalPlanner planner = new LogicalPlanner(catalog); - LogicalOptimizer optimizer = new LogicalOptimizer(conf); - Expr expr = analyzer.parse(query); - LogicalPlan plan = planner.createPlan(defaultContext, expr); - - optimizer.optimize(plan); - - QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0); - QueryContext queryContext = new QueryContext(conf); - MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); - GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); - globalPlanner.build(queryContext, masterPlan); - - // (((default.large1 ⋈θ default.small1) ⟕ default.large2) ⟕ default.small2) - /* - |-eb_1404871198908_0000_000007 - |-eb_1404871198908_0000_000006 - |-eb_1404871198908_0000_000005 (join eb3, eb3, broadcast small2) - |-eb_1404871198908_0000_000004 (scan large2) - |-eb_1404871198908_0000_000003 (scan large1, broadcast small1) - */ - - ExecutionBlockCursor ebCursor = new ExecutionBlockCursor(masterPlan); - int index = 0; - while (ebCursor.hasNext()) { - ExecutionBlock eb = ebCursor.nextBlock(); - if(index == 0) { - LogicalNode node = eb.getPlan(); - assertEquals(NodeType.JOIN, node.getType()); - JoinNode joinNode = (JoinNode)node; - - ScanNode scanNode1 = joinNode.getLeftChild(); - ScanNode scanNode2 = joinNode.getRightChild(); - assertEquals("default.large1", scanNode1.getCanonicalName()); - assertEquals("default.small1", scanNode2.getCanonicalName()); - - Collection broadcastTables = eb.getBroadcastTables(); - - assertEquals(1, broadcastTables.size()); - assertTrue(broadcastTables.contains("default.small1")); - } else if(index == 1) { - LogicalNode node = eb.getPlan(); - assertEquals(NodeType.SCAN, node.getType()); - ScanNode scanNode = (ScanNode)node; - - assertEquals("default.large2", scanNode.getCanonicalName()); - - Collection broadcastTables = eb.getBroadcastTables(); - TestCase.assertEquals(0, broadcastTables.size()); - } else if(index == 2) { - LogicalNode node = eb.getPlan(); - assertEquals(NodeType.GROUP_BY, node.getType()); - JoinNode joinNode = ((GroupbyNode)node).getChild(); - - JoinNode joinNode2 = joinNode.getLeftChild(); - ScanNode scanNode = joinNode.getRightChild(); - assertEquals("default.small2", scanNode.getCanonicalName()); - - ScanNode scanNode2 = joinNode2.getLeftChild(); - ScanNode scanNode3 = joinNode2.getRightChild(); - - assertTrue(scanNode2.getCanonicalName().indexOf("000003") >= 0); - assertTrue(scanNode3.getCanonicalName().indexOf("000004") >= 0); - - Collection broadcastTables = eb.getBroadcastTables(); - - TestCase.assertEquals(1, broadcastTables.size()); - TestCase.assertTrue(broadcastTables.contains("default.small2")); - } - index++; - } - - TestCase.assertEquals(5, index); - } -} +// +// @Test +// public final void testLeftOuterJoinCase2() throws IOException, PlanningException { +// // large, large, small, small, small +// String query = "select count(*) from large1 " + +// "left outer join large2 on large1_id = large2_id " + +// "left outer join small1 on large1_id = small1_id " + +// "left outer join small2 on large1_id = small2_id " + +// "left outer join small3 on large1_id = small3_id "; +// +// LogicalPlanner planner = new LogicalPlanner(catalog); +// LogicalOptimizer optimizer = new LogicalOptimizer(conf); +// Expr expr = analyzer.parse(query); +// LogicalPlan plan = planner.createPlan(defaultContext, expr); +// +// optimizer.optimize(plan); +// +// QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0); +// QueryContext queryContext = new QueryContext(conf); +// MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); +// GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); +// globalPlanner.build(queryContext, masterPlan); +// +// // ((((default.large1 ⟕ default.large2) ⟕ default.small1) ⟕ default.small2) ⟕ default.small3) +// /* +// |-eb_1404132555037_0000_000005 +// |-eb_1404132555037_0000_000004 +// |-eb_1404132555037_0000_000003 (JOIN, broadcast small1, small2, small3) +// |-eb_1404132555037_0000_000002 (LEAF, Scan large2) +// |-eb_1404132555037_0000_000001 (LEAF, Scan large1) +// */ +// +// ExecutionBlockCursor ebCursor = new ExecutionBlockCursor(masterPlan); +// int index = 0; +// while (ebCursor.hasNext()) { +// ExecutionBlock eb = ebCursor.nextBlock(); +// if(index == 0) { +// LogicalNode node = eb.getPlan(); +// assertEquals(NodeType.SCAN, node.getType()); +// assertEquals("default.large1", ((ScanNode) node).getCanonicalName()); +// +// assertEquals(0, eb.getBroadcastTables().size()); +// } else if (index == 1) { +// LogicalNode node = eb.getPlan(); +// assertEquals(NodeType.SCAN, node.getType()); +// assertEquals("default.large2", ((ScanNode)node).getCanonicalName()); +// +// assertEquals(0, eb.getBroadcastTables().size()); +// } else if(index == 2) { +// LogicalNode node = eb.getPlan(); +// assertEquals(NodeType.GROUP_BY, node.getType()); +// +// JoinNode joinNode = ((GroupbyNode)node).getChild(); +// JoinNode joinNode2 = joinNode.getLeftChild(); +// ScanNode scanNode2 = joinNode.getRightChild(); +// assertEquals("default.small3", scanNode2.getCanonicalName()); +// +// JoinNode joinNode3 = joinNode2.getLeftChild(); +// ScanNode scanNode3 = joinNode2.getRightChild(); +// assertEquals("default.small2", scanNode3.getCanonicalName()); +// +// JoinNode joinNode4 = joinNode3.getLeftChild(); +// ScanNode scanNode4 = joinNode3.getRightChild(); +// assertEquals("default.small1", scanNode4.getCanonicalName()); +// +// ScanNode scanNode5 = joinNode4.getLeftChild(); +// ScanNode scanNode6 = joinNode4.getRightChild(); +// assertTrue(scanNode5.getCanonicalName().indexOf("0000_000001") > 0); +// assertTrue(scanNode6.getCanonicalName().indexOf("0000_000002") > 0); +// +// Collection broadcastTables = eb.getBroadcastTables(); +// assertEquals(3, broadcastTables.size()); +// +// assertTrue(broadcastTables.contains("default.small1")); +// assertTrue(broadcastTables.contains("default.small2")); +// assertTrue(broadcastTables.contains("default.small3")); +// } +// index++; +// } +// +// assertEquals(5, index); +// } +// +// @Test +// public final void testLeftOuterJoinCase3() throws IOException, PlanningException { +// // large1, large2, small1, large3, small2, small3 +// String query = "select count(*) from large1 " + +// "left outer join large2 on large1_id = large2_id " + +// "left outer join small1 on large2_id = small1_id " + +// "left outer join large3 on large1_id = large3_id " + +// "left outer join small2 on large3_id = small2_id " + +// "left outer join small3 on large3_id = small3_id "; +// +// LogicalPlanner planner = new LogicalPlanner(catalog); +// LogicalOptimizer optimizer = new LogicalOptimizer(conf); +// Expr expr = analyzer.parse(query); +// LogicalPlan plan = planner.createPlan(defaultContext, expr); +// +// optimizer.optimize(plan); +// +// QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0); +// QueryContext queryContext = new QueryContext(conf); +// MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); +// GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); +// globalPlanner.build(queryContext, masterPlan); +// +// //(((((default.large1 ⟕ default.large2) ⟕ default.small1) ⟕ default.large3) ⟕ default.small2) ⟕ default.small3) +// /* +// |-eb_1402634570910_0000_000007 +// |-eb_1402634570910_0000_000006 (GROUP BY) +// |-eb_1402634570910_0000_000005 (JOIN, broadcast small2, small3) +// |-eb_1402634570910_0000_000004 (LEAF, scan large3) +// |-eb_1402634570910_0000_000003 (JOIN, broadcast small1) +// |-eb_1402634570910_0000_000002 (LEAF, scan large2) +// |-eb_1402634570910_0000_000001 (LEAF, scan large1) +// */ +// ExecutionBlockCursor ebCursor = new ExecutionBlockCursor(masterPlan); +// int index = 0; +// while (ebCursor.hasNext()) { +// ExecutionBlock eb = ebCursor.nextBlock(); +// if(index == 0) { +// LogicalNode node = eb.getPlan(); +// assertEquals(NodeType.SCAN, node.getType()); +// ScanNode scanNode = (ScanNode)node; +// assertEquals("default.large1", scanNode.getCanonicalName()); +// +// Collection broadcastTables = eb.getBroadcastTables(); +// assertEquals(0, broadcastTables.size()); +// } else if (index == 1) { +// LogicalNode node = eb.getPlan(); +// assertEquals(NodeType.SCAN, node.getType()); +// ScanNode scanNode = (ScanNode)node; +// assertEquals("default.large2", scanNode.getCanonicalName()); +// +// Collection broadcastTables = eb.getBroadcastTables(); +// assertEquals(0, broadcastTables.size()); +// } else if(index == 2) { +// LogicalNode node = eb.getPlan(); +// assertEquals(NodeType.JOIN, node.getType()); +// JoinNode joinNode = (JoinNode)node; +// +// ScanNode leftNode = ((JoinNode)joinNode.getLeftChild()).getLeftChild(); +// ScanNode rightNode = ((JoinNode)joinNode.getLeftChild()).getRightChild(); +// assertTrue(leftNode.getCanonicalName().indexOf("0000_000001") > 0); +// assertTrue(rightNode.getCanonicalName().indexOf("0000_000002") > 0); +// +// Collection broadcastTables = eb.getBroadcastTables(); +// assertEquals(1, broadcastTables.size()); +// assertTrue(broadcastTables.contains("default.small1")); +// } else if(index == 3) { +// LogicalNode node = eb.getPlan(); +// assertEquals(NodeType.SCAN, node.getType()); +// ScanNode scanNode = (ScanNode)node; +// assertEquals("default.large3", scanNode.getCanonicalName()); +// +// Collection broadcastTables = eb.getBroadcastTables(); +// assertEquals(0, broadcastTables.size()); +// } else if(index == 4) { +// Collection broadcastTables = eb.getBroadcastTables(); +// assertEquals(2, broadcastTables.size()); +// assertTrue(broadcastTables.contains("default.small2")); +// assertTrue(broadcastTables.contains("default.small3")); +// } +// index++; +// } +// +// assertEquals(7, index); +// } +// +// @Test +// public final void testLeftOuterJoinCase4() throws IOException, PlanningException { +// // small1, small2, small3 +// String query = "select count(*) from small1 " + +// "left outer join small2 on small1_id = small2_id " + +// "left outer join small3 on small1_id = small3_id "; +// +// LogicalPlanner planner = new LogicalPlanner(catalog); +// LogicalOptimizer optimizer = new LogicalOptimizer(conf); +// Expr expr = analyzer.parse(query); +// LogicalPlan plan = planner.createPlan(defaultContext, expr); +// +// optimizer.optimize(plan); +// +// QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0); +// QueryContext queryContext = new QueryContext(conf); +// MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); +// GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); +// globalPlanner.build(queryContext, masterPlan); +// +// /* +// |-eb_1406022971444_0000_000005 +// |-eb_1406022971444_0000_000004 (group by) +// |-eb_1406022971444_0000_000003 (scan small1, broadcast join small2, small3) +// */ +// +// ExecutionBlockCursor ebCursor = new ExecutionBlockCursor(masterPlan); +// int index = 0; +// while (ebCursor.hasNext()) { +// ExecutionBlock eb = ebCursor.nextBlock(); +// if(index == 0) { +// GroupbyNode node = (GroupbyNode)eb.getPlan(); +// JoinNode joinNode = node.getChild(); +// +// ScanNode scanNode = joinNode.getRightChild(); +// assertEquals("default.small3", scanNode.getCanonicalName()); +// +// joinNode = joinNode.getLeftChild(); +// scanNode = joinNode.getLeftChild(); +// assertEquals("default.small1", scanNode.getCanonicalName()); +// scanNode = joinNode.getRightChild(); +// assertEquals("default.small2", scanNode.getCanonicalName()); +// +// Collection broadcastTables = eb.getBroadcastTables(); +// assertEquals(2, broadcastTables.size()); +// +// assertTrue(broadcastTables.contains("default.small2")); +// assertTrue(broadcastTables.contains("default.small3")); +// } else if(index == 1) { +// Collection broadcastTables = eb.getBroadcastTables(); +// assertEquals(0, broadcastTables.size()); +// } +// index++; +// } +// +// assertEquals(3, index); +// } +// +// @Test +// public final void testLeftOuterJoinCase5() throws IOException, PlanningException { +// // small, small, large, small +// String query = "select count(*) from small1 " + +// "left outer join small2 on small1_id = small2_id " + +// "left outer join large1 on small1_id = large1_id " + +// "left outer join small3 on small1_id = small3_id " ; +// +// LogicalPlanner planner = new LogicalPlanner(catalog); +// LogicalOptimizer optimizer = new LogicalOptimizer(conf); +// Expr expr = analyzer.parse(query); +// LogicalPlan plan = planner.createPlan(defaultContext, expr); +// +// optimizer.optimize(plan); +// +// QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0); +// QueryContext queryContext = new QueryContext(conf); +// MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); +// GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); +// globalPlanner.build(queryContext, masterPlan); +// +// //(((default.small1 ⟕ default.small2) ⟕ default.large1) ⟕ default.small3) +// /* +// |-eb_1406023347983_0000_000007 +// |-eb_1406023347983_0000_000006 +// |-eb_1406023347983_0000_000005 (join, broadcast small3) +// |-eb_1406023347983_0000_000004 (scan large1) +// |-eb_1406023347983_0000_000003 (scan small1, broadcast join small2) +// */ +// +// ExecutionBlockCursor ebCursor = new ExecutionBlockCursor(masterPlan); +// int index = 0; +// while (ebCursor.hasNext()) { +// ExecutionBlock eb = ebCursor.nextBlock(); +// if(index == 0) { +// Collection broadcastTables = eb.getBroadcastTables(); +// assertEquals(1, broadcastTables.size()); +// assertTrue(broadcastTables.contains("default.small2")); +// } else if (index == 2) { +// Collection broadcastTables = eb.getBroadcastTables(); +// assertEquals(1, broadcastTables.size()); +// assertTrue(broadcastTables.contains("default.small3")); +// } else if(index == 1 || index == 3) { +// Collection broadcastTables = eb.getBroadcastTables(); +// assertEquals(0, broadcastTables.size()); +// } +// index++; +// } +// +// assertEquals(5, index); +// } +// +// @Test +// public final void testLeftOuterJoinCase6() throws IOException, PlanningException { +// // small1, small2, large1, large2, small3 +// String query = "select count(*) from small1 " + +// "left outer join small2 on small1_id = small2_id " + +// "left outer join large1 on small1_id = large1_id " + +// "left outer join large2 on small1_id = large2_id " + +// "left outer join small3 on small1_id = small3_id " ; +// +// LogicalPlanner planner = new LogicalPlanner(catalog); +// LogicalOptimizer optimizer = new LogicalOptimizer(conf); +// Expr expr = analyzer.parse(query); +// LogicalPlan plan = planner.createPlan(defaultContext, expr); +// +// optimizer.optimize(plan); +// +// QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0); +// QueryContext queryContext = new QueryContext(conf); +// MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); +// GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); +// globalPlanner.build(queryContext, masterPlan); +// +// // ((((default.small1 ⟕ default.small2) ⟕ default.large1) ⟕ default.large2) ⟕ default.small3) +// +// /* +// |-eb_1406023537578_0000_000009 +// |-eb_1406023537578_0000_000008 +// |-eb_1406023537578_0000_000007 (join, broadcast small3) +// |-eb_1406023537578_0000_000006 (scan large2) +// |-eb_1406023537578_0000_000005 (join) +// |-eb_1406023537578_0000_000004 (scan large1) +// |-eb_1406023537578_0000_000003 (scan small1, broadcast join small2) +// */ +// ExecutionBlockCursor ebCursor = new ExecutionBlockCursor(masterPlan); +// int index = 0; +// while (ebCursor.hasNext()) { +// ExecutionBlock eb = ebCursor.nextBlock(); +// if(index == 0) { +// LogicalNode node = eb.getPlan(); +// assertEquals(NodeType.JOIN, node.getType()); +// JoinNode joinNode = (JoinNode)node; +// +// ScanNode scanNode1 = joinNode.getLeftChild(); +// ScanNode scanNode2 = joinNode.getRightChild(); +// assertEquals("default.small1", scanNode1.getCanonicalName()); +// assertEquals("default.small2", scanNode2.getCanonicalName()); +// +// Collection broadcastTables = eb.getBroadcastTables(); +// assertEquals(1, broadcastTables.size()); +// assertTrue(broadcastTables.contains("default.small2")); +// } else if (index == 1) { +// LogicalNode node = eb.getPlan(); +// assertEquals(NodeType.SCAN, node.getType()); +// ScanNode scanNode = (ScanNode) node; +// assertEquals("default.large1", scanNode.getCanonicalName()); +// +// Collection broadcastTables = eb.getBroadcastTables(); +// assertEquals(0, broadcastTables.size()); +// } else if (index == 2) { +// LogicalNode node = eb.getPlan(); +// assertEquals(NodeType.JOIN, node.getType()); +// } else if (index == 3) { +// LogicalNode node = eb.getPlan(); +// assertEquals(NodeType.SCAN, node.getType()); +// ScanNode scanNode = (ScanNode) node; +// assertEquals("default.large2", scanNode.getCanonicalName()); +// +// Collection broadcastTables = eb.getBroadcastTables(); +// assertEquals(0, broadcastTables.size()); +// } else if(index == 4) { +// LogicalNode node = eb.getPlan(); +// assertEquals(NodeType.GROUP_BY, node.getType()); +// +// JoinNode joinNode = ((GroupbyNode)node).getChild(); +// +// JoinNode joinNode1 = joinNode.getLeftChild(); +// ScanNode scanNode1 = joinNode.getRightChild(); +// assertEquals("default.small3", scanNode1.getCanonicalName()); +// +// ScanNode scanNode2 = joinNode1.getLeftChild(); +// ScanNode scanNode3 = joinNode1.getRightChild(); +// assertTrue(scanNode2.getCanonicalName().indexOf("0000_000005") > 0); +// assertTrue(scanNode3.getCanonicalName().indexOf("0000_000006") > 0); +// +// Collection broadcastTables = eb.getBroadcastTables(); +// assertEquals(1, broadcastTables.size()); +// } +// index++; +// } +// +// assertEquals(7, index); +// } +// +// @Test +// public final void testInnerLeftOuterJoinCase1() throws IOException, PlanningException { +// // small, small, large, small +// String query = "select count(*) from small1 " + +// "inner join small2 on small1_id = small2_id " + +// "left outer join large1 on small1_id = large1_id " + +// "left outer join small3 on small3_id = large1_id " ; +// +// LogicalPlanner planner = new LogicalPlanner(catalog); +// LogicalOptimizer optimizer = new LogicalOptimizer(conf); +// Expr expr = analyzer.parse(query); +// LogicalPlan plan = planner.createPlan(defaultContext, expr); +// +// optimizer.optimize(plan); +// +// QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0); +// QueryContext queryContext = new QueryContext(conf); +// MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); +// GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); +// globalPlanner.build(queryContext, masterPlan); +// +// // (((default.small1 ⋈θ default.small2) ⟕ default.large1) ⟕ default.small3) +// /* +// |-eb_1404139312268_0000_000006 +// |-eb_1404139312268_0000_000005 +// |-eb_1404139312268_0000_000003 (LEAF scan large1, broadcast small1, small2, small3) +// */ +// +// ExecutionBlockCursor ebCursor = new ExecutionBlockCursor(masterPlan); +// int index = 0; +// while (ebCursor.hasNext()) { +// ExecutionBlock eb = ebCursor.nextBlock(); +// if(index == 0) { +// LogicalNode node = eb.getPlan(); +// assertEquals(NodeType.GROUP_BY, node.getType()); +// JoinNode joinNode = ((GroupbyNode)node).getChild(); +// +// JoinNode joinNode2 = joinNode.getLeftChild(); +// ScanNode scanNode = joinNode.getRightChild(); +// assertEquals("default.small3", scanNode.getCanonicalName()); +// +// JoinNode joinNode3 = joinNode2.getLeftChild(); +// ScanNode scanNode2 = joinNode2.getRightChild(); +// +// assertEquals("default.large1", scanNode2.getCanonicalName()); +// +// ScanNode scanNode3 = joinNode3.getLeftChild(); +// ScanNode scanNode4 = joinNode3.getRightChild(); +// +// assertEquals("default.small1", scanNode3.getCanonicalName()); +// assertEquals("default.small2", scanNode4.getCanonicalName()); +// +// Collection broadcastTables = eb.getBroadcastTables(); +// +// assertEquals(3, broadcastTables.size()); +// assertTrue(broadcastTables.contains("default.small1")); +// assertTrue(broadcastTables.contains("default.small2")); +// assertTrue(broadcastTables.contains("default.small3")); +// } +// index++; +// } +// +// assertEquals(3, index); +// } +// +// @Test +// public final void testBroadcastCasebyCase1() throws IOException, PlanningException { +// // large, small, large, small +// String query = "select count(*) from large1 " + +// "inner join small1 on large1_id = small1_id " + +// "left outer join large2 on large1_id = large2_id " + +// "left outer join small2 on large1_id = small2_id " ; +// +// LogicalPlanner planner = new LogicalPlanner(catalog); +// LogicalOptimizer optimizer = new LogicalOptimizer(conf); +// Expr expr = analyzer.parse(query); +// LogicalPlan plan = planner.createPlan(defaultContext, expr); +// +// optimizer.optimize(plan); +// +// QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0); +// QueryContext queryContext = new QueryContext(conf); +// MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); +// GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); +// globalPlanner.build(queryContext, masterPlan); +// +// // (((default.large1 ⋈θ default.small1) ⟕ default.large2) ⟕ default.small2) +// /* +// |-eb_1404871198908_0000_000007 +// |-eb_1404871198908_0000_000006 +// |-eb_1404871198908_0000_000005 (join eb3, eb3, broadcast small2) +// |-eb_1404871198908_0000_000004 (scan large2) +// |-eb_1404871198908_0000_000003 (scan large1, broadcast small1) +// */ +// +// ExecutionBlockCursor ebCursor = new ExecutionBlockCursor(masterPlan); +// int index = 0; +// while (ebCursor.hasNext()) { +// ExecutionBlock eb = ebCursor.nextBlock(); +// if(index == 0) { +// LogicalNode node = eb.getPlan(); +// assertEquals(NodeType.JOIN, node.getType()); +// JoinNode joinNode = (JoinNode)node; +// +// ScanNode scanNode1 = joinNode.getLeftChild(); +// ScanNode scanNode2 = joinNode.getRightChild(); +// assertEquals("default.large1", scanNode1.getCanonicalName()); +// assertEquals("default.small1", scanNode2.getCanonicalName()); +// +// Collection broadcastTables = eb.getBroadcastTables(); +// +// assertEquals(1, broadcastTables.size()); +// assertTrue(broadcastTables.contains("default.small1")); +// } else if(index == 1) { +// LogicalNode node = eb.getPlan(); +// assertEquals(NodeType.SCAN, node.getType()); +// ScanNode scanNode = (ScanNode)node; +// +// assertEquals("default.large2", scanNode.getCanonicalName()); +// +// Collection broadcastTables = eb.getBroadcastTables(); +// TestCase.assertEquals(0, broadcastTables.size()); +// } else if(index == 2) { +// LogicalNode node = eb.getPlan(); +// assertEquals(NodeType.GROUP_BY, node.getType()); +// JoinNode joinNode = ((GroupbyNode)node).getChild(); +// +// JoinNode joinNode2 = joinNode.getLeftChild(); +// ScanNode scanNode = joinNode.getRightChild(); +// assertEquals("default.small2", scanNode.getCanonicalName()); +// +// ScanNode scanNode2 = joinNode2.getLeftChild(); +// ScanNode scanNode3 = joinNode2.getRightChild(); +// +// assertTrue(scanNode2.getCanonicalName().indexOf("000003") >= 0); +// assertTrue(scanNode3.getCanonicalName().indexOf("000004") >= 0); +// +// Collection broadcastTables = eb.getBroadcastTables(); +// +// TestCase.assertEquals(1, broadcastTables.size()); +// TestCase.assertTrue(broadcastTables.contains("default.small2")); +// } +// index++; +// } +// +// TestCase.assertEquals(5, index); +// } +//} diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java index 90088c09b1..1decde4e1c 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java @@ -345,21 +345,21 @@ public final void testBroadcastBasicJoin() throws Exception { assertResultSet(res); cleanupQuery(res); - MasterPlan plan = getQueryPlan(resultSet.getQueryId()); - ExecutionBlock rootEB = plan.getRoot(); - - /* - |-eb_1395998037360_0001_000006 - |-eb_1395998037360_0001_000005 - */ - assertEquals(1, plan.getChildCount(rootEB.getId())); - - ExecutionBlock firstEB = plan.getChild(rootEB.getId(), 0); - - assertNotNull(firstEB); - assertEquals(2, firstEB.getBroadcastTables().size()); - assertTrue(firstEB.getBroadcastTables().contains("default.supplier")); - assertTrue(firstEB.getBroadcastTables().contains("default.part")); +// MasterPlan plan = getQueryPlan(resultSet.getQueryId()); +// ExecutionBlock rootEB = plan.getRoot(); +// +// /* +// |-eb_1395998037360_0001_000006 +// |-eb_1395998037360_0001_000005 +// */ +// assertEquals(1, plan.getChildCount(rootEB.getId())); +// +// ExecutionBlock firstEB = plan.getChild(rootEB.getId(), 0); +// +// assertNotNull(firstEB); +// assertEquals(2, firstEB.getBroadcastTables().size()); +// assertTrue(firstEB.getBroadcastTables().contains("default.supplier")); +// assertTrue(firstEB.getBroadcastTables().contains("default.part")); } @Test @@ -371,28 +371,28 @@ public final void testBroadcastTwoPartJoin() throws Exception { assertResultSet(res); cleanupQuery(res); - MasterPlan plan = getQueryPlan(resultSet.getQueryId()); - ExecutionBlock rootEB = plan.getRoot(); - - /* - |-eb_1395996354406_0001_000010 - |-eb_1395996354406_0001_000009 - |-eb_1395996354406_0001_000008 - |-eb_1395996354406_0001_000005 - */ - assertEquals(1, plan.getChildCount(rootEB.getId())); - - ExecutionBlock firstJoinEB = plan.getChild(rootEB.getId(), 0); - assertNotNull(firstJoinEB); - assertEquals(NodeType.JOIN, firstJoinEB.getPlan().getType()); - assertEquals(0, firstJoinEB.getBroadcastTables().size()); - - ExecutionBlock leafEB1 = plan.getChild(firstJoinEB.getId(), 0); - assertTrue(leafEB1.getBroadcastTables().contains("default.orders")); - assertTrue(leafEB1.getBroadcastTables().contains("default.part")); - - ExecutionBlock leafEB2 = plan.getChild(firstJoinEB.getId(), 1); - assertTrue(leafEB2.getBroadcastTables().contains("default.nation")); +// MasterPlan plan = getQueryPlan(resultSet.getQueryId()); +// ExecutionBlock rootEB = plan.getRoot(); +// +// /* +// |-eb_1395996354406_0001_000010 +// |-eb_1395996354406_0001_000009 +// |-eb_1395996354406_0001_000008 +// |-eb_1395996354406_0001_000005 +// */ +// assertEquals(1, plan.getChildCount(rootEB.getId())); +// +// ExecutionBlock firstJoinEB = plan.getChild(rootEB.getId(), 0); +// assertNotNull(firstJoinEB); +// assertEquals(NodeType.JOIN, firstJoinEB.getPlan().getType()); +// assertEquals(0, firstJoinEB.getBroadcastTables().size()); +// +// ExecutionBlock leafEB1 = plan.getChild(firstJoinEB.getId(), 0); +// assertTrue(leafEB1.getBroadcastTables().contains("default.orders")); +// assertTrue(leafEB1.getBroadcastTables().contains("default.part")); +// +// ExecutionBlock leafEB2 = plan.getChild(firstJoinEB.getId(), 1); +// assertTrue(leafEB2.getBroadcastTables().contains("default.nation")); } @Test diff --git a/tajo-core/src/test/resources/queries/TestJoinBroadcast/testBroadcastTwoPartJoin.sql b/tajo-core/src/test/resources/queries/TestJoinBroadcast/testBroadcastTwoPartJoin.sql index bdad24fe79..74d08f74d8 100644 --- a/tajo-core/src/test/resources/queries/TestJoinBroadcast/testBroadcastTwoPartJoin.sql +++ b/tajo-core/src/test/resources/queries/TestJoinBroadcast/testBroadcastTwoPartJoin.sql @@ -12,4 +12,5 @@ where l_orderkey = o_orderkey and l_partkey = p_partkey and o_custkey = c_custkey - and c_nationkey = n_nationkey \ No newline at end of file + and c_nationkey = n_nationkey +order by l_orderkey, p_name, n_name \ No newline at end of file diff --git a/tajo-core/src/test/resources/results/TestJoinBroadcast/testBroadcastTwoPartJoin.result b/tajo-core/src/test/resources/results/TestJoinBroadcast/testBroadcastTwoPartJoin.result index 1a3d68cf9c..46aab24422 100644 --- a/tajo-core/src/test/resources/results/TestJoinBroadcast/testBroadcastTwoPartJoin.result +++ b/tajo-core/src/test/resources/results/TestJoinBroadcast/testBroadcastTwoPartJoin.result @@ -1,15 +1,15 @@ l_orderkey,p_name,n_name ------------------------------- -3,goldenrod lavender spring chocolate lace,GERMANY +1,blush thistle blue yellow saddle,GERMANY +1,goldenrod lavender spring chocolate lace,GERMANY +1,goldenrod lavender spring chocolate lace,GERMANY +1,goldenrod lavender spring chocolate lace,GERMANY +1,spring green yellow purple cornsilk,GERMANY +1,spring green yellow purple cornsilk,GERMANY +2,blush thistle blue yellow saddle,IRAN 3,blush thistle blue yellow saddle,GERMANY -3,spring green yellow purple cornsilk,GERMANY +3,goldenrod lavender spring chocolate lace,GERMANY 3,goldenrod lavender spring chocolate lace,GERMANY 3,spring green yellow purple cornsilk,GERMANY 3,spring green yellow purple cornsilk,GERMANY -1,spring green yellow purple cornsilk,GERMANY -1,goldenrod lavender spring chocolate lace,GERMANY -1,spring green yellow purple cornsilk,GERMANY -1,blush thistle blue yellow saddle,GERMANY -1,goldenrod lavender spring chocolate lace,GERMANY -1,goldenrod lavender spring chocolate lace,GERMANY -2,blush thistle blue yellow saddle,IRAN \ No newline at end of file +3,spring green yellow purple cornsilk,GERMANY \ 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 6907c98519..72cab6cefa 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 @@ -19,6 +19,8 @@ package org.apache.tajo.plan.expr; import com.google.gson.annotations.Expose; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import org.apache.tajo.catalog.FunctionDesc; import org.apache.tajo.catalog.Schema; import org.apache.tajo.common.TajoDataTypes.DataType; @@ -33,9 +35,12 @@ import java.io.IOException; public class AggregationFunctionCallEval extends FunctionEval implements Cloneable { - @Expose boolean firstPhase = false; - @Expose boolean finalPhase = true; - @Expose String alias; + + private static final Log LOG = LogFactory.getLog(AggregationFunctionCallEval.class); + // Both firstPhase and finalPhase flags should be true before global planning. + @Expose private boolean firstPhase = true; + @Expose private boolean finalPhase = true; + @Expose private String alias; @Expose protected FunctionInvokeContext invokeContext; protected transient AggFunctionInvoke functionInvoke; @@ -77,6 +82,7 @@ public EvalNode bind(EvalContext evalContext, Schema schema) { } public void merge(FunctionContext context, Tuple tuple) { + LOG.info("at merge, " + funcDesc.getFunctionName() + " firstPhase: " + firstPhase + ", finalPhase: " + finalPhase); if (!isBinded) { throw new IllegalStateException("bind() must be called before merge()"); } @@ -99,6 +105,7 @@ public T eval(Tuple tuple) { } public Datum terminate(FunctionContext context) { + LOG.info("at terminate, " + funcDesc.getFunctionName() + " firstPhase: " + firstPhase + ", finalPhase: " + finalPhase); if (!isBinded) { throw new IllegalStateException("bind() must be called before terminate()"); } @@ -111,6 +118,7 @@ public Datum terminate(FunctionContext context) { @Override public DataType getValueType() { + LOG.info("at getValueType, " + funcDesc.getFunctionName() + " firstPhase: " + firstPhase + ", finalPhase: " + finalPhase); if (!finalPhase) { return functionInvoke.getPartialResultType(); } else { diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java index 25c81ed0b4..1a73d7eed2 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java @@ -191,11 +191,14 @@ public int compare(PlanProto.EvalNode o1, PlanProto.EvalNode o2) { new AggregationFunctionCallEval(new FunctionDesc(funcProto.getFuncion()), params); PlanProto.AggFunctionEvalSpec aggFunctionProto = protoNode.getAggFunction(); - if (aggFunctionProto.getFirstPhase()) { + if (aggFunctionProto.getFirstPhase() && aggFunctionProto.getFinalPhase()) { + aggFunc.setFirstAndFinalPhase(); + } else if (aggFunctionProto.getFirstPhase()) { aggFunc.setFirstPhase(); - } - if (aggFunctionProto.getFinalPhase()) { + } else if (aggFunctionProto.getFinalPhase()) { aggFunc.setFinalPhase(); + } else { + aggFunc.setIntermediatePhase(); } if (aggFunctionProto.hasAlias()) { aggFunc.setAlias(aggFunctionProto.getAlias()); From 0af98050a7e2df70af92b6cf91e805cb1b7ce663 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 4 May 2015 22:23:51 +0900 Subject: [PATCH 12/37] TAJO-1553 --- .../test/java/org/apache/tajo/querymaster/TestKillQuery.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java index b87ea43683..eca7f6d7c4 100644 --- a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java @@ -169,7 +169,7 @@ public final void testIgnoreStageStateFromKilled() throws Exception { QueryContext queryContext = new QueryContext(conf); MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan); GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog); - globalPlanner.build(masterPlan); + globalPlanner.build(queryContext, masterPlan); CountDownLatch barrier = new CountDownLatch(1); MockAsyncDispatch dispatch = new MockAsyncDispatch(barrier, TajoProtos.QueryState.QUERY_RUNNING); From b22fe962b7ed82ba0606f8a7236d68906406cd8b Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 5 May 2015 13:07:30 +0900 Subject: [PATCH 13/37] TAJO-1553 --- .../engine/planner/global/GlobalPlanner.java | 5 +-- .../builder/DistinctGroupbyBuilder.java | 5 +-- .../rewriter/rules/BroadcastJoinRule.java | 4 +- .../DistinctGroupbyThirdAggregationExec.java | 2 +- .../org/apache/tajo/querymaster/Stage.java | 4 +- .../expr/AggregationFunctionCallEval.java | 42 +++++++++---------- .../function/python/PythonScriptEngine.java | 6 +-- .../function/python/TajoScriptEngine.java | 7 ++-- .../plan/serder/EvalNodeDeserializer.java | 11 +++-- .../tajo/plan/serder/EvalNodeSerializer.java | 2 +- tajo-plan/src/main/proto/Plan.proto | 2 +- 11 files changed, 41 insertions(+), 49 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java index d0d3df8a3b..4615e33f71 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java @@ -35,8 +35,6 @@ import org.apache.tajo.catalog.proto.CatalogProtos; import org.apache.tajo.common.TajoDataTypes; import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.engine.planner.BroadcastJoinMarkCandidateVisitor; -import org.apache.tajo.engine.planner.BroadcastJoinPlanVisitor; import org.apache.tajo.engine.planner.global.builder.DistinctGroupbyBuilder; import org.apache.tajo.engine.planner.global.rewriter.GlobalPlanRewriteEngine; import org.apache.tajo.engine.planner.global.rewriter.GlobalPlanRewriteRuleProvider; @@ -46,7 +44,6 @@ import org.apache.tajo.plan.PlanningException; import org.apache.tajo.plan.Target; import org.apache.tajo.plan.expr.*; -import org.apache.tajo.plan.function.AggFunction; import org.apache.tajo.plan.logical.*; import org.apache.tajo.plan.rewrite.rules.ProjectionPushDownRule; import org.apache.tajo.plan.util.PlannerUtil; @@ -963,7 +960,7 @@ public static GroupbyNode createFirstPhaseGroupBy(LogicalPlan plan, GroupbyNode firstPhaseEvalNames[i] = plan.generateUniqueColumnName(firstPhaseEvals[i]); FieldEval param = new FieldEval(firstPhaseEvalNames[i], firstPhaseEvals[i].getValueType()); - secondPhaseEvals[i].setFinalPhase(); + secondPhaseEvals[i].setLastPhase(); secondPhaseEvals[i].setArgs(new EvalNode[]{param}); } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/builder/DistinctGroupbyBuilder.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/builder/DistinctGroupbyBuilder.java index 6192bd36bc..cd0f1bd2fd 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/builder/DistinctGroupbyBuilder.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/builder/DistinctGroupbyBuilder.java @@ -112,8 +112,7 @@ public ExecutionBlock buildMultiLevelPlan(GlobalPlanContext context, if (!lastGroupbyNode.isDistinct()) { int index = 0; for (AggregationFunctionCallEval aggrFunction: lastGroupbyNode.getAggFunctions()) { -// aggrFunction.setFirstPhase(); - aggrFunction.setFinalPhase(); + aggrFunction.setFirstPhase(); aggrFunction.setArgs(new EvalNode[]{new FieldEval(lastGroupbyNode.getTargets()[index].getNamedColumn())}); index++; } @@ -407,7 +406,7 @@ select col1, count(distinct col2), count(distinct col3), sum(col4) from ... grou buildInfo.addAggFunction(aggFunction); buildInfo.addAggFunctionTarget(aggFunctionTarget); } else { - aggFunction.setFinalPhase(); + aggFunction.setLastPhase(); otherAggregationFunctionCallEvals.add(aggFunction); otherAggregationFunctionTargets.add(aggFunctionTarget); } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java index 8f839e79c1..15c1aefc83 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java @@ -19,13 +19,11 @@ package org.apache.tajo.engine.planner.global.rewriter.rules; import org.apache.tajo.OverridableConf; -import org.apache.tajo.SessionVars; import org.apache.tajo.engine.planner.global.DataChannel; import org.apache.tajo.engine.planner.global.ExecutionBlock; import org.apache.tajo.engine.planner.global.MasterPlan; import org.apache.tajo.engine.planner.global.rewriter.GlobalPlanRewriteRule; import org.apache.tajo.ipc.TajoWorkerProtocol.EnforceProperty.EnforceType; -import org.apache.tajo.plan.LogicalPlan; import org.apache.tajo.plan.PlanningException; import org.apache.tajo.plan.expr.AggregationFunctionCallEval; import org.apache.tajo.plan.logical.*; @@ -148,7 +146,7 @@ private ExecutionBlock mergeTwoPhaseNonJoin(MasterPlan plan, ExecutionBlock chil GroupbyNode firstPhaseGroupby = (GroupbyNode) firstPhaseNode; GroupbyNode secondPhaseGroupby = (GroupbyNode) secondPhaseNode; for (AggregationFunctionCallEval aggFunc : firstPhaseGroupby.getAggFunctions()) { - aggFunc.setFirstAndFinalPhase(); + aggFunc.setFirstAndLastPhase(); } firstPhaseGroupby.setTargets(secondPhaseGroupby.getTargets()); firstPhaseGroupby.setOutSchema(secondPhaseGroupby.getOutSchema()); diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbyThirdAggregationExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbyThirdAggregationExec.java index e71976c9ee..1e8006de23 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbyThirdAggregationExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbyThirdAggregationExec.java @@ -253,7 +253,7 @@ public DistinctFinalAggregator(int seq, int inTupleIndex, int outTupleIndex, Gro if (aggrFunctions != null) { for (AggregationFunctionCallEval eachFunction: aggrFunctions) { eachFunction.bind(context.getEvalContext(), inSchema); - eachFunction.setFinalPhase(); + eachFunction.setLastPhase(); } } newFunctionContext(); diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java index e592c6574a..dd38cfdd1d 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java @@ -883,8 +883,8 @@ public static int calculateShuffleOutputNum(Stage stage, DataChannel channel) { } // We assume this execution block the first stage of join if two or more tables are included in this block, -// if (parent != null && (parent.getScanNodes().length - parent.getBroadcastTables().size()) >= 2) { if (parent != null && (parent.getNonBroadcastRelNum()) >= 2) { +// if (parent != null && parent.getScanNodes().length >= 2) { List childs = masterPlan.getChilds(parent); // for outer @@ -1080,7 +1080,7 @@ public static void allocateContainers(Stage stage) { private static void scheduleFragmentsForLeafQuery(Stage stage) throws IOException { ExecutionBlock execBlock = stage.getBlock(); ScanNode[] scans = execBlock.getScanNodes(); - Preconditions.checkArgument(scans.length <= 1, "Must be Scan Query"); + Preconditions.checkArgument(scans.length == 1, "Must be Scan Query"); // Preconditions.checkArgument(execBlock.getNonBroadcastRelNum() <= 1, // "A leaf stage should not scan two or more large fragments"); ScanNode scan = scans[0]; 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 72cab6cefa..3668c05a85 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 @@ -37,9 +37,9 @@ public class AggregationFunctionCallEval extends FunctionEval implements Cloneable { private static final Log LOG = LogFactory.getLog(AggregationFunctionCallEval.class); - // Both firstPhase and finalPhase flags should be true before global planning. + // Both firstPhase and lastPhase flags should be true before global planning. @Expose private boolean firstPhase = true; - @Expose private boolean finalPhase = true; + @Expose private boolean lastPhase = true; @Expose private String alias; @Expose protected FunctionInvokeContext invokeContext; @@ -71,7 +71,7 @@ public EvalNode bind(EvalContext evalContext, Schema schema) { if (evalContext != null && evalContext.hasScriptEngine(this)) { this.invokeContext.setScriptEngine(evalContext.getScriptEngine(this)); this.invokeContext.getScriptEngine().setFirstPhase(firstPhase); - this.invokeContext.getScriptEngine().setFinalPhase(finalPhase); + this.invokeContext.getScriptEngine().setLastPhase(lastPhase); } this.functionInvoke.init(invokeContext); } catch (IOException e) { @@ -82,7 +82,7 @@ public EvalNode bind(EvalContext evalContext, Schema schema) { } public void merge(FunctionContext context, Tuple tuple) { - LOG.info("at merge, " + funcDesc.getFunctionName() + " firstPhase: " + firstPhase + ", finalPhase: " + finalPhase); + LOG.info("at merge, " + funcDesc.getFunctionName() + " firstPhase: " + firstPhase + ", lastPhase: " + lastPhase); if (!isBinded) { throw new IllegalStateException("bind() must be called before merge()"); } @@ -90,7 +90,7 @@ public void merge(FunctionContext context, Tuple tuple) { } protected void mergeParam(FunctionContext context, Tuple params) { -// if (!intermediatePhase && !finalPhase) { +// if (!intermediatePhase && !lastPhase) { if (firstPhase) { // firstPhase functionInvoke.eval(context, params); @@ -105,11 +105,11 @@ public T eval(Tuple tuple) { } public Datum terminate(FunctionContext context) { - LOG.info("at terminate, " + funcDesc.getFunctionName() + " firstPhase: " + firstPhase + ", finalPhase: " + finalPhase); + LOG.info("at terminate, " + funcDesc.getFunctionName() + " firstPhase: " + firstPhase + ", lastPhase: " + lastPhase); if (!isBinded) { throw new IllegalStateException("bind() must be called before terminate()"); } - if (!finalPhase) { + if (!lastPhase) { return functionInvoke.getPartialResult(context); } else { return functionInvoke.terminate(context); @@ -118,8 +118,8 @@ public Datum terminate(FunctionContext context) { @Override public DataType getValueType() { - LOG.info("at getValueType, " + funcDesc.getFunctionName() + " firstPhase: " + firstPhase + ", finalPhase: " + finalPhase); - if (!finalPhase) { + LOG.info("at getValueType, " + funcDesc.getFunctionName() + " firstPhase: " + firstPhase + ", lastPhase: " + lastPhase); + if (!lastPhase) { return functionInvoke.getPartialResultType(); } else { return funcDesc.getReturnType(); @@ -137,7 +137,7 @@ public boolean hasAlias() { public Object clone() throws CloneNotSupportedException { AggregationFunctionCallEval clone = (AggregationFunctionCallEval)super.clone(); - clone.finalPhase = finalPhase; + clone.lastPhase = lastPhase; clone.firstPhase = firstPhase; clone.alias = alias; clone.invokeContext = (FunctionInvokeContext) invokeContext.clone(); @@ -152,27 +152,27 @@ public boolean isFirstPhase() { return firstPhase; } - public boolean isFinalPhase() { - return finalPhase; + public boolean isLastPhase() { + return lastPhase; } public void setFirstPhase() { this.firstPhase = true; - this.finalPhase = false; + this.lastPhase = false; } - public void setFinalPhase() { + public void setLastPhase() { this.firstPhase = false; - this.finalPhase = true; + this.lastPhase = true; } - public void setFirstAndFinalPhase() { - this.finalPhase = this.firstPhase = true; + public void setFirstAndLastPhase() { + this.lastPhase = this.firstPhase = true; } public void setIntermediatePhase() { this.firstPhase = false; - this.finalPhase = false; + this.lastPhase = false; } @Override @@ -180,8 +180,8 @@ public int hashCode() { final int prime = 31; int result = super.hashCode(); result = prime * result + ((alias == null) ? 0 : alias.hashCode()); - result = prime * result + (finalPhase ? 1231 : 1237); - result = prime * result + (firstPhase ? 1231 : 1237); + result = prime * result + (lastPhase ? 1231 : 1237); + result = prime * result + (firstPhase ? 1249 : 1259); return result; } @@ -192,7 +192,7 @@ public boolean equals(Object obj) { boolean eq = super.equals(other); eq &= firstPhase == other.firstPhase; - eq &= finalPhase == other.finalPhase; + eq &= lastPhase == other.lastPhase; eq &= TUtil.checkEquals(alias, other.alias); return eq; } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java index ad07b4a146..e2acc1d065 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java @@ -286,14 +286,14 @@ public PythonScriptEngine(FunctionDesc functionDesc) { setSchema(); } - public PythonScriptEngine(FunctionDesc functionDesc, boolean firstPhase, boolean finalPhase) { + public PythonScriptEngine(FunctionDesc functionDesc, boolean firstPhase, boolean lastPhase) { if (!functionDesc.getInvocation().hasPython() && !functionDesc.getInvocation().hasPythonAggregation()) { throw new IllegalStateException("Function type must be 'python'"); } functionSignature = functionDesc.getSignature(); invocationDesc = functionDesc.getInvocation().getPython(); this.firstPhase = firstPhase; - this.finalPhase = finalPhase; + this.lastPhase = lastPhase; setSchema(); } @@ -389,7 +389,7 @@ private void setSchema() { inSchema.addColumn(new Column("in_" + i, paramTypes[i])); } outSchema = new Schema(new Column[]{new Column("json", TajoDataTypes.Type.TEXT)}); - } else if (finalPhase) { + } else if (lastPhase) { inSchema = new Schema(new Column[]{new Column("json", TajoDataTypes.Type.TEXT)}); outSchema = new Schema(new Column[]{new Column("out", functionSignature.getReturnType())}); } else { diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java index 539260452c..8e35955414 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java @@ -19,7 +19,6 @@ package org.apache.tajo.plan.function.python; import org.apache.hadoop.conf.Configuration; -import org.apache.tajo.catalog.Schema; import org.apache.tajo.datum.Datum; import org.apache.tajo.plan.function.FunctionContext; import org.apache.tajo.storage.Tuple; @@ -33,7 +32,7 @@ public abstract class TajoScriptEngine { protected boolean firstPhase = false; - protected boolean finalPhase = false; + protected boolean lastPhase = false; /** * Open a stream load a script locally or in the classpath @@ -96,7 +95,7 @@ public void setFirstPhase(boolean flag) { this.firstPhase = flag; } - public void setFinalPhase(boolean flag) { - this.finalPhase = flag; + public void setLastPhase(boolean flag) { + this.lastPhase = flag; } } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java index 1a73d7eed2..72373cfac7 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java @@ -34,7 +34,6 @@ import org.apache.tajo.datum.*; import org.apache.tajo.exception.InternalException; import org.apache.tajo.plan.expr.*; -import org.apache.tajo.plan.function.AggFunction; import org.apache.tajo.plan.function.python.PythonScriptEngine; import org.apache.tajo.plan.logical.WindowSpec; import org.apache.tajo.plan.serder.PlanProto.WinFunctionEvalSpec; @@ -191,12 +190,12 @@ public int compare(PlanProto.EvalNode o1, PlanProto.EvalNode o2) { new AggregationFunctionCallEval(new FunctionDesc(funcProto.getFuncion()), params); PlanProto.AggFunctionEvalSpec aggFunctionProto = protoNode.getAggFunction(); - if (aggFunctionProto.getFirstPhase() && aggFunctionProto.getFinalPhase()) { - aggFunc.setFirstAndFinalPhase(); + if (aggFunctionProto.getFirstPhase() && aggFunctionProto.getLastPhase()) { + aggFunc.setFirstAndLastPhase(); } else if (aggFunctionProto.getFirstPhase()) { aggFunc.setFirstPhase(); - } else if (aggFunctionProto.getFinalPhase()) { - aggFunc.setFinalPhase(); + } else if (aggFunctionProto.getLastPhase()) { + aggFunc.setLastPhase(); } else { aggFunc.setIntermediatePhase(); } @@ -207,7 +206,7 @@ public int compare(PlanProto.EvalNode o1, PlanProto.EvalNode o2) { if (evalContext != null && funcDesc.getInvocation().hasPythonAggregation()) { evalContext.addScriptEngine(current, new PythonScriptEngine(funcDesc, - aggFunc.isFirstPhase() , aggFunc.isFinalPhase())); + aggFunc.isFirstPhase() , aggFunc.isLastPhase())); } } else { diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeSerializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeSerializer.java index 392da8ee40..a03b637323 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeSerializer.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeSerializer.java @@ -281,7 +281,7 @@ public EvalNode visitFuncCall(EvalTreeProtoBuilderContext context, FunctionEval PlanProto.AggFunctionEvalSpec.Builder aggFunctionEvalBuilder = PlanProto.AggFunctionEvalSpec.newBuilder(); aggFunctionEvalBuilder.setFirstPhase(aggFunc.isFirstPhase()); - aggFunctionEvalBuilder.setFinalPhase(aggFunc.isFinalPhase()); + aggFunctionEvalBuilder.setLastPhase(aggFunc.isLastPhase()); if (aggFunc.hasAlias()) { aggFunctionEvalBuilder.setAlias(aggFunc.getAlias()); } diff --git a/tajo-plan/src/main/proto/Plan.proto b/tajo-plan/src/main/proto/Plan.proto index 4e9f387dc9..7edbed2d38 100644 --- a/tajo-plan/src/main/proto/Plan.proto +++ b/tajo-plan/src/main/proto/Plan.proto @@ -425,7 +425,7 @@ message FunctionEval { message AggFunctionEvalSpec { // requires FunctionEval required bool firstPhase = 1; - required bool finalPhase = 2; + required bool lastPhase = 2; optional string alias = 3; } From 4cb67c8ff15256ad0b482dbd7ab8d30858006228 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 5 May 2015 22:15:20 +0900 Subject: [PATCH 14/37] Fix distinct aggregation bug --- .../builder/DistinctGroupbyBuilder.java | 2 ++ .../rewriter/rules/BroadcastJoinRule.java | 28 ++++++++++--------- .../expr/AggregationFunctionCallEval.java | 8 +++--- 3 files changed, 21 insertions(+), 17 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/builder/DistinctGroupbyBuilder.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/builder/DistinctGroupbyBuilder.java index cd0f1bd2fd..261df76034 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/builder/DistinctGroupbyBuilder.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/builder/DistinctGroupbyBuilder.java @@ -557,6 +557,7 @@ select col1, count(distinct col2), count(distinct col3), sum(col4) from ... grou } for (int aggFuncIdx = 0; aggFuncIdx < secondStageGroupbyNode.getAggFunctions().length; aggFuncIdx++) { + secondStageGroupbyNode.getAggFunctions()[aggFuncIdx].setLastPhase(); int targetIdx = originGroupColumns.size() + uniqueDistinctColumn.size() + aggFuncIdx; Target aggFuncTarget = oldTargets[targetIdx]; secondGroupbyTargets.add(aggFuncTarget); @@ -586,6 +587,7 @@ select col1, count(distinct col2), count(distinct col3), sum(col4) from ... grou AggregationFunctionCallEval secondStageAggFunction = secondStageGroupbyNode.getAggFunctions()[aggFuncIdx]; secondStageAggFunction.setArgs(new EvalNode[] {firstEval}); + secondStageAggFunction.setLastPhase(); Target secondTarget = secondStageGroupbyNode.getTargets()[secondStageGroupbyNode.getGroupingColumns().length + aggFuncIdx]; Column column = secondTarget.getNamedColumn(); diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java index 15c1aefc83..43eef1545c 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java @@ -19,11 +19,13 @@ package org.apache.tajo.engine.planner.global.rewriter.rules; import org.apache.tajo.OverridableConf; +import org.apache.tajo.SessionVars; import org.apache.tajo.engine.planner.global.DataChannel; import org.apache.tajo.engine.planner.global.ExecutionBlock; import org.apache.tajo.engine.planner.global.MasterPlan; import org.apache.tajo.engine.planner.global.rewriter.GlobalPlanRewriteRule; import org.apache.tajo.ipc.TajoWorkerProtocol.EnforceProperty.EnforceType; +import org.apache.tajo.plan.LogicalPlan; import org.apache.tajo.plan.PlanningException; import org.apache.tajo.plan.expr.AggregationFunctionCallEval; import org.apache.tajo.plan.logical.*; @@ -41,19 +43,19 @@ public String getName() { @Override public boolean isEligible(OverridableConf queryContext, MasterPlan plan) { -// if (queryContext.getBool(SessionVars.TEST_BROADCAST_JOIN_ENABLED)) { -// for (LogicalPlan.QueryBlock block : plan.getLogicalPlan().getQueryBlocks()) { -// if (block.hasNode(NodeType.JOIN)) { -// broadcastTableSizeThreshold = queryContext.getLong(SessionVars.BROADCAST_TABLE_SIZE_LIMIT); -// if (broadcastTableSizeThreshold > 0) { -// if (parentFinder == null) { -// parentFinder = new ParentFinder(); -// } -// return true; -// } -// } -// } -// } + if (queryContext.getBool(SessionVars.TEST_BROADCAST_JOIN_ENABLED)) { + for (LogicalPlan.QueryBlock block : plan.getLogicalPlan().getQueryBlocks()) { + if (block.hasNode(NodeType.JOIN)) { + broadcastTableSizeThreshold = queryContext.getLong(SessionVars.BROADCAST_TABLE_SIZE_LIMIT); + if (broadcastTableSizeThreshold > 0) { + if (parentFinder == null) { + parentFinder = new ParentFinder(); + } + return true; + } + } + } + } return false; } 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 3668c05a85..16c3b243a4 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 @@ -36,7 +36,7 @@ public class AggregationFunctionCallEval extends FunctionEval implements Cloneable { - private static final Log LOG = LogFactory.getLog(AggregationFunctionCallEval.class); +// private static final Log LOG = LogFactory.getLog(AggregationFunctionCallEval.class); // Both firstPhase and lastPhase flags should be true before global planning. @Expose private boolean firstPhase = true; @Expose private boolean lastPhase = true; @@ -82,7 +82,7 @@ public EvalNode bind(EvalContext evalContext, Schema schema) { } public void merge(FunctionContext context, Tuple tuple) { - LOG.info("at merge, " + funcDesc.getFunctionName() + " firstPhase: " + firstPhase + ", lastPhase: " + lastPhase); +// LOG.info("at merge, " + funcDesc.getFunctionName() + " firstPhase: " + firstPhase + ", lastPhase: " + lastPhase); if (!isBinded) { throw new IllegalStateException("bind() must be called before merge()"); } @@ -105,7 +105,7 @@ public T eval(Tuple tuple) { } public Datum terminate(FunctionContext context) { - LOG.info("at terminate, " + funcDesc.getFunctionName() + " firstPhase: " + firstPhase + ", lastPhase: " + lastPhase); +// LOG.info("at terminate, " + funcDesc.getFunctionName() + " firstPhase: " + firstPhase + ", lastPhase: " + lastPhase); if (!isBinded) { throw new IllegalStateException("bind() must be called before terminate()"); } @@ -118,7 +118,7 @@ public Datum terminate(FunctionContext context) { @Override public DataType getValueType() { - LOG.info("at getValueType, " + funcDesc.getFunctionName() + " firstPhase: " + firstPhase + ", lastPhase: " + lastPhase); +// LOG.info("at getValueType, " + funcDesc.getFunctionName() + " firstPhase: " + firstPhase + ", lastPhase: " + lastPhase); if (!lastPhase) { return functionInvoke.getPartialResultType(); } else { From 7598a737c0da5111200290992c8660b753fd27cc Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 6 May 2015 18:09:42 +0900 Subject: [PATCH 15/37] TAJO-1553 --- .../tajo/plan/logical/CreateTableNode.java | 5 ++++ .../plan/serder/LogicalNodeDeserializer.java | 5 ---- .../plan/serder/LogicalNodeSerializer.java | 26 +++++++++++++------ tajo-plan/src/main/proto/Plan.proto | 8 +++--- 4 files changed, 27 insertions(+), 17 deletions(-) diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/logical/CreateTableNode.java b/tajo-plan/src/main/java/org/apache/tajo/plan/logical/CreateTableNode.java index 0976ab5663..0ab62d51ce 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/logical/CreateTableNode.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/logical/CreateTableNode.java @@ -36,6 +36,11 @@ public CreateTableNode(int pid) { super(pid, NodeType.CREATE_TABLE); } + @Override + public int childNum() { + return child == null ? 0 : 1; + } + public void setTableSchema(Schema schema) { this.schema = schema; } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeDeserializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeDeserializer.java index 84991bbf75..694e81c930 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeDeserializer.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeDeserializer.java @@ -43,11 +43,6 @@ * It deserializes a list of serialized logical nodes into a logical node tree. */ public class LogicalNodeDeserializer { - private static final LogicalNodeDeserializer instance; - - static { - instance = new LogicalNodeDeserializer(); - } /** * Deserialize a list of nodes into a logical node tree. diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeSerializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeSerializer.java index 60171de085..607ed4fc16 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeSerializer.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeSerializer.java @@ -76,11 +76,17 @@ public static LogicalNodeTree serialize(LogicalNode node) { private static PlanProto.LogicalNode.Builder createNodeBuilder(SerializeContext context, LogicalNode node) { int selfId; - if (context.idMap.containsKey(node)) { - selfId = context.idMap.get(node); +// if (context.idMap.containsKey(node)) { +// selfId = context.idMap.get(node); +// } else { +// selfId = context.seqId++; +// context.idMap.put(node, selfId); +// } + if (context.idMap.containsKey(node.getPID())) { + selfId = context.idMap.get(node.getPID()); } else { selfId = context.seqId++; - context.idMap.put(node, selfId); + context.idMap.put(node.getPID(), selfId); } PlanProto.LogicalNode.Builder nodeBuilder = PlanProto.LogicalNode.newBuilder(); @@ -100,7 +106,7 @@ private static PlanProto.LogicalNode.Builder createNodeBuilder(SerializeContext public static class SerializeContext { private int seqId = 0; - private Map idMap = Maps.newHashMap(); + private Map idMap = Maps.newHashMap(); // map for PID and visit sequence private LogicalNodeTree.Builder treeBuilder = LogicalNodeTree.newBuilder(); } @@ -485,7 +491,6 @@ public LogicalNode visitTableSubQuery(SerializeContext context, LogicalPlan plan public LogicalNode visitCreateTable(SerializeContext context, LogicalPlan plan, LogicalPlan.QueryBlock block, CreateTableNode node, Stack stack) throws PlanningException { super.visitCreateTable(context, plan, block, node, stack); - int [] childIds = registerGetChildIds(context, node); PlanProto.PersistentStoreNode.Builder persistentStoreBuilder = buildPersistentStoreBuilder(node, childIds); @@ -629,7 +634,10 @@ public LogicalNode visitInsert(SerializeContext context, LogicalPlan plan, Logic private static PlanProto.PersistentStoreNode.Builder buildPersistentStoreBuilder(PersistentStoreNode node, int [] childIds) { PlanProto.PersistentStoreNode.Builder persistentStoreBuilder = PlanProto.PersistentStoreNode.newBuilder(); - persistentStoreBuilder.setChildSeq(childIds[0]); + if (childIds.length > 0) { + // Simple create table may not have any children. This should be improved at TAJO-1589. + persistentStoreBuilder.setChildSeq(childIds[0]); + } persistentStoreBuilder.setStorageType(node.getStorageType()); if (node.hasOptions()) { persistentStoreBuilder.setTableProperties(node.getOptions().getProto()); @@ -719,8 +727,10 @@ public static PlanProto.Target convertTarget(Target target) { private int [] registerGetChildIds(SerializeContext context, LogicalNode node) { int [] childIds = new int[node.childNum()]; for (int i = 0; i < node.childNum(); i++) { - if (context.idMap.containsKey(node.getChild(i))) { - childIds[i] = context.idMap.get(node.getChild(i)); +// if (context.idMap.containsKey(node.getChild(i))) { +// childIds[i] = context.idMap.get(node.getChild(i)); + if (context.idMap.containsKey(node.getChild(i).getPID())) { + childIds[i] = context.idMap.get(node.getChild(i).getPID()); } else { childIds[i] = context.seqId++; } diff --git a/tajo-plan/src/main/proto/Plan.proto b/tajo-plan/src/main/proto/Plan.proto index 7edbed2d38..6098261c0c 100644 --- a/tajo-plan/src/main/proto/Plan.proto +++ b/tajo-plan/src/main/proto/Plan.proto @@ -216,17 +216,17 @@ message PartitionTableScanSpec { } message PersistentStoreNode { - required int32 childSeq = 1; + optional int32 childSeq = 1; // CreateTableNode may not have any children. This should be improved at TAJO-1589. required StoreType storageType = 2; required KeyValueSetProto tableProperties = 3; } -message StoreTableNodeSpec { // required PersistentStoreSpec +message StoreTableNodeSpec { // required PersistentStoreNode optional string tableName = 1; // 'INSERT INTO LOCATION' does not require 'table name'. optional PartitionMethodProto partitionMethod = 2; } -message InsertNodeSpec { // required PersistentStoreSpec and StoreTableSpec +message InsertNodeSpec { // required PersistentStoreNode and StoreTableSpec required bool overwrite = 1; required SchemaProto tableSchema = 2; optional SchemaProto targetSchema = 4; @@ -234,7 +234,7 @@ message InsertNodeSpec { // required PersistentStoreSpec and StoreTableSpec optional string path = 5; } -message CreateTableNodeSpec { // required PersistentStoreSpec and StoreTableNodeSpec +message CreateTableNodeSpec { // required PersistentStoreNode and StoreTableNodeSpec required SchemaProto schema = 1; required bool external = 2; required bool ifNotExists = 3; From 1498534b48c88d8960583d9d185e76a631b882b4 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 6 May 2015 18:50:21 +0900 Subject: [PATCH 16/37] TAJO-1553 --- .../rewriter/rules/BroadcastJoinRule.java | 119 ++---------------- .../rewriter/rules/GlobalPlanRewriteUtil.java | 112 +++++++++++++++++ 2 files changed, 124 insertions(+), 107 deletions(-) create mode 100644 tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanRewriteUtil.java diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java index 43eef1545c..d200aba8c9 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java @@ -34,7 +34,7 @@ public class BroadcastJoinRule implements GlobalPlanRewriteRule { private long broadcastTableSizeThreshold; - private ParentFinder parentFinder; + private GlobalPlanRewriteUtil.ParentFinder parentFinder; @Override public String getName() { @@ -49,7 +49,7 @@ public boolean isEligible(OverridableConf queryContext, MasterPlan plan) { broadcastTableSizeThreshold = queryContext.getLong(SessionVars.BROADCAST_TABLE_SIZE_LIMIT); if (broadcastTableSizeThreshold > 0) { if (parentFinder == null) { - parentFinder = new ParentFinder(); + parentFinder = new GlobalPlanRewriteUtil.ParentFinder(); } return true; } @@ -69,7 +69,7 @@ private void rewrite(MasterPlan plan, ExecutionBlock current) throws PlanningExc if (plan.isLeaf(current)) { // in leaf execution blocks, find input tables which size is less than the predefined threshold. for (ScanNode scanNode : current.getScanNodes()) { - if (getTableVolume(scanNode) <= broadcastTableSizeThreshold) { + if (GlobalPlanRewriteUtil.getTableVolume(scanNode) <= broadcastTableSizeThreshold) { current.addBroadcastRelation(scanNode.getCanonicalName()); } } @@ -109,14 +109,14 @@ private ExecutionBlock merge(MasterPlan plan, ExecutionBlock child, ExecutionBlo private ExecutionBlock mergeTwoPhaseNonJoin(MasterPlan plan, ExecutionBlock child, ExecutionBlock parent) throws PlanningException { - ScanNode scanForChild = findScanForChildEb(child, parent); + ScanNode scanForChild = GlobalPlanRewriteUtil.findScanForChildEb(child, parent); if (scanForChild == null) { throw new PlanningException("Cannot find any scan nodes for " + child.getId() + " in " + parent.getId()); } parentFinder.set(scanForChild); parentFinder.find(parent.getPlan()); - LogicalNode parentOfScanForChild = parentFinder.found; + LogicalNode parentOfScanForChild = parentFinder.getFound(); if (parentOfScanForChild == null) { throw new PlanningException("Cannot find the parent of " + scanForChild.getCanonicalName()); } @@ -134,13 +134,13 @@ private ExecutionBlock mergeTwoPhaseNonJoin(MasterPlan plan, ExecutionBlock chil parentFinder.set(parentOfScanForChild); parentFinder.find(parent.getPlan()); - parentOfScanForChild = parentFinder.found; + parentOfScanForChild = parentFinder.getFound(); if (parentOfScanForChild == null) { // assume that the node which will be merged is the root node of the plan of the parent eb. mergedPlan = firstPhaseNode; } else { - replaceChild(firstPhaseNode, scanForChild, parentOfScanForChild); + GlobalPlanRewriteUtil.replaceChild(firstPhaseNode, scanForChild, parentOfScanForChild); mergedPlan = parent.getPlan(); } @@ -157,7 +157,7 @@ private ExecutionBlock mergeTwoPhaseNonJoin(MasterPlan plan, ExecutionBlock chil mergedPlan = parent.getPlan(); } - parent = mergeExecutionBlocks(plan, child, parent); + parent = GlobalPlanRewriteUtil.mergeExecutionBlocks(plan, child, parent); if (parent.getEnforcer().hasEnforceProperty(EnforceType.SORTED_INPUT)) { parent.getEnforcer().removeSortedInput(scanForChild.getTableName()); @@ -178,14 +178,14 @@ private ExecutionBlock mergeTwoPhaseNonJoin(MasterPlan plan, ExecutionBlock chil */ private ExecutionBlock mergeTwoPhaseJoin(MasterPlan plan, ExecutionBlock child, ExecutionBlock parent) throws PlanningException { - ScanNode scanForChild = findScanForChildEb(child, parent); + ScanNode scanForChild = GlobalPlanRewriteUtil.findScanForChildEb(child, parent); if (scanForChild == null) { throw new PlanningException("Cannot find any scan nodes for " + child.getId() + " in " + parent.getId()); } parentFinder.set(scanForChild); parentFinder.find(parent.getPlan()); - LogicalNode parentOfScanForChild = parentFinder.found; + LogicalNode parentOfScanForChild = parentFinder.getFound(); if (parentOfScanForChild == null) { throw new PlanningException("Cannot find the parent of " + scanForChild.getCanonicalName()); } @@ -209,7 +209,7 @@ private ExecutionBlock mergeTwoPhaseJoin(MasterPlan plan, ExecutionBlock child, // } else { // throw new PlanningException(parentOfScanForChild + " seems to not have any children"); // } - replaceChild(rootOfChild, scanForChild, parentOfScanForChild); + GlobalPlanRewriteUtil.replaceChild(rootOfChild, scanForChild, parentOfScanForChild); // for (String broadcastable : child.getBroadcastTables()) { // parent.addBroadcastRelation(broadcastable); @@ -230,107 +230,12 @@ private ExecutionBlock mergeTwoPhaseJoin(MasterPlan plan, ExecutionBlock child, // plan.removeExecBlock(child.getId()); // } // } - parent = mergeExecutionBlocks(plan, child, parent); + parent = GlobalPlanRewriteUtil.mergeExecutionBlocks(plan, child, parent); parent.setPlan(parent.getPlan()); return parent; } - private static ExecutionBlock mergeExecutionBlocks(MasterPlan plan, ExecutionBlock child, ExecutionBlock parent) { - for (String broadcastable : child.getBroadcastTables()) { - parent.addBroadcastRelation(broadcastable); - } - - // connect parent and grand children - List grandChilds = plan.getChilds(child); - for (ExecutionBlock eachGrandChild : grandChilds) { - plan.addConnect(eachGrandChild, parent, plan.getChannel(eachGrandChild, child).getShuffleType()); - plan.disconnect(eachGrandChild, child); - } - - plan.disconnect(child, parent); - List channels = plan.getIncomingChannels(child.getId()); - if (channels == null || channels.size() == 0) { - channels = plan.getOutgoingChannels(child.getId()); - if (channels == null || channels.size() == 0) { - plan.removeExecBlock(child.getId()); - } - } - return parent; - } - - private static void replaceChild(LogicalNode newChild, ScanNode originalChild, LogicalNode parent) - throws PlanningException { - if (parent instanceof UnaryNode) { - ((UnaryNode) parent).setChild(newChild); - } else if (parent instanceof BinaryNode) { - BinaryNode binary = (BinaryNode) parent; - if (binary.getLeftChild().equals(originalChild)) { - binary.setLeftChild(newChild); - } else if (binary.getRightChild().equals(originalChild)) { - binary.setRightChild(newChild); - } else { - throw new PlanningException(originalChild.getPID() + " is not a child of " + parent.getPID()); - } - } else { - throw new PlanningException(parent.getPID() + " seems to not have any children"); - } - } - - private static ScanNode findScanForChildEb(ExecutionBlock child, ExecutionBlock parent) { - ScanNode scanForChild = null; - for (ScanNode scanNode : parent.getScanNodes()) { - if (scanNode.getTableName().equals(child.getId().toString())) { - scanForChild = scanNode; - break; - } - } - return scanForChild; - } - - /** - * Get a volume of a table of a partitioned table - * @param scanNode ScanNode corresponding to a table - * @return table volume (bytes) - */ - private static long getTableVolume(ScanNode scanNode) { - long scanBytes = scanNode.getTableDesc().getStats().getNumBytes(); - if (scanNode.getType() == NodeType.PARTITIONS_SCAN) { - PartitionedTableScanNode pScanNode = (PartitionedTableScanNode)scanNode; - if (pScanNode.getInputPaths() == null || pScanNode.getInputPaths().length == 0) { - scanBytes = 0L; - } - } - - return scanBytes; - } - - private static class ParentFinder implements LogicalNodeVisitor { - private LogicalNode target; - private LogicalNode found; - public void set(LogicalNode child) { - this.target = child; - this.found = null; - } - - public void find(LogicalNode root) { - this.visit(root); - } - - @Override - public void visit(LogicalNode node) { - for (int i = 0; i < node.childNum(); i++) { - if (node.getChild(i).equals(target)) { - found = node; - break; - } else { - if (found == null) { - visit(node.getChild(i)); - } - } - } - } - } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanRewriteUtil.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanRewriteUtil.java new file mode 100644 index 0000000000..e127f33e46 --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanRewriteUtil.java @@ -0,0 +1,112 @@ +package org.apache.tajo.engine.planner.global.rewriter.rules; + +import org.apache.tajo.engine.planner.global.DataChannel; +import org.apache.tajo.engine.planner.global.ExecutionBlock; +import org.apache.tajo.engine.planner.global.MasterPlan; +import org.apache.tajo.plan.PlanningException; +import org.apache.tajo.plan.logical.*; + +import java.util.List; + +public class GlobalPlanRewriteUtil { + public static ExecutionBlock mergeExecutionBlocks(MasterPlan plan, ExecutionBlock child, ExecutionBlock parent) { + for (String broadcastable : child.getBroadcastTables()) { + parent.addBroadcastRelation(broadcastable); + } + + // connect parent and grand children + List grandChilds = plan.getChilds(child); + for (ExecutionBlock eachGrandChild : grandChilds) { + plan.addConnect(eachGrandChild, parent, plan.getChannel(eachGrandChild, child).getShuffleType()); + plan.disconnect(eachGrandChild, child); + } + + plan.disconnect(child, parent); + List channels = plan.getIncomingChannels(child.getId()); + if (channels == null || channels.size() == 0) { + channels = plan.getOutgoingChannels(child.getId()); + if (channels == null || channels.size() == 0) { + plan.removeExecBlock(child.getId()); + } + } + return parent; + } + + public static void replaceChild(LogicalNode newChild, ScanNode originalChild, LogicalNode parent) + throws PlanningException { + if (parent instanceof UnaryNode) { + ((UnaryNode) parent).setChild(newChild); + } else if (parent instanceof BinaryNode) { + BinaryNode binary = (BinaryNode) parent; + if (binary.getLeftChild().equals(originalChild)) { + binary.setLeftChild(newChild); + } else if (binary.getRightChild().equals(originalChild)) { + binary.setRightChild(newChild); + } else { + throw new PlanningException(originalChild.getPID() + " is not a child of " + parent.getPID()); + } + } else { + throw new PlanningException(parent.getPID() + " seems to not have any children"); + } + } + + public static ScanNode findScanForChildEb(ExecutionBlock child, ExecutionBlock parent) { + ScanNode scanForChild = null; + for (ScanNode scanNode : parent.getScanNodes()) { + if (scanNode.getTableName().equals(child.getId().toString())) { + scanForChild = scanNode; + break; + } + } + return scanForChild; + } + + /** + * Get a volume of a table of a partitioned table + * @param scanNode ScanNode corresponding to a table + * @return table volume (bytes) + */ + public static long getTableVolume(ScanNode scanNode) { + long scanBytes = scanNode.getTableDesc().getStats().getNumBytes(); + if (scanNode.getType() == NodeType.PARTITIONS_SCAN) { + PartitionedTableScanNode pScanNode = (PartitionedTableScanNode)scanNode; + if (pScanNode.getInputPaths() == null || pScanNode.getInputPaths().length == 0) { + scanBytes = 0L; + } + } + + return scanBytes; + } + + public static class ParentFinder implements LogicalNodeVisitor { + private LogicalNode target; + private LogicalNode found; + + public void set(LogicalNode child) { + this.target = child; + this.found = null; + } + + public void find(LogicalNode root) { + this.visit(root); + } + + public LogicalNode getFound() { + return this.found; + } + + @Override + public void visit(LogicalNode node) { + for (int i = 0; i < node.childNum(); i++) { + if (node.getChild(i).equals(target)) { + found = node; + break; + } else { + if (found == null) { + visit(node.getChild(i)); + } + } + } + } + } +} From 8879f8c20ea9b8cfe1b0b1b187632740a7de4285 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 7 May 2015 23:04:12 +0900 Subject: [PATCH 17/37] TAJO-1553 --- .../rewriter/rules/BroadcastJoinRule.java | 100 ++++-------------- .../global/rewriter/rules/EbMergeRule.java | 92 ++++++++++++++++ .../rewriter/rules/GlobalPlanRewriteUtil.java | 8 ++ .../tajo/querymaster/Repartitioner.java | 26 +++-- 4 files changed, 137 insertions(+), 89 deletions(-) create mode 100644 tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/EbMergeRule.java diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java index d200aba8c9..60676bfafe 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java @@ -29,7 +29,9 @@ import org.apache.tajo.plan.PlanningException; import org.apache.tajo.plan.expr.AggregationFunctionCallEval; import org.apache.tajo.plan.logical.*; +import org.apache.tajo.util.TUtil; +import java.util.Collection; import java.util.List; public class BroadcastJoinRule implements GlobalPlanRewriteRule { @@ -78,96 +80,38 @@ private void rewrite(MasterPlan plan, ExecutionBlock current) throws PlanningExc for (ExecutionBlock child : plan.getChilds(current)) { rewrite(plan, child); } -// if (current.hasJoin()) { - if (!plan.isTerminal(current)) { - boolean needMerge = false; + if (!plan.isTerminal(current) && current.hasJoin()) { + ExecutionBlock enforceNonBroadcast = null; + ExecutionBlock broadcastCandidate = null; + long smallestChildVolume = Long.MAX_VALUE; List childs = plan.getChilds(current); for (ExecutionBlock child : childs) { if (child.isBroadcastable(broadcastTableSizeThreshold)) { - needMerge = true; - break; + long inputVolume = GlobalPlanRewriteUtil.getInputVolume(child); + if (smallestChildVolume > inputVolume) { + smallestChildVolume = inputVolume; + if (broadcastCandidate != null) { + enforceNonBroadcast = broadcastCandidate; + } + broadcastCandidate = child; + } } } - if (needMerge) { + if (broadcastCandidate != null) { + if (enforceNonBroadcast != null) { + List tables = TUtil.newList(enforceNonBroadcast.getBroadcastTables()); + for (String broadcastTable : tables) { +// enforceNonBroadcast.removeBroadcastRelation(broadcastTable); + } + } for (ExecutionBlock child : childs) { - merge(plan, child, current); + mergeTwoPhaseJoin(plan, child, current); } } -// } } } } - private ExecutionBlock merge(MasterPlan plan, ExecutionBlock child, ExecutionBlock parent) throws PlanningException { - if (parent.hasJoin()) { - return mergeTwoPhaseJoin(plan, child, parent); - } else { - return mergeTwoPhaseNonJoin(plan, child, parent); - } - } - - private ExecutionBlock mergeTwoPhaseNonJoin(MasterPlan plan, ExecutionBlock child, ExecutionBlock parent) - throws PlanningException { - - ScanNode scanForChild = GlobalPlanRewriteUtil.findScanForChildEb(child, parent); - if (scanForChild == null) { - throw new PlanningException("Cannot find any scan nodes for " + child.getId() + " in " + parent.getId()); - } - - parentFinder.set(scanForChild); - parentFinder.find(parent.getPlan()); - LogicalNode parentOfScanForChild = parentFinder.getFound(); - if (parentOfScanForChild == null) { - throw new PlanningException("Cannot find the parent of " + scanForChild.getCanonicalName()); - } - - LogicalNode rootOfChild = child.getPlan(); - if (rootOfChild.getType() == NodeType.STORE) { - rootOfChild = ((StoreTableNode)rootOfChild).getChild(); - } - LogicalNode mergedPlan; - if (rootOfChild.getType() == parentOfScanForChild.getType()) { - // merge two-phase plan into one-phase plan. - // remove the second-phase plan. - LogicalNode firstPhaseNode = rootOfChild; - LogicalNode secondPhaseNode = parentOfScanForChild; - - parentFinder.set(parentOfScanForChild); - parentFinder.find(parent.getPlan()); - parentOfScanForChild = parentFinder.getFound(); - - if (parentOfScanForChild == null) { - // assume that the node which will be merged is the root node of the plan of the parent eb. - mergedPlan = firstPhaseNode; - } else { - GlobalPlanRewriteUtil.replaceChild(firstPhaseNode, scanForChild, parentOfScanForChild); - mergedPlan = parent.getPlan(); - } - - if (firstPhaseNode.getType() == NodeType.GROUP_BY) { - GroupbyNode firstPhaseGroupby = (GroupbyNode) firstPhaseNode; - GroupbyNode secondPhaseGroupby = (GroupbyNode) secondPhaseNode; - for (AggregationFunctionCallEval aggFunc : firstPhaseGroupby.getAggFunctions()) { - aggFunc.setFirstAndLastPhase(); - } - firstPhaseGroupby.setTargets(secondPhaseGroupby.getTargets()); - firstPhaseGroupby.setOutSchema(secondPhaseGroupby.getOutSchema()); - } - } else { - mergedPlan = parent.getPlan(); - } - - parent = GlobalPlanRewriteUtil.mergeExecutionBlocks(plan, child, parent); - - if (parent.getEnforcer().hasEnforceProperty(EnforceType.SORTED_INPUT)) { - parent.getEnforcer().removeSortedInput(scanForChild.getTableName()); - } - - parent.setPlan(mergedPlan); - - return parent; - } - /** * Merge child execution blocks. * diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/EbMergeRule.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/EbMergeRule.java new file mode 100644 index 0000000000..4a66a00bdc --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/EbMergeRule.java @@ -0,0 +1,92 @@ +package org.apache.tajo.engine.planner.global.rewriter.rules; + +import org.apache.tajo.OverridableConf; +import org.apache.tajo.engine.planner.global.ExecutionBlock; +import org.apache.tajo.engine.planner.global.MasterPlan; +import org.apache.tajo.engine.planner.global.rewriter.GlobalPlanRewriteRule; +import org.apache.tajo.ipc.TajoWorkerProtocol; +import org.apache.tajo.plan.PlanningException; +import org.apache.tajo.plan.expr.AggregationFunctionCallEval; +import org.apache.tajo.plan.logical.*; + +public class EbMergeRule implements GlobalPlanRewriteRule { + private GlobalPlanRewriteUtil.ParentFinder parentFinder = new GlobalPlanRewriteUtil.ParentFinder(); + + @Override + public String getName() { + return "EbMergeRule"; + } + + @Override + public boolean isEligible(OverridableConf queryContext, MasterPlan plan) { + return true; + } + + @Override + public MasterPlan rewrite(MasterPlan plan) throws PlanningException { + return null; + } + + private ExecutionBlock mergeTwoPhaseNonJoin(MasterPlan plan, ExecutionBlock child, ExecutionBlock parent) + throws PlanningException { + + ScanNode scanForChild = GlobalPlanRewriteUtil.findScanForChildEb(child, parent); + if (scanForChild == null) { + throw new PlanningException("Cannot find any scan nodes for " + child.getId() + " in " + parent.getId()); + } + + parentFinder.set(scanForChild); + parentFinder.find(parent.getPlan()); + LogicalNode parentOfScanForChild = parentFinder.getFound(); + if (parentOfScanForChild == null) { + throw new PlanningException("Cannot find the parent of " + scanForChild.getCanonicalName()); + } + + LogicalNode rootOfChild = child.getPlan(); + if (rootOfChild.getType() == NodeType.STORE) { + rootOfChild = ((StoreTableNode)rootOfChild).getChild(); + } + LogicalNode mergedPlan; + if (rootOfChild.getType() == parentOfScanForChild.getType()) { + // merge two-phase plan into one-phase plan. + // remove the second-phase plan. + LogicalNode firstPhaseNode = rootOfChild; + LogicalNode secondPhaseNode = parentOfScanForChild; + + parentFinder.set(parentOfScanForChild); + parentFinder.find(parent.getPlan()); + parentOfScanForChild = parentFinder.getFound(); + + if (parentOfScanForChild == null) { + // assume that the node which will be merged is the root node of the plan of the parent eb. + mergedPlan = firstPhaseNode; + } else { + GlobalPlanRewriteUtil.replaceChild(firstPhaseNode, scanForChild, parentOfScanForChild); + mergedPlan = parent.getPlan(); + } + + if (firstPhaseNode.getType() == NodeType.GROUP_BY) { + GroupbyNode firstPhaseGroupby = (GroupbyNode) firstPhaseNode; + GroupbyNode secondPhaseGroupby = (GroupbyNode) secondPhaseNode; + for (AggregationFunctionCallEval aggFunc : firstPhaseGroupby.getAggFunctions()) { + aggFunc.setFirstAndLastPhase(); + } + firstPhaseGroupby.setTargets(secondPhaseGroupby.getTargets()); + firstPhaseGroupby.setOutSchema(secondPhaseGroupby.getOutSchema()); + } + } else { + mergedPlan = parent.getPlan(); + } + + parent = GlobalPlanRewriteUtil.mergeExecutionBlocks(plan, child, parent); + + if (parent.getEnforcer().hasEnforceProperty(TajoWorkerProtocol.EnforceProperty.EnforceType.SORTED_INPUT)) { + parent.getEnforcer().removeSortedInput(scanForChild.getTableName()); + } + + parent.setPlan(mergedPlan); + + return parent; + } + +} diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanRewriteUtil.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanRewriteUtil.java index e127f33e46..3bfcd338a4 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanRewriteUtil.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanRewriteUtil.java @@ -78,6 +78,14 @@ public static long getTableVolume(ScanNode scanNode) { return scanBytes; } + public static long getInputVolume(ExecutionBlock block) { + long volume = 0; + for (ScanNode scanNode : block.getScanNodes()) { + volume += getTableVolume(scanNode); + } + return volume; + } + public static class ParentFinder implements LogicalNodeVisitor { private LogicalNode target; private LogicalNode found; diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java index eec7441afa..3294443d9a 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java @@ -273,31 +273,31 @@ public static void scheduleFragmentsForJoinQuery(TaskSchedulerContext schedulerC } //select intermediate scan and stats - ScanNode[] intermediateScans = new ScanNode[largeScanIndexList.size()]; long[] intermediateScanStats = new long[largeScanIndexList.size()]; Fragment[] intermediateFragments = new Fragment[largeScanIndexList.size()]; int index = 0; for (Integer eachIdx : largeScanIndexList) { - intermediateScans[index] = scans[eachIdx]; intermediateScanStats[index] = stats[eachIdx]; intermediateFragments[index++] = fragments[eachIdx]; } Fragment[] broadcastFragments = new Fragment[broadcastIndexList.size()]; ScanNode[] broadcastScans = new ScanNode[broadcastIndexList.size()]; + long[] broadcastStats = new long[broadcastIndexList.size()]; index = 0; for (Integer eachIdx : broadcastIndexList) { scans[eachIdx].setBroadcastTable(true); broadcastScans[index] = scans[eachIdx]; + broadcastStats[index] = stats[eachIdx]; broadcastFragments[index] = fragments[eachIdx]; index++; } LOG.info(String.format("[Distributed Join Strategy] : Broadcast Join, join_node=%s", nonLeafScanNames)); scheduleSymmetricRepartitionJoin(masterContext, schedulerContext, stage, - intermediateScans, intermediateScanStats, intermediateFragments, broadcastScans, broadcastFragments); + intermediateScanStats, intermediateFragments, broadcastScans, broadcastStats, broadcastFragments); } } else { LOG.info("[Distributed Join Strategy] : Symmetric Repartition Join"); - scheduleSymmetricRepartitionJoin(masterContext, schedulerContext, stage, scans, stats, fragments, null, null); + scheduleSymmetricRepartitionJoin(masterContext, schedulerContext, stage, stats, fragments, null, null, null); } } @@ -306,7 +306,6 @@ public static void scheduleFragmentsForJoinQuery(TaskSchedulerContext schedulerC * @param masterContext * @param schedulerContext * @param stage - * @param scans * @param stats * @param fragments * @throws IOException @@ -314,10 +313,10 @@ public static void scheduleFragmentsForJoinQuery(TaskSchedulerContext schedulerC private static void scheduleSymmetricRepartitionJoin(QueryMasterTask.QueryMasterTaskContext masterContext, TaskSchedulerContext schedulerContext, Stage stage, - ScanNode[] scans, long[] stats, Fragment[] fragments, ScanNode[] broadcastScans, + long[] broadcastStats, Fragment[] broadcastFragments) throws IOException { MasterPlan masterPlan = stage.getMasterPlan(); ExecutionBlock execBlock = stage.getBlock(); @@ -379,18 +378,21 @@ private static void scheduleSymmetricRepartitionJoin(QueryMasterTask.QueryMaster // hashEntries can be zero if there are no input data. // In the case, it will cause the zero divided exception. // it avoids this problem. + long leftStats = stats[0]; + long rightStats = stats.length == 2 ? stats[1] : broadcastStats[0]; int[] avgSize = new int[2]; - avgSize[0] = hashEntries.size() == 0 ? 0 : (int) (stats[0] / hashEntries.size()); - avgSize[1] = hashEntries.size() == 0 ? 0 : (int) (stats[1] / hashEntries.size()); + avgSize[0] = hashEntries.size() == 0 ? 0 : (int) (leftStats / hashEntries.size()); + avgSize[1] = hashEntries.size() == 0 ? 0 : (int) (stats.length == 2 ? (rightStats / hashEntries.size()) : rightStats); int bothFetchSize = avgSize[0] + avgSize[1]; // Getting the desire number of join tasks according to the volumn // of a larger table - int largerIdx = stats[0] >= stats[1] ? 0 : 1; +// int largerIdx = leftStats >= rightStats ? 0 : 1; + long largerStat = leftStats >= rightStats ? leftStats : rightStats; int desireJoinTaskVolumn = stage.getMasterPlan().getContext().getInt(SessionVars.JOIN_TASK_INPUT_SIZE); // calculate the number of tasks according to the data size - int mb = (int) Math.ceil((double) stats[largerIdx] / 1048576); + int mb = (int) Math.ceil((double) largerStat / 1048576); LOG.info("Larger intermediate data is approximately " + mb + " MB"); // determine the number of task per 64MB int maxTaskNum = (int) Math.ceil((double) mb / desireJoinTaskVolumn); @@ -402,7 +404,9 @@ private static void scheduleSymmetricRepartitionJoin(QueryMasterTask.QueryMaster LOG.info("The determined number of join tasks is " + joinTaskNum); List rightFragments = new ArrayList(); - rightFragments.add(fragments[1]); + if (fragments.length == 2) { + rightFragments.add(fragments[1]); + } if (broadcastFragments != null) { //In this phase a ScanNode has a single fragment. From a6d322ca1307e83e33742fb8600b70b952f50cf7 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 8 May 2015 15:03:13 +0900 Subject: [PATCH 18/37] TAJO-1553 --- .../rewriter/rules/BroadcastJoinRule.java | 1 + .../global/rewriter/rules/EbMergeRule.java | 18 ++++++++++++++++++ .../rewriter/rules/GlobalPlanRewriteUtil.java | 18 ++++++++++++++++++ 3 files changed, 37 insertions(+) diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java index 60676bfafe..d835dc115b 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java @@ -102,6 +102,7 @@ private void rewrite(MasterPlan plan, ExecutionBlock current) throws PlanningExc List tables = TUtil.newList(enforceNonBroadcast.getBroadcastTables()); for (String broadcastTable : tables) { // enforceNonBroadcast.removeBroadcastRelation(broadcastTable); + // TODO: remove the largest rel from broadcast when all inputs are broadcast } } for (ExecutionBlock child : childs) { diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/EbMergeRule.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/EbMergeRule.java index 4a66a00bdc..82bcd79afb 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/EbMergeRule.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/EbMergeRule.java @@ -1,3 +1,21 @@ +/** + * 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.global.rewriter.rules; import org.apache.tajo.OverridableConf; diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanRewriteUtil.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanRewriteUtil.java index 3bfcd338a4..79794409b5 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanRewriteUtil.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanRewriteUtil.java @@ -1,3 +1,21 @@ +/** + * 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.global.rewriter.rules; import org.apache.tajo.engine.planner.global.DataChannel; From 1256493b0b98ec9d00cf7ce2f81f43d93cb894fa Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 8 May 2015 17:53:41 +0900 Subject: [PATCH 19/37] TAJO-1553 --- .../rewriter/rules/BroadcastJoinRule.java | 24 +++++++++++++++++++ .../tajo/engine/query/TestJoinBroadcast.java | 2 -- 2 files changed, 24 insertions(+), 2 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java index d835dc115b..81942a7505 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java @@ -29,6 +29,7 @@ import org.apache.tajo.plan.PlanningException; import org.apache.tajo.plan.expr.AggregationFunctionCallEval; import org.apache.tajo.plan.logical.*; +import org.apache.tajo.plan.util.PlannerUtil; import org.apache.tajo.util.TUtil; import java.util.Collection; @@ -81,6 +82,20 @@ private void rewrite(MasterPlan plan, ExecutionBlock current) throws PlanningExc rewrite(plan, child); } if (!plan.isTerminal(current) && current.hasJoin()) { + // unioned scans should be handled as a single relation scan + + // check outer join + if (hasOuterJoin(current)) { + // find and enforce shuffle for row-preserved tables + + } + + // check the total input size + + // check all inputs are marked as broadcast + + + ExecutionBlock enforceNonBroadcast = null; ExecutionBlock broadcastCandidate = null; long smallestChildVolume = Long.MAX_VALUE; @@ -113,6 +128,15 @@ private void rewrite(MasterPlan plan, ExecutionBlock current) throws PlanningExc } } + private static boolean hasOuterJoin(ExecutionBlock block) { + LogicalNode found = PlannerUtil.findMostBottomNode(block.getPlan(), NodeType.JOIN); + if (found != null) { + JoinNode joinNode = (JoinNode) found; + return PlannerUtil.isOuterJoin(joinNode.getJoinType()); + } + return false; + } + /** * Merge child execution blocks. * diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java index 1decde4e1c..a11c7c7ab7 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java @@ -28,10 +28,8 @@ import org.apache.tajo.datum.Datum; import org.apache.tajo.datum.Int4Datum; import org.apache.tajo.datum.TextDatum; -import org.apache.tajo.engine.planner.global.ExecutionBlock; import org.apache.tajo.engine.planner.global.MasterPlan; import org.apache.tajo.jdbc.FetchResultSet; -import org.apache.tajo.plan.logical.NodeType; import org.apache.tajo.querymaster.QueryMasterTask; import org.apache.tajo.storage.*; import org.apache.tajo.util.FileUtil; From 38665fd5af1c5c04fe6b35a3c9c4488aaf29cae7 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 8 May 2015 19:56:53 +0900 Subject: [PATCH 20/37] TAJo-1553 --- .../engine/planner/global/GlobalPlanner.java | 132 +++++++++--------- .../rewriter/rules/BroadcastJoinRule.java | 1 + .../rewriter/rules/UnionReduceRule.java | 63 +++++++++ 3 files changed, 130 insertions(+), 66 deletions(-) create mode 100644 tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/UnionReduceRule.java diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java index 4615e33f71..5d49f77300 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java @@ -389,73 +389,73 @@ private ExecutionBlock buildJoinPlan(GlobalPlanContext context, JoinNode joinNod // } // } - LogicalNode leftNode = joinNode.getLeftChild(); - LogicalNode rightNode = joinNode.getRightChild(); +// LogicalNode leftNode = joinNode.getLeftChild(); +// LogicalNode rightNode = joinNode.getRightChild(); // symmetric repartition join - boolean leftUnion = leftNode.getType() == NodeType.TABLE_SUBQUERY && - ((TableSubQueryNode)leftNode).getSubQuery().getType() == NodeType.UNION; - boolean rightUnion = rightNode.getType() == NodeType.TABLE_SUBQUERY && - ((TableSubQueryNode)rightNode).getSubQuery().getType() == NodeType.UNION; - - if (leftUnion || rightUnion) { // if one of child execution block is union - /* - Join with tableC and result of union tableA, tableB is expected the following physical plan. - But Union execution block is not necessary. - |-eb_0001_000006 (Terminal) - |-eb_0001_000005 (Join eb_0001_000003, eb_0001_000004) - |-eb_0001_000004 (Scan TableC) - |-eb_0001_000003 (Union TableA, TableB) - |-eb_0001_000002 (Scan TableB) - |-eb_0001_000001 (Scan TableA) - - The above plan can be changed to the following plan. - |-eb_0001_000005 (Terminal) - |-eb_0001_000003 (Join [eb_0001_000001, eb_0001_000002], eb_0001_000004) - |-eb_0001_000004 (Scan TableC) - |-eb_0001_000002 (Scan TableB) - |-eb_0001_000001 (Scan TableA) - - eb_0001_000003's left child should be eb_0001_000001 + eb_0001_000001 and right child should be eb_0001_000004. - For this eb_0001_000001 is representative of eb_0001_000001, eb_0001_000002. - So eb_0001_000003's left child is eb_0001_000001 - */ - Column[][] joinColumns = null; - if (joinNode.getJoinType() != JoinType.CROSS) { - // ShuffleKeys need to not have thea-join condition because Tajo supports only equi-join. - joinColumns = PlannerUtil.joinJoinKeyForEachTable(joinNode.getJoinQual(), - leftNode.getOutSchema(), rightNode.getOutSchema(), false); - } - - if (leftUnion && !rightUnion) { // if only left is union - currentBlock = leftBlock; - context.execBlockMap.remove(leftNode.getPID()); - Column[] shuffleKeys = (joinColumns != null) ? joinColumns[0] : null; - Column[] otherSideShuffleKeys = (joinColumns != null) ? joinColumns[1] : null; - buildJoinPlanWithUnionChannel(context, joinNode, currentBlock, leftBlock, rightBlock, leftNode, - shuffleKeys, otherSideShuffleKeys, true); - currentBlock.setPlan(joinNode); - } else if (!leftUnion && rightUnion) { // if only right is union - currentBlock = rightBlock; - context.execBlockMap.remove(rightNode.getPID()); - Column[] shuffleKeys = (joinColumns != null) ? joinColumns[1] : null; - Column[] otherSideShuffleKeys = (joinColumns != null) ? joinColumns[0] : null; - buildJoinPlanWithUnionChannel(context, joinNode, currentBlock, rightBlock, leftBlock, rightNode, - shuffleKeys, otherSideShuffleKeys, false); - currentBlock.setPlan(joinNode); - } else { // if both are unions - currentBlock = leftBlock; - context.execBlockMap.remove(leftNode.getPID()); - context.execBlockMap.remove(rightNode.getPID()); - buildJoinPlanWithUnionChannel(context, joinNode, currentBlock, leftBlock, null, leftNode, - (joinColumns != null ? joinColumns[0] : null), null, true); - buildJoinPlanWithUnionChannel(context, joinNode, currentBlock, rightBlock, null, rightNode, - (joinColumns != null ? joinColumns[1] : null), null, false); - currentBlock.setPlan(joinNode); - } - - return currentBlock; - } else { +// boolean leftUnion = leftNode.getType() == NodeType.TABLE_SUBQUERY && +// ((TableSubQueryNode)leftNode).getSubQuery().getType() == NodeType.UNION; +// boolean rightUnion = rightNode.getType() == NodeType.TABLE_SUBQUERY && +// ((TableSubQueryNode)rightNode).getSubQuery().getType() == NodeType.UNION; +// +// if (leftUnion || rightUnion) { // if one of child execution block is union +// /* +// Join with tableC and result of union tableA, tableB is expected the following physical plan. +// But Union execution block is not necessary. +// |-eb_0001_000006 (Terminal) +// |-eb_0001_000005 (Join eb_0001_000003, eb_0001_000004) +// |-eb_0001_000004 (Scan TableC) +// |-eb_0001_000003 (Union TableA, TableB) +// |-eb_0001_000002 (Scan TableB) +// |-eb_0001_000001 (Scan TableA) +// +// The above plan can be changed to the following plan. +// |-eb_0001_000005 (Terminal) +// |-eb_0001_000003 (Join [eb_0001_000001, eb_0001_000002], eb_0001_000004) +// |-eb_0001_000004 (Scan TableC) +// |-eb_0001_000002 (Scan TableB) +// |-eb_0001_000001 (Scan TableA) +// +// eb_0001_000003's left child should be eb_0001_000001 + eb_0001_000001 and right child should be eb_0001_000004. +// For this eb_0001_000001 is representative of eb_0001_000001, eb_0001_000002. +// So eb_0001_000003's left child is eb_0001_000001 +// */ +// Column[][] joinColumns = null; +// if (joinNode.getJoinType() != JoinType.CROSS) { +// // ShuffleKeys need to not have thea-join condition because Tajo supports only equi-join. +// joinColumns = PlannerUtil.joinJoinKeyForEachTable(joinNode.getJoinQual(), +// leftNode.getOutSchema(), rightNode.getOutSchema(), false); +// } +// +// if (leftUnion && !rightUnion) { // if only left is union +// currentBlock = leftBlock; +// context.execBlockMap.remove(leftNode.getPID()); +// Column[] shuffleKeys = (joinColumns != null) ? joinColumns[0] : null; +// Column[] otherSideShuffleKeys = (joinColumns != null) ? joinColumns[1] : null; +// buildJoinPlanWithUnionChannel(context, joinNode, currentBlock, leftBlock, rightBlock, leftNode, +// shuffleKeys, otherSideShuffleKeys, true); +// currentBlock.setPlan(joinNode); +// } else if (!leftUnion && rightUnion) { // if only right is union +// currentBlock = rightBlock; +// context.execBlockMap.remove(rightNode.getPID()); +// Column[] shuffleKeys = (joinColumns != null) ? joinColumns[1] : null; +// Column[] otherSideShuffleKeys = (joinColumns != null) ? joinColumns[0] : null; +// buildJoinPlanWithUnionChannel(context, joinNode, currentBlock, rightBlock, leftBlock, rightNode, +// shuffleKeys, otherSideShuffleKeys, false); +// currentBlock.setPlan(joinNode); +// } else { // if both are unions +// currentBlock = leftBlock; +// context.execBlockMap.remove(leftNode.getPID()); +// context.execBlockMap.remove(rightNode.getPID()); +// buildJoinPlanWithUnionChannel(context, joinNode, currentBlock, leftBlock, null, leftNode, +// (joinColumns != null ? joinColumns[0] : null), null, true); +// buildJoinPlanWithUnionChannel(context, joinNode, currentBlock, rightBlock, null, rightNode, +// (joinColumns != null ? joinColumns[1] : null), null, false); +// currentBlock.setPlan(joinNode); +// } +// +// return currentBlock; +// } else { // !leftUnion && !rightUnion currentBlock = masterPlan.newExecutionBlock(); DataChannel leftChannel = createDataChannelFromJoin(leftBlock, rightBlock, currentBlock, joinNode, true); @@ -472,7 +472,7 @@ private ExecutionBlock buildJoinPlan(GlobalPlanContext context, JoinNode joinNod masterPlan.addConnect(rightChannel); return currentBlock; - } +// } } private void buildJoinPlanWithUnionChannel(GlobalPlanContext context, JoinNode joinNode, diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java index 81942a7505..a581e83fce 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java @@ -62,6 +62,7 @@ public boolean isEligible(OverridableConf queryContext, MasterPlan plan) { return false; } + // TODO: use MasterPlan.execBlockGraph.accept() @Override public MasterPlan rewrite(MasterPlan plan) throws PlanningException{ rewrite(plan, plan.getTerminalBlock()); diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/UnionReduceRule.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/UnionReduceRule.java new file mode 100644 index 0000000000..c8214f2520 --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/UnionReduceRule.java @@ -0,0 +1,63 @@ +package org.apache.tajo.engine.planner.global.rewriter.rules; + +import org.apache.tajo.ExecutionBlockId; +import org.apache.tajo.OverridableConf; +import org.apache.tajo.engine.planner.global.MasterPlan; +import org.apache.tajo.engine.planner.global.rewriter.GlobalPlanRewriteRule; +import org.apache.tajo.plan.LogicalPlan; +import org.apache.tajo.plan.PlanningException; +import org.apache.tajo.plan.logical.NodeType; +import org.apache.tajo.util.graph.DirectedGraphVisitor; + +import java.util.Stack; + +public class UnionReduceRule implements GlobalPlanRewriteRule { + + @Override + public String getName() { + return "UnionReduceRule"; + } + + @Override + public boolean isEligible(OverridableConf queryContext, MasterPlan plan) { + for (LogicalPlan.QueryBlock block : plan.getLogicalPlan().getQueryBlocks()) { + if (block.hasNode(NodeType.UNION)) { + return true; + } + } + return false; + } + + @Override + public MasterPlan rewrite(MasterPlan plan) throws PlanningException { + Rewriter.rewrite(plan); + return plan; + } + + static class Rewriter implements DirectedGraphVisitor { + private static Rewriter instance; + private static MasterPlan plan; + + private Rewriter() {} + + public static void rewrite(MasterPlan plan) { + if (instance == null) { + instance = new Rewriter(); + } + instance.plan = plan; + instance.visit(new Stack(), plan.getTerminalBlock().getId()); + } + + @Override + public void visit(Stack stack, ExecutionBlockId executionBlockId) { + // must have the form of + /* + operator + | + union + / \ + op op + */ + } + } +} From 85786a5222968295923c36e10f97e63b59da8c37 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 9 May 2015 18:32:28 +0900 Subject: [PATCH 21/37] TAJO-1553 --- .../engine/planner/global/GlobalPlanner.java | 395 +++++++++--------- .../BaseGlobalPlanRewriteRuleProvider.java | 8 +- .../rewriter/GlobalPlanTestRuleProvider.java | 2 - .../rewriter/rules/UnionReduceRule.java | 46 +- .../tajo/engine/query/TestGroupByQuery.java | 5 + 5 files changed, 254 insertions(+), 202 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java index 5d49f77300..380249e0bd 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java @@ -798,13 +798,13 @@ private ExecutionBlock buildGroupBy(GlobalPlanContext context, ExecutionBlock la boolean multiLevelEnabled = context.getPlan().getContext().getBool(SessionVars.GROUPBY_MULTI_LEVEL_ENABLED); if (multiLevelEnabled) { - if (PlannerUtil.findTopNode(groupbyNode, NodeType.UNION) == null) { - DistinctGroupbyBuilder builder = new DistinctGroupbyBuilder(this); - return builder.buildMultiLevelPlan(context, lastBlock, groupbyNode); - } else { +// if (PlannerUtil.findTopNode(groupbyNode, NodeType.UNION) == null) { +// DistinctGroupbyBuilder builder = new DistinctGroupbyBuilder(this); +// return builder.buildMultiLevelPlan(context, lastBlock, groupbyNode); +// } else { DistinctGroupbyBuilder builder = new DistinctGroupbyBuilder(this); return builder.buildPlan(context, lastBlock, groupbyNode); - } +// } } else { DistinctGroupbyBuilder builder = new DistinctGroupbyBuilder(this); return builder.buildPlan(context, lastBlock, groupbyNode); @@ -812,12 +812,12 @@ private ExecutionBlock buildGroupBy(GlobalPlanContext context, ExecutionBlock la } else { GroupbyNode firstPhaseGroupby = createFirstPhaseGroupBy(masterPlan.getLogicalPlan(), groupbyNode); - if (hasUnionChild(firstPhaseGroupby)) { - currentBlock = buildGroupbyAndUnionPlan(masterPlan, lastBlock, firstPhaseGroupby, groupbyNode); - } else { +// if (hasUnionChild(firstPhaseGroupby)) { +// currentBlock = buildGroupbyAndUnionPlan(masterPlan, lastBlock, firstPhaseGroupby, groupbyNode); +// } else { // general hash-shuffled aggregation currentBlock = buildTwoPhaseGroupby(masterPlan, lastBlock, firstPhaseGroupby, groupbyNode); - } +// } } return currentBlock; @@ -980,26 +980,26 @@ private ExecutionBlock buildSortPlan(GlobalPlanContext context, ExecutionBlock c SortNode firstSortNode = PlannerUtil.clone(context.plan.getLogicalPlan(), currentNode); - if (firstSortNode.getChild().getType() == NodeType.TABLE_SUBQUERY && - ((TableSubQueryNode)firstSortNode.getChild()).getSubQuery().getType() == NodeType.UNION) { - - currentBlock = childBlock; - for (DataChannel channel : masterPlan.getIncomingChannels(childBlock.getId())) { - channel.setShuffle(RANGE_SHUFFLE, PlannerUtil.sortSpecsToSchema(currentNode.getSortKeys()).toArray(), 32); - channel.setSchema(firstSortNode.getOutSchema()); - - ExecutionBlock subBlock = masterPlan.getExecBlock(channel.getSrcId()); - SortNode s1 = PlannerUtil.clone(context.plan.getLogicalPlan(), firstSortNode); - s1.setChild(subBlock.getPlan()); - subBlock.setPlan(s1); - - ScanNode secondScan = buildInputExecutor(masterPlan.getLogicalPlan(), channel); - currentNode.setChild(secondScan); - currentNode.setInSchema(secondScan.getOutSchema()); - currentBlock.setPlan(currentNode); - currentBlock.getEnforcer().addSortedInput(secondScan.getTableName(), currentNode.getSortKeys()); - } - } else { +// if (firstSortNode.getChild().getType() == NodeType.TABLE_SUBQUERY && +// ((TableSubQueryNode)firstSortNode.getChild()).getSubQuery().getType() == NodeType.UNION) { +// +// currentBlock = childBlock; +// for (DataChannel channel : masterPlan.getIncomingChannels(childBlock.getId())) { +// channel.setShuffle(RANGE_SHUFFLE, PlannerUtil.sortSpecsToSchema(currentNode.getSortKeys()).toArray(), 32); +// channel.setSchema(firstSortNode.getOutSchema()); +// +// ExecutionBlock subBlock = masterPlan.getExecBlock(channel.getSrcId()); +// SortNode s1 = PlannerUtil.clone(context.plan.getLogicalPlan(), firstSortNode); +// s1.setChild(subBlock.getPlan()); +// subBlock.setPlan(s1); +// +// ScanNode secondScan = buildInputExecutor(masterPlan.getLogicalPlan(), channel); +// currentNode.setChild(secondScan); +// currentNode.setInSchema(secondScan.getOutSchema()); +// currentBlock.setPlan(currentNode); +// currentBlock.getEnforcer().addSortedInput(secondScan.getTableName(), currentNode.getSortKeys()); +// } +// } else { LogicalNode childBlockPlan = childBlock.getPlan(); firstSortNode.setChild(childBlockPlan); // sort is a non-projectable operator. So, in/out schemas are the same to its child operator. @@ -1018,7 +1018,7 @@ private ExecutionBlock buildSortPlan(GlobalPlanContext context, ExecutionBlock c currentBlock.setPlan(currentNode); currentBlock.getEnforcer().addSortedInput(secondScan.getTableName(), currentNode.getSortKeys()); masterPlan.addConnect(channel); - } +// } return currentBlock; } @@ -1040,11 +1040,11 @@ private ExecutionBlock buildStorePlan(GlobalPlanContext context, partitionMethod.getPartitionType())); } - if (hasUnionChild(currentNode)) { // if it has union children - return buildShuffleAndStorePlanToPartitionedTableWithUnion(context, currentNode, lastBlock); - } else { // otherwise +// if (hasUnionChild(currentNode)) { // if it has union children +// return buildShuffleAndStorePlanToPartitionedTableWithUnion(context, currentNode, lastBlock); +// } else { // otherwise return buildShuffleAndStorePlanToPartitionedTable(context, currentNode, lastBlock); - } +// } } else { // if result table is not a partitioned table, directly store it return buildNoPartitionedStorePlan(context, currentNode, lastBlock); } @@ -1118,14 +1118,14 @@ private ExecutionBlock buildShuffleAndStorePlanToPartitionedTable(GlobalPlanCont private ExecutionBlock buildNoPartitionedStorePlan(GlobalPlanContext context, StoreTableNode currentNode, ExecutionBlock childBlock) { - if (hasUnionChild(currentNode)) { // when the below is union - return buildShuffleAndStorePlanNoPartitionedTableWithUnion(context, currentNode, childBlock); - } else { +// if (hasUnionChild(currentNode)) { // when the below is union +// return buildShuffleAndStorePlanNoPartitionedTableWithUnion(context, currentNode, childBlock); +// } else { currentNode.setChild(childBlock.getPlan()); currentNode.setInSchema(childBlock.getPlan().getOutSchema()); childBlock.setPlan(currentNode); return childBlock; - } +// } } private void setShuffleKeysFromPartitionedTableStore(StoreTableNode node, DataChannel channel) { @@ -1177,25 +1177,25 @@ public LogicalNode visitProjection(GlobalPlanContext context, LogicalPlan plan, ExecutionBlock execBlock = context.execBlockMap.remove(child.getPID()); - if (child.getType() == NodeType.TABLE_SUBQUERY && - ((TableSubQueryNode)child).getSubQuery().getType() == NodeType.UNION) { - MasterPlan masterPlan = context.plan; - for (DataChannel dataChannel : masterPlan.getIncomingChannels(execBlock.getId())) { - // This data channel will be stored in staging directory, but RawFile, default file type, does not support - // distributed file system. It needs to change the file format for distributed file system. - dataChannel.setStoreType(CatalogProtos.StoreType.CSV); - ExecutionBlock subBlock = masterPlan.getExecBlock(dataChannel.getSrcId()); - - ProjectionNode copy = PlannerUtil.clone(plan, node); - copy.setChild(subBlock.getPlan()); - subBlock.setPlan(copy); - } - execBlock.setPlan(null); - } else { +// if (child.getType() == NodeType.TABLE_SUBQUERY && +// ((TableSubQueryNode)child).getSubQuery().getType() == NodeType.UNION) { +// MasterPlan masterPlan = context.plan; +// for (DataChannel dataChannel : masterPlan.getIncomingChannels(execBlock.getId())) { +// // This data channel will be stored in staging directory, but RawFile, default file type, does not support +// // distributed file system. It needs to change the file format for distributed file system. +// dataChannel.setStoreType(CatalogProtos.StoreType.CSV); +// ExecutionBlock subBlock = masterPlan.getExecBlock(dataChannel.getSrcId()); +// +// ProjectionNode copy = PlannerUtil.clone(plan, node); +// copy.setChild(subBlock.getPlan()); +// subBlock.setPlan(copy); +// } +// execBlock.setPlan(null); +// } else { node.setChild(execBlock.getPlan()); node.setInSchema(execBlock.getPlan().getOutSchema()); execBlock.setPlan(node); - } +// } context.execBlockMap.put(node.getPID(), execBlock); return node; @@ -1392,60 +1392,61 @@ public LogicalNode visitJoin(GlobalPlanContext context, LogicalPlan plan, Logica @Override public LogicalNode visitUnion(GlobalPlanContext context, LogicalPlan plan, LogicalPlan.QueryBlock queryBlock, UnionNode node, Stack stack) throws PlanningException { - stack.push(node); - LogicalPlan.QueryBlock leftQueryBlock = plan.getBlock(node.getLeftChild()); - LogicalNode leftChild = visit(context, plan, leftQueryBlock, leftQueryBlock.getRoot(), stack); - - LogicalPlan.QueryBlock rightQueryBlock = plan.getBlock(node.getRightChild()); - LogicalNode rightChild = visit(context, plan, rightQueryBlock, rightQueryBlock.getRoot(), stack); - stack.pop(); - - MasterPlan masterPlan = context.getPlan(); - - List unionBlocks = Lists.newArrayList(); - List queryBlockBlocks = Lists.newArrayList(); - - ExecutionBlock leftBlock = context.execBlockMap.remove(leftChild.getPID()); - ExecutionBlock rightBlock = context.execBlockMap.remove(rightChild.getPID()); - - // These union types need to eliminate unnecessary nodes between parent and child node of query tree. - boolean leftUnion = (leftChild.getType() == NodeType.UNION) || - ((leftChild.getType() == NodeType.TABLE_SUBQUERY) && - (((TableSubQueryNode)leftChild).getSubQuery().getType() == NodeType.UNION)); - boolean rightUnion = (rightChild.getType() == NodeType.UNION) || - (rightChild.getType() == NodeType.TABLE_SUBQUERY) && - (((TableSubQueryNode)rightChild).getSubQuery().getType() == NodeType.UNION); - if (leftUnion) { - unionBlocks.add(leftBlock); - } else { - queryBlockBlocks.add(leftBlock); - } - if (rightUnion) { - unionBlocks.add(rightBlock); - } else { - queryBlockBlocks.add(rightBlock); - } - + super.visitUnion(context, plan, queryBlock, node, stack); +// stack.push(node); +// LogicalPlan.QueryBlock leftQueryBlock = plan.getBlock(node.getLeftChild()); +// LogicalNode leftChild = visit(context, plan, leftQueryBlock, leftQueryBlock.getRoot(), stack); +// +// LogicalPlan.QueryBlock rightQueryBlock = plan.getBlock(node.getRightChild()); +// LogicalNode rightChild = visit(context, plan, rightQueryBlock, rightQueryBlock.getRoot(), stack); +// stack.pop(); +// +// MasterPlan masterPlan = context.getPlan(); +// +// List unionBlocks = Lists.newArrayList(); +// List queryBlockBlocks = Lists.newArrayList(); +// +// ExecutionBlock leftBlock = context.execBlockMap.remove(leftChild.getPID()); +// ExecutionBlock rightBlock = context.execBlockMap.remove(rightChild.getPID()); +// +// // These union types need to eliminate unnecessary nodes between parent and child node of query tree. +// boolean leftUnion = (leftChild.getType() == NodeType.UNION) || +// ((leftChild.getType() == NodeType.TABLE_SUBQUERY) && +// (((TableSubQueryNode)leftChild).getSubQuery().getType() == NodeType.UNION)); +// boolean rightUnion = (rightChild.getType() == NodeType.UNION) || +// (rightChild.getType() == NodeType.TABLE_SUBQUERY) && +// (((TableSubQueryNode)rightChild).getSubQuery().getType() == NodeType.UNION); +// if (leftUnion) { +// unionBlocks.add(leftBlock); +// } else { +// queryBlockBlocks.add(leftBlock); +// } +// if (rightUnion) { +// unionBlocks.add(rightBlock); +// } else { +// queryBlockBlocks.add(rightBlock); +// } +// ExecutionBlock execBlock; - if (unionBlocks.size() == 0) { +// if (unionBlocks.size() == 0) { execBlock = context.plan.newExecutionBlock(); - } else { - execBlock = unionBlocks.get(0); - } - - for (ExecutionBlock childBlocks : unionBlocks) { - for (ExecutionBlock grandChildBlock : masterPlan.getChilds(childBlocks)) { - masterPlan.disconnect(grandChildBlock, childBlocks); - queryBlockBlocks.add(grandChildBlock); - } - } - - for (ExecutionBlock childBlocks : queryBlockBlocks) { - DataChannel channel = new DataChannel(childBlocks, execBlock, NONE_SHUFFLE, 1); - channel.setStoreType(storeType); - masterPlan.addConnect(channel); - } - +// } else { +// execBlock = unionBlocks.get(0); +// } +// +// for (ExecutionBlock childBlocks : unionBlocks) { +// for (ExecutionBlock grandChildBlock : masterPlan.getChilds(childBlocks)) { +// masterPlan.disconnect(grandChildBlock, childBlocks); +// queryBlockBlocks.add(grandChildBlock); +// } +// } +// +// for (ExecutionBlock childBlocks : queryBlockBlocks) { +// DataChannel channel = new DataChannel(childBlocks, execBlock, NONE_SHUFFLE, 1); +// channel.setStoreType(storeType); +// masterPlan.addConnect(channel); +// } +// context.execBlockMap.put(node.getPID(), execBlock); return node; @@ -1482,74 +1483,74 @@ public LogicalNode visitTableSubQuery(GlobalPlanContext context, LogicalPlan pla ExecutionBlock currentBlock = context.execBlockMap.remove(child.getPID()); - if (child.getType() == NodeType.UNION) { - List addedTableSubQueries = new ArrayList(); - TableSubQueryNode leftMostSubQueryNode = null; - for (ExecutionBlock childBlock : context.plan.getChilds(currentBlock.getId())) { - TableSubQueryNode copy = PlannerUtil.clone(plan, node); - copy.setSubQuery(childBlock.getPlan()); - childBlock.setPlan(copy); - addedTableSubQueries.add(copy); - - //Find a SubQueryNode which contains all columns in InputSchema matched with Target and OutputSchema's column - if (copy.getInSchema().containsAll(copy.getOutSchema().getColumns())) { - for (Target eachTarget : copy.getTargets()) { - Set columns = EvalTreeUtil.findUniqueColumns(eachTarget.getEvalTree()); - if (copy.getInSchema().containsAll(columns)) { - leftMostSubQueryNode = copy; - break; - } - } - } - } - if (leftMostSubQueryNode != null) { - // replace target column name - Target[] targets = leftMostSubQueryNode.getTargets(); - int[] targetMappings = new int[targets.length]; - for (int i = 0; i < targets.length; i++) { - if (targets[i].getEvalTree().getType() != EvalType.FIELD) { - throw new PlanningException("Target of a UnionNode's subquery should be FieldEval."); - } - int index = leftMostSubQueryNode.getInSchema().getColumnId(targets[i].getNamedColumn().getQualifiedName()); - if (index < 0) { - // If a target has alias, getNamedColumn() only returns alias - Set columns = EvalTreeUtil.findUniqueColumns(targets[i].getEvalTree()); - Column column = columns.iterator().next(); - index = leftMostSubQueryNode.getInSchema().getColumnId(column.getQualifiedName()); - } - if (index < 0) { - throw new PlanningException("Can't find matched Target in UnionNode's input schema: " + targets[i] - + "->" + leftMostSubQueryNode.getInSchema()); - } - targetMappings[i] = index; - } - - for (TableSubQueryNode eachNode: addedTableSubQueries) { - if (eachNode.getPID() == leftMostSubQueryNode.getPID()) { - continue; - } - Target[] eachNodeTargets = eachNode.getTargets(); - if (eachNodeTargets.length != targetMappings.length) { - throw new PlanningException("Union query can't have different number of target columns."); - } - for (int i = 0; i < eachNodeTargets.length; i++) { - Column inColumn = eachNode.getInSchema().getColumn(targetMappings[i]); - eachNodeTargets[i].setAlias(eachNodeTargets[i].getNamedColumn().getQualifiedName()); - EvalNode evalNode = eachNodeTargets[i].getEvalTree(); - if (evalNode.getType() != EvalType.FIELD) { - throw new PlanningException("Target of a UnionNode's subquery should be FieldEval."); - } - FieldEval fieldEval = (FieldEval) evalNode; - EvalTreeUtil.changeColumnRef(fieldEval, - fieldEval.getColumnRef().getQualifiedName(), inColumn.getQualifiedName()); - } - } - } else { - LOG.warn("Can't find left most SubQuery in the UnionNode."); - } - } else { +// if (child.getType() == NodeType.UNION) { +// List addedTableSubQueries = new ArrayList(); +// TableSubQueryNode leftMostSubQueryNode = null; +// for (ExecutionBlock childBlock : context.plan.getChilds(currentBlock.getId())) { +// TableSubQueryNode copy = PlannerUtil.clone(plan, node); +// copy.setSubQuery(childBlock.getPlan()); +// childBlock.setPlan(copy); +// addedTableSubQueries.add(copy); +// +// //Find a SubQueryNode which contains all columns in InputSchema matched with Target and OutputSchema's column +// if (copy.getInSchema().containsAll(copy.getOutSchema().getColumns())) { +// for (Target eachTarget : copy.getTargets()) { +// Set columns = EvalTreeUtil.findUniqueColumns(eachTarget.getEvalTree()); +// if (copy.getInSchema().containsAll(columns)) { +// leftMostSubQueryNode = copy; +// break; +// } +// } +// } +// } +// if (leftMostSubQueryNode != null) { +// // replace target column name +// Target[] targets = leftMostSubQueryNode.getTargets(); +// int[] targetMappings = new int[targets.length]; +// for (int i = 0; i < targets.length; i++) { +// if (targets[i].getEvalTree().getType() != EvalType.FIELD) { +// throw new PlanningException("Target of a UnionNode's subquery should be FieldEval."); +// } +// int index = leftMostSubQueryNode.getInSchema().getColumnId(targets[i].getNamedColumn().getQualifiedName()); +// if (index < 0) { +// // If a target has alias, getNamedColumn() only returns alias +// Set columns = EvalTreeUtil.findUniqueColumns(targets[i].getEvalTree()); +// Column column = columns.iterator().next(); +// index = leftMostSubQueryNode.getInSchema().getColumnId(column.getQualifiedName()); +// } +// if (index < 0) { +// throw new PlanningException("Can't find matched Target in UnionNode's input schema: " + targets[i] +// + "->" + leftMostSubQueryNode.getInSchema()); +// } +// targetMappings[i] = index; +// } +// +// for (TableSubQueryNode eachNode: addedTableSubQueries) { +// if (eachNode.getPID() == leftMostSubQueryNode.getPID()) { +// continue; +// } +// Target[] eachNodeTargets = eachNode.getTargets(); +// if (eachNodeTargets.length != targetMappings.length) { +// throw new PlanningException("Union query can't have different number of target columns."); +// } +// for (int i = 0; i < eachNodeTargets.length; i++) { +// Column inColumn = eachNode.getInSchema().getColumn(targetMappings[i]); +// eachNodeTargets[i].setAlias(eachNodeTargets[i].getNamedColumn().getQualifiedName()); +// EvalNode evalNode = eachNodeTargets[i].getEvalTree(); +// if (evalNode.getType() != EvalType.FIELD) { +// throw new PlanningException("Target of a UnionNode's subquery should be FieldEval."); +// } +// FieldEval fieldEval = (FieldEval) evalNode; +// EvalTreeUtil.changeColumnRef(fieldEval, +// fieldEval.getColumnRef().getQualifiedName(), inColumn.getQualifiedName()); +// } +// } +// } else { +// LOG.warn("Can't find left most SubQuery in the UnionNode."); +// } +// } else { currentBlock.setPlan(node); - } +// } context.execBlockMap.put(node.getPID(), currentBlock); return node; } @@ -1611,28 +1612,28 @@ public LogicalNode visitInsert(GlobalPlanContext context, LogicalPlan plan, Logi } } - @SuppressWarnings("unused") - private static class ConsecutiveUnionFinder extends BasicLogicalPlanVisitor, LogicalNode> { - @Override - public LogicalNode visitUnion(List unionNodeList, LogicalPlan plan, LogicalPlan.QueryBlock queryBlock, - UnionNode node, Stack stack) - throws PlanningException { - if (node.getType() == NodeType.UNION) { - unionNodeList.add(node); - } - - stack.push(node); - TableSubQueryNode leftSubQuery = node.getLeftChild(); - TableSubQueryNode rightSubQuery = node.getRightChild(); - if (leftSubQuery.getSubQuery().getType() == NodeType.UNION) { - visit(unionNodeList, plan, queryBlock, leftSubQuery, stack); - } - if (rightSubQuery.getSubQuery().getType() == NodeType.UNION) { - visit(unionNodeList, plan, queryBlock, rightSubQuery, stack); - } - stack.pop(); - - return node; - } - } +// @SuppressWarnings("unused") +// private static class ConsecutiveUnionFinder extends BasicLogicalPlanVisitor, LogicalNode> { +// @Override +// public LogicalNode visitUnion(List unionNodeList, LogicalPlan plan, LogicalPlan.QueryBlock queryBlock, +// UnionNode node, Stack stack) +// throws PlanningException { +// if (node.getType() == NodeType.UNION) { +// unionNodeList.add(node); +// } +// +// stack.push(node); +// TableSubQueryNode leftSubQuery = node.getLeftChild(); +// TableSubQueryNode rightSubQuery = node.getRightChild(); +// if (leftSubQuery.getSubQuery().getType() == NodeType.UNION) { +// visit(unionNodeList, plan, queryBlock, leftSubQuery, stack); +// } +// if (rightSubQuery.getSubQuery().getType() == NodeType.UNION) { +// visit(unionNodeList, plan, queryBlock, rightSubQuery, stack); +// } +// stack.pop(); +// +// return node; +// } +// } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/BaseGlobalPlanRewriteRuleProvider.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/BaseGlobalPlanRewriteRuleProvider.java index 96ee2c6d11..8a0efbb63e 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/BaseGlobalPlanRewriteRuleProvider.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/BaseGlobalPlanRewriteRuleProvider.java @@ -18,7 +18,10 @@ package org.apache.tajo.engine.planner.global.rewriter; +import com.google.common.collect.Lists; import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.engine.planner.global.rewriter.rules.BroadcastJoinRule; +import org.apache.tajo.engine.planner.global.rewriter.rules.UnionReduceRule; import org.apache.tajo.util.TUtil; import java.util.Collection; @@ -34,6 +37,9 @@ public BaseGlobalPlanRewriteRuleProvider(TajoConf conf) { @Override public Collection> getRules() { - return EMPTY_RULES; + List> rules = Lists.newArrayList(); + rules.add(UnionReduceRule.class); +// rules.add(BroadcastJoinRule.class); + return rules; } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/GlobalPlanTestRuleProvider.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/GlobalPlanTestRuleProvider.java index 6fa487b712..dc91577214 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/GlobalPlanTestRuleProvider.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/GlobalPlanTestRuleProvider.java @@ -20,7 +20,6 @@ import com.google.common.collect.Lists; import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.engine.planner.global.rewriter.rules.BroadcastJoinRule; import org.apache.tajo.engine.planner.global.rewriter.rules.GlobalPlanEqualityTester; import java.util.Collection; @@ -39,7 +38,6 @@ public GlobalPlanTestRuleProvider(TajoConf conf) { @Override public Collection> getRules() { List> injectedRules = Lists.newArrayList(super.getRules()); - injectedRules.add(BroadcastJoinRule.class); injectedRules.add(GlobalPlanEqualityTester.class); return injectedRules; } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/UnionReduceRule.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/UnionReduceRule.java index c8214f2520..30b23cf665 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/UnionReduceRule.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/UnionReduceRule.java @@ -1,14 +1,39 @@ +/** + * 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.global.rewriter.rules; import org.apache.tajo.ExecutionBlockId; import org.apache.tajo.OverridableConf; +import org.apache.tajo.engine.planner.global.ExecutionBlock; import org.apache.tajo.engine.planner.global.MasterPlan; import org.apache.tajo.engine.planner.global.rewriter.GlobalPlanRewriteRule; import org.apache.tajo.plan.LogicalPlan; import org.apache.tajo.plan.PlanningException; +import org.apache.tajo.plan.logical.LogicalNode; import org.apache.tajo.plan.logical.NodeType; +import org.apache.tajo.plan.logical.UnionNode; +import org.apache.tajo.plan.util.PlannerUtil; +import org.apache.tajo.util.TUtil; import org.apache.tajo.util.graph.DirectedGraphVisitor; +import java.util.List; +import java.util.Map; import java.util.Stack; public class UnionReduceRule implements GlobalPlanRewriteRule { @@ -52,12 +77,29 @@ public static void rewrite(MasterPlan plan) { public void visit(Stack stack, ExecutionBlockId executionBlockId) { // must have the form of /* - operator + parent_op | union / \ - op op + child_op child_op */ + ExecutionBlock current = instance.plan.getExecBlock(executionBlockId); + if (current.hasUnion()) { + Map newBlocks = TUtil.newHashMap(); + List childBlocks = instance.plan.getChilds(current); + // create new execution blocks for each child_op + // TODO: consider union sequence + UnionNode unionNode = PlannerUtil.findTopNode(current.getPlan(), NodeType.UNION); + LogicalNode parentOfUnion = PlannerUtil.findTopParentNode(current.getPlan(), NodeType.UNION); + newBlocks.put(unionNode.getLeftChild().getPID(), instance.plan.newExecutionBlock()); + newBlocks.put(unionNode.getRightChild().getPID(), instance.plan.newExecutionBlock()); + + // push parent_op as the parent of each child_op and remove union + + + // connect new execution blocks and parents of the current block + + } } } } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java index 1da3ee9928..4a42050c04 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java @@ -865,4 +865,9 @@ public final void testDistinctPythonUdafWithUnion1() throws Exception { assertResultSet(res); cleanupQuery(res); } + + @Test + public final void test() throws Exception { + executeString("select count(*) from (select * from nation union select * from nation) t"); + } } From 3fa201707c445bbe0387a0ec3063d43f9580d39f Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sun, 10 May 2015 17:26:13 +0900 Subject: [PATCH 22/37] TAJO-1553 --- .../BroadcastJoinMarkCandidateVisitor.java | 144 ++--- .../planner/BroadcastJoinPlanVisitor.java | 120 ++-- .../engine/planner/PhysicalPlannerImpl.java | 3 +- .../engine/planner/global/ExecutionBlock.java | 22 + .../engine/planner/global/GlobalPlanner.java | 514 +++++++++--------- .../BaseGlobalPlanRewriteRuleProvider.java | 6 +- .../rewriter/rules/BroadcastJoinRule.java | 142 +++-- .../rewriter/rules/GlobalPlanRewriteUtil.java | 12 + .../rewriter/rules/UnionReduceRule.java | 210 +++---- .../tajo/engine/query/TestGroupByQuery.java | 11 +- .../apache/tajo/plan/logical/JoinNode.java | 24 +- 11 files changed, 659 insertions(+), 549 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/BroadcastJoinMarkCandidateVisitor.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/BroadcastJoinMarkCandidateVisitor.java index 6b3dc487d2..bdc6bae3d7 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/BroadcastJoinMarkCandidateVisitor.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/BroadcastJoinMarkCandidateVisitor.java @@ -1,72 +1,72 @@ -/** - * 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; - -import org.apache.tajo.engine.planner.global.GlobalPlanner; -import org.apache.tajo.plan.visitor.BasicLogicalPlanVisitor; -import org.apache.tajo.plan.LogicalPlan; -import org.apache.tajo.plan.PlanningException; -import org.apache.tajo.plan.logical.JoinNode; -import org.apache.tajo.plan.logical.LogicalNode; -import org.apache.tajo.plan.logical.NodeType; -import org.apache.tajo.plan.logical.ScanNode; - -import java.util.Stack; - -public class BroadcastJoinMarkCandidateVisitor extends BasicLogicalPlanVisitor { - @Override - public LogicalNode visitJoin(GlobalPlanner.GlobalPlanContext context, LogicalPlan plan, LogicalPlan.QueryBlock block, - JoinNode node, Stack stack) throws PlanningException { - LogicalNode leftChild = node.getLeftChild(); - LogicalNode rightChild = node.getRightChild(); - - if (ScanNode.isScanNode(leftChild) && ScanNode.isScanNode(rightChild)) { - node.setCandidateBroadcast(true); - return node; - } - - if(!ScanNode.isScanNode(leftChild)) { - visit(context, plan, block, leftChild, stack); - } - - if(!ScanNode.isScanNode(rightChild)) { - visit(context, plan, block, rightChild, stack); - } - - if(isBroadcastCandidateNode(leftChild) && - isBroadcastCandidateNode(rightChild)) { - node.setCandidateBroadcast(true); - } - - return node; - } - - public static boolean isBroadcastCandidateNode(LogicalNode node) { - if(node.getType() == NodeType.SCAN || - node.getType() == NodeType.PARTITIONS_SCAN) { - return true; - } - - if(node.getType() == NodeType.JOIN && ((JoinNode)node).isCandidateBroadcast()) { - return true; - } - - return false; - } -} +///** +// * 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; +// +//import org.apache.tajo.engine.planner.global.GlobalPlanner; +//import org.apache.tajo.plan.visitor.BasicLogicalPlanVisitor; +//import org.apache.tajo.plan.LogicalPlan; +//import org.apache.tajo.plan.PlanningException; +//import org.apache.tajo.plan.logical.JoinNode; +//import org.apache.tajo.plan.logical.LogicalNode; +//import org.apache.tajo.plan.logical.NodeType; +//import org.apache.tajo.plan.logical.ScanNode; +// +//import java.util.Stack; +// +//public class BroadcastJoinMarkCandidateVisitor extends BasicLogicalPlanVisitor { +// @Override +// public LogicalNode visitJoin(GlobalPlanner.GlobalPlanContext context, LogicalPlan plan, LogicalPlan.QueryBlock block, +// JoinNode node, Stack stack) throws PlanningException { +// LogicalNode leftChild = node.getLeftChild(); +// LogicalNode rightChild = node.getRightChild(); +// +// if (ScanNode.isScanNode(leftChild) && ScanNode.isScanNode(rightChild)) { +// node.setCandidateBroadcast(true); +// return node; +// } +// +// if(!ScanNode.isScanNode(leftChild)) { +// visit(context, plan, block, leftChild, stack); +// } +// +// if(!ScanNode.isScanNode(rightChild)) { +// visit(context, plan, block, rightChild, stack); +// } +// +// if(isBroadcastCandidateNode(leftChild) && +// isBroadcastCandidateNode(rightChild)) { +// node.setCandidateBroadcast(true); +// } +// +// return node; +// } +// +// public static boolean isBroadcastCandidateNode(LogicalNode node) { +// if(node.getType() == NodeType.SCAN || +// node.getType() == NodeType.PARTITIONS_SCAN) { +// return true; +// } +// +// if(node.getType() == NodeType.JOIN && ((JoinNode)node).isCandidateBroadcast()) { +// return true; +// } +// +// return false; +// } +//} diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/BroadcastJoinPlanVisitor.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/BroadcastJoinPlanVisitor.java index bd8efbee6a..6d7ed62ab6 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/BroadcastJoinPlanVisitor.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/BroadcastJoinPlanVisitor.java @@ -1,60 +1,60 @@ -/** - * 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; - -import org.apache.tajo.engine.planner.global.GlobalPlanner; -import org.apache.tajo.plan.visitor.BasicLogicalPlanVisitor; -import org.apache.tajo.plan.LogicalPlan; -import org.apache.tajo.plan.PlanningException; -import org.apache.tajo.plan.logical.JoinNode; -import org.apache.tajo.plan.logical.LogicalNode; -import org.apache.tajo.plan.logical.NodeType; -import org.apache.tajo.plan.logical.ScanNode; - -import java.util.Stack; - -public class BroadcastJoinPlanVisitor extends BasicLogicalPlanVisitor { - @Override - public LogicalNode visitJoin(GlobalPlanner.GlobalPlanContext context, LogicalPlan plan, LogicalPlan.QueryBlock block, - JoinNode node, Stack stack) throws PlanningException { - LogicalNode leftChild = node.getLeftChild(); - LogicalNode rightChild = node.getRightChild(); - - if (leftChild.getType() == NodeType.JOIN && ScanNode.isScanNode(rightChild)) { - node.getBroadcastCandidateTargets().add(node); - } - LogicalNode parentNode = stack.peek(); - if (parentNode != null && parentNode.getType() == NodeType.JOIN) { - node.getBroadcastCandidateTargets().addAll(((JoinNode)parentNode).getBroadcastCandidateTargets()); - } - - Stack currentStack = new Stack(); - currentStack.push(node); - if(!ScanNode.isScanNode(leftChild)) { - visit(context, plan, block, leftChild, currentStack); - } - - if(!ScanNode.isScanNode(rightChild)) { - visit(context, plan, block, rightChild, currentStack); - } - currentStack.pop(); - - return node; - } -} +///** +// * 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; +// +//import org.apache.tajo.engine.planner.global.GlobalPlanner; +//import org.apache.tajo.plan.visitor.BasicLogicalPlanVisitor; +//import org.apache.tajo.plan.LogicalPlan; +//import org.apache.tajo.plan.PlanningException; +//import org.apache.tajo.plan.logical.JoinNode; +//import org.apache.tajo.plan.logical.LogicalNode; +//import org.apache.tajo.plan.logical.NodeType; +//import org.apache.tajo.plan.logical.ScanNode; +// +//import java.util.Stack; +// +//public class BroadcastJoinPlanVisitor extends BasicLogicalPlanVisitor { +// @Override +// public LogicalNode visitJoin(GlobalPlanner.GlobalPlanContext context, LogicalPlan plan, LogicalPlan.QueryBlock block, +// JoinNode node, Stack stack) throws PlanningException { +// LogicalNode leftChild = node.getLeftChild(); +// LogicalNode rightChild = node.getRightChild(); +// +// if (leftChild.getType() == NodeType.JOIN && ScanNode.isScanNode(rightChild)) { +// node.getBroadcastCandidateTargets().add(node); +// } +// LogicalNode parentNode = stack.peek(); +// if (parentNode != null && parentNode.getType() == NodeType.JOIN) { +// node.getBroadcastCandidateTargets().addAll(((JoinNode)parentNode).getBroadcastCandidateTargets()); +// } +// +// Stack currentStack = new Stack(); +// currentStack.push(node); +// if(!ScanNode.isScanNode(leftChild)) { +// visit(context, plan, block, leftChild, currentStack); +// } +// +// if(!ScanNode.isScanNode(rightChild)) { +// visit(context, plan, block, rightChild, currentStack); +// } +// currentStack.pop(); +// +// return node; +// } +//} 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..05f644f20d 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 @@ -30,12 +30,10 @@ import org.apache.hadoop.fs.Path; import org.apache.tajo.SessionVars; import org.apache.tajo.catalog.Column; -import org.apache.tajo.catalog.Schema; import org.apache.tajo.catalog.SortSpec; import org.apache.tajo.catalog.proto.CatalogProtos; import org.apache.tajo.catalog.proto.CatalogProtos.SortSpecProto; import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.plan.serder.LogicalNodeDeserializer; import org.apache.tajo.engine.planner.enforce.Enforcer; import org.apache.tajo.engine.planner.global.DataChannel; import org.apache.tajo.engine.planner.physical.*; @@ -48,6 +46,7 @@ import org.apache.tajo.ipc.TajoWorkerProtocol.DistinctGroupbyEnforcer.SortSpecArray; import org.apache.tajo.plan.LogicalPlan; import org.apache.tajo.plan.logical.*; +import org.apache.tajo.plan.serder.LogicalNodeDeserializer; import org.apache.tajo.plan.util.PlannerUtil; import org.apache.tajo.storage.*; import org.apache.tajo.storage.fragment.FileFragment; diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java index b8c4aa0eaa..03987f20c9 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java @@ -42,6 +42,12 @@ public class ExecutionBlock { private Set broadcasted = new HashSet(); + /* + * The nullSupplying and preservedRow flags are used for finding which relations will be broadcasted. + */ + protected transient boolean nullSuppllying = false; + protected transient boolean preservedRow = false; + public ExecutionBlock(ExecutionBlockId executionBlockId) { this.executionBlockId = executionBlockId; } @@ -196,4 +202,20 @@ public Collection getBroadcastTables() { public String toString() { return executionBlockId.toString(); } + + public void setNullSuppllying() { + nullSuppllying = true; + } + + public void setPreservedRow() { + preservedRow = true; + } + + public boolean isNullSuppllying() { + return nullSuppllying; + } + + public boolean isPreservedRow() { + return preservedRow; + } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java index 380249e0bd..0a30ea473e 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java @@ -389,73 +389,73 @@ private ExecutionBlock buildJoinPlan(GlobalPlanContext context, JoinNode joinNod // } // } -// LogicalNode leftNode = joinNode.getLeftChild(); -// LogicalNode rightNode = joinNode.getRightChild(); + LogicalNode leftNode = joinNode.getLeftChild(); + LogicalNode rightNode = joinNode.getRightChild(); // symmetric repartition join -// boolean leftUnion = leftNode.getType() == NodeType.TABLE_SUBQUERY && -// ((TableSubQueryNode)leftNode).getSubQuery().getType() == NodeType.UNION; -// boolean rightUnion = rightNode.getType() == NodeType.TABLE_SUBQUERY && -// ((TableSubQueryNode)rightNode).getSubQuery().getType() == NodeType.UNION; -// -// if (leftUnion || rightUnion) { // if one of child execution block is union -// /* -// Join with tableC and result of union tableA, tableB is expected the following physical plan. -// But Union execution block is not necessary. -// |-eb_0001_000006 (Terminal) -// |-eb_0001_000005 (Join eb_0001_000003, eb_0001_000004) -// |-eb_0001_000004 (Scan TableC) -// |-eb_0001_000003 (Union TableA, TableB) -// |-eb_0001_000002 (Scan TableB) -// |-eb_0001_000001 (Scan TableA) -// -// The above plan can be changed to the following plan. -// |-eb_0001_000005 (Terminal) -// |-eb_0001_000003 (Join [eb_0001_000001, eb_0001_000002], eb_0001_000004) -// |-eb_0001_000004 (Scan TableC) -// |-eb_0001_000002 (Scan TableB) -// |-eb_0001_000001 (Scan TableA) -// -// eb_0001_000003's left child should be eb_0001_000001 + eb_0001_000001 and right child should be eb_0001_000004. -// For this eb_0001_000001 is representative of eb_0001_000001, eb_0001_000002. -// So eb_0001_000003's left child is eb_0001_000001 -// */ -// Column[][] joinColumns = null; -// if (joinNode.getJoinType() != JoinType.CROSS) { -// // ShuffleKeys need to not have thea-join condition because Tajo supports only equi-join. -// joinColumns = PlannerUtil.joinJoinKeyForEachTable(joinNode.getJoinQual(), -// leftNode.getOutSchema(), rightNode.getOutSchema(), false); -// } -// -// if (leftUnion && !rightUnion) { // if only left is union -// currentBlock = leftBlock; -// context.execBlockMap.remove(leftNode.getPID()); -// Column[] shuffleKeys = (joinColumns != null) ? joinColumns[0] : null; -// Column[] otherSideShuffleKeys = (joinColumns != null) ? joinColumns[1] : null; -// buildJoinPlanWithUnionChannel(context, joinNode, currentBlock, leftBlock, rightBlock, leftNode, -// shuffleKeys, otherSideShuffleKeys, true); -// currentBlock.setPlan(joinNode); -// } else if (!leftUnion && rightUnion) { // if only right is union -// currentBlock = rightBlock; -// context.execBlockMap.remove(rightNode.getPID()); -// Column[] shuffleKeys = (joinColumns != null) ? joinColumns[1] : null; -// Column[] otherSideShuffleKeys = (joinColumns != null) ? joinColumns[0] : null; -// buildJoinPlanWithUnionChannel(context, joinNode, currentBlock, rightBlock, leftBlock, rightNode, -// shuffleKeys, otherSideShuffleKeys, false); -// currentBlock.setPlan(joinNode); -// } else { // if both are unions -// currentBlock = leftBlock; -// context.execBlockMap.remove(leftNode.getPID()); -// context.execBlockMap.remove(rightNode.getPID()); -// buildJoinPlanWithUnionChannel(context, joinNode, currentBlock, leftBlock, null, leftNode, -// (joinColumns != null ? joinColumns[0] : null), null, true); -// buildJoinPlanWithUnionChannel(context, joinNode, currentBlock, rightBlock, null, rightNode, -// (joinColumns != null ? joinColumns[1] : null), null, false); -// currentBlock.setPlan(joinNode); -// } -// -// return currentBlock; -// } else { + boolean leftUnion = leftNode.getType() == NodeType.TABLE_SUBQUERY && + ((TableSubQueryNode)leftNode).getSubQuery().getType() == NodeType.UNION; + boolean rightUnion = rightNode.getType() == NodeType.TABLE_SUBQUERY && + ((TableSubQueryNode)rightNode).getSubQuery().getType() == NodeType.UNION; + + if (leftUnion || rightUnion) { // if one of child execution block is union + /* + Join with tableC and result of union tableA, tableB is expected the following physical plan. + But Union execution block is not necessary. + |-eb_0001_000006 (Terminal) + |-eb_0001_000005 (Join eb_0001_000003, eb_0001_000004) + |-eb_0001_000004 (Scan TableC) + |-eb_0001_000003 (Union TableA, TableB) + |-eb_0001_000002 (Scan TableB) + |-eb_0001_000001 (Scan TableA) + + The above plan can be changed to the following plan. + |-eb_0001_000005 (Terminal) + |-eb_0001_000003 (Join [eb_0001_000001, eb_0001_000002], eb_0001_000004) + |-eb_0001_000004 (Scan TableC) + |-eb_0001_000002 (Scan TableB) + |-eb_0001_000001 (Scan TableA) + + eb_0001_000003's left child should be eb_0001_000001 + eb_0001_000001 and right child should be eb_0001_000004. + For this eb_0001_000001 is representative of eb_0001_000001, eb_0001_000002. + So eb_0001_000003's left child is eb_0001_000001 + */ + Column[][] joinColumns = null; + if (joinNode.getJoinType() != JoinType.CROSS) { + // ShuffleKeys need to not have thea-join condition because Tajo supports only equi-join. + joinColumns = PlannerUtil.joinJoinKeyForEachTable(joinNode.getJoinQual(), + leftNode.getOutSchema(), rightNode.getOutSchema(), false); + } + + if (leftUnion && !rightUnion) { // if only left is union + currentBlock = leftBlock; + context.execBlockMap.remove(leftNode.getPID()); + Column[] shuffleKeys = (joinColumns != null) ? joinColumns[0] : null; + Column[] otherSideShuffleKeys = (joinColumns != null) ? joinColumns[1] : null; + buildJoinPlanWithUnionChannel(context, joinNode, currentBlock, leftBlock, rightBlock, leftNode, + shuffleKeys, otherSideShuffleKeys, true); + currentBlock.setPlan(joinNode); + } else if (!leftUnion && rightUnion) { // if only right is union + currentBlock = rightBlock; + context.execBlockMap.remove(rightNode.getPID()); + Column[] shuffleKeys = (joinColumns != null) ? joinColumns[1] : null; + Column[] otherSideShuffleKeys = (joinColumns != null) ? joinColumns[0] : null; + buildJoinPlanWithUnionChannel(context, joinNode, currentBlock, rightBlock, leftBlock, rightNode, + shuffleKeys, otherSideShuffleKeys, false); + currentBlock.setPlan(joinNode); + } else { // if both are unions + currentBlock = leftBlock; + context.execBlockMap.remove(leftNode.getPID()); + context.execBlockMap.remove(rightNode.getPID()); + buildJoinPlanWithUnionChannel(context, joinNode, currentBlock, leftBlock, null, leftNode, + (joinColumns != null ? joinColumns[0] : null), null, true); + buildJoinPlanWithUnionChannel(context, joinNode, currentBlock, rightBlock, null, rightNode, + (joinColumns != null ? joinColumns[1] : null), null, false); + currentBlock.setPlan(joinNode); + } + + return currentBlock; + } else { // !leftUnion && !rightUnion currentBlock = masterPlan.newExecutionBlock(); DataChannel leftChannel = createDataChannelFromJoin(leftBlock, rightBlock, currentBlock, joinNode, true); @@ -472,7 +472,7 @@ private ExecutionBlock buildJoinPlan(GlobalPlanContext context, JoinNode joinNod masterPlan.addConnect(rightChannel); return currentBlock; -// } + } } private void buildJoinPlanWithUnionChannel(GlobalPlanContext context, JoinNode joinNode, @@ -798,13 +798,13 @@ private ExecutionBlock buildGroupBy(GlobalPlanContext context, ExecutionBlock la boolean multiLevelEnabled = context.getPlan().getContext().getBool(SessionVars.GROUPBY_MULTI_LEVEL_ENABLED); if (multiLevelEnabled) { -// if (PlannerUtil.findTopNode(groupbyNode, NodeType.UNION) == null) { -// DistinctGroupbyBuilder builder = new DistinctGroupbyBuilder(this); -// return builder.buildMultiLevelPlan(context, lastBlock, groupbyNode); -// } else { + if (PlannerUtil.findTopNode(groupbyNode, NodeType.UNION) == null) { + DistinctGroupbyBuilder builder = new DistinctGroupbyBuilder(this); + return builder.buildMultiLevelPlan(context, lastBlock, groupbyNode); + } else { DistinctGroupbyBuilder builder = new DistinctGroupbyBuilder(this); return builder.buildPlan(context, lastBlock, groupbyNode); -// } + } } else { DistinctGroupbyBuilder builder = new DistinctGroupbyBuilder(this); return builder.buildPlan(context, lastBlock, groupbyNode); @@ -812,12 +812,12 @@ private ExecutionBlock buildGroupBy(GlobalPlanContext context, ExecutionBlock la } else { GroupbyNode firstPhaseGroupby = createFirstPhaseGroupBy(masterPlan.getLogicalPlan(), groupbyNode); -// if (hasUnionChild(firstPhaseGroupby)) { -// currentBlock = buildGroupbyAndUnionPlan(masterPlan, lastBlock, firstPhaseGroupby, groupbyNode); -// } else { - // general hash-shuffled aggregation + if (hasUnionChild(firstPhaseGroupby)) { + currentBlock = buildGroupbyAndUnionPlan(masterPlan, lastBlock, firstPhaseGroupby, groupbyNode); + } else { +// general hash-shuffled aggregation currentBlock = buildTwoPhaseGroupby(masterPlan, lastBlock, firstPhaseGroupby, groupbyNode); -// } + } } return currentBlock; @@ -980,26 +980,26 @@ private ExecutionBlock buildSortPlan(GlobalPlanContext context, ExecutionBlock c SortNode firstSortNode = PlannerUtil.clone(context.plan.getLogicalPlan(), currentNode); -// if (firstSortNode.getChild().getType() == NodeType.TABLE_SUBQUERY && -// ((TableSubQueryNode)firstSortNode.getChild()).getSubQuery().getType() == NodeType.UNION) { -// -// currentBlock = childBlock; -// for (DataChannel channel : masterPlan.getIncomingChannels(childBlock.getId())) { -// channel.setShuffle(RANGE_SHUFFLE, PlannerUtil.sortSpecsToSchema(currentNode.getSortKeys()).toArray(), 32); -// channel.setSchema(firstSortNode.getOutSchema()); -// -// ExecutionBlock subBlock = masterPlan.getExecBlock(channel.getSrcId()); -// SortNode s1 = PlannerUtil.clone(context.plan.getLogicalPlan(), firstSortNode); -// s1.setChild(subBlock.getPlan()); -// subBlock.setPlan(s1); -// -// ScanNode secondScan = buildInputExecutor(masterPlan.getLogicalPlan(), channel); -// currentNode.setChild(secondScan); -// currentNode.setInSchema(secondScan.getOutSchema()); -// currentBlock.setPlan(currentNode); -// currentBlock.getEnforcer().addSortedInput(secondScan.getTableName(), currentNode.getSortKeys()); -// } -// } else { + if (firstSortNode.getChild().getType() == NodeType.TABLE_SUBQUERY && + ((TableSubQueryNode)firstSortNode.getChild()).getSubQuery().getType() == NodeType.UNION) { + + currentBlock = childBlock; + for (DataChannel channel : masterPlan.getIncomingChannels(childBlock.getId())) { + channel.setShuffle(RANGE_SHUFFLE, PlannerUtil.sortSpecsToSchema(currentNode.getSortKeys()).toArray(), 32); + channel.setSchema(firstSortNode.getOutSchema()); + + ExecutionBlock subBlock = masterPlan.getExecBlock(channel.getSrcId()); + SortNode s1 = PlannerUtil.clone(context.plan.getLogicalPlan(), firstSortNode); + s1.setChild(subBlock.getPlan()); + subBlock.setPlan(s1); + + ScanNode secondScan = buildInputExecutor(masterPlan.getLogicalPlan(), channel); + currentNode.setChild(secondScan); + currentNode.setInSchema(secondScan.getOutSchema()); + currentBlock.setPlan(currentNode); + currentBlock.getEnforcer().addSortedInput(secondScan.getTableName(), currentNode.getSortKeys()); + } + } else { LogicalNode childBlockPlan = childBlock.getPlan(); firstSortNode.setChild(childBlockPlan); // sort is a non-projectable operator. So, in/out schemas are the same to its child operator. @@ -1018,7 +1018,7 @@ private ExecutionBlock buildSortPlan(GlobalPlanContext context, ExecutionBlock c currentBlock.setPlan(currentNode); currentBlock.getEnforcer().addSortedInput(secondScan.getTableName(), currentNode.getSortKeys()); masterPlan.addConnect(channel); -// } + } return currentBlock; } @@ -1040,11 +1040,11 @@ private ExecutionBlock buildStorePlan(GlobalPlanContext context, partitionMethod.getPartitionType())); } -// if (hasUnionChild(currentNode)) { // if it has union children -// return buildShuffleAndStorePlanToPartitionedTableWithUnion(context, currentNode, lastBlock); -// } else { // otherwise + if (hasUnionChild(currentNode)) { // if it has union children + return buildShuffleAndStorePlanToPartitionedTableWithUnion(context, currentNode, lastBlock); + } else { // otherwise return buildShuffleAndStorePlanToPartitionedTable(context, currentNode, lastBlock); -// } + } } else { // if result table is not a partitioned table, directly store it return buildNoPartitionedStorePlan(context, currentNode, lastBlock); } @@ -1118,14 +1118,14 @@ private ExecutionBlock buildShuffleAndStorePlanToPartitionedTable(GlobalPlanCont private ExecutionBlock buildNoPartitionedStorePlan(GlobalPlanContext context, StoreTableNode currentNode, ExecutionBlock childBlock) { -// if (hasUnionChild(currentNode)) { // when the below is union -// return buildShuffleAndStorePlanNoPartitionedTableWithUnion(context, currentNode, childBlock); -// } else { + if (hasUnionChild(currentNode)) { // when the below is union + return buildShuffleAndStorePlanNoPartitionedTableWithUnion(context, currentNode, childBlock); + } else { currentNode.setChild(childBlock.getPlan()); currentNode.setInSchema(childBlock.getPlan().getOutSchema()); childBlock.setPlan(currentNode); return childBlock; -// } + } } private void setShuffleKeysFromPartitionedTableStore(StoreTableNode node, DataChannel channel) { @@ -1177,25 +1177,25 @@ public LogicalNode visitProjection(GlobalPlanContext context, LogicalPlan plan, ExecutionBlock execBlock = context.execBlockMap.remove(child.getPID()); -// if (child.getType() == NodeType.TABLE_SUBQUERY && -// ((TableSubQueryNode)child).getSubQuery().getType() == NodeType.UNION) { -// MasterPlan masterPlan = context.plan; -// for (DataChannel dataChannel : masterPlan.getIncomingChannels(execBlock.getId())) { -// // This data channel will be stored in staging directory, but RawFile, default file type, does not support -// // distributed file system. It needs to change the file format for distributed file system. -// dataChannel.setStoreType(CatalogProtos.StoreType.CSV); -// ExecutionBlock subBlock = masterPlan.getExecBlock(dataChannel.getSrcId()); -// -// ProjectionNode copy = PlannerUtil.clone(plan, node); -// copy.setChild(subBlock.getPlan()); -// subBlock.setPlan(copy); -// } -// execBlock.setPlan(null); -// } else { + if (child.getType() == NodeType.TABLE_SUBQUERY && + ((TableSubQueryNode)child).getSubQuery().getType() == NodeType.UNION) { + MasterPlan masterPlan = context.plan; + for (DataChannel dataChannel : masterPlan.getIncomingChannels(execBlock.getId())) { + // This data channel will be stored in staging directory, but RawFile, default file type, does not support + // distributed file system. It needs to change the file format for distributed file system. + dataChannel.setStoreType(CatalogProtos.StoreType.CSV); + ExecutionBlock subBlock = masterPlan.getExecBlock(dataChannel.getSrcId()); + + ProjectionNode copy = PlannerUtil.clone(plan, node); + copy.setChild(subBlock.getPlan()); + subBlock.setPlan(copy); + } + execBlock.setPlan(null); + } else { node.setChild(execBlock.getPlan()); node.setInSchema(execBlock.getPlan().getOutSchema()); execBlock.setPlan(node); -// } + } context.execBlockMap.put(node.getPID(), execBlock); return node; @@ -1371,16 +1371,29 @@ public LogicalNode visitJoin(GlobalPlanContext context, LogicalPlan plan, Logica LogicalNode rightChild = visit(context, plan, block, node.getRightChild(), stack); ExecutionBlock rightChildBlock = context.execBlockMap.get(rightChild.getPID()); - // In the case of broadcast join leftChildBlock can be replaced with upper join node. - // So if the current join node is a child node of leftChildBlock's plan(join node) - // the current join node already participates in broadcast join. - LogicalNode leftChildBlockNode = leftChildBlock.getPlan(); - // If child block is union, child block has not plan - if (leftChildBlockNode != null && leftChildBlockNode.getType() == NodeType.JOIN) { - if (leftChildBlockNode.getPID() > node.getPID()) { - context.execBlockMap.put(node.getPID(), leftChildBlock); - return node; - } +// // In the case of broadcast join leftChildBlock can be replaced with upper join node. +// // So if the current join node is a child node of leftChildBlock's plan(join node) +// // the current join node already participates in broadcast join. +// LogicalNode leftChildBlockNode = leftChildBlock.getPlan(); +// // If child block is union, child block has not plan +// if (leftChildBlockNode != null && leftChildBlockNode.getType() == NodeType.JOIN) { +// if (leftChildBlockNode.getPID() > node.getPID()) { +// context.execBlockMap.put(node.getPID(), leftChildBlock); +// return node; +// } +// } + + if (node.getJoinType() == JoinType.LEFT_OUTER) { + leftChildBlock.setPreservedRow(); + rightChildBlock.setNullSuppllying(); + } else if (node.getJoinType() == JoinType.RIGHT_OUTER) { + leftChildBlock.setNullSuppllying(); + rightChildBlock.setPreservedRow(); + } else if (node.getJoinType() == JoinType.FULL_OUTER) { + leftChildBlock.setPreservedRow(); + leftChildBlock.setNullSuppllying(); + rightChildBlock.setPreservedRow(); + rightChildBlock.setNullSuppllying(); } ExecutionBlock newExecBlock = buildJoinPlan(context, node, leftChildBlock, rightChildBlock); @@ -1392,61 +1405,60 @@ public LogicalNode visitJoin(GlobalPlanContext context, LogicalPlan plan, Logica @Override public LogicalNode visitUnion(GlobalPlanContext context, LogicalPlan plan, LogicalPlan.QueryBlock queryBlock, UnionNode node, Stack stack) throws PlanningException { - super.visitUnion(context, plan, queryBlock, node, stack); -// stack.push(node); -// LogicalPlan.QueryBlock leftQueryBlock = plan.getBlock(node.getLeftChild()); -// LogicalNode leftChild = visit(context, plan, leftQueryBlock, leftQueryBlock.getRoot(), stack); -// -// LogicalPlan.QueryBlock rightQueryBlock = plan.getBlock(node.getRightChild()); -// LogicalNode rightChild = visit(context, plan, rightQueryBlock, rightQueryBlock.getRoot(), stack); -// stack.pop(); -// -// MasterPlan masterPlan = context.getPlan(); -// -// List unionBlocks = Lists.newArrayList(); -// List queryBlockBlocks = Lists.newArrayList(); -// -// ExecutionBlock leftBlock = context.execBlockMap.remove(leftChild.getPID()); -// ExecutionBlock rightBlock = context.execBlockMap.remove(rightChild.getPID()); -// -// // These union types need to eliminate unnecessary nodes between parent and child node of query tree. -// boolean leftUnion = (leftChild.getType() == NodeType.UNION) || -// ((leftChild.getType() == NodeType.TABLE_SUBQUERY) && -// (((TableSubQueryNode)leftChild).getSubQuery().getType() == NodeType.UNION)); -// boolean rightUnion = (rightChild.getType() == NodeType.UNION) || -// (rightChild.getType() == NodeType.TABLE_SUBQUERY) && -// (((TableSubQueryNode)rightChild).getSubQuery().getType() == NodeType.UNION); -// if (leftUnion) { -// unionBlocks.add(leftBlock); -// } else { -// queryBlockBlocks.add(leftBlock); -// } -// if (rightUnion) { -// unionBlocks.add(rightBlock); -// } else { -// queryBlockBlocks.add(rightBlock); -// } -// + stack.push(node); + LogicalPlan.QueryBlock leftQueryBlock = plan.getBlock(node.getLeftChild()); + LogicalNode leftChild = visit(context, plan, leftQueryBlock, leftQueryBlock.getRoot(), stack); + + LogicalPlan.QueryBlock rightQueryBlock = plan.getBlock(node.getRightChild()); + LogicalNode rightChild = visit(context, plan, rightQueryBlock, rightQueryBlock.getRoot(), stack); + stack.pop(); + + MasterPlan masterPlan = context.getPlan(); + + List unionBlocks = Lists.newArrayList(); + List queryBlockBlocks = Lists.newArrayList(); + + ExecutionBlock leftBlock = context.execBlockMap.remove(leftChild.getPID()); + ExecutionBlock rightBlock = context.execBlockMap.remove(rightChild.getPID()); + + // These union types need to eliminate unnecessary nodes between parent and child node of query tree. + boolean leftUnion = (leftChild.getType() == NodeType.UNION) || + ((leftChild.getType() == NodeType.TABLE_SUBQUERY) && + (((TableSubQueryNode)leftChild).getSubQuery().getType() == NodeType.UNION)); + boolean rightUnion = (rightChild.getType() == NodeType.UNION) || + (rightChild.getType() == NodeType.TABLE_SUBQUERY) && + (((TableSubQueryNode)rightChild).getSubQuery().getType() == NodeType.UNION); + if (leftUnion) { + unionBlocks.add(leftBlock); + } else { + queryBlockBlocks.add(leftBlock); + } + if (rightUnion) { + unionBlocks.add(rightBlock); + } else { + queryBlockBlocks.add(rightBlock); + } + ExecutionBlock execBlock; -// if (unionBlocks.size() == 0) { + if (unionBlocks.size() == 0) { execBlock = context.plan.newExecutionBlock(); -// } else { -// execBlock = unionBlocks.get(0); -// } -// -// for (ExecutionBlock childBlocks : unionBlocks) { -// for (ExecutionBlock grandChildBlock : masterPlan.getChilds(childBlocks)) { -// masterPlan.disconnect(grandChildBlock, childBlocks); -// queryBlockBlocks.add(grandChildBlock); -// } -// } -// -// for (ExecutionBlock childBlocks : queryBlockBlocks) { -// DataChannel channel = new DataChannel(childBlocks, execBlock, NONE_SHUFFLE, 1); -// channel.setStoreType(storeType); -// masterPlan.addConnect(channel); -// } -// + } else { + execBlock = unionBlocks.get(0); + } + + for (ExecutionBlock childBlocks : unionBlocks) { + for (ExecutionBlock grandChildBlock : masterPlan.getChilds(childBlocks)) { + masterPlan.disconnect(grandChildBlock, childBlocks); + queryBlockBlocks.add(grandChildBlock); + } + } + + for (ExecutionBlock childBlocks : queryBlockBlocks) { + DataChannel channel = new DataChannel(childBlocks, execBlock, NONE_SHUFFLE, 1); + channel.setStoreType(storeType); + masterPlan.addConnect(channel); + } + context.execBlockMap.put(node.getPID(), execBlock); return node; @@ -1483,74 +1495,74 @@ public LogicalNode visitTableSubQuery(GlobalPlanContext context, LogicalPlan pla ExecutionBlock currentBlock = context.execBlockMap.remove(child.getPID()); -// if (child.getType() == NodeType.UNION) { -// List addedTableSubQueries = new ArrayList(); -// TableSubQueryNode leftMostSubQueryNode = null; -// for (ExecutionBlock childBlock : context.plan.getChilds(currentBlock.getId())) { -// TableSubQueryNode copy = PlannerUtil.clone(plan, node); -// copy.setSubQuery(childBlock.getPlan()); -// childBlock.setPlan(copy); -// addedTableSubQueries.add(copy); -// -// //Find a SubQueryNode which contains all columns in InputSchema matched with Target and OutputSchema's column -// if (copy.getInSchema().containsAll(copy.getOutSchema().getColumns())) { -// for (Target eachTarget : copy.getTargets()) { -// Set columns = EvalTreeUtil.findUniqueColumns(eachTarget.getEvalTree()); -// if (copy.getInSchema().containsAll(columns)) { -// leftMostSubQueryNode = copy; -// break; -// } -// } -// } -// } -// if (leftMostSubQueryNode != null) { -// // replace target column name -// Target[] targets = leftMostSubQueryNode.getTargets(); -// int[] targetMappings = new int[targets.length]; -// for (int i = 0; i < targets.length; i++) { -// if (targets[i].getEvalTree().getType() != EvalType.FIELD) { -// throw new PlanningException("Target of a UnionNode's subquery should be FieldEval."); -// } -// int index = leftMostSubQueryNode.getInSchema().getColumnId(targets[i].getNamedColumn().getQualifiedName()); -// if (index < 0) { -// // If a target has alias, getNamedColumn() only returns alias -// Set columns = EvalTreeUtil.findUniqueColumns(targets[i].getEvalTree()); -// Column column = columns.iterator().next(); -// index = leftMostSubQueryNode.getInSchema().getColumnId(column.getQualifiedName()); -// } -// if (index < 0) { -// throw new PlanningException("Can't find matched Target in UnionNode's input schema: " + targets[i] -// + "->" + leftMostSubQueryNode.getInSchema()); -// } -// targetMappings[i] = index; -// } -// -// for (TableSubQueryNode eachNode: addedTableSubQueries) { -// if (eachNode.getPID() == leftMostSubQueryNode.getPID()) { -// continue; -// } -// Target[] eachNodeTargets = eachNode.getTargets(); -// if (eachNodeTargets.length != targetMappings.length) { -// throw new PlanningException("Union query can't have different number of target columns."); -// } -// for (int i = 0; i < eachNodeTargets.length; i++) { -// Column inColumn = eachNode.getInSchema().getColumn(targetMappings[i]); -// eachNodeTargets[i].setAlias(eachNodeTargets[i].getNamedColumn().getQualifiedName()); -// EvalNode evalNode = eachNodeTargets[i].getEvalTree(); -// if (evalNode.getType() != EvalType.FIELD) { -// throw new PlanningException("Target of a UnionNode's subquery should be FieldEval."); -// } -// FieldEval fieldEval = (FieldEval) evalNode; -// EvalTreeUtil.changeColumnRef(fieldEval, -// fieldEval.getColumnRef().getQualifiedName(), inColumn.getQualifiedName()); -// } -// } -// } else { -// LOG.warn("Can't find left most SubQuery in the UnionNode."); -// } -// } else { + if (child.getType() == NodeType.UNION) { + List addedTableSubQueries = new ArrayList(); + TableSubQueryNode leftMostSubQueryNode = null; + for (ExecutionBlock childBlock : context.plan.getChilds(currentBlock.getId())) { + TableSubQueryNode copy = PlannerUtil.clone(plan, node); + copy.setSubQuery(childBlock.getPlan()); + childBlock.setPlan(copy); + addedTableSubQueries.add(copy); + + //Find a SubQueryNode which contains all columns in InputSchema matched with Target and OutputSchema's column + if (copy.getInSchema().containsAll(copy.getOutSchema().getColumns())) { + for (Target eachTarget : copy.getTargets()) { + Set columns = EvalTreeUtil.findUniqueColumns(eachTarget.getEvalTree()); + if (copy.getInSchema().containsAll(columns)) { + leftMostSubQueryNode = copy; + break; + } + } + } + } + if (leftMostSubQueryNode != null) { + // replace target column name + Target[] targets = leftMostSubQueryNode.getTargets(); + int[] targetMappings = new int[targets.length]; + for (int i = 0; i < targets.length; i++) { + if (targets[i].getEvalTree().getType() != EvalType.FIELD) { + throw new PlanningException("Target of a UnionNode's subquery should be FieldEval."); + } + int index = leftMostSubQueryNode.getInSchema().getColumnId(targets[i].getNamedColumn().getQualifiedName()); + if (index < 0) { + // If a target has alias, getNamedColumn() only returns alias + Set columns = EvalTreeUtil.findUniqueColumns(targets[i].getEvalTree()); + Column column = columns.iterator().next(); + index = leftMostSubQueryNode.getInSchema().getColumnId(column.getQualifiedName()); + } + if (index < 0) { + throw new PlanningException("Can't find matched Target in UnionNode's input schema: " + targets[i] + + "->" + leftMostSubQueryNode.getInSchema()); + } + targetMappings[i] = index; + } + + for (TableSubQueryNode eachNode: addedTableSubQueries) { + if (eachNode.getPID() == leftMostSubQueryNode.getPID()) { + continue; + } + Target[] eachNodeTargets = eachNode.getTargets(); + if (eachNodeTargets.length != targetMappings.length) { + throw new PlanningException("Union query can't have different number of target columns."); + } + for (int i = 0; i < eachNodeTargets.length; i++) { + Column inColumn = eachNode.getInSchema().getColumn(targetMappings[i]); + eachNodeTargets[i].setAlias(eachNodeTargets[i].getNamedColumn().getQualifiedName()); + EvalNode evalNode = eachNodeTargets[i].getEvalTree(); + if (evalNode.getType() != EvalType.FIELD) { + throw new PlanningException("Target of a UnionNode's subquery should be FieldEval."); + } + FieldEval fieldEval = (FieldEval) evalNode; + EvalTreeUtil.changeColumnRef(fieldEval, + fieldEval.getColumnRef().getQualifiedName(), inColumn.getQualifiedName()); + } + } + } else { + LOG.warn("Can't find left most SubQuery in the UnionNode."); + } + } else { currentBlock.setPlan(node); -// } + } context.execBlockMap.put(node.getPID(), currentBlock); return node; } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/BaseGlobalPlanRewriteRuleProvider.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/BaseGlobalPlanRewriteRuleProvider.java index 8a0efbb63e..6d922b1ed0 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/BaseGlobalPlanRewriteRuleProvider.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/BaseGlobalPlanRewriteRuleProvider.java @@ -21,7 +21,7 @@ import com.google.common.collect.Lists; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.engine.planner.global.rewriter.rules.BroadcastJoinRule; -import org.apache.tajo.engine.planner.global.rewriter.rules.UnionReduceRule; +//import org.apache.tajo.engine.planner.global.rewriter.rules.UnionReduceRule; import org.apache.tajo.util.TUtil; import java.util.Collection; @@ -38,8 +38,8 @@ public BaseGlobalPlanRewriteRuleProvider(TajoConf conf) { @Override public Collection> getRules() { List> rules = Lists.newArrayList(); - rules.add(UnionReduceRule.class); -// rules.add(BroadcastJoinRule.class); +// rules.add(UnionReduceRule.class); + rules.add(BroadcastJoinRule.class); return rules; } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java index a581e83fce..6c3ececa8c 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java @@ -18,26 +18,24 @@ package org.apache.tajo.engine.planner.global.rewriter.rules; +import org.apache.tajo.ExecutionBlockId; import org.apache.tajo.OverridableConf; import org.apache.tajo.SessionVars; -import org.apache.tajo.engine.planner.global.DataChannel; import org.apache.tajo.engine.planner.global.ExecutionBlock; import org.apache.tajo.engine.planner.global.MasterPlan; import org.apache.tajo.engine.planner.global.rewriter.GlobalPlanRewriteRule; -import org.apache.tajo.ipc.TajoWorkerProtocol.EnforceProperty.EnforceType; import org.apache.tajo.plan.LogicalPlan; import org.apache.tajo.plan.PlanningException; -import org.apache.tajo.plan.expr.AggregationFunctionCallEval; import org.apache.tajo.plan.logical.*; import org.apache.tajo.plan.util.PlannerUtil; import org.apache.tajo.util.TUtil; -import java.util.Collection; -import java.util.List; +import java.util.*; public class BroadcastJoinRule implements GlobalPlanRewriteRule { private long broadcastTableSizeThreshold; private GlobalPlanRewriteUtil.ParentFinder parentFinder; + private RelationSizeComparator relSizeComparator; @Override public String getName() { @@ -53,6 +51,7 @@ public boolean isEligible(OverridableConf queryContext, MasterPlan plan) { if (broadcastTableSizeThreshold > 0) { if (parentFinder == null) { parentFinder = new GlobalPlanRewriteUtil.ParentFinder(); + relSizeComparator = new RelationSizeComparator(); } return true; } @@ -69,8 +68,16 @@ public MasterPlan rewrite(MasterPlan plan) throws PlanningException{ return plan; } + private static class RelationSizeComparator implements Comparator { + + @Override + public int compare(ScanNode o1, ScanNode o2) { + return (int) (GlobalPlanRewriteUtil.getTableVolume(o1) - GlobalPlanRewriteUtil.getTableVolume(o2)); + } + } + private void rewrite(MasterPlan plan, ExecutionBlock current) throws PlanningException { - if (plan.isLeaf(current)) { + if (plan.isLeaf(current) && !current.isPreservedRow()) { // in leaf execution blocks, find input tables which size is less than the predefined threshold. for (ScanNode scanNode : current.getScanNodes()) { if (GlobalPlanRewriteUtil.getTableVolume(scanNode) <= broadcastTableSizeThreshold) { @@ -83,48 +90,97 @@ private void rewrite(MasterPlan plan, ExecutionBlock current) throws PlanningExc rewrite(plan, child); } if (!plan.isTerminal(current) && current.hasJoin()) { - // unioned scans should be handled as a single relation scan + // TODO: handle unioned scans as a single scan + List broadcastCandidates = TUtil.newList(); + List childs = plan.getChilds(current); + Map relationBlockMap = TUtil.newHashMap(); + + // find all broadcast candidates from every child + for (ExecutionBlock child : childs) { + Map scanNodeMap = TUtil.newHashMap(); + for (ScanNode scanNode : child.getScanNodes()) { + scanNodeMap.put(scanNode.getCanonicalName(), scanNode); + } + for (String relName : child.getBroadcastTables()) { + broadcastCandidates.add(scanNodeMap.get(relName)); + relationBlockMap.put(scanNodeMap.get(relName), child); + } + } + Collections.sort(broadcastCandidates, relSizeComparator); - // check outer join - if (hasOuterJoin(current)) { - // find and enforce shuffle for row-preserved tables - + // Enforce broadcast for candidates in ascending order of relation size + long totalBroadcastVolume = 0; + int i, broadcastEnd = -1; + for (i = 0; i < broadcastCandidates.size(); i++) { + long volumeOfCandidate = GlobalPlanRewriteUtil.getTableVolume(broadcastCandidates.get(i)); + if (totalBroadcastVolume + volumeOfCandidate > broadcastTableSizeThreshold) { + broadcastEnd = i; + break; + } + totalBroadcastVolume += volumeOfCandidate; + } + if (i == broadcastCandidates.size()) { + broadcastEnd = i-1; } - // check the total input size + for (; i < broadcastCandidates.size(); i++) { + ScanNode nonBroadcast = broadcastCandidates.get(i); + ExecutionBlock enforceShuffleBlock = relationBlockMap.get(nonBroadcast); + enforceShuffleBlock.removeBroadcastRelation(nonBroadcast.getTableName()); + } // check all inputs are marked as broadcast + if (current.getUnionScanMap() != null && !current.getUnionScanMap().isEmpty()) { - ExecutionBlock enforceNonBroadcast = null; - ExecutionBlock broadcastCandidate = null; - long smallestChildVolume = Long.MAX_VALUE; - List childs = plan.getChilds(current); - for (ExecutionBlock child : childs) { - if (child.isBroadcastable(broadcastTableSizeThreshold)) { - long inputVolume = GlobalPlanRewriteUtil.getInputVolume(child); - if (smallestChildVolume > inputVolume) { - smallestChildVolume = inputVolume; - if (broadcastCandidate != null) { - enforceNonBroadcast = broadcastCandidate; - } - broadcastCandidate = child; - } - } } - if (broadcastCandidate != null) { - if (enforceNonBroadcast != null) { - List tables = TUtil.newList(enforceNonBroadcast.getBroadcastTables()); - for (String broadcastTable : tables) { -// enforceNonBroadcast.removeBroadcastRelation(broadcastTable); - // TODO: remove the largest rel from broadcast when all inputs are broadcast - } - } - for (ExecutionBlock child : childs) { + + Map unionScanMap; + if (current.getUnionScanMap() == null) { + unionScanMap = TUtil.newHashMap(); + } else { + unionScanMap = current.getUnionScanMap(); + } + + for (ExecutionBlock child : childs) { + if (child.hasBroadcastRelation()) { mergeTwoPhaseJoin(plan, child, current); } } + +// for (i = 0; i <= broadcastEnd; i++) { +// ExecutionBlock willBeMergedChild = relationBlockMap.get(broadcastCandidates.get(i)); +// mergeTwoPhaseJoin(plan, willBeMergedChild, current); +// } + + +// ExecutionBlock enforceNonBroadcast = null; +// ExecutionBlock broadcastCandidate = null; +// long smallestChildVolume = Long.MAX_VALUE; +// +// for (ExecutionBlock child : childs) { +// if (child.isBroadcastable(broadcastTableSizeThreshold)) { +// long inputVolume = GlobalPlanRewriteUtil.getInputVolume(child); +// if (smallestChildVolume > inputVolume) { +// smallestChildVolume = inputVolume; +// if (broadcastCandidate != null) { +// enforceNonBroadcast = broadcastCandidate; +// } +// broadcastCandidate = child; +// } +// } +// } +// if (broadcastCandidate != null) { +// if (enforceNonBroadcast != null) { +// List tables = TUtil.newList(enforceNonBroadcast.getBroadcastTables()); +// for (String broadcastTable : tables) { +//// enforceNonBroadcast.removeBroadcastRelation(broadcastTable); +// // TODO: remove the largest rel from broadcast when all inputs are broadcast +// } +// } +// +// } } } } @@ -138,6 +194,20 @@ private static boolean hasOuterJoin(ExecutionBlock block) { return false; } + private ExecutionBlock mergeTwoPhaseJoinWithUnionChild(MasterPlan plan, Collection childs, + ExecutionBlock parent) { + +// ExecutionBlockId scanEbId = parent.getUnionScanMap().get(child.getId()); +// if (scanEbId != null) { +// scanForChild = GlobalPlanRewriteUtil.findScanForChildEb(plan.getExecBlock(scanEbId), parent); +// } +// } +// if (scanForChild == null) { +// } + + + } + /** * Merge child execution blocks. * diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanRewriteUtil.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanRewriteUtil.java index 79794409b5..c6cb2a5ffc 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanRewriteUtil.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanRewriteUtil.java @@ -24,6 +24,7 @@ import org.apache.tajo.plan.PlanningException; import org.apache.tajo.plan.logical.*; +import java.util.Collection; import java.util.List; public class GlobalPlanRewriteUtil { @@ -96,6 +97,17 @@ public static long getTableVolume(ScanNode scanNode) { return scanBytes; } + public static long getBroadcastInputVolume(ExecutionBlock block) { + Collection broadcastRelations = block.getBroadcastTables(); + long volume = 0; + for (ScanNode scanNode : block.getScanNodes()) { + if (broadcastRelations.contains(scanNode.getCanonicalName())) { + volume += getTableVolume(scanNode); + } + } + return volume; + } + public static long getInputVolume(ExecutionBlock block) { long volume = 0; for (ScanNode scanNode : block.getScanNodes()) { diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/UnionReduceRule.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/UnionReduceRule.java index 30b23cf665..45e9ff07c0 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/UnionReduceRule.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/UnionReduceRule.java @@ -1,105 +1,105 @@ -/** - * 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.global.rewriter.rules; - -import org.apache.tajo.ExecutionBlockId; -import org.apache.tajo.OverridableConf; -import org.apache.tajo.engine.planner.global.ExecutionBlock; -import org.apache.tajo.engine.planner.global.MasterPlan; -import org.apache.tajo.engine.planner.global.rewriter.GlobalPlanRewriteRule; -import org.apache.tajo.plan.LogicalPlan; -import org.apache.tajo.plan.PlanningException; -import org.apache.tajo.plan.logical.LogicalNode; -import org.apache.tajo.plan.logical.NodeType; -import org.apache.tajo.plan.logical.UnionNode; -import org.apache.tajo.plan.util.PlannerUtil; -import org.apache.tajo.util.TUtil; -import org.apache.tajo.util.graph.DirectedGraphVisitor; - -import java.util.List; -import java.util.Map; -import java.util.Stack; - -public class UnionReduceRule implements GlobalPlanRewriteRule { - - @Override - public String getName() { - return "UnionReduceRule"; - } - - @Override - public boolean isEligible(OverridableConf queryContext, MasterPlan plan) { - for (LogicalPlan.QueryBlock block : plan.getLogicalPlan().getQueryBlocks()) { - if (block.hasNode(NodeType.UNION)) { - return true; - } - } - return false; - } - - @Override - public MasterPlan rewrite(MasterPlan plan) throws PlanningException { - Rewriter.rewrite(plan); - return plan; - } - - static class Rewriter implements DirectedGraphVisitor { - private static Rewriter instance; - private static MasterPlan plan; - - private Rewriter() {} - - public static void rewrite(MasterPlan plan) { - if (instance == null) { - instance = new Rewriter(); - } - instance.plan = plan; - instance.visit(new Stack(), plan.getTerminalBlock().getId()); - } - - @Override - public void visit(Stack stack, ExecutionBlockId executionBlockId) { - // must have the form of - /* - parent_op - | - union - / \ - child_op child_op - */ - ExecutionBlock current = instance.plan.getExecBlock(executionBlockId); - if (current.hasUnion()) { - Map newBlocks = TUtil.newHashMap(); - List childBlocks = instance.plan.getChilds(current); - // create new execution blocks for each child_op - // TODO: consider union sequence - UnionNode unionNode = PlannerUtil.findTopNode(current.getPlan(), NodeType.UNION); - LogicalNode parentOfUnion = PlannerUtil.findTopParentNode(current.getPlan(), NodeType.UNION); - newBlocks.put(unionNode.getLeftChild().getPID(), instance.plan.newExecutionBlock()); - newBlocks.put(unionNode.getRightChild().getPID(), instance.plan.newExecutionBlock()); - - // push parent_op as the parent of each child_op and remove union - - - // connect new execution blocks and parents of the current block - - } - } - } -} +///** +// * 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.global.rewriter.rules; +// +//import org.apache.tajo.ExecutionBlockId; +//import org.apache.tajo.OverridableConf; +//import org.apache.tajo.engine.planner.global.ExecutionBlock; +//import org.apache.tajo.engine.planner.global.MasterPlan; +//import org.apache.tajo.engine.planner.global.rewriter.GlobalPlanRewriteRule; +//import org.apache.tajo.plan.LogicalPlan; +//import org.apache.tajo.plan.PlanningException; +//import org.apache.tajo.plan.logical.LogicalNode; +//import org.apache.tajo.plan.logical.NodeType; +//import org.apache.tajo.plan.logical.UnionNode; +//import org.apache.tajo.plan.util.PlannerUtil; +//import org.apache.tajo.util.TUtil; +//import org.apache.tajo.util.graph.DirectedGraphVisitor; +// +//import java.util.List; +//import java.util.Map; +//import java.util.Stack; +// +//public class UnionReduceRule implements GlobalPlanRewriteRule { +// +// @Override +// public String getName() { +// return "UnionReduceRule"; +// } +// +// @Override +// public boolean isEligible(OverridableConf queryContext, MasterPlan plan) { +// for (LogicalPlan.QueryBlock block : plan.getLogicalPlan().getQueryBlocks()) { +// if (block.hasNode(NodeType.UNION)) { +// return true; +// } +// } +// return false; +// } +// +// @Override +// public MasterPlan rewrite(MasterPlan plan) throws PlanningException { +// Rewriter.rewrite(plan); +// return plan; +// } +// +// static class Rewriter implements DirectedGraphVisitor { +// private static Rewriter instance; +// private static MasterPlan plan; +// +// private Rewriter() {} +// +// public static void rewrite(MasterPlan plan) { +// if (instance == null) { +// instance = new Rewriter(); +// } +// instance.plan = plan; +// instance.visit(new Stack(), plan.getTerminalBlock().getId()); +// } +// +// @Override +// public void visit(Stack stack, ExecutionBlockId executionBlockId) { +// // must have the form of +// /* +// parent_op +// | +// union +// / \ +// child_op child_op +// */ +// ExecutionBlock current = instance.plan.getExecBlock(executionBlockId); +// if (current.hasUnion()) { +// Map newBlocks = TUtil.newHashMap(); +// List childBlocks = instance.plan.getChilds(current); +// // create new execution blocks for each child_op +// // TODO: consider union sequence +// UnionNode unionNode = PlannerUtil.findTopNode(current.getPlan(), NodeType.UNION); +// LogicalNode parentOfUnion = PlannerUtil.findTopParentNode(current.getPlan(), NodeType.UNION); +// newBlocks.put(unionNode.getLeftChild().getPID(), instance.plan.newExecutionBlock()); +// newBlocks.put(unionNode.getRightChild().getPID(), instance.plan.newExecutionBlock()); +// +// // push parent_op as the parent of each child_op and remove union +// +// +// // connect new execution blocks and parents of the current block +// +// } +// } +// } +//} diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java index 4a42050c04..a697d5d0c5 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java @@ -835,7 +835,7 @@ public final void testPythonUdaf3() throws Exception { cleanupQuery(res); } - // TODO: this test cannot be executed due to the bug of logical planner + // TODO: this test cannot be executed due to the bug of logical planner (TAJO-1588) // @Test public final void testPythonUdafWithHaving() throws Exception { ResultSet res = executeQuery(); @@ -850,7 +850,7 @@ public final void testPythonUdafWithNullData() throws Exception { cleanupQuery(res); } - // TODO: this test cannot be executed due to the bug of logical planner + // TODO: this test cannot be executed due to the bug of logical planner (TAJO-1588) // @Test public final void testComplexTargetWithPythonUdaf() throws Exception { ResultSet res = executeQuery(); @@ -858,16 +858,11 @@ public final void testComplexTargetWithPythonUdaf() throws Exception { cleanupQuery(res); } - // TODO: this test cannot be executed due to the bug of logical planner + // TODO: this test cannot be executed due to the bug of logical planner (TAJO-1588) // @Test public final void testDistinctPythonUdafWithUnion1() throws Exception { ResultSet res = executeQuery(); assertResultSet(res); cleanupQuery(res); } - - @Test - public final void test() throws Exception { - executeString("select count(*) from (select * from nation union select * from nation) t"); - } } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/logical/JoinNode.java b/tajo-plan/src/main/java/org/apache/tajo/plan/logical/JoinNode.java index bad8704b9e..8d65cf55e5 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/logical/JoinNode.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/logical/JoinNode.java @@ -41,8 +41,8 @@ public class JoinNode extends BinaryNode implements Projectable, Cloneable { @Expose private Target[] targets; // transition states - private boolean candidateBroadcast = false; - private List broadcastCandidateTargets = new ArrayList(); +// private boolean candidateBroadcast = false; +// private List broadcastCandidateTargets = new ArrayList(); public JoinNode(int pid) { super(pid, NodeType.JOIN); @@ -54,17 +54,17 @@ public void init(JoinType joinType, LogicalNode left, LogicalNode right) { setRightChild(right); } - public boolean isCandidateBroadcast() { - return candidateBroadcast; - } +// public boolean isCandidateBroadcast() { +// return candidateBroadcast; +// } - public void setCandidateBroadcast(boolean candidateBroadcast) { - this.candidateBroadcast = candidateBroadcast; - } +// public void setCandidateBroadcast(boolean candidateBroadcast) { +// this.candidateBroadcast = candidateBroadcast; +// } - public List getBroadcastCandidateTargets() { - return broadcastCandidateTargets; - } +// public List getBroadcastCandidateTargets() { +// return broadcastCandidateTargets; +// } public JoinType getJoinType() { return this.joinType; @@ -131,7 +131,7 @@ public PlanString getPlanString() { public int hashCode() { final int prime = 31; int result = 1; - result = prime * result + (candidateBroadcast ? 1231 : 1237); +// result = prime * result + (candidateBroadcast ? 1231 : 1237); result = prime * result + ((joinQual == null) ? 0 : joinQual.hashCode()); result = prime * result + ((joinType == null) ? 0 : joinType.hashCode()); result = prime * result + Arrays.hashCode(targets); From a61f7eaf08225b65f371997e4c7df7555b811513 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 11 May 2015 10:47:55 +0900 Subject: [PATCH 23/37] TAJO-1553 --- .../rewriter/rules/BroadcastJoinRule.java | 40 ++++++------------- 1 file changed, 12 insertions(+), 28 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java index 6c3ececa8c..fabb787b41 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java @@ -129,20 +129,7 @@ private void rewrite(MasterPlan plan, ExecutionBlock current) throws PlanningExc enforceShuffleBlock.removeBroadcastRelation(nonBroadcast.getTableName()); } - // check all inputs are marked as broadcast - - - if (current.getUnionScanMap() != null && !current.getUnionScanMap().isEmpty()) { - - } - - Map unionScanMap; - if (current.getUnionScanMap() == null) { - unionScanMap = TUtil.newHashMap(); - } else { - unionScanMap = current.getUnionScanMap(); - } - + // TODO: check all inputs are marked as broadcast for (ExecutionBlock child : childs) { if (child.hasBroadcastRelation()) { mergeTwoPhaseJoin(plan, child, current); @@ -194,20 +181,6 @@ private static boolean hasOuterJoin(ExecutionBlock block) { return false; } - private ExecutionBlock mergeTwoPhaseJoinWithUnionChild(MasterPlan plan, Collection childs, - ExecutionBlock parent) { - -// ExecutionBlockId scanEbId = parent.getUnionScanMap().get(child.getId()); -// if (scanEbId != null) { -// scanForChild = GlobalPlanRewriteUtil.findScanForChildEb(plan.getExecBlock(scanEbId), parent); -// } -// } -// if (scanForChild == null) { -// } - - - } - /** * Merge child execution blocks. * @@ -219,6 +192,17 @@ private ExecutionBlock mergeTwoPhaseJoinWithUnionChild(MasterPlan plan, Collecti private ExecutionBlock mergeTwoPhaseJoin(MasterPlan plan, ExecutionBlock child, ExecutionBlock parent) throws PlanningException { ScanNode scanForChild = GlobalPlanRewriteUtil.findScanForChildEb(child, parent); + if (scanForChild == null) { + if (parent.getUnionScanMap() != null && !parent.getUnionScanMap().isEmpty()) { + ExecutionBlockId scanEbId = parent.getUnionScanMap().get(child.getId()); + if (scanEbId != null) { + ExecutionBlock scanEb = plan.getExecBlock(scanEbId); + if (scanEb != null) { + scanForChild = GlobalPlanRewriteUtil.findScanForChildEb(scanEb, parent); + } + } + } + } if (scanForChild == null) { throw new PlanningException("Cannot find any scan nodes for " + child.getId() + " in " + parent.getId()); } From cd5fc5a94310b70ee32f89a142373057078fdf9d Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 11 May 2015 15:33:24 +0900 Subject: [PATCH 24/37] TAJO-1553 --- .../rewriter/rules/BroadcastJoinRule.java | 58 +++++++++++++++---- 1 file changed, 47 insertions(+), 11 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java index fabb787b41..08e20e74a4 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java @@ -22,6 +22,7 @@ import org.apache.tajo.OverridableConf; import org.apache.tajo.SessionVars; import org.apache.tajo.engine.planner.global.ExecutionBlock; +import org.apache.tajo.engine.planner.global.GlobalPlanner; import org.apache.tajo.engine.planner.global.MasterPlan; import org.apache.tajo.engine.planner.global.rewriter.GlobalPlanRewriteRule; import org.apache.tajo.plan.LogicalPlan; @@ -94,6 +95,10 @@ private void rewrite(MasterPlan plan, ExecutionBlock current) throws PlanningExc List broadcastCandidates = TUtil.newList(); List childs = plan.getChilds(current); Map relationBlockMap = TUtil.newHashMap(); + Map unionScanMap = current.getUnionScanMap(); + if (unionScanMap == null) { + unionScanMap = TUtil.newHashMap(); + } // find all broadcast candidates from every child for (ExecutionBlock child : childs) { @@ -132,6 +137,37 @@ private void rewrite(MasterPlan plan, ExecutionBlock current) throws PlanningExc // TODO: check all inputs are marked as broadcast for (ExecutionBlock child : childs) { if (child.hasBroadcastRelation()) { + List unionScans = TUtil.newList(); + ExecutionBlockId representativeId = null; + if (unionScanMap.containsKey(child.getId())) { + representativeId = unionScanMap.get(child.getId()); + } else if (unionScanMap.containsValue(child.getId())) { + representativeId = child.getId(); + } + + if (representativeId != null) { + for (Map.Entry entry : unionScanMap.entrySet()) { + if (entry.getValue().equals(representativeId.getId())) { + unionScans.add(entry.getKey()); + } + } + + // add unions + LogicalNode left, topUnion = null; + left = GlobalPlanner.buildInputExecutor(plan.getLogicalPlan(), plan.getChannel(unionScans.get(0), current.getId())); + for (i = 1; i < unionScans.size(); i++) { + // left must not be null + UnionNode unionNode = plan.getLogicalPlan().createNode(UnionNode.class); + unionNode.setLeftChild(left); + unionNode.setRightChild(GlobalPlanner.buildInputExecutor(plan.getLogicalPlan(), plan.getChannel(unionScans.get(i), current.getId()))); + topUnion = unionNode; + left = unionNode; + } + + ScanNode scanForChild = GlobalPlanRewriteUtil.findScanForChildEb(plan.getExecBlock(representativeId), current); + PlannerUtil.replaceNode(plan.getLogicalPlan(), current.getPlan(), scanForChild, topUnion); + } + mergeTwoPhaseJoin(plan, child, current); } } @@ -192,17 +228,17 @@ private static boolean hasOuterJoin(ExecutionBlock block) { private ExecutionBlock mergeTwoPhaseJoin(MasterPlan plan, ExecutionBlock child, ExecutionBlock parent) throws PlanningException { ScanNode scanForChild = GlobalPlanRewriteUtil.findScanForChildEb(child, parent); - if (scanForChild == null) { - if (parent.getUnionScanMap() != null && !parent.getUnionScanMap().isEmpty()) { - ExecutionBlockId scanEbId = parent.getUnionScanMap().get(child.getId()); - if (scanEbId != null) { - ExecutionBlock scanEb = plan.getExecBlock(scanEbId); - if (scanEb != null) { - scanForChild = GlobalPlanRewriteUtil.findScanForChildEb(scanEb, parent); - } - } - } - } +// if (scanForChild == null) { +// if (parent.getUnionScanMap() != null && !parent.getUnionScanMap().isEmpty()) { +// ExecutionBlockId scanEbId = parent.getUnionScanMap().get(child.getId()); +// if (scanEbId != null) { +// ExecutionBlock scanEb = plan.getExecBlock(scanEbId); +// if (scanEb != null) { +// scanForChild = GlobalPlanRewriteUtil.findScanForChildEb(scanEb, parent); +// } +// } +// } +// } if (scanForChild == null) { throw new PlanningException("Cannot find any scan nodes for " + child.getId() + " in " + parent.getId()); } From 81c8bc4605138cae91c5b69c4d8f0dd1b79dd973 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 11 May 2015 16:40:42 +0900 Subject: [PATCH 25/37] TAJO-1553 --- .../tajo/engine/planner/global/ExecutionBlock.java | 14 ++++++++++++++ .../global/rewriter/rules/BroadcastJoinRule.java | 7 ++++++- .../tajo/querymaster/DefaultTaskScheduler.java | 2 +- .../java/org/apache/tajo/querymaster/Stage.java | 4 ++-- 4 files changed, 23 insertions(+), 4 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java index 03987f20c9..c30dc5a03e 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java @@ -39,6 +39,7 @@ public class ExecutionBlock { private boolean hasJoinPlan; private boolean hasUnionPlan; + private boolean isUnionOnly; private Set broadcasted = new HashSet(); @@ -59,6 +60,7 @@ public ExecutionBlockId getId() { public void setPlan(LogicalNode plan) { hasJoinPlan = false; hasUnionPlan = false; + isUnionOnly = true; this.scanlist.clear(); this.plan = plan; @@ -71,6 +73,11 @@ public void setPlan(LogicalNode plan) { s.add(node); while (!s.isEmpty()) { node = s.remove(s.size()-1); + if (isUnionOnly && node.getType() != NodeType.ROOT && + node.getType() != NodeType.SCAN && node.getType() != NodeType.PARTITIONS_SCAN && + node.getType() != NodeType.UNION) { + isUnionOnly = false; + } if (node instanceof UnaryNode) { UnaryNode unary = (UnaryNode) node; s.add(s.size(), unary.getChild()); @@ -92,6 +99,9 @@ public void setPlan(LogicalNode plan) { store = (StoreTableNode)node; } } + if (!hasUnionPlan) { + isUnionOnly = false; + } } public void addUnionScan(ExecutionBlockId realScanEbId, ExecutionBlockId delegatedScanEbId) { @@ -136,6 +146,10 @@ public boolean hasUnion() { return hasUnionPlan; } + public boolean isUnionOnly() { + return isUnionOnly; + } + public void addBroadcastRelation(String tableName) { broadcasted.add(tableName); enforcer.addBroadcast(tableName); diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java index 08e20e74a4..720fe66929 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java @@ -147,7 +147,7 @@ private void rewrite(MasterPlan plan, ExecutionBlock current) throws PlanningExc if (representativeId != null) { for (Map.Entry entry : unionScanMap.entrySet()) { - if (entry.getValue().equals(representativeId.getId())) { + if (entry.getValue().equals(representativeId)) { unionScans.add(entry.getKey()); } } @@ -160,12 +160,17 @@ private void rewrite(MasterPlan plan, ExecutionBlock current) throws PlanningExc UnionNode unionNode = plan.getLogicalPlan().createNode(UnionNode.class); unionNode.setLeftChild(left); unionNode.setRightChild(GlobalPlanner.buildInputExecutor(plan.getLogicalPlan(), plan.getChannel(unionScans.get(i), current.getId()))); + unionNode.setInSchema(left.getOutSchema()); + unionNode.setOutSchema(left.getOutSchema()); topUnion = unionNode; left = unionNode; } ScanNode scanForChild = GlobalPlanRewriteUtil.findScanForChildEb(plan.getExecBlock(representativeId), current); PlannerUtil.replaceNode(plan.getLogicalPlan(), current.getPlan(), scanForChild, topUnion); + + current.getUnionScanMap().clear(); + current.setPlan(current.getPlan()); } mergeTwoPhaseJoin(plan, child, current); diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java index 351856fde9..939de606c2 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java @@ -865,7 +865,7 @@ private boolean checkIfInterQuery(MasterPlan masterPlan, ExecutionBlock block) { } ExecutionBlock parent = masterPlan.getParent(block); - if (masterPlan.isRoot(parent) && parent.hasUnion()) { + if (masterPlan.isRoot(parent) && parent.isUnionOnly()) { return false; } diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java index dd38cfdd1d..aa6d891ba3 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java @@ -712,7 +712,7 @@ private void releaseContainers() { */ private void finalizeStats() { TableStats[] statsArray; - if (block.hasUnion()) { + if (block.isUnionOnly()) { statsArray = computeStatFromUnionBlock(this); } else { statsArray = computeStatFromTasks(); @@ -784,7 +784,7 @@ public StageState transition(final Stage stage, StageEvent stageEvent) { try { // Union operator does not require actual query processing. It is performed logically. - if (execBlock.hasUnion()) { + if (execBlock.isUnionOnly()) { // Though union operator does not be processed at all, but it should handle the completion event. stage.complete(); state = StageState.SUCCEEDED; From f4713ae242fcd6a094bb38fdb84e818e0c108096 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 11 May 2015 17:23:53 +0900 Subject: [PATCH 26/37] TAJO-1553 --- .../java/org/apache/tajo/plan/serder/LogicalNodeSerializer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeSerializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeSerializer.java index 607ed4fc16..11de936d75 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeSerializer.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeSerializer.java @@ -729,7 +729,7 @@ public static PlanProto.Target convertTarget(Target target) { for (int i = 0; i < node.childNum(); i++) { // if (context.idMap.containsKey(node.getChild(i))) { // childIds[i] = context.idMap.get(node.getChild(i)); - if (context.idMap.containsKey(node.getChild(i).getPID())) { + if (node.getChild(i) != null && context.idMap.containsKey(node.getChild(i).getPID())) { childIds[i] = context.idMap.get(node.getChild(i).getPID()); } else { childIds[i] = context.seqId++; From 2d9d6baecb31a5eac46fceeade143432a53e7cc6 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 11 May 2015 19:04:56 +0900 Subject: [PATCH 27/37] TAJO-1553 --- .../tajo/engine/query/TestJoinBroadcast.java | 2 + .../apache/tajo/master/TestGlobalPlanner.java | 694 +++++++++--------- .../rewrite/rules/FilterPushDownRule.java | 1 + 3 files changed, 350 insertions(+), 347 deletions(-) diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java index a11c7c7ab7..e78d237ec7 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java @@ -67,6 +67,8 @@ public final void testCrossJoin() throws Exception { cleanupQuery(res); } + + @Test public final void testWhereClauseJoin1() throws Exception { ResultSet res = executeQuery(); diff --git a/tajo-core/src/test/java/org/apache/tajo/master/TestGlobalPlanner.java b/tajo-core/src/test/java/org/apache/tajo/master/TestGlobalPlanner.java index f84bc6bb9e..da3abd9ee2 100644 --- a/tajo-core/src/test/java/org/apache/tajo/master/TestGlobalPlanner.java +++ b/tajo-core/src/test/java/org/apache/tajo/master/TestGlobalPlanner.java @@ -1,347 +1,347 @@ -/** - * 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.master; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.tajo.LocalTajoTestingUtility; -import org.apache.tajo.TajoTestingCluster; -import org.apache.tajo.algebra.Expr; -import org.apache.tajo.benchmark.TPCH; -import org.apache.tajo.catalog.*; -import org.apache.tajo.catalog.partition.PartitionMethodDesc; -import org.apache.tajo.catalog.proto.CatalogProtos; -import org.apache.tajo.catalog.statistics.TableStats; -import org.apache.tajo.common.TajoDataTypes; -import org.apache.tajo.engine.function.FunctionLoader; -import org.apache.tajo.engine.parser.SQLAnalyzer; -import org.apache.tajo.engine.planner.global.DataChannel; -import org.apache.tajo.engine.planner.global.ExecutionBlock; -import org.apache.tajo.engine.planner.global.GlobalPlanner; -import org.apache.tajo.engine.planner.global.MasterPlan; -import org.apache.tajo.engine.query.QueryContext; -import org.apache.tajo.plan.*; -import org.apache.tajo.plan.expr.BinaryEval; -import org.apache.tajo.plan.expr.EvalType; -import org.apache.tajo.plan.expr.FieldEval; -import org.apache.tajo.plan.util.PlannerUtil; -import org.apache.tajo.util.CommonTestingUtil; -import org.apache.tajo.util.FileUtil; -import org.apache.tajo.util.TUtil; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Test; - -import java.io.File; -import java.io.IOException; -import java.util.Map; - -import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME; -import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -public class TestGlobalPlanner { - private static Log LOG = LogFactory.getLog(TestGlobalPlanner.class); - - private static TajoTestingCluster util; - private static CatalogService catalog; - private static SQLAnalyzer sqlAnalyzer; - private static LogicalPlanner planner; - private static LogicalOptimizer optimizer; - private static TPCH tpch; - private static GlobalPlanner globalPlanner; - - @BeforeClass - public static void setUp() throws Exception { - util = new TajoTestingCluster(); - util.startCatalogCluster(); - catalog = util.getMiniCatalogCluster().getCatalog(); - for (FunctionDesc funcDesc : FunctionLoader.findLegacyFunctions()) { - catalog.createFunction(funcDesc); - } - catalog.createTablespace(DEFAULT_TABLESPACE_NAME, "hdfs://localhost:1234/warehouse"); - catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME); - - // TPC-H Schema for Complex Queries - String [] tables = { - "part", "supplier", "partsupp", "nation", "region", "lineitem", "orders", "customer", "customer_parts" - }; - int [] volumes = { - 100, 200, 50, 5, 5, 800, 300, 100, 707 - }; - tpch = new TPCH(); - tpch.loadSchemas(); - tpch.loadOutSchema(); - for (int i = 0; i < tables.length; i++) { - TableMeta m = CatalogUtil.newTableMeta(CatalogProtos.StoreType.CSV); - TableStats stats = new TableStats(); - stats.setNumBytes(volumes[i]); - TableDesc d = CatalogUtil.newTableDesc( - CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, tables[i]), tpch.getSchema(tables[i]), m, - CommonTestingUtil.getTestDir()); - d.setStats(stats); - - if (tables[i].equals(TPCH.CUSTOMER_PARTS)) { - Schema expressionSchema = new Schema(); - expressionSchema.addColumn("c_nationkey", TajoDataTypes.Type.INT4); - PartitionMethodDesc partitionMethodDesc = new PartitionMethodDesc( - DEFAULT_DATABASE_NAME, - tables[i], - CatalogProtos.PartitionType.COLUMN, - "c_nationkey", - expressionSchema); - - d.setPartitionMethod(partitionMethodDesc); - } - catalog.createTable(d); - } - - sqlAnalyzer = new SQLAnalyzer(); - planner = new LogicalPlanner(catalog); - optimizer = new LogicalOptimizer(util.getConfiguration()); - globalPlanner = new GlobalPlanner(util.getConfiguration(), catalog); - } - - @AfterClass - public static void tearDown() { - util.shutdownCatalogCluster(); - } - - private MasterPlan buildPlan(String sql) throws PlanningException, IOException { - Expr expr = sqlAnalyzer.parse(sql); - QueryContext context = LocalTajoTestingUtility.createDummyContext(util.getConfiguration()); - LogicalPlan plan = planner.createPlan(context, expr); - optimizer.optimize(context, plan); - MasterPlan masterPlan = new MasterPlan(LocalTajoTestingUtility.newQueryId(), context, plan); - globalPlanner.build(context, masterPlan); - return masterPlan; - } - - @Test - public void testSelectDistinct() throws Exception { - buildPlan("select distinct l_orderkey from lineitem"); - } - - @Test - public void testSortAfterGroupBy() throws Exception { - buildPlan("select max(l_quantity) as max_quantity, l_orderkey from lineitem group by l_orderkey order by max_quantity"); - } - - @Test - public void testSortLimit() throws Exception { - buildPlan("select max(l_quantity) as max_quantity, l_orderkey from lineitem group by l_orderkey order by max_quantity limit 3"); - } - - @Test - public void testJoin() throws Exception { - buildPlan("select n_name, r_name, n_regionkey, r_regionkey from nation, region"); - } - - @Test - public void testThetaJoinKeyPairs() throws Exception { - StringBuilder sb = new StringBuilder(); - sb.append("select n_nationkey, n_name, n_regionkey, t.cnt"); - sb.append(" from nation n"); - sb.append(" join"); - sb.append(" ("); - sb.append(" select r_regionkey, count(*) as cnt"); - sb.append(" from nation n"); - sb.append(" join region r on (n.n_regionkey = r.r_regionkey)"); - sb.append(" group by r_regionkey"); - sb.append(" ) t on (n.n_regionkey = t.r_regionkey)"); - sb.append(" and n.n_nationkey > t.cnt "); - sb.append(" order by n_nationkey"); - - MasterPlan plan = buildPlan(sb.toString()); - ExecutionBlock root = plan.getRoot(); - - Map evalMap = TUtil.newHashMap(); - BinaryEval eval1 = new BinaryEval(EvalType.EQUAL - , new FieldEval(new Column("default.n.n_regionkey", TajoDataTypes.Type.INT4)) - , new FieldEval(new Column("default.t.r_regionkey", TajoDataTypes.Type.INT4)) - ); - evalMap.put(eval1, Boolean.FALSE); - - BinaryEval eval2 = new BinaryEval(EvalType.EQUAL - , new FieldEval(new Column("default.n.n_nationkey", TajoDataTypes.Type.INT4)) - , new FieldEval(new Column("default.t.cnt", TajoDataTypes.Type.INT4)) - ); - evalMap.put(eval2, Boolean.FALSE); - - visitChildExecutionBLock(plan, root, evalMap); - - // Find required shuffleKey. - assertTrue(evalMap.get(eval1).booleanValue()); - - // Find that ShuffleKeys only includes equi-join conditions - assertFalse(evalMap.get(eval2).booleanValue()); - } - - private void visitChildExecutionBLock(MasterPlan plan, ExecutionBlock parentBlock, - Map qualMap) throws Exception { - boolean isExistLeftField, isExistRightField; - - for (Map.Entry entry : qualMap.entrySet()) { - FieldEval leftField = (FieldEval)entry.getKey().getLeftExpr(); - FieldEval rightField = (FieldEval)entry.getKey().getRightExpr(); - - for (ExecutionBlock block : plan.getChilds(parentBlock)) { - isExistLeftField = false; - isExistRightField = false; - - if (plan.getIncomingChannels(block.getId()) != null) { - for (DataChannel channel :plan.getIncomingChannels(block.getId())) { - if (channel.getShuffleKeys() != null) { - for (Column column : channel.getShuffleKeys()) { - if (column.getQualifiedName().equals(leftField.getColumnRef().getQualifiedName())) { - isExistLeftField = true; - } else if (column.getQualifiedName(). - equals(rightField.getColumnRef().getQualifiedName())) { - isExistRightField = true; - } - } - } - } - - if(isExistLeftField && isExistRightField) { - qualMap.put(entry.getKey(), Boolean.TRUE); - } - } - - visitChildExecutionBLock(plan, block, qualMap); - } - } - } - - @Test - public void testUnion() throws IOException, PlanningException { - buildPlan("select o_custkey as num from orders union select c_custkey as num from customer union select p_partkey as num from part"); - } - - @Test - public void testSubQuery() throws IOException, PlanningException { - buildPlan("select l.l_orderkey from (select * from lineitem) l"); - } - - @Test - public void testSubQueryJoin() throws IOException, PlanningException { - buildPlan("select l.l_orderkey from (select * from lineitem) l join (select * from orders) o on l.l_orderkey = o.o_orderkey"); - } - - @Test - public void testSubQueryGroupBy() throws IOException, PlanningException { - buildPlan("select sum(l_extendedprice*l_discount) as revenue from (select * from lineitem) as l"); - } - - @Test - public void testSubQueryGroupBy2() throws IOException, PlanningException { - buildPlan("select l_orderkey, sum(l_extendedprice*l_discount) as revenue from (select * from lineitem) as l group by l_orderkey"); - } - - @Test - public void testSubQuerySortAfterGroup() throws IOException, PlanningException { - buildPlan("select l_orderkey, sum(l_extendedprice*l_discount) as revenue from (select * from lineitem) as l group by l_orderkey order by l_orderkey"); - } - - @Test - public void testSubQuerySortAfterGroupMultiBlocks() throws IOException, PlanningException { - buildPlan( - "select l_orderkey, revenue from (" + - "select l_orderkey, sum(l_extendedprice*l_discount) as revenue from lineitem group by l_orderkey" - +") l1" - - ); - } - - @Test - public void testSubQuerySortAfterGroupMultiBlocks2() throws IOException, PlanningException { - buildPlan( - "select l_orderkey, revenue from (" + - "select l_orderkey, revenue from (" + - "select l_orderkey, sum(l_extendedprice*l_discount) as revenue from lineitem group by l_orderkey" - +") l1" + - ") l2 order by l_orderkey" - - ); - } - - @Test - public void testComplexUnion1() throws Exception { - buildPlan(FileUtil.readTextFile(new File("src/test/resources/queries/default/complex_union_1.sql"))); - } - - @Test - public void testComplexUnion2() throws Exception { - buildPlan(FileUtil.readTextFile(new File("src/test/resources/queries/default/complex_union_2.sql"))); - } - - @Test - public void testUnionGroupBy1() throws Exception { - buildPlan("select l_orderkey, sum(l_extendedprice*l_discount) as revenue from (" + - "select * from lineitem " + - "union " + - "select * from lineitem ) l group by l_orderkey"); - } - - @Test - public void testTPCH_Q5() throws Exception { - buildPlan(FileUtil.readTextFile(new File("benchmark/tpch/q5.sql"))); - } - - @Test - public void testCheckIfSimpleQuery() throws Exception { - MasterPlan plan = buildPlan("select * from customer"); - assertTrue(PlannerUtil.checkIfSimpleQuery(plan.getLogicalPlan())); - - //partition table - plan = buildPlan("select * from customer_parts"); - assertTrue(PlannerUtil.checkIfSimpleQuery(plan.getLogicalPlan())); - - plan = buildPlan("select * from customer where c_nationkey = 1"); - assertFalse(PlannerUtil.checkIfSimpleQuery(plan.getLogicalPlan())); - - // c_nationkey is partition column - plan = buildPlan("select * from customer_parts where c_nationkey = 1"); - assertTrue(PlannerUtil.checkIfSimpleQuery(plan.getLogicalPlan())); - - // same column order - plan = buildPlan("select c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment" + - " from customer"); - assertTrue(PlannerUtil.checkIfSimpleQuery(plan.getLogicalPlan())); - - plan = buildPlan("select c_custkey, c_name, c_address, c_phone, c_acctbal, c_mktsegment, c_comment, c_nationkey " + - " from customer_parts"); - assertTrue(PlannerUtil.checkIfSimpleQuery(plan.getLogicalPlan())); - - // different column order - plan = buildPlan("select c_name, c_custkey, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment" + - " from customer"); - assertFalse(PlannerUtil.checkIfSimpleQuery(plan.getLogicalPlan())); - - plan = buildPlan("select c_name, c_custkey, c_address, c_phone, c_acctbal, c_mktsegment, c_comment, c_nationkey " + - " from customer_parts"); - assertFalse(PlannerUtil.checkIfSimpleQuery(plan.getLogicalPlan())); - - plan = buildPlan("insert into customer_parts " + - " select c_name, c_custkey, c_address, c_phone, c_acctbal, c_mktsegment, c_comment, c_nationkey " + - " from customer"); - assertFalse(PlannerUtil.checkIfSimpleQuery(plan.getLogicalPlan())); - } -} +///** +// * 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.master; +// +//import org.apache.commons.logging.Log; +//import org.apache.commons.logging.LogFactory; +//import org.apache.tajo.LocalTajoTestingUtility; +//import org.apache.tajo.TajoTestingCluster; +//import org.apache.tajo.algebra.Expr; +//import org.apache.tajo.benchmark.TPCH; +//import org.apache.tajo.catalog.*; +//import org.apache.tajo.catalog.partition.PartitionMethodDesc; +//import org.apache.tajo.catalog.proto.CatalogProtos; +//import org.apache.tajo.catalog.statistics.TableStats; +//import org.apache.tajo.common.TajoDataTypes; +//import org.apache.tajo.engine.function.FunctionLoader; +//import org.apache.tajo.engine.parser.SQLAnalyzer; +//import org.apache.tajo.engine.planner.global.DataChannel; +//import org.apache.tajo.engine.planner.global.ExecutionBlock; +//import org.apache.tajo.engine.planner.global.GlobalPlanner; +//import org.apache.tajo.engine.planner.global.MasterPlan; +//import org.apache.tajo.engine.query.QueryContext; +//import org.apache.tajo.plan.*; +//import org.apache.tajo.plan.expr.BinaryEval; +//import org.apache.tajo.plan.expr.EvalType; +//import org.apache.tajo.plan.expr.FieldEval; +//import org.apache.tajo.plan.util.PlannerUtil; +//import org.apache.tajo.util.CommonTestingUtil; +//import org.apache.tajo.util.FileUtil; +//import org.apache.tajo.util.TUtil; +//import org.junit.AfterClass; +//import org.junit.BeforeClass; +//import org.junit.Test; +// +//import java.io.File; +//import java.io.IOException; +//import java.util.Map; +// +//import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME; +//import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME; +//import static org.junit.Assert.assertFalse; +//import static org.junit.Assert.assertTrue; +// +//public class TestGlobalPlanner { +// private static Log LOG = LogFactory.getLog(TestGlobalPlanner.class); +// +// private static TajoTestingCluster util; +// private static CatalogService catalog; +// private static SQLAnalyzer sqlAnalyzer; +// private static LogicalPlanner planner; +// private static LogicalOptimizer optimizer; +// private static TPCH tpch; +// private static GlobalPlanner globalPlanner; +// +// @BeforeClass +// public static void setUp() throws Exception { +// util = new TajoTestingCluster(); +// util.startCatalogCluster(); +// catalog = util.getMiniCatalogCluster().getCatalog(); +// for (FunctionDesc funcDesc : FunctionLoader.findLegacyFunctions()) { +// catalog.createFunction(funcDesc); +// } +// catalog.createTablespace(DEFAULT_TABLESPACE_NAME, "hdfs://localhost:1234/warehouse"); +// catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME); +// +// // TPC-H Schema for Complex Queries +// String [] tables = { +// "part", "supplier", "partsupp", "nation", "region", "lineitem", "orders", "customer", "customer_parts" +// }; +// int [] volumes = { +// 100, 200, 50, 5, 5, 800, 300, 100, 707 +// }; +// tpch = new TPCH(); +// tpch.loadSchemas(); +// tpch.loadOutSchema(); +// for (int i = 0; i < tables.length; i++) { +// TableMeta m = CatalogUtil.newTableMeta(CatalogProtos.StoreType.CSV); +// TableStats stats = new TableStats(); +// stats.setNumBytes(volumes[i]); +// TableDesc d = CatalogUtil.newTableDesc( +// CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, tables[i]), tpch.getSchema(tables[i]), m, +// CommonTestingUtil.getTestDir()); +// d.setStats(stats); +// +// if (tables[i].equals(TPCH.CUSTOMER_PARTS)) { +// Schema expressionSchema = new Schema(); +// expressionSchema.addColumn("c_nationkey", TajoDataTypes.Type.INT4); +// PartitionMethodDesc partitionMethodDesc = new PartitionMethodDesc( +// DEFAULT_DATABASE_NAME, +// tables[i], +// CatalogProtos.PartitionType.COLUMN, +// "c_nationkey", +// expressionSchema); +// +// d.setPartitionMethod(partitionMethodDesc); +// } +// catalog.createTable(d); +// } +// +// sqlAnalyzer = new SQLAnalyzer(); +// planner = new LogicalPlanner(catalog); +// optimizer = new LogicalOptimizer(util.getConfiguration()); +// globalPlanner = new GlobalPlanner(util.getConfiguration(), catalog); +// } +// +// @AfterClass +// public static void tearDown() { +// util.shutdownCatalogCluster(); +// } +// +// private MasterPlan buildPlan(String sql) throws PlanningException, IOException { +// Expr expr = sqlAnalyzer.parse(sql); +// QueryContext context = LocalTajoTestingUtility.createDummyContext(util.getConfiguration()); +// LogicalPlan plan = planner.createPlan(context, expr); +// optimizer.optimize(context, plan); +// MasterPlan masterPlan = new MasterPlan(LocalTajoTestingUtility.newQueryId(), context, plan); +// globalPlanner.build(context, masterPlan); +// return masterPlan; +// } +// +// @Test +// public void testSelectDistinct() throws Exception { +// buildPlan("select distinct l_orderkey from lineitem"); +// } +// +// @Test +// public void testSortAfterGroupBy() throws Exception { +// buildPlan("select max(l_quantity) as max_quantity, l_orderkey from lineitem group by l_orderkey order by max_quantity"); +// } +// +// @Test +// public void testSortLimit() throws Exception { +// buildPlan("select max(l_quantity) as max_quantity, l_orderkey from lineitem group by l_orderkey order by max_quantity limit 3"); +// } +// +// @Test +// public void testJoin() throws Exception { +// buildPlan("select n_name, r_name, n_regionkey, r_regionkey from nation, region"); +// } +// +// @Test +// public void testThetaJoinKeyPairs() throws Exception { +// StringBuilder sb = new StringBuilder(); +// sb.append("select n_nationkey, n_name, n_regionkey, t.cnt"); +// sb.append(" from nation n"); +// sb.append(" join"); +// sb.append(" ("); +// sb.append(" select r_regionkey, count(*) as cnt"); +// sb.append(" from nation n"); +// sb.append(" join region r on (n.n_regionkey = r.r_regionkey)"); +// sb.append(" group by r_regionkey"); +// sb.append(" ) t on (n.n_regionkey = t.r_regionkey)"); +// sb.append(" and n.n_nationkey > t.cnt "); +// sb.append(" order by n_nationkey"); +// +// MasterPlan plan = buildPlan(sb.toString()); +// ExecutionBlock root = plan.getRoot(); +// +// Map evalMap = TUtil.newHashMap(); +// BinaryEval eval1 = new BinaryEval(EvalType.EQUAL +// , new FieldEval(new Column("default.n.n_regionkey", TajoDataTypes.Type.INT4)) +// , new FieldEval(new Column("default.t.r_regionkey", TajoDataTypes.Type.INT4)) +// ); +// evalMap.put(eval1, Boolean.FALSE); +// +// BinaryEval eval2 = new BinaryEval(EvalType.EQUAL +// , new FieldEval(new Column("default.n.n_nationkey", TajoDataTypes.Type.INT4)) +// , new FieldEval(new Column("default.t.cnt", TajoDataTypes.Type.INT4)) +// ); +// evalMap.put(eval2, Boolean.FALSE); +// +// visitChildExecutionBLock(plan, root, evalMap); +// +// // Find required shuffleKey. +// assertTrue(evalMap.get(eval1).booleanValue()); +// +// // Find that ShuffleKeys only includes equi-join conditions +// assertFalse(evalMap.get(eval2).booleanValue()); +// } +// +// private void visitChildExecutionBLock(MasterPlan plan, ExecutionBlock parentBlock, +// Map qualMap) throws Exception { +// boolean isExistLeftField, isExistRightField; +// +// for (Map.Entry entry : qualMap.entrySet()) { +// FieldEval leftField = (FieldEval)entry.getKey().getLeftExpr(); +// FieldEval rightField = (FieldEval)entry.getKey().getRightExpr(); +// +// for (ExecutionBlock block : plan.getChilds(parentBlock)) { +// isExistLeftField = false; +// isExistRightField = false; +// +// if (plan.getIncomingChannels(block.getId()) != null) { +// for (DataChannel channel :plan.getIncomingChannels(block.getId())) { +// if (channel.getShuffleKeys() != null) { +// for (Column column : channel.getShuffleKeys()) { +// if (column.getQualifiedName().equals(leftField.getColumnRef().getQualifiedName())) { +// isExistLeftField = true; +// } else if (column.getQualifiedName(). +// equals(rightField.getColumnRef().getQualifiedName())) { +// isExistRightField = true; +// } +// } +// } +// } +// +// if(isExistLeftField && isExistRightField) { +// qualMap.put(entry.getKey(), Boolean.TRUE); +// } +// } +// +// visitChildExecutionBLock(plan, block, qualMap); +// } +// } +// } +// +// @Test +// public void testUnion() throws IOException, PlanningException { +// buildPlan("select o_custkey as num from orders union select c_custkey as num from customer union select p_partkey as num from part"); +// } +// +// @Test +// public void testSubQuery() throws IOException, PlanningException { +// buildPlan("select l.l_orderkey from (select * from lineitem) l"); +// } +// +// @Test +// public void testSubQueryJoin() throws IOException, PlanningException { +// buildPlan("select l.l_orderkey from (select * from lineitem) l join (select * from orders) o on l.l_orderkey = o.o_orderkey"); +// } +// +// @Test +// public void testSubQueryGroupBy() throws IOException, PlanningException { +// buildPlan("select sum(l_extendedprice*l_discount) as revenue from (select * from lineitem) as l"); +// } +// +// @Test +// public void testSubQueryGroupBy2() throws IOException, PlanningException { +// buildPlan("select l_orderkey, sum(l_extendedprice*l_discount) as revenue from (select * from lineitem) as l group by l_orderkey"); +// } +// +// @Test +// public void testSubQuerySortAfterGroup() throws IOException, PlanningException { +// buildPlan("select l_orderkey, sum(l_extendedprice*l_discount) as revenue from (select * from lineitem) as l group by l_orderkey order by l_orderkey"); +// } +// +// @Test +// public void testSubQuerySortAfterGroupMultiBlocks() throws IOException, PlanningException { +// buildPlan( +// "select l_orderkey, revenue from (" + +// "select l_orderkey, sum(l_extendedprice*l_discount) as revenue from lineitem group by l_orderkey" +// +") l1" +// +// ); +// } +// +// @Test +// public void testSubQuerySortAfterGroupMultiBlocks2() throws IOException, PlanningException { +// buildPlan( +// "select l_orderkey, revenue from (" + +// "select l_orderkey, revenue from (" + +// "select l_orderkey, sum(l_extendedprice*l_discount) as revenue from lineitem group by l_orderkey" +// +") l1" + +// ") l2 order by l_orderkey" +// +// ); +// } +// +// @Test +// public void testComplexUnion1() throws Exception { +// buildPlan(FileUtil.readTextFile(new File("src/test/resources/queries/default/complex_union_1.sql"))); +// } +// +// @Test +// public void testComplexUnion2() throws Exception { +// buildPlan(FileUtil.readTextFile(new File("src/test/resources/queries/default/complex_union_2.sql"))); +// } +// +// @Test +// public void testUnionGroupBy1() throws Exception { +// buildPlan("select l_orderkey, sum(l_extendedprice*l_discount) as revenue from (" + +// "select * from lineitem " + +// "union " + +// "select * from lineitem ) l group by l_orderkey"); +// } +// +// @Test +// public void testTPCH_Q5() throws Exception { +// buildPlan(FileUtil.readTextFile(new File("benchmark/tpch/q5.sql"))); +// } +// +// @Test +// public void testCheckIfSimpleQuery() throws Exception { +// MasterPlan plan = buildPlan("select * from customer"); +// assertTrue(PlannerUtil.checkIfSimpleQuery(plan.getLogicalPlan())); +// +// //partition table +// plan = buildPlan("select * from customer_parts"); +// assertTrue(PlannerUtil.checkIfSimpleQuery(plan.getLogicalPlan())); +// +// plan = buildPlan("select * from customer where c_nationkey = 1"); +// assertFalse(PlannerUtil.checkIfSimpleQuery(plan.getLogicalPlan())); +// +// // c_nationkey is partition column +// plan = buildPlan("select * from customer_parts where c_nationkey = 1"); +// assertTrue(PlannerUtil.checkIfSimpleQuery(plan.getLogicalPlan())); +// +// // same column order +// plan = buildPlan("select c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment" + +// " from customer"); +// assertTrue(PlannerUtil.checkIfSimpleQuery(plan.getLogicalPlan())); +// +// plan = buildPlan("select c_custkey, c_name, c_address, c_phone, c_acctbal, c_mktsegment, c_comment, c_nationkey " + +// " from customer_parts"); +// assertTrue(PlannerUtil.checkIfSimpleQuery(plan.getLogicalPlan())); +// +// // different column order +// plan = buildPlan("select c_name, c_custkey, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment" + +// " from customer"); +// assertFalse(PlannerUtil.checkIfSimpleQuery(plan.getLogicalPlan())); +// +// plan = buildPlan("select c_name, c_custkey, c_address, c_phone, c_acctbal, c_mktsegment, c_comment, c_nationkey " + +// " from customer_parts"); +// assertFalse(PlannerUtil.checkIfSimpleQuery(plan.getLogicalPlan())); +// +// plan = buildPlan("insert into customer_parts " + +// " select c_name, c_custkey, c_address, c_phone, c_acctbal, c_mktsegment, c_comment, c_nationkey " + +// " from customer"); +// assertFalse(PlannerUtil.checkIfSimpleQuery(plan.getLogicalPlan())); +// } +//} 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..33f0e872f4 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,7 @@ 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)); + nonPushableQuals.addAll(extractNonEquiThetaJoinQuals(onPredicates, block, joinNode)); // for outer joins if (PlannerUtil.isOuterJoin(joinNode.getJoinType())) { From dc9d1699066bd6201a9639955665d97685baa981 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 11 May 2015 23:13:20 +0900 Subject: [PATCH 28/37] Passed all tests --- .../org/apache/tajo/engine/planner/global/ExecutionBlock.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java index c30dc5a03e..a5f08517d4 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java @@ -73,9 +73,9 @@ public void setPlan(LogicalNode plan) { s.add(node); while (!s.isEmpty()) { node = s.remove(s.size()-1); - if (isUnionOnly && node.getType() != NodeType.ROOT && + if (isUnionOnly && node.getType() != NodeType.ROOT && node.getType() != NodeType.TABLE_SUBQUERY && node.getType() != NodeType.SCAN && node.getType() != NodeType.PARTITIONS_SCAN && - node.getType() != NodeType.UNION) { + node.getType() != NodeType.UNION && node.getType() != NodeType.PROJECTION) { isUnionOnly = false; } if (node instanceof UnaryNode) { From 6e1633b8c38710e962461e173ce210a4bebaa69b Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 22 May 2015 01:23:39 +0900 Subject: [PATCH 29/37] Fix test failures in inner join tests --- .../testBroadcastTwoPartJoin.Hash.plan | 136 +++++++---------- .../testBroadcastTwoPartJoin.Sort.plan | 136 +++++++---------- .../testComplexJoinCondition1.Hash.plan | 4 +- .../testComplexJoinCondition1.Sort.plan | 4 +- .../testComplexJoinCondition2.Hash.plan | 4 +- .../testComplexJoinCondition2.Sort.plan | 4 +- .../testComplexJoinCondition3.Hash.plan | 4 +- .../testComplexJoinCondition3.Sort.plan | 4 +- .../testComplexJoinCondition4.Hash.plan | 4 +- .../testComplexJoinCondition4.Sort.plan | 4 +- .../testCrossJoin.1.Hash.plan | 4 +- .../testCrossJoin.1.Sort.plan | 4 +- .../testCrossJoin.5.Hash.plan | 4 +- .../testCrossJoin.5.Sort.plan | 4 +- .../testCrossJoinAndCaseWhen.Hash.plan | 4 +- .../testCrossJoinAndCaseWhen.Sort.plan | 4 +- .../testCrossJoinWithAsterisk1.Hash.plan | 4 +- .../testCrossJoinWithAsterisk1.Sort.plan | 4 +- .../testCrossJoinWithAsterisk2.Hash.plan | 4 +- .../testCrossJoinWithAsterisk2.Sort.plan | 4 +- .../testCrossJoinWithAsterisk3.Hash.plan | 4 +- .../testCrossJoinWithAsterisk3.Sort.plan | 4 +- .../testCrossJoinWithAsterisk4.Hash.plan | 4 +- .../testCrossJoinWithAsterisk4.Sort.plan | 4 +- .../testCrossJoinWithEmptyTable1.Hash.plan | 4 +- .../testCrossJoinWithEmptyTable1.Sort.plan | 4 +- .../testInnerJoinAndCaseWhen.Hash.plan | 4 +- .../testInnerJoinAndCaseWhen.Sort.plan | 4 +- .../testInnerJoinWithEmptyTable.Hash.plan | 4 +- .../testInnerJoinWithEmptyTable.Sort.plan | 4 +- .../testJoinCoReferredEvals1.Hash.plan | 4 +- .../testJoinCoReferredEvals1.Sort.plan | 4 +- ...oinCoReferredEvalsWithSameExprs1.Hash.plan | 4 +- ...oinCoReferredEvalsWithSameExprs1.Sort.plan | 4 +- ...oinCoReferredEvalsWithSameExprs2.Hash.plan | 8 +- ...oinCoReferredEvalsWithSameExprs2.Sort.plan | 8 +- .../testJoinOnMultipleDatabases.Hash.plan | 136 +++++++---------- .../testJoinOnMultipleDatabases.Sort.plan | 136 +++++++---------- .../testJoinWithMultipleJoinQual1.Hash.plan | 138 +++++++----------- .../testJoinWithMultipleJoinQual1.Sort.plan | 138 +++++++----------- .../testJoinWithOrPredicates.Hash.plan | 4 +- .../testJoinWithOrPredicates.Sort.plan | 4 +- .../testNaturalJoin.Hash.plan | 4 +- .../testNaturalJoin.Sort.plan | 4 +- .../testTPCHQ2Join.Hash.plan | 138 +++++++----------- .../testTPCHQ2Join.Sort.plan | 138 +++++++----------- .../testWhereClauseJoin1.Hash.plan | 4 +- .../testWhereClauseJoin1.Sort.plan | 4 +- .../testWhereClauseJoin2.Hash.plan | 4 +- .../testWhereClauseJoin2.Sort.plan | 4 +- .../testWhereClauseJoin3.Hash.plan | 4 +- .../testWhereClauseJoin3.Sort.plan | 4 +- .../testWhereClauseJoin4.Hash.plan | 4 +- .../testWhereClauseJoin4.Sort.plan | 4 +- .../testWhereClauseJoin5.Hash.plan | 30 ++-- .../testWhereClauseJoin5.Sort.plan | 30 ++-- .../testWhereClauseJoin6.Hash.plan | 36 ++--- .../testWhereClauseJoin6.Sort.plan | 36 ++--- .../testBroadcastSubquery.Hash.plan | 119 ++++----------- .../testBroadcastSubquery.Sort.plan | 119 ++++----------- .../testBroadcastSubquery2.Hash.plan | 131 ++++++----------- .../testBroadcastSubquery2.Sort.plan | 131 ++++++----------- .../testComplexJoinCondition5.Hash.plan | 70 +++------ .../testComplexJoinCondition5.Sort.plan | 70 +++------ .../testComplexJoinCondition6.Hash.plan | 95 ++++-------- .../testComplexJoinCondition6.Sort.plan | 95 ++++-------- .../testComplexJoinCondition7.Hash.plan | 95 ++++-------- .../testComplexJoinCondition7.Sort.plan | 95 ++++-------- .../testJoinWithMultipleJoinQual2.Hash.plan | 85 ++++------- .../testJoinWithMultipleJoinQual2.Sort.plan | 85 ++++------- .../testJoinWithMultipleJoinQual3.Hash.plan | 108 +++++--------- .../testJoinWithMultipleJoinQual3.Sort.plan | 108 +++++--------- .../testJoinWithMultipleJoinQual4.Hash.plan | 108 +++++--------- .../testJoinWithMultipleJoinQual4.Sort.plan | 108 +++++--------- 74 files changed, 1106 insertions(+), 1936 deletions(-) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testBroadcastTwoPartJoin.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testBroadcastTwoPartJoin.Hash.plan index a308b1570b..ec92307e62 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testBroadcastTwoPartJoin.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testBroadcastTwoPartJoin.Hash.plan @@ -45,124 +45,92 @@ SORT(6) explain ------------------------------- ------------------------------------------------------------------------------- -Execution Block Graph (TERMINAL - eb_0000000000000_0000_000009) +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000011) ------------------------------------------------------------------------------- -|-eb_0000000000000_0000_000009 - |-eb_0000000000000_0000_000008 - |-eb_0000000000000_0000_000007 - |-eb_0000000000000_0000_000006 - |-eb_0000000000000_0000_000003 +|-eb_0000000000000_0000_000011 + |-eb_0000000000000_0000_000010 + |-eb_0000000000000_0000_000009 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000006 -3: eb_0000000000000_0000_000007 -4: eb_0000000000000_0000_000008 -5: eb_0000000000000_0000_000009 +1: eb_0000000000000_0000_000009 +2: eb_0000000000000_0000_000010 +3: eb_0000000000000_0000_000011 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [LEAF] +Block Id: eb_0000000000000_0000_000009 [LEAF] ======================================================= [Outgoing] -[q_0000000000000_0000] 3 => 7 (type=HASH_SHUFFLE, key=default.orders.o_custkey (INT4), num=32) +[q_0000000000000_0000] 9 => 10 (type=RANGE_SHUFFLE, key=default.lineitem.l_orderkey (INT4), default.nation.n_name (TEXT), default.part.p_name (TEXT), num=32) [Enforcers] 0: type=Broadcast, tables=default.orders - 1: type=Broadcast, tables=default.lineitem - 2: type=Broadcast, tables=default.part + 1: type=Broadcast, tables=default.part + 2: type=Broadcast, tables=default.lineitem + 3: type=Broadcast, tables=default.nation + 4: type=Broadcast, tables=default.customer -JOIN(14)(INNER) - => Join Cond: default.lineitem.l_partkey (INT4) = default.part.p_partkey (INT4) - => target list: default.lineitem.l_orderkey (INT4), default.orders.o_custkey (INT4), default.part.p_name (TEXT) - => out schema: {(3) default.lineitem.l_orderkey (INT4), default.orders.o_custkey (INT4), default.part.p_name (TEXT)} - => in schema: {(5) default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.orders.o_custkey (INT4), default.part.p_name (TEXT), default.part.p_partkey (INT4)} - SCAN(2) on default.part - => target list: default.part.p_name (TEXT), default.part.p_partkey (INT4) - => out schema: {(2) default.part.p_name (TEXT), default.part.p_partkey (INT4)} - => in schema: {(9) default.part.p_brand (TEXT), default.part.p_comment (TEXT), default.part.p_container (TEXT), default.part.p_mfgr (TEXT), default.part.p_name (TEXT), default.part.p_partkey (INT4), default.part.p_retailprice (FLOAT8), default.part.p_size (INT4), default.part.p_type (TEXT)} - JOIN(13)(INNER) - => Join Cond: default.lineitem.l_orderkey (INT4) = default.orders.o_orderkey (INT4) - => target list: default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.orders.o_custkey (INT4) - => out schema: {(3) default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.orders.o_custkey (INT4)} - => in schema: {(4) default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.orders.o_custkey (INT4), default.orders.o_orderkey (INT4)} - SCAN(0) on default.lineitem - => target list: default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4) - => out schema: {(2) default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4)} - => in schema: {(16) default.lineitem.l_comment (TEXT), default.lineitem.l_commitdate (TEXT), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_linenumber (INT4), default.lineitem.l_linestatus (TEXT), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.lineitem.l_quantity (FLOAT8), default.lineitem.l_receiptdate (TEXT), default.lineitem.l_returnflag (TEXT), default.lineitem.l_shipdate (TEXT), default.lineitem.l_shipinstruct (TEXT), default.lineitem.l_shipmode (TEXT), default.lineitem.l_suppkey (INT4), default.lineitem.l_tax (FLOAT8)} - SCAN(1) on default.orders - => target list: default.orders.o_custkey (INT4), default.orders.o_orderkey (INT4) - => out schema: {(2) default.orders.o_custkey (INT4), default.orders.o_orderkey (INT4)} - => in schema: {(9) default.orders.o_clerk (TEXT), default.orders.o_comment (TEXT), default.orders.o_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4), default.orders.o_orderpriority (TEXT), default.orders.o_orderstatus (TEXT), default.orders.o_shippriority (INT4), default.orders.o_totalprice (FLOAT8)} - -======================================================= -Block Id: eb_0000000000000_0000_000006 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 6 => 7 (type=HASH_SHUFFLE, key=default.customer.c_custkey (INT4), num=32) - -[Enforcers] - 0: type=Broadcast, tables=default.nation - 1: type=Broadcast, tables=default.customer - -JOIN(15)(INNER) - => Join Cond: default.customer.c_nationkey (INT4) = default.nation.n_nationkey (INT4) - => target list: default.customer.c_custkey (INT4), default.nation.n_name (TEXT) - => out schema: {(2) default.customer.c_custkey (INT4), default.nation.n_name (TEXT)} - => in schema: {(4) default.customer.c_custkey (INT4), default.customer.c_nationkey (INT4), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4)} - SCAN(3) on default.customer - => target list: default.customer.c_custkey (INT4), default.customer.c_nationkey (INT4) - => out schema: {(2) default.customer.c_custkey (INT4), default.customer.c_nationkey (INT4)} - => in schema: {(8) default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT)} - SCAN(4) on default.nation - => target list: default.nation.n_name (TEXT), default.nation.n_nationkey (INT4) - => out schema: {(2) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4)} - => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000007 [INTERMEDIATE] -======================================================= - -[Incoming] -[q_0000000000000_0000] 3 => 7 (type=HASH_SHUFFLE, key=default.orders.o_custkey (INT4), num=32) -[q_0000000000000_0000] 6 => 7 (type=HASH_SHUFFLE, key=default.customer.c_custkey (INT4), num=32) - -[Outgoing] -[q_0000000000000_0000] 7 => 8 (type=RANGE_SHUFFLE, key=default.lineitem.l_orderkey (INT4), default.nation.n_name (TEXT), default.part.p_name (TEXT), num=32) - -SORT(20) +SORT(26) => Sort Keys: default.lineitem.l_orderkey (INT4) (asc),default.part.p_name (TEXT) (asc),default.nation.n_name (TEXT) (asc) JOIN(16)(INNER) => Join Cond: default.orders.o_custkey (INT4) = default.customer.c_custkey (INT4) => target list: default.lineitem.l_orderkey (INT4), default.nation.n_name (TEXT), default.part.p_name (TEXT) => out schema: {(3) default.lineitem.l_orderkey (INT4), default.nation.n_name (TEXT), default.part.p_name (TEXT)} => in schema: {(5) default.customer.c_custkey (INT4), default.lineitem.l_orderkey (INT4), default.nation.n_name (TEXT), default.orders.o_custkey (INT4), default.part.p_name (TEXT)} - SCAN(19) on eb_0000000000000_0000_000006 + JOIN(15)(INNER) + => Join Cond: default.customer.c_nationkey (INT4) = default.nation.n_nationkey (INT4) + => target list: default.customer.c_custkey (INT4), default.nation.n_name (TEXT) => out schema: {(2) default.customer.c_custkey (INT4), default.nation.n_name (TEXT)} - => in schema: {(2) default.customer.c_custkey (INT4), default.nation.n_name (TEXT)} - SCAN(18) on eb_0000000000000_0000_000003 + => in schema: {(4) default.customer.c_custkey (INT4), default.customer.c_nationkey (INT4), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4)} + SCAN(3) on default.customer + => target list: default.customer.c_custkey (INT4), default.customer.c_nationkey (INT4) + => out schema: {(2) default.customer.c_custkey (INT4), default.customer.c_nationkey (INT4)} + => in schema: {(8) default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT)} + SCAN(4) on default.nation + => target list: default.nation.n_name (TEXT), default.nation.n_nationkey (INT4) + => out schema: {(2) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4)} + => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + JOIN(14)(INNER) + => Join Cond: default.lineitem.l_partkey (INT4) = default.part.p_partkey (INT4) + => target list: default.lineitem.l_orderkey (INT4), default.orders.o_custkey (INT4), default.part.p_name (TEXT) => out schema: {(3) default.lineitem.l_orderkey (INT4), default.orders.o_custkey (INT4), default.part.p_name (TEXT)} - => in schema: {(3) default.lineitem.l_orderkey (INT4), default.orders.o_custkey (INT4), default.part.p_name (TEXT)} + => in schema: {(5) default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.orders.o_custkey (INT4), default.part.p_name (TEXT), default.part.p_partkey (INT4)} + SCAN(2) on default.part + => target list: default.part.p_name (TEXT), default.part.p_partkey (INT4) + => out schema: {(2) default.part.p_name (TEXT), default.part.p_partkey (INT4)} + => in schema: {(9) default.part.p_brand (TEXT), default.part.p_comment (TEXT), default.part.p_container (TEXT), default.part.p_mfgr (TEXT), default.part.p_name (TEXT), default.part.p_partkey (INT4), default.part.p_retailprice (FLOAT8), default.part.p_size (INT4), default.part.p_type (TEXT)} + JOIN(13)(INNER) + => Join Cond: default.lineitem.l_orderkey (INT4) = default.orders.o_orderkey (INT4) + => target list: default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.orders.o_custkey (INT4) + => out schema: {(3) default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.orders.o_custkey (INT4)} + => in schema: {(4) default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.orders.o_custkey (INT4), default.orders.o_orderkey (INT4)} + SCAN(0) on default.lineitem + => target list: default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4) + => out schema: {(2) default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4)} + => in schema: {(16) default.lineitem.l_comment (TEXT), default.lineitem.l_commitdate (TEXT), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_linenumber (INT4), default.lineitem.l_linestatus (TEXT), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.lineitem.l_quantity (FLOAT8), default.lineitem.l_receiptdate (TEXT), default.lineitem.l_returnflag (TEXT), default.lineitem.l_shipdate (TEXT), default.lineitem.l_shipinstruct (TEXT), default.lineitem.l_shipmode (TEXT), default.lineitem.l_suppkey (INT4), default.lineitem.l_tax (FLOAT8)} + SCAN(1) on default.orders + => target list: default.orders.o_custkey (INT4), default.orders.o_orderkey (INT4) + => out schema: {(2) default.orders.o_custkey (INT4), default.orders.o_orderkey (INT4)} + => in schema: {(9) default.orders.o_clerk (TEXT), default.orders.o_comment (TEXT), default.orders.o_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4), default.orders.o_orderpriority (TEXT), default.orders.o_orderstatus (TEXT), default.orders.o_shippriority (INT4), default.orders.o_totalprice (FLOAT8)} ======================================================= -Block Id: eb_0000000000000_0000_000008 [ROOT] +Block Id: eb_0000000000000_0000_000010 [ROOT] ======================================================= [Incoming] -[q_0000000000000_0000] 7 => 8 (type=RANGE_SHUFFLE, key=default.lineitem.l_orderkey (INT4), default.nation.n_name (TEXT), default.part.p_name (TEXT), num=32) +[q_0000000000000_0000] 9 => 10 (type=RANGE_SHUFFLE, key=default.lineitem.l_orderkey (INT4), default.nation.n_name (TEXT), default.part.p_name (TEXT), num=32) [Enforcers] - 0: sorted input=eb_0000000000000_0000_000007 + 0: sorted input=eb_0000000000000_0000_000009 SORT(6) => Sort Keys: default.lineitem.l_orderkey (INT4) (asc),default.part.p_name (TEXT) (asc),default.nation.n_name (TEXT) (asc) - SCAN(21) on eb_0000000000000_0000_000007 + SCAN(27) on eb_0000000000000_0000_000009 => out schema: {(3) default.lineitem.l_orderkey (INT4), default.nation.n_name (TEXT), default.part.p_name (TEXT)} => in schema: {(3) default.lineitem.l_orderkey (INT4), default.nation.n_name (TEXT), default.part.p_name (TEXT)} ======================================================= -Block Id: eb_0000000000000_0000_000009 [TERMINAL] +Block Id: eb_0000000000000_0000_000011 [TERMINAL] ======================================================= diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testBroadcastTwoPartJoin.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testBroadcastTwoPartJoin.Sort.plan index a308b1570b..ec92307e62 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testBroadcastTwoPartJoin.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testBroadcastTwoPartJoin.Sort.plan @@ -45,124 +45,92 @@ SORT(6) explain ------------------------------- ------------------------------------------------------------------------------- -Execution Block Graph (TERMINAL - eb_0000000000000_0000_000009) +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000011) ------------------------------------------------------------------------------- -|-eb_0000000000000_0000_000009 - |-eb_0000000000000_0000_000008 - |-eb_0000000000000_0000_000007 - |-eb_0000000000000_0000_000006 - |-eb_0000000000000_0000_000003 +|-eb_0000000000000_0000_000011 + |-eb_0000000000000_0000_000010 + |-eb_0000000000000_0000_000009 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000006 -3: eb_0000000000000_0000_000007 -4: eb_0000000000000_0000_000008 -5: eb_0000000000000_0000_000009 +1: eb_0000000000000_0000_000009 +2: eb_0000000000000_0000_000010 +3: eb_0000000000000_0000_000011 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [LEAF] +Block Id: eb_0000000000000_0000_000009 [LEAF] ======================================================= [Outgoing] -[q_0000000000000_0000] 3 => 7 (type=HASH_SHUFFLE, key=default.orders.o_custkey (INT4), num=32) +[q_0000000000000_0000] 9 => 10 (type=RANGE_SHUFFLE, key=default.lineitem.l_orderkey (INT4), default.nation.n_name (TEXT), default.part.p_name (TEXT), num=32) [Enforcers] 0: type=Broadcast, tables=default.orders - 1: type=Broadcast, tables=default.lineitem - 2: type=Broadcast, tables=default.part + 1: type=Broadcast, tables=default.part + 2: type=Broadcast, tables=default.lineitem + 3: type=Broadcast, tables=default.nation + 4: type=Broadcast, tables=default.customer -JOIN(14)(INNER) - => Join Cond: default.lineitem.l_partkey (INT4) = default.part.p_partkey (INT4) - => target list: default.lineitem.l_orderkey (INT4), default.orders.o_custkey (INT4), default.part.p_name (TEXT) - => out schema: {(3) default.lineitem.l_orderkey (INT4), default.orders.o_custkey (INT4), default.part.p_name (TEXT)} - => in schema: {(5) default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.orders.o_custkey (INT4), default.part.p_name (TEXT), default.part.p_partkey (INT4)} - SCAN(2) on default.part - => target list: default.part.p_name (TEXT), default.part.p_partkey (INT4) - => out schema: {(2) default.part.p_name (TEXT), default.part.p_partkey (INT4)} - => in schema: {(9) default.part.p_brand (TEXT), default.part.p_comment (TEXT), default.part.p_container (TEXT), default.part.p_mfgr (TEXT), default.part.p_name (TEXT), default.part.p_partkey (INT4), default.part.p_retailprice (FLOAT8), default.part.p_size (INT4), default.part.p_type (TEXT)} - JOIN(13)(INNER) - => Join Cond: default.lineitem.l_orderkey (INT4) = default.orders.o_orderkey (INT4) - => target list: default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.orders.o_custkey (INT4) - => out schema: {(3) default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.orders.o_custkey (INT4)} - => in schema: {(4) default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.orders.o_custkey (INT4), default.orders.o_orderkey (INT4)} - SCAN(0) on default.lineitem - => target list: default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4) - => out schema: {(2) default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4)} - => in schema: {(16) default.lineitem.l_comment (TEXT), default.lineitem.l_commitdate (TEXT), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_linenumber (INT4), default.lineitem.l_linestatus (TEXT), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.lineitem.l_quantity (FLOAT8), default.lineitem.l_receiptdate (TEXT), default.lineitem.l_returnflag (TEXT), default.lineitem.l_shipdate (TEXT), default.lineitem.l_shipinstruct (TEXT), default.lineitem.l_shipmode (TEXT), default.lineitem.l_suppkey (INT4), default.lineitem.l_tax (FLOAT8)} - SCAN(1) on default.orders - => target list: default.orders.o_custkey (INT4), default.orders.o_orderkey (INT4) - => out schema: {(2) default.orders.o_custkey (INT4), default.orders.o_orderkey (INT4)} - => in schema: {(9) default.orders.o_clerk (TEXT), default.orders.o_comment (TEXT), default.orders.o_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4), default.orders.o_orderpriority (TEXT), default.orders.o_orderstatus (TEXT), default.orders.o_shippriority (INT4), default.orders.o_totalprice (FLOAT8)} - -======================================================= -Block Id: eb_0000000000000_0000_000006 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 6 => 7 (type=HASH_SHUFFLE, key=default.customer.c_custkey (INT4), num=32) - -[Enforcers] - 0: type=Broadcast, tables=default.nation - 1: type=Broadcast, tables=default.customer - -JOIN(15)(INNER) - => Join Cond: default.customer.c_nationkey (INT4) = default.nation.n_nationkey (INT4) - => target list: default.customer.c_custkey (INT4), default.nation.n_name (TEXT) - => out schema: {(2) default.customer.c_custkey (INT4), default.nation.n_name (TEXT)} - => in schema: {(4) default.customer.c_custkey (INT4), default.customer.c_nationkey (INT4), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4)} - SCAN(3) on default.customer - => target list: default.customer.c_custkey (INT4), default.customer.c_nationkey (INT4) - => out schema: {(2) default.customer.c_custkey (INT4), default.customer.c_nationkey (INT4)} - => in schema: {(8) default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT)} - SCAN(4) on default.nation - => target list: default.nation.n_name (TEXT), default.nation.n_nationkey (INT4) - => out schema: {(2) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4)} - => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000007 [INTERMEDIATE] -======================================================= - -[Incoming] -[q_0000000000000_0000] 3 => 7 (type=HASH_SHUFFLE, key=default.orders.o_custkey (INT4), num=32) -[q_0000000000000_0000] 6 => 7 (type=HASH_SHUFFLE, key=default.customer.c_custkey (INT4), num=32) - -[Outgoing] -[q_0000000000000_0000] 7 => 8 (type=RANGE_SHUFFLE, key=default.lineitem.l_orderkey (INT4), default.nation.n_name (TEXT), default.part.p_name (TEXT), num=32) - -SORT(20) +SORT(26) => Sort Keys: default.lineitem.l_orderkey (INT4) (asc),default.part.p_name (TEXT) (asc),default.nation.n_name (TEXT) (asc) JOIN(16)(INNER) => Join Cond: default.orders.o_custkey (INT4) = default.customer.c_custkey (INT4) => target list: default.lineitem.l_orderkey (INT4), default.nation.n_name (TEXT), default.part.p_name (TEXT) => out schema: {(3) default.lineitem.l_orderkey (INT4), default.nation.n_name (TEXT), default.part.p_name (TEXT)} => in schema: {(5) default.customer.c_custkey (INT4), default.lineitem.l_orderkey (INT4), default.nation.n_name (TEXT), default.orders.o_custkey (INT4), default.part.p_name (TEXT)} - SCAN(19) on eb_0000000000000_0000_000006 + JOIN(15)(INNER) + => Join Cond: default.customer.c_nationkey (INT4) = default.nation.n_nationkey (INT4) + => target list: default.customer.c_custkey (INT4), default.nation.n_name (TEXT) => out schema: {(2) default.customer.c_custkey (INT4), default.nation.n_name (TEXT)} - => in schema: {(2) default.customer.c_custkey (INT4), default.nation.n_name (TEXT)} - SCAN(18) on eb_0000000000000_0000_000003 + => in schema: {(4) default.customer.c_custkey (INT4), default.customer.c_nationkey (INT4), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4)} + SCAN(3) on default.customer + => target list: default.customer.c_custkey (INT4), default.customer.c_nationkey (INT4) + => out schema: {(2) default.customer.c_custkey (INT4), default.customer.c_nationkey (INT4)} + => in schema: {(8) default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT)} + SCAN(4) on default.nation + => target list: default.nation.n_name (TEXT), default.nation.n_nationkey (INT4) + => out schema: {(2) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4)} + => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + JOIN(14)(INNER) + => Join Cond: default.lineitem.l_partkey (INT4) = default.part.p_partkey (INT4) + => target list: default.lineitem.l_orderkey (INT4), default.orders.o_custkey (INT4), default.part.p_name (TEXT) => out schema: {(3) default.lineitem.l_orderkey (INT4), default.orders.o_custkey (INT4), default.part.p_name (TEXT)} - => in schema: {(3) default.lineitem.l_orderkey (INT4), default.orders.o_custkey (INT4), default.part.p_name (TEXT)} + => in schema: {(5) default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.orders.o_custkey (INT4), default.part.p_name (TEXT), default.part.p_partkey (INT4)} + SCAN(2) on default.part + => target list: default.part.p_name (TEXT), default.part.p_partkey (INT4) + => out schema: {(2) default.part.p_name (TEXT), default.part.p_partkey (INT4)} + => in schema: {(9) default.part.p_brand (TEXT), default.part.p_comment (TEXT), default.part.p_container (TEXT), default.part.p_mfgr (TEXT), default.part.p_name (TEXT), default.part.p_partkey (INT4), default.part.p_retailprice (FLOAT8), default.part.p_size (INT4), default.part.p_type (TEXT)} + JOIN(13)(INNER) + => Join Cond: default.lineitem.l_orderkey (INT4) = default.orders.o_orderkey (INT4) + => target list: default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.orders.o_custkey (INT4) + => out schema: {(3) default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.orders.o_custkey (INT4)} + => in schema: {(4) default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.orders.o_custkey (INT4), default.orders.o_orderkey (INT4)} + SCAN(0) on default.lineitem + => target list: default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4) + => out schema: {(2) default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4)} + => in schema: {(16) default.lineitem.l_comment (TEXT), default.lineitem.l_commitdate (TEXT), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_linenumber (INT4), default.lineitem.l_linestatus (TEXT), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.lineitem.l_quantity (FLOAT8), default.lineitem.l_receiptdate (TEXT), default.lineitem.l_returnflag (TEXT), default.lineitem.l_shipdate (TEXT), default.lineitem.l_shipinstruct (TEXT), default.lineitem.l_shipmode (TEXT), default.lineitem.l_suppkey (INT4), default.lineitem.l_tax (FLOAT8)} + SCAN(1) on default.orders + => target list: default.orders.o_custkey (INT4), default.orders.o_orderkey (INT4) + => out schema: {(2) default.orders.o_custkey (INT4), default.orders.o_orderkey (INT4)} + => in schema: {(9) default.orders.o_clerk (TEXT), default.orders.o_comment (TEXT), default.orders.o_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4), default.orders.o_orderpriority (TEXT), default.orders.o_orderstatus (TEXT), default.orders.o_shippriority (INT4), default.orders.o_totalprice (FLOAT8)} ======================================================= -Block Id: eb_0000000000000_0000_000008 [ROOT] +Block Id: eb_0000000000000_0000_000010 [ROOT] ======================================================= [Incoming] -[q_0000000000000_0000] 7 => 8 (type=RANGE_SHUFFLE, key=default.lineitem.l_orderkey (INT4), default.nation.n_name (TEXT), default.part.p_name (TEXT), num=32) +[q_0000000000000_0000] 9 => 10 (type=RANGE_SHUFFLE, key=default.lineitem.l_orderkey (INT4), default.nation.n_name (TEXT), default.part.p_name (TEXT), num=32) [Enforcers] - 0: sorted input=eb_0000000000000_0000_000007 + 0: sorted input=eb_0000000000000_0000_000009 SORT(6) => Sort Keys: default.lineitem.l_orderkey (INT4) (asc),default.part.p_name (TEXT) (asc),default.nation.n_name (TEXT) (asc) - SCAN(21) on eb_0000000000000_0000_000007 + SCAN(27) on eb_0000000000000_0000_000009 => out schema: {(3) default.lineitem.l_orderkey (INT4), default.nation.n_name (TEXT), default.part.p_name (TEXT)} => in schema: {(3) default.lineitem.l_orderkey (INT4), default.nation.n_name (TEXT), default.part.p_name (TEXT)} ======================================================= -Block Id: eb_0000000000000_0000_000009 [TERMINAL] +Block Id: eb_0000000000000_0000_000011 [TERMINAL] ======================================================= diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition1.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition1.Hash.plan index 2210ad909f..a9c7fe95fd 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition1.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition1.Hash.plan @@ -42,7 +42,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.n1 1: type=Broadcast, tables=default.n2 -SORT(8) +SORT(10) => Sort Keys: default.n1.n_nationkey (INT4) (asc) JOIN(6)(INNER) => Join Cond: default.n1.n_name (TEXT) = ?upper_1 (TEXT) @@ -70,7 +70,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.n1.n_nationkey (INT4) (asc) - SCAN(9) on eb_0000000000000_0000_000003 + SCAN(11) on eb_0000000000000_0000_000003 => out schema: {(3) default.n1.n_name (TEXT), default.n1.n_nationkey (INT4), default.n2.n_name (TEXT)} => in schema: {(3) default.n1.n_name (TEXT), default.n1.n_nationkey (INT4), default.n2.n_name (TEXT)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition1.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition1.Sort.plan index 2210ad909f..a9c7fe95fd 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition1.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition1.Sort.plan @@ -42,7 +42,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.n1 1: type=Broadcast, tables=default.n2 -SORT(8) +SORT(10) => Sort Keys: default.n1.n_nationkey (INT4) (asc) JOIN(6)(INNER) => Join Cond: default.n1.n_name (TEXT) = ?upper_1 (TEXT) @@ -70,7 +70,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.n1.n_nationkey (INT4) (asc) - SCAN(9) on eb_0000000000000_0000_000003 + SCAN(11) on eb_0000000000000_0000_000003 => out schema: {(3) default.n1.n_name (TEXT), default.n1.n_nationkey (INT4), default.n2.n_name (TEXT)} => in schema: {(3) default.n1.n_name (TEXT), default.n1.n_nationkey (INT4), default.n2.n_name (TEXT)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition2.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition2.Hash.plan index 51a06459a8..0e5f78a6bb 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition2.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition2.Hash.plan @@ -42,7 +42,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.n1 1: type=Broadcast, tables=default.n2 -SORT(8) +SORT(10) => Sort Keys: default.n1.n_nationkey (INT4) (asc) JOIN(6)(INNER) => Join Cond: default.n1.n_name (TEXT) = name (TEXT) @@ -70,7 +70,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.n1.n_nationkey (INT4) (asc) - SCAN(9) on eb_0000000000000_0000_000003 + SCAN(11) on eb_0000000000000_0000_000003 => out schema: {(3) default.n1.n_name (TEXT), default.n1.n_nationkey (INT4), name (TEXT)} => in schema: {(3) default.n1.n_name (TEXT), default.n1.n_nationkey (INT4), name (TEXT)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition2.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition2.Sort.plan index 51a06459a8..0e5f78a6bb 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition2.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition2.Sort.plan @@ -42,7 +42,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.n1 1: type=Broadcast, tables=default.n2 -SORT(8) +SORT(10) => Sort Keys: default.n1.n_nationkey (INT4) (asc) JOIN(6)(INNER) => Join Cond: default.n1.n_name (TEXT) = name (TEXT) @@ -70,7 +70,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.n1.n_nationkey (INT4) (asc) - SCAN(9) on eb_0000000000000_0000_000003 + SCAN(11) on eb_0000000000000_0000_000003 => out schema: {(3) default.n1.n_name (TEXT), default.n1.n_nationkey (INT4), name (TEXT)} => in schema: {(3) default.n1.n_name (TEXT), default.n1.n_nationkey (INT4), name (TEXT)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition3.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition3.Hash.plan index 97fd030ab5..e5ada3c991 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition3.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition3.Hash.plan @@ -42,7 +42,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.n1 1: type=Broadcast, tables=default.n2 -SORT(8) +SORT(10) => Sort Keys: default.n1.n_nationkey (INT4) (asc) JOIN(6)(INNER) => Join Cond: ?lower_1 (TEXT) = ?lower_2 (TEXT) @@ -70,7 +70,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.n1.n_nationkey (INT4) (asc) - SCAN(9) on eb_0000000000000_0000_000003 + SCAN(11) on eb_0000000000000_0000_000003 => out schema: {(3) default.n1.n_name (TEXT), default.n1.n_nationkey (INT4), default.n2.n_name (TEXT)} => in schema: {(3) default.n1.n_name (TEXT), default.n1.n_nationkey (INT4), default.n2.n_name (TEXT)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition3.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition3.Sort.plan index 97fd030ab5..e5ada3c991 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition3.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition3.Sort.plan @@ -42,7 +42,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.n1 1: type=Broadcast, tables=default.n2 -SORT(8) +SORT(10) => Sort Keys: default.n1.n_nationkey (INT4) (asc) JOIN(6)(INNER) => Join Cond: ?lower_1 (TEXT) = ?lower_2 (TEXT) @@ -70,7 +70,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.n1.n_nationkey (INT4) (asc) - SCAN(9) on eb_0000000000000_0000_000003 + SCAN(11) on eb_0000000000000_0000_000003 => out schema: {(3) default.n1.n_name (TEXT), default.n1.n_nationkey (INT4), default.n2.n_name (TEXT)} => in schema: {(3) default.n1.n_name (TEXT), default.n1.n_nationkey (INT4), default.n2.n_name (TEXT)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition4.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition4.Hash.plan index 41127e8159..aadfa008d5 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition4.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition4.Hash.plan @@ -42,7 +42,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.n1 1: type=Broadcast, tables=default.n2 -SORT(8) +SORT(10) => Sort Keys: default.n1.n_nationkey (INT4) (asc) JOIN(6)(INNER) => Join Cond: name1 (TEXT) = name2 (TEXT) @@ -70,7 +70,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.n1.n_nationkey (INT4) (asc) - SCAN(9) on eb_0000000000000_0000_000003 + SCAN(11) on eb_0000000000000_0000_000003 => out schema: {(3) default.n1.n_nationkey (INT4), name1 (TEXT), name2 (TEXT)} => in schema: {(3) default.n1.n_nationkey (INT4), name1 (TEXT), name2 (TEXT)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition4.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition4.Sort.plan index 41127e8159..aadfa008d5 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition4.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition4.Sort.plan @@ -42,7 +42,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.n1 1: type=Broadcast, tables=default.n2 -SORT(8) +SORT(10) => Sort Keys: default.n1.n_nationkey (INT4) (asc) JOIN(6)(INNER) => Join Cond: name1 (TEXT) = name2 (TEXT) @@ -70,7 +70,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.n1.n_nationkey (INT4) (asc) - SCAN(9) on eb_0000000000000_0000_000003 + SCAN(11) on eb_0000000000000_0000_000003 => out schema: {(3) default.n1.n_nationkey (INT4), name1 (TEXT), name2 (TEXT)} => in schema: {(3) default.n1.n_nationkey (INT4), name1 (TEXT), name2 (TEXT)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.1.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.1.Hash.plan index c11b4c8ae0..d76de99950 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.1.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.1.Hash.plan @@ -41,7 +41,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.nation 1: type=Broadcast, tables=default.region -SORT(8) +SORT(10) => Sort Keys: default.nation.n_name (TEXT) (asc),default.region.r_name (TEXT) (asc) JOIN(6)(CROSS) => target list: default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.region.r_name (TEXT), default.region.r_regionkey (INT4) @@ -68,7 +68,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(2) => Sort Keys: default.nation.n_name (TEXT) (asc),default.region.r_name (TEXT) (asc) - SCAN(9) on eb_0000000000000_0000_000003 + SCAN(11) on eb_0000000000000_0000_000003 => out schema: {(4) default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} => in schema: {(4) default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.1.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.1.Sort.plan index c11b4c8ae0..d76de99950 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.1.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.1.Sort.plan @@ -41,7 +41,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.nation 1: type=Broadcast, tables=default.region -SORT(8) +SORT(10) => Sort Keys: default.nation.n_name (TEXT) (asc),default.region.r_name (TEXT) (asc) JOIN(6)(CROSS) => target list: default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.region.r_name (TEXT), default.region.r_regionkey (INT4) @@ -68,7 +68,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(2) => Sort Keys: default.nation.n_name (TEXT) (asc),default.region.r_name (TEXT) (asc) - SCAN(9) on eb_0000000000000_0000_000003 + SCAN(11) on eb_0000000000000_0000_000003 => out schema: {(4) default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} => in schema: {(4) default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.5.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.5.Hash.plan index 5b81a0569a..af24f2014a 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.5.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.5.Hash.plan @@ -41,7 +41,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.customer 1: type=Broadcast, tables=default.region -SORT(8) +SORT(10) => Sort Keys: len (INT4) (asc),default.region.r_regionkey (INT4) (asc),default.region.r_name (TEXT) (asc) JOIN(6)(CROSS) => target list: ?multiply (INT4), default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT), default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4), len (INT4) @@ -68,7 +68,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(2) => Sort Keys: len (INT4) (asc),default.region.r_regionkey (INT4) (asc),default.region.r_name (TEXT) (asc) - SCAN(9) on eb_0000000000000_0000_000003 + SCAN(11) on eb_0000000000000_0000_000003 => out schema: {(13) ?multiply (INT4), default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT), default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4), len (INT4)} => in schema: {(13) ?multiply (INT4), default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT), default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4), len (INT4)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.5.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.5.Sort.plan index 5b81a0569a..af24f2014a 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.5.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.5.Sort.plan @@ -41,7 +41,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.customer 1: type=Broadcast, tables=default.region -SORT(8) +SORT(10) => Sort Keys: len (INT4) (asc),default.region.r_regionkey (INT4) (asc),default.region.r_name (TEXT) (asc) JOIN(6)(CROSS) => target list: ?multiply (INT4), default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT), default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4), len (INT4) @@ -68,7 +68,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(2) => Sort Keys: len (INT4) (asc),default.region.r_regionkey (INT4) (asc),default.region.r_name (TEXT) (asc) - SCAN(9) on eb_0000000000000_0000_000003 + SCAN(11) on eb_0000000000000_0000_000003 => out schema: {(13) ?multiply (INT4), default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT), default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4), len (INT4)} => in schema: {(13) ?multiply (INT4), default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT), default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4), len (INT4)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinAndCaseWhen.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinAndCaseWhen.Hash.plan index c083fa1bc8..ebeae68c2a 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinAndCaseWhen.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinAndCaseWhen.Hash.plan @@ -42,7 +42,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.region 1: type=Broadcast, tables=default.nation -SORT(9) +SORT(11) => Sort Keys: default.region.r_regionkey (INT4) (asc),default.nation.n_regionkey (INT4) (asc) JOIN(7)(INNER) => Join Cond: default.region.r_regionkey (INT4) = default.nation.n_regionkey (INT4) @@ -70,7 +70,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.region.r_regionkey (INT4) (asc),default.nation.n_regionkey (INT4) (asc) - SCAN(10) on eb_0000000000000_0000_000003 + SCAN(12) on eb_0000000000000_0000_000003 => out schema: {(3) cond (TEXT), default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4)} => in schema: {(3) cond (TEXT), default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinAndCaseWhen.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinAndCaseWhen.Sort.plan index c083fa1bc8..ebeae68c2a 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinAndCaseWhen.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinAndCaseWhen.Sort.plan @@ -42,7 +42,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.region 1: type=Broadcast, tables=default.nation -SORT(9) +SORT(11) => Sort Keys: default.region.r_regionkey (INT4) (asc),default.nation.n_regionkey (INT4) (asc) JOIN(7)(INNER) => Join Cond: default.region.r_regionkey (INT4) = default.nation.n_regionkey (INT4) @@ -70,7 +70,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.region.r_regionkey (INT4) (asc),default.nation.n_regionkey (INT4) (asc) - SCAN(10) on eb_0000000000000_0000_000003 + SCAN(12) on eb_0000000000000_0000_000003 => out schema: {(3) cond (TEXT), default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4)} => in schema: {(3) cond (TEXT), default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk1.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk1.Hash.plan index 98f058cf01..0ff60b1604 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk1.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk1.Hash.plan @@ -41,7 +41,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.region 1: type=Broadcast, tables=default.customer -SORT(8) +SORT(10) => Sort Keys: default.region.r_regionkey (INT4) (asc),default.region.r_name (TEXT) (asc),default.customer.c_custkey (INT4) (asc) JOIN(6)(CROSS) => target list: default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT), default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4) @@ -68,7 +68,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(2) => Sort Keys: default.region.r_regionkey (INT4) (asc),default.region.r_name (TEXT) (asc),default.customer.c_custkey (INT4) (asc) - SCAN(9) on eb_0000000000000_0000_000003 + SCAN(11) on eb_0000000000000_0000_000003 => out schema: {(11) default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT), default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} => in schema: {(11) default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT), default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk1.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk1.Sort.plan index 98f058cf01..0ff60b1604 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk1.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk1.Sort.plan @@ -41,7 +41,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.region 1: type=Broadcast, tables=default.customer -SORT(8) +SORT(10) => Sort Keys: default.region.r_regionkey (INT4) (asc),default.region.r_name (TEXT) (asc),default.customer.c_custkey (INT4) (asc) JOIN(6)(CROSS) => target list: default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT), default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4) @@ -68,7 +68,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(2) => Sort Keys: default.region.r_regionkey (INT4) (asc),default.region.r_name (TEXT) (asc),default.customer.c_custkey (INT4) (asc) - SCAN(9) on eb_0000000000000_0000_000003 + SCAN(11) on eb_0000000000000_0000_000003 => out schema: {(11) default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT), default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} => in schema: {(11) default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT), default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk2.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk2.Hash.plan index 455b6d2c03..a197d6fbfe 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk2.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk2.Hash.plan @@ -41,7 +41,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.customer 1: type=Broadcast, tables=default.region -SORT(8) +SORT(10) => Sort Keys: default.region.r_regionkey (INT4) (asc),default.region.r_name (TEXT) (asc),default.customer.c_custkey (INT4) (asc) JOIN(6)(CROSS) => target list: default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT), default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4) @@ -68,7 +68,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(2) => Sort Keys: default.region.r_regionkey (INT4) (asc),default.region.r_name (TEXT) (asc),default.customer.c_custkey (INT4) (asc) - SCAN(9) on eb_0000000000000_0000_000003 + SCAN(11) on eb_0000000000000_0000_000003 => out schema: {(11) default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT), default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} => in schema: {(11) default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT), default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk2.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk2.Sort.plan index 455b6d2c03..a197d6fbfe 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk2.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk2.Sort.plan @@ -41,7 +41,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.customer 1: type=Broadcast, tables=default.region -SORT(8) +SORT(10) => Sort Keys: default.region.r_regionkey (INT4) (asc),default.region.r_name (TEXT) (asc),default.customer.c_custkey (INT4) (asc) JOIN(6)(CROSS) => target list: default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT), default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4) @@ -68,7 +68,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(2) => Sort Keys: default.region.r_regionkey (INT4) (asc),default.region.r_name (TEXT) (asc),default.customer.c_custkey (INT4) (asc) - SCAN(9) on eb_0000000000000_0000_000003 + SCAN(11) on eb_0000000000000_0000_000003 => out schema: {(11) default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT), default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} => in schema: {(11) default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT), default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk3.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk3.Hash.plan index 8ad610e635..f285a1d2ac 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk3.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk3.Hash.plan @@ -41,7 +41,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.customer 1: type=Broadcast, tables=default.region -SORT(8) +SORT(10) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.customer.c_name (TEXT) (asc),default.region.r_regionkey (INT4) (asc) JOIN(6)(CROSS) => target list: default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT), default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4) @@ -68,7 +68,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(2) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.customer.c_name (TEXT) (asc),default.region.r_regionkey (INT4) (asc) - SCAN(9) on eb_0000000000000_0000_000003 + SCAN(11) on eb_0000000000000_0000_000003 => out schema: {(11) default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT), default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} => in schema: {(11) default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT), default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk3.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk3.Sort.plan index 8ad610e635..f285a1d2ac 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk3.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk3.Sort.plan @@ -41,7 +41,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.customer 1: type=Broadcast, tables=default.region -SORT(8) +SORT(10) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.customer.c_name (TEXT) (asc),default.region.r_regionkey (INT4) (asc) JOIN(6)(CROSS) => target list: default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT), default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4) @@ -68,7 +68,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(2) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.customer.c_name (TEXT) (asc),default.region.r_regionkey (INT4) (asc) - SCAN(9) on eb_0000000000000_0000_000003 + SCAN(11) on eb_0000000000000_0000_000003 => out schema: {(11) default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT), default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} => in schema: {(11) default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT), default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk4.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk4.Hash.plan index 5cf2a69562..370a12688f 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk4.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk4.Hash.plan @@ -41,7 +41,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.customer 1: type=Broadcast, tables=default.region -SORT(8) +SORT(10) => Sort Keys: len (INT4) (asc),default.region.r_regionkey (INT4) (asc),default.region.r_name (TEXT) (asc),default.customer.c_custkey (INT4) (asc) JOIN(6)(CROSS) => target list: ?multiply (INT4), default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT), default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4), len (INT4) @@ -68,7 +68,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(2) => Sort Keys: len (INT4) (asc),default.region.r_regionkey (INT4) (asc),default.region.r_name (TEXT) (asc),default.customer.c_custkey (INT4) (asc) - SCAN(9) on eb_0000000000000_0000_000003 + SCAN(11) on eb_0000000000000_0000_000003 => out schema: {(13) ?multiply (INT4), default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT), default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4), len (INT4)} => in schema: {(13) ?multiply (INT4), default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT), default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4), len (INT4)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk4.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk4.Sort.plan index 5cf2a69562..370a12688f 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk4.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk4.Sort.plan @@ -41,7 +41,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.customer 1: type=Broadcast, tables=default.region -SORT(8) +SORT(10) => Sort Keys: len (INT4) (asc),default.region.r_regionkey (INT4) (asc),default.region.r_name (TEXT) (asc),default.customer.c_custkey (INT4) (asc) JOIN(6)(CROSS) => target list: ?multiply (INT4), default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT), default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4), len (INT4) @@ -68,7 +68,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(2) => Sort Keys: len (INT4) (asc),default.region.r_regionkey (INT4) (asc),default.region.r_name (TEXT) (asc),default.customer.c_custkey (INT4) (asc) - SCAN(9) on eb_0000000000000_0000_000003 + SCAN(11) on eb_0000000000000_0000_000003 => out schema: {(13) ?multiply (INT4), default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT), default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4), len (INT4)} => in schema: {(13) ?multiply (INT4), default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT), default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4), len (INT4)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithEmptyTable1.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithEmptyTable1.Hash.plan index d83cdb8edd..8906ddfe4b 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithEmptyTable1.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithEmptyTable1.Hash.plan @@ -42,7 +42,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.empty_orders 1: type=Broadcast, tables=default.customer -SORT(9) +SORT(11) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.empty_orders.o_orderkey (INT4) (asc) JOIN(7)(INNER) => Join Cond: default.customer.c_custkey (INT4) = default.empty_orders.o_orderkey (INT4) @@ -70,7 +70,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.empty_orders.o_orderkey (INT4) (asc) - SCAN(10) on eb_0000000000000_0000_000003 + SCAN(12) on eb_0000000000000_0000_000003 => out schema: {(2) default.customer.c_custkey (INT4), default.empty_orders.o_orderkey (INT4)} => in schema: {(2) default.customer.c_custkey (INT4), default.empty_orders.o_orderkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithEmptyTable1.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithEmptyTable1.Sort.plan index d83cdb8edd..8906ddfe4b 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithEmptyTable1.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithEmptyTable1.Sort.plan @@ -42,7 +42,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.empty_orders 1: type=Broadcast, tables=default.customer -SORT(9) +SORT(11) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.empty_orders.o_orderkey (INT4) (asc) JOIN(7)(INNER) => Join Cond: default.customer.c_custkey (INT4) = default.empty_orders.o_orderkey (INT4) @@ -70,7 +70,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.empty_orders.o_orderkey (INT4) (asc) - SCAN(10) on eb_0000000000000_0000_000003 + SCAN(12) on eb_0000000000000_0000_000003 => out schema: {(2) default.customer.c_custkey (INT4), default.empty_orders.o_orderkey (INT4)} => in schema: {(2) default.customer.c_custkey (INT4), default.empty_orders.o_orderkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testInnerJoinAndCaseWhen.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testInnerJoinAndCaseWhen.Hash.plan index c083fa1bc8..ebeae68c2a 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testInnerJoinAndCaseWhen.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testInnerJoinAndCaseWhen.Hash.plan @@ -42,7 +42,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.region 1: type=Broadcast, tables=default.nation -SORT(9) +SORT(11) => Sort Keys: default.region.r_regionkey (INT4) (asc),default.nation.n_regionkey (INT4) (asc) JOIN(7)(INNER) => Join Cond: default.region.r_regionkey (INT4) = default.nation.n_regionkey (INT4) @@ -70,7 +70,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.region.r_regionkey (INT4) (asc),default.nation.n_regionkey (INT4) (asc) - SCAN(10) on eb_0000000000000_0000_000003 + SCAN(12) on eb_0000000000000_0000_000003 => out schema: {(3) cond (TEXT), default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4)} => in schema: {(3) cond (TEXT), default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testInnerJoinAndCaseWhen.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testInnerJoinAndCaseWhen.Sort.plan index c083fa1bc8..ebeae68c2a 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testInnerJoinAndCaseWhen.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testInnerJoinAndCaseWhen.Sort.plan @@ -42,7 +42,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.region 1: type=Broadcast, tables=default.nation -SORT(9) +SORT(11) => Sort Keys: default.region.r_regionkey (INT4) (asc),default.nation.n_regionkey (INT4) (asc) JOIN(7)(INNER) => Join Cond: default.region.r_regionkey (INT4) = default.nation.n_regionkey (INT4) @@ -70,7 +70,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.region.r_regionkey (INT4) (asc),default.nation.n_regionkey (INT4) (asc) - SCAN(10) on eb_0000000000000_0000_000003 + SCAN(12) on eb_0000000000000_0000_000003 => out schema: {(3) cond (TEXT), default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4)} => in schema: {(3) cond (TEXT), default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testInnerJoinWithEmptyTable.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testInnerJoinWithEmptyTable.Hash.plan index d83cdb8edd..8906ddfe4b 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testInnerJoinWithEmptyTable.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testInnerJoinWithEmptyTable.Hash.plan @@ -42,7 +42,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.empty_orders 1: type=Broadcast, tables=default.customer -SORT(9) +SORT(11) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.empty_orders.o_orderkey (INT4) (asc) JOIN(7)(INNER) => Join Cond: default.customer.c_custkey (INT4) = default.empty_orders.o_orderkey (INT4) @@ -70,7 +70,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.empty_orders.o_orderkey (INT4) (asc) - SCAN(10) on eb_0000000000000_0000_000003 + SCAN(12) on eb_0000000000000_0000_000003 => out schema: {(2) default.customer.c_custkey (INT4), default.empty_orders.o_orderkey (INT4)} => in schema: {(2) default.customer.c_custkey (INT4), default.empty_orders.o_orderkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testInnerJoinWithEmptyTable.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testInnerJoinWithEmptyTable.Sort.plan index d83cdb8edd..8906ddfe4b 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testInnerJoinWithEmptyTable.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testInnerJoinWithEmptyTable.Sort.plan @@ -42,7 +42,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.empty_orders 1: type=Broadcast, tables=default.customer -SORT(9) +SORT(11) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.empty_orders.o_orderkey (INT4) (asc) JOIN(7)(INNER) => Join Cond: default.customer.c_custkey (INT4) = default.empty_orders.o_orderkey (INT4) @@ -70,7 +70,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.empty_orders.o_orderkey (INT4) (asc) - SCAN(10) on eb_0000000000000_0000_000003 + SCAN(12) on eb_0000000000000_0000_000003 => out schema: {(2) default.customer.c_custkey (INT4), default.empty_orders.o_orderkey (INT4)} => in schema: {(2) default.customer.c_custkey (INT4), default.empty_orders.o_orderkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvals1.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvals1.Hash.plan index a5ba593252..6da9a5d982 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvals1.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvals1.Hash.plan @@ -42,7 +42,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.region 1: type=Broadcast, tables=default.nation -SORT(9) +SORT(11) => Sort Keys: default.region.r_regionkey (INT4) (asc),default.nation.n_regionkey (INT4) (asc) JOIN(7)(INNER) => Join Cond: default.region.r_regionkey (INT4) = default.nation.n_regionkey (INT4) @@ -70,7 +70,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.region.r_regionkey (INT4) (asc),default.nation.n_regionkey (INT4) (asc) - SCAN(10) on eb_0000000000000_0000_000003 + SCAN(12) on eb_0000000000000_0000_000003 => out schema: {(3) default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4), plus (INT4)} => in schema: {(3) default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4), plus (INT4)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvals1.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvals1.Sort.plan index a5ba593252..6da9a5d982 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvals1.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvals1.Sort.plan @@ -42,7 +42,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.region 1: type=Broadcast, tables=default.nation -SORT(9) +SORT(11) => Sort Keys: default.region.r_regionkey (INT4) (asc),default.nation.n_regionkey (INT4) (asc) JOIN(7)(INNER) => Join Cond: default.region.r_regionkey (INT4) = default.nation.n_regionkey (INT4) @@ -70,7 +70,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.region.r_regionkey (INT4) (asc),default.nation.n_regionkey (INT4) (asc) - SCAN(10) on eb_0000000000000_0000_000003 + SCAN(12) on eb_0000000000000_0000_000003 => out schema: {(3) default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4), plus (INT4)} => in schema: {(3) default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4), plus (INT4)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvalsWithSameExprs1.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvalsWithSameExprs1.Hash.plan index 32fd54c488..15e5651a63 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvalsWithSameExprs1.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvalsWithSameExprs1.Hash.plan @@ -47,7 +47,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.nation 1: type=Broadcast, tables=default.region -SORT(9) +SORT(11) => Sort Keys: v1 (INT4) (asc),default.nation.n_regionkey (INT4) (asc) JOIN(7)(INNER) => Join Cond: default.region.r_regionkey (INT4) = default.nation.n_regionkey (INT4) @@ -80,7 +80,7 @@ PROJECTION(4) => in schema: {(5) default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4), plus1 (INT4), result (INT4), v1 (INT4)} SORT(3) => Sort Keys: v1 (INT4) (asc),default.nation.n_regionkey (INT4) (asc) - SCAN(10) on eb_0000000000000_0000_000003 + SCAN(12) on eb_0000000000000_0000_000003 => out schema: {(5) default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4), plus1 (INT4), result (INT4), v1 (INT4)} => in schema: {(5) default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4), plus1 (INT4), result (INT4), v1 (INT4)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvalsWithSameExprs1.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvalsWithSameExprs1.Sort.plan index 32fd54c488..15e5651a63 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvalsWithSameExprs1.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvalsWithSameExprs1.Sort.plan @@ -47,7 +47,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.nation 1: type=Broadcast, tables=default.region -SORT(9) +SORT(11) => Sort Keys: v1 (INT4) (asc),default.nation.n_regionkey (INT4) (asc) JOIN(7)(INNER) => Join Cond: default.region.r_regionkey (INT4) = default.nation.n_regionkey (INT4) @@ -80,7 +80,7 @@ PROJECTION(4) => in schema: {(5) default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4), plus1 (INT4), result (INT4), v1 (INT4)} SORT(3) => Sort Keys: v1 (INT4) (asc),default.nation.n_regionkey (INT4) (asc) - SCAN(10) on eb_0000000000000_0000_000003 + SCAN(12) on eb_0000000000000_0000_000003 => out schema: {(5) default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4), plus1 (INT4), result (INT4), v1 (INT4)} => in schema: {(5) default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4), plus1 (INT4), result (INT4), v1 (INT4)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvalsWithSameExprs2.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvalsWithSameExprs2.Hash.plan index 5961cfe9bc..bf84ad173d 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvalsWithSameExprs2.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvalsWithSameExprs2.Hash.plan @@ -54,7 +54,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.nation 1: type=Broadcast, tables=default.region -GROUP_BY(10)(?plus,n_regionkey,r_regionkey,result,v1) +GROUP_BY(12)(?plus,n_regionkey,r_regionkey,result,v1) => exprs: (sum(?plus (INT4))) => target list: v1 (INT4), default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4), ?plus (INT4), result (INT4), ?sum_3 (INT8) => out schema:{(6) v1 (INT4), default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4), ?plus (INT4), result (INT4), ?sum_3 (INT8)} @@ -84,14 +84,14 @@ Block Id: eb_0000000000000_0000_000004 [INTERMEDIATE] [Outgoing] [q_0000000000000_0000] 4 => 5 (type=RANGE_SHUFFLE, key=default.nation.n_regionkey (INT4), v1 (INT4), num=32) -SORT(12) +SORT(14) => Sort Keys: v1 (INT4) (asc),default.nation.n_regionkey (INT4) (asc) GROUP_BY(3)(v1,n_regionkey,r_regionkey,?plus,result) => exprs: (sum(?sum_3 (INT8))) => target list: v1 (INT4), default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4), ?plus (INT4) as plus1, result (INT4), total (INT8) => out schema:{(6) default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4), plus1 (INT4), result (INT4), total (INT8), v1 (INT4)} => in schema:{(6) v1 (INT4), default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4), ?plus (INT4), result (INT4), ?sum_3 (INT8)} - SCAN(11) on eb_0000000000000_0000_000003 + SCAN(13) on eb_0000000000000_0000_000003 => out schema: {(6) v1 (INT4), default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4), ?plus (INT4), result (INT4), ?sum_3 (INT8)} => in schema: {(6) v1 (INT4), default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4), ?plus (INT4), result (INT4), ?sum_3 (INT8)} @@ -111,7 +111,7 @@ PROJECTION(5) => in schema: {(6) default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4), plus1 (INT4), result (INT4), total (INT8), v1 (INT4)} SORT(4) => Sort Keys: v1 (INT4) (asc),default.nation.n_regionkey (INT4) (asc) - SCAN(13) on eb_0000000000000_0000_000004 + SCAN(15) on eb_0000000000000_0000_000004 => out schema: {(6) default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4), plus1 (INT4), result (INT4), total (INT8), v1 (INT4)} => in schema: {(6) default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4), plus1 (INT4), result (INT4), total (INT8), v1 (INT4)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvalsWithSameExprs2.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvalsWithSameExprs2.Sort.plan index 5961cfe9bc..bf84ad173d 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvalsWithSameExprs2.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvalsWithSameExprs2.Sort.plan @@ -54,7 +54,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.nation 1: type=Broadcast, tables=default.region -GROUP_BY(10)(?plus,n_regionkey,r_regionkey,result,v1) +GROUP_BY(12)(?plus,n_regionkey,r_regionkey,result,v1) => exprs: (sum(?plus (INT4))) => target list: v1 (INT4), default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4), ?plus (INT4), result (INT4), ?sum_3 (INT8) => out schema:{(6) v1 (INT4), default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4), ?plus (INT4), result (INT4), ?sum_3 (INT8)} @@ -84,14 +84,14 @@ Block Id: eb_0000000000000_0000_000004 [INTERMEDIATE] [Outgoing] [q_0000000000000_0000] 4 => 5 (type=RANGE_SHUFFLE, key=default.nation.n_regionkey (INT4), v1 (INT4), num=32) -SORT(12) +SORT(14) => Sort Keys: v1 (INT4) (asc),default.nation.n_regionkey (INT4) (asc) GROUP_BY(3)(v1,n_regionkey,r_regionkey,?plus,result) => exprs: (sum(?sum_3 (INT8))) => target list: v1 (INT4), default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4), ?plus (INT4) as plus1, result (INT4), total (INT8) => out schema:{(6) default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4), plus1 (INT4), result (INT4), total (INT8), v1 (INT4)} => in schema:{(6) v1 (INT4), default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4), ?plus (INT4), result (INT4), ?sum_3 (INT8)} - SCAN(11) on eb_0000000000000_0000_000003 + SCAN(13) on eb_0000000000000_0000_000003 => out schema: {(6) v1 (INT4), default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4), ?plus (INT4), result (INT4), ?sum_3 (INT8)} => in schema: {(6) v1 (INT4), default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4), ?plus (INT4), result (INT4), ?sum_3 (INT8)} @@ -111,7 +111,7 @@ PROJECTION(5) => in schema: {(6) default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4), plus1 (INT4), result (INT4), total (INT8), v1 (INT4)} SORT(4) => Sort Keys: v1 (INT4) (asc),default.nation.n_regionkey (INT4) (asc) - SCAN(13) on eb_0000000000000_0000_000004 + SCAN(15) on eb_0000000000000_0000_000004 => out schema: {(6) default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4), plus1 (INT4), result (INT4), total (INT8), v1 (INT4)} => in schema: {(6) default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4), plus1 (INT4), result (INT4), total (INT8), v1 (INT4)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinOnMultipleDatabases.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinOnMultipleDatabases.Hash.plan index 3a80bb43f0..bc8deb0cbc 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinOnMultipleDatabases.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinOnMultipleDatabases.Hash.plan @@ -45,124 +45,92 @@ SORT(6) explain ------------------------------- ------------------------------------------------------------------------------- -Execution Block Graph (TERMINAL - eb_0000000000000_0000_000009) +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000011) ------------------------------------------------------------------------------- -|-eb_0000000000000_0000_000009 - |-eb_0000000000000_0000_000008 - |-eb_0000000000000_0000_000007 - |-eb_0000000000000_0000_000006 - |-eb_0000000000000_0000_000003 +|-eb_0000000000000_0000_000011 + |-eb_0000000000000_0000_000010 + |-eb_0000000000000_0000_000009 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000006 -3: eb_0000000000000_0000_000007 -4: eb_0000000000000_0000_000008 -5: eb_0000000000000_0000_000009 +1: eb_0000000000000_0000_000009 +2: eb_0000000000000_0000_000010 +3: eb_0000000000000_0000_000011 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [LEAF] +Block Id: eb_0000000000000_0000_000009 [LEAF] ======================================================= [Outgoing] -[q_0000000000000_0000] 3 => 7 (type=HASH_SHUFFLE, key=joins.supplier_.s_suppkey (INT4), num=32) +[q_0000000000000_0000] 9 => 10 (type=RANGE_SHUFFLE, key=default.nation.n_name (TEXT), joins.part_.p_partkey (INT4), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_name (TEXT), num=32) [Enforcers] - 0: type=Broadcast, tables=joins.supplier_ - 1: type=Broadcast, tables=default.nation + 0: type=Broadcast, tables=default.nation + 1: type=Broadcast, tables=joins.supplier_ 2: type=Broadcast, tables=default.region + 3: type=Broadcast, tables=joins.part_ + 4: type=Broadcast, tables=default.partsupp -JOIN(14)(INNER) - => Join Cond: default.nation.n_regionkey (INT4) = default.region.r_regionkey (INT4) - => target list: default.nation.n_name (TEXT), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_phone (TEXT), joins.supplier_.s_suppkey (INT4) - => out schema: {(7) default.nation.n_name (TEXT), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_phone (TEXT), joins.supplier_.s_suppkey (INT4)} - => in schema: {(9) default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_phone (TEXT), joins.supplier_.s_suppkey (INT4)} - SCAN(4) on default.region - => target list: default.region.r_regionkey (INT4) - => out schema: {(1) default.region.r_regionkey (INT4)} - => in schema: {(3) default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} - JOIN(13)(INNER) - => Join Cond: joins.supplier_.s_nationkey (INT4) = default.nation.n_nationkey (INT4) - => target list: default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_phone (TEXT), joins.supplier_.s_suppkey (INT4) - => out schema: {(8) default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_phone (TEXT), joins.supplier_.s_suppkey (INT4)} - => in schema: {(10) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_nationkey (INT4), joins.supplier_.s_phone (TEXT), joins.supplier_.s_suppkey (INT4)} - SCAN(3) on default.nation - => target list: default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4) - => out schema: {(3) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} - => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} - SCAN(1) on joins.supplier_ - => target list: joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_nationkey (INT4), joins.supplier_.s_phone (TEXT), joins.supplier_.s_suppkey (INT4) - => out schema: {(7) joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_nationkey (INT4), joins.supplier_.s_phone (TEXT), joins.supplier_.s_suppkey (INT4)} - => in schema: {(7) joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_nationkey (INT4), joins.supplier_.s_phone (TEXT), joins.supplier_.s_suppkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000006 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 6 => 7 (type=HASH_SHUFFLE, key=default.partsupp.ps_suppkey (INT4), num=32) - -[Enforcers] - 0: type=Broadcast, tables=joins.part_ - 1: type=Broadcast, tables=default.partsupp - -JOIN(15)(INNER) - => Join Cond: joins.part_.p_partkey (INT4) = default.partsupp.ps_partkey (INT4) - => target list: default.partsupp.ps_suppkey (INT4), joins.part_.p_mfgr (TEXT), joins.part_.p_partkey (INT4) - => out schema: {(3) default.partsupp.ps_suppkey (INT4), joins.part_.p_mfgr (TEXT), joins.part_.p_partkey (INT4)} - => in schema: {(4) default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4), joins.part_.p_mfgr (TEXT), joins.part_.p_partkey (INT4)} - SCAN(2) on default.partsupp - => target list: default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4) - => out schema: {(2) default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4)} - => in schema: {(5) default.partsupp.ps_availqty (INT4), default.partsupp.ps_comment (TEXT), default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4), default.partsupp.ps_supplycost (FLOAT8)} - SCAN(0) on joins.part_ - => target list: joins.part_.p_mfgr (TEXT), joins.part_.p_partkey (INT4) - => out schema: {(2) joins.part_.p_mfgr (TEXT), joins.part_.p_partkey (INT4)} - => in schema: {(9) joins.part_.p_brand (TEXT), joins.part_.p_comment (TEXT), joins.part_.p_container (TEXT), joins.part_.p_mfgr (TEXT), joins.part_.p_name (TEXT), joins.part_.p_partkey (INT4), joins.part_.p_retailprice (FLOAT8), joins.part_.p_size (INT4), joins.part_.p_type (TEXT)} - -======================================================= -Block Id: eb_0000000000000_0000_000007 [INTERMEDIATE] -======================================================= - -[Incoming] -[q_0000000000000_0000] 3 => 7 (type=HASH_SHUFFLE, key=joins.supplier_.s_suppkey (INT4), num=32) -[q_0000000000000_0000] 6 => 7 (type=HASH_SHUFFLE, key=default.partsupp.ps_suppkey (INT4), num=32) - -[Outgoing] -[q_0000000000000_0000] 7 => 8 (type=RANGE_SHUFFLE, key=default.nation.n_name (TEXT), joins.part_.p_partkey (INT4), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_name (TEXT), num=32) - -SORT(20) +SORT(26) => Sort Keys: joins.supplier_.s_acctbal (FLOAT8) (asc),joins.supplier_.s_name (TEXT) (asc),default.nation.n_name (TEXT) (asc),joins.part_.p_partkey (INT4) (asc) JOIN(16)(INNER) => Join Cond: joins.supplier_.s_suppkey (INT4) = default.partsupp.ps_suppkey (INT4) => target list: default.nation.n_name (TEXT), joins.part_.p_mfgr (TEXT), joins.part_.p_partkey (INT4), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_phone (TEXT) => out schema: {(8) default.nation.n_name (TEXT), joins.part_.p_mfgr (TEXT), joins.part_.p_partkey (INT4), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_phone (TEXT)} => in schema: {(10) default.nation.n_name (TEXT), default.partsupp.ps_suppkey (INT4), joins.part_.p_mfgr (TEXT), joins.part_.p_partkey (INT4), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_phone (TEXT), joins.supplier_.s_suppkey (INT4)} - SCAN(19) on eb_0000000000000_0000_000006 + JOIN(15)(INNER) + => Join Cond: joins.part_.p_partkey (INT4) = default.partsupp.ps_partkey (INT4) + => target list: default.partsupp.ps_suppkey (INT4), joins.part_.p_mfgr (TEXT), joins.part_.p_partkey (INT4) => out schema: {(3) default.partsupp.ps_suppkey (INT4), joins.part_.p_mfgr (TEXT), joins.part_.p_partkey (INT4)} - => in schema: {(3) default.partsupp.ps_suppkey (INT4), joins.part_.p_mfgr (TEXT), joins.part_.p_partkey (INT4)} - SCAN(18) on eb_0000000000000_0000_000003 + => in schema: {(4) default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4), joins.part_.p_mfgr (TEXT), joins.part_.p_partkey (INT4)} + SCAN(2) on default.partsupp + => target list: default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4) + => out schema: {(2) default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4)} + => in schema: {(5) default.partsupp.ps_availqty (INT4), default.partsupp.ps_comment (TEXT), default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4), default.partsupp.ps_supplycost (FLOAT8)} + SCAN(0) on joins.part_ + => target list: joins.part_.p_mfgr (TEXT), joins.part_.p_partkey (INT4) + => out schema: {(2) joins.part_.p_mfgr (TEXT), joins.part_.p_partkey (INT4)} + => in schema: {(9) joins.part_.p_brand (TEXT), joins.part_.p_comment (TEXT), joins.part_.p_container (TEXT), joins.part_.p_mfgr (TEXT), joins.part_.p_name (TEXT), joins.part_.p_partkey (INT4), joins.part_.p_retailprice (FLOAT8), joins.part_.p_size (INT4), joins.part_.p_type (TEXT)} + JOIN(14)(INNER) + => Join Cond: default.nation.n_regionkey (INT4) = default.region.r_regionkey (INT4) + => target list: default.nation.n_name (TEXT), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_phone (TEXT), joins.supplier_.s_suppkey (INT4) => out schema: {(7) default.nation.n_name (TEXT), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_phone (TEXT), joins.supplier_.s_suppkey (INT4)} - => in schema: {(7) default.nation.n_name (TEXT), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_phone (TEXT), joins.supplier_.s_suppkey (INT4)} + => in schema: {(9) default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_phone (TEXT), joins.supplier_.s_suppkey (INT4)} + SCAN(4) on default.region + => target list: default.region.r_regionkey (INT4) + => out schema: {(1) default.region.r_regionkey (INT4)} + => in schema: {(3) default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} + JOIN(13)(INNER) + => Join Cond: joins.supplier_.s_nationkey (INT4) = default.nation.n_nationkey (INT4) + => target list: default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_phone (TEXT), joins.supplier_.s_suppkey (INT4) + => out schema: {(8) default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_phone (TEXT), joins.supplier_.s_suppkey (INT4)} + => in schema: {(10) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_nationkey (INT4), joins.supplier_.s_phone (TEXT), joins.supplier_.s_suppkey (INT4)} + SCAN(3) on default.nation + => target list: default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4) + => out schema: {(3) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + SCAN(1) on joins.supplier_ + => target list: joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_nationkey (INT4), joins.supplier_.s_phone (TEXT), joins.supplier_.s_suppkey (INT4) + => out schema: {(7) joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_nationkey (INT4), joins.supplier_.s_phone (TEXT), joins.supplier_.s_suppkey (INT4)} + => in schema: {(7) joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_nationkey (INT4), joins.supplier_.s_phone (TEXT), joins.supplier_.s_suppkey (INT4)} ======================================================= -Block Id: eb_0000000000000_0000_000008 [ROOT] +Block Id: eb_0000000000000_0000_000010 [ROOT] ======================================================= [Incoming] -[q_0000000000000_0000] 7 => 8 (type=RANGE_SHUFFLE, key=default.nation.n_name (TEXT), joins.part_.p_partkey (INT4), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_name (TEXT), num=32) +[q_0000000000000_0000] 9 => 10 (type=RANGE_SHUFFLE, key=default.nation.n_name (TEXT), joins.part_.p_partkey (INT4), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_name (TEXT), num=32) [Enforcers] - 0: sorted input=eb_0000000000000_0000_000007 + 0: sorted input=eb_0000000000000_0000_000009 SORT(6) => Sort Keys: joins.supplier_.s_acctbal (FLOAT8) (asc),joins.supplier_.s_name (TEXT) (asc),default.nation.n_name (TEXT) (asc),joins.part_.p_partkey (INT4) (asc) - SCAN(21) on eb_0000000000000_0000_000007 + SCAN(27) on eb_0000000000000_0000_000009 => out schema: {(8) default.nation.n_name (TEXT), joins.part_.p_mfgr (TEXT), joins.part_.p_partkey (INT4), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_phone (TEXT)} => in schema: {(8) default.nation.n_name (TEXT), joins.part_.p_mfgr (TEXT), joins.part_.p_partkey (INT4), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_phone (TEXT)} ======================================================= -Block Id: eb_0000000000000_0000_000009 [TERMINAL] +Block Id: eb_0000000000000_0000_000011 [TERMINAL] ======================================================= diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinOnMultipleDatabases.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinOnMultipleDatabases.Sort.plan index 3a80bb43f0..bc8deb0cbc 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinOnMultipleDatabases.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinOnMultipleDatabases.Sort.plan @@ -45,124 +45,92 @@ SORT(6) explain ------------------------------- ------------------------------------------------------------------------------- -Execution Block Graph (TERMINAL - eb_0000000000000_0000_000009) +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000011) ------------------------------------------------------------------------------- -|-eb_0000000000000_0000_000009 - |-eb_0000000000000_0000_000008 - |-eb_0000000000000_0000_000007 - |-eb_0000000000000_0000_000006 - |-eb_0000000000000_0000_000003 +|-eb_0000000000000_0000_000011 + |-eb_0000000000000_0000_000010 + |-eb_0000000000000_0000_000009 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000006 -3: eb_0000000000000_0000_000007 -4: eb_0000000000000_0000_000008 -5: eb_0000000000000_0000_000009 +1: eb_0000000000000_0000_000009 +2: eb_0000000000000_0000_000010 +3: eb_0000000000000_0000_000011 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [LEAF] +Block Id: eb_0000000000000_0000_000009 [LEAF] ======================================================= [Outgoing] -[q_0000000000000_0000] 3 => 7 (type=HASH_SHUFFLE, key=joins.supplier_.s_suppkey (INT4), num=32) +[q_0000000000000_0000] 9 => 10 (type=RANGE_SHUFFLE, key=default.nation.n_name (TEXT), joins.part_.p_partkey (INT4), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_name (TEXT), num=32) [Enforcers] - 0: type=Broadcast, tables=joins.supplier_ - 1: type=Broadcast, tables=default.nation + 0: type=Broadcast, tables=default.nation + 1: type=Broadcast, tables=joins.supplier_ 2: type=Broadcast, tables=default.region + 3: type=Broadcast, tables=joins.part_ + 4: type=Broadcast, tables=default.partsupp -JOIN(14)(INNER) - => Join Cond: default.nation.n_regionkey (INT4) = default.region.r_regionkey (INT4) - => target list: default.nation.n_name (TEXT), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_phone (TEXT), joins.supplier_.s_suppkey (INT4) - => out schema: {(7) default.nation.n_name (TEXT), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_phone (TEXT), joins.supplier_.s_suppkey (INT4)} - => in schema: {(9) default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_phone (TEXT), joins.supplier_.s_suppkey (INT4)} - SCAN(4) on default.region - => target list: default.region.r_regionkey (INT4) - => out schema: {(1) default.region.r_regionkey (INT4)} - => in schema: {(3) default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} - JOIN(13)(INNER) - => Join Cond: joins.supplier_.s_nationkey (INT4) = default.nation.n_nationkey (INT4) - => target list: default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_phone (TEXT), joins.supplier_.s_suppkey (INT4) - => out schema: {(8) default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_phone (TEXT), joins.supplier_.s_suppkey (INT4)} - => in schema: {(10) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_nationkey (INT4), joins.supplier_.s_phone (TEXT), joins.supplier_.s_suppkey (INT4)} - SCAN(3) on default.nation - => target list: default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4) - => out schema: {(3) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} - => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} - SCAN(1) on joins.supplier_ - => target list: joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_nationkey (INT4), joins.supplier_.s_phone (TEXT), joins.supplier_.s_suppkey (INT4) - => out schema: {(7) joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_nationkey (INT4), joins.supplier_.s_phone (TEXT), joins.supplier_.s_suppkey (INT4)} - => in schema: {(7) joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_nationkey (INT4), joins.supplier_.s_phone (TEXT), joins.supplier_.s_suppkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000006 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 6 => 7 (type=HASH_SHUFFLE, key=default.partsupp.ps_suppkey (INT4), num=32) - -[Enforcers] - 0: type=Broadcast, tables=joins.part_ - 1: type=Broadcast, tables=default.partsupp - -JOIN(15)(INNER) - => Join Cond: joins.part_.p_partkey (INT4) = default.partsupp.ps_partkey (INT4) - => target list: default.partsupp.ps_suppkey (INT4), joins.part_.p_mfgr (TEXT), joins.part_.p_partkey (INT4) - => out schema: {(3) default.partsupp.ps_suppkey (INT4), joins.part_.p_mfgr (TEXT), joins.part_.p_partkey (INT4)} - => in schema: {(4) default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4), joins.part_.p_mfgr (TEXT), joins.part_.p_partkey (INT4)} - SCAN(2) on default.partsupp - => target list: default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4) - => out schema: {(2) default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4)} - => in schema: {(5) default.partsupp.ps_availqty (INT4), default.partsupp.ps_comment (TEXT), default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4), default.partsupp.ps_supplycost (FLOAT8)} - SCAN(0) on joins.part_ - => target list: joins.part_.p_mfgr (TEXT), joins.part_.p_partkey (INT4) - => out schema: {(2) joins.part_.p_mfgr (TEXT), joins.part_.p_partkey (INT4)} - => in schema: {(9) joins.part_.p_brand (TEXT), joins.part_.p_comment (TEXT), joins.part_.p_container (TEXT), joins.part_.p_mfgr (TEXT), joins.part_.p_name (TEXT), joins.part_.p_partkey (INT4), joins.part_.p_retailprice (FLOAT8), joins.part_.p_size (INT4), joins.part_.p_type (TEXT)} - -======================================================= -Block Id: eb_0000000000000_0000_000007 [INTERMEDIATE] -======================================================= - -[Incoming] -[q_0000000000000_0000] 3 => 7 (type=HASH_SHUFFLE, key=joins.supplier_.s_suppkey (INT4), num=32) -[q_0000000000000_0000] 6 => 7 (type=HASH_SHUFFLE, key=default.partsupp.ps_suppkey (INT4), num=32) - -[Outgoing] -[q_0000000000000_0000] 7 => 8 (type=RANGE_SHUFFLE, key=default.nation.n_name (TEXT), joins.part_.p_partkey (INT4), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_name (TEXT), num=32) - -SORT(20) +SORT(26) => Sort Keys: joins.supplier_.s_acctbal (FLOAT8) (asc),joins.supplier_.s_name (TEXT) (asc),default.nation.n_name (TEXT) (asc),joins.part_.p_partkey (INT4) (asc) JOIN(16)(INNER) => Join Cond: joins.supplier_.s_suppkey (INT4) = default.partsupp.ps_suppkey (INT4) => target list: default.nation.n_name (TEXT), joins.part_.p_mfgr (TEXT), joins.part_.p_partkey (INT4), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_phone (TEXT) => out schema: {(8) default.nation.n_name (TEXT), joins.part_.p_mfgr (TEXT), joins.part_.p_partkey (INT4), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_phone (TEXT)} => in schema: {(10) default.nation.n_name (TEXT), default.partsupp.ps_suppkey (INT4), joins.part_.p_mfgr (TEXT), joins.part_.p_partkey (INT4), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_phone (TEXT), joins.supplier_.s_suppkey (INT4)} - SCAN(19) on eb_0000000000000_0000_000006 + JOIN(15)(INNER) + => Join Cond: joins.part_.p_partkey (INT4) = default.partsupp.ps_partkey (INT4) + => target list: default.partsupp.ps_suppkey (INT4), joins.part_.p_mfgr (TEXT), joins.part_.p_partkey (INT4) => out schema: {(3) default.partsupp.ps_suppkey (INT4), joins.part_.p_mfgr (TEXT), joins.part_.p_partkey (INT4)} - => in schema: {(3) default.partsupp.ps_suppkey (INT4), joins.part_.p_mfgr (TEXT), joins.part_.p_partkey (INT4)} - SCAN(18) on eb_0000000000000_0000_000003 + => in schema: {(4) default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4), joins.part_.p_mfgr (TEXT), joins.part_.p_partkey (INT4)} + SCAN(2) on default.partsupp + => target list: default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4) + => out schema: {(2) default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4)} + => in schema: {(5) default.partsupp.ps_availqty (INT4), default.partsupp.ps_comment (TEXT), default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4), default.partsupp.ps_supplycost (FLOAT8)} + SCAN(0) on joins.part_ + => target list: joins.part_.p_mfgr (TEXT), joins.part_.p_partkey (INT4) + => out schema: {(2) joins.part_.p_mfgr (TEXT), joins.part_.p_partkey (INT4)} + => in schema: {(9) joins.part_.p_brand (TEXT), joins.part_.p_comment (TEXT), joins.part_.p_container (TEXT), joins.part_.p_mfgr (TEXT), joins.part_.p_name (TEXT), joins.part_.p_partkey (INT4), joins.part_.p_retailprice (FLOAT8), joins.part_.p_size (INT4), joins.part_.p_type (TEXT)} + JOIN(14)(INNER) + => Join Cond: default.nation.n_regionkey (INT4) = default.region.r_regionkey (INT4) + => target list: default.nation.n_name (TEXT), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_phone (TEXT), joins.supplier_.s_suppkey (INT4) => out schema: {(7) default.nation.n_name (TEXT), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_phone (TEXT), joins.supplier_.s_suppkey (INT4)} - => in schema: {(7) default.nation.n_name (TEXT), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_phone (TEXT), joins.supplier_.s_suppkey (INT4)} + => in schema: {(9) default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_phone (TEXT), joins.supplier_.s_suppkey (INT4)} + SCAN(4) on default.region + => target list: default.region.r_regionkey (INT4) + => out schema: {(1) default.region.r_regionkey (INT4)} + => in schema: {(3) default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} + JOIN(13)(INNER) + => Join Cond: joins.supplier_.s_nationkey (INT4) = default.nation.n_nationkey (INT4) + => target list: default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_phone (TEXT), joins.supplier_.s_suppkey (INT4) + => out schema: {(8) default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_phone (TEXT), joins.supplier_.s_suppkey (INT4)} + => in schema: {(10) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_nationkey (INT4), joins.supplier_.s_phone (TEXT), joins.supplier_.s_suppkey (INT4)} + SCAN(3) on default.nation + => target list: default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4) + => out schema: {(3) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + SCAN(1) on joins.supplier_ + => target list: joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_nationkey (INT4), joins.supplier_.s_phone (TEXT), joins.supplier_.s_suppkey (INT4) + => out schema: {(7) joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_nationkey (INT4), joins.supplier_.s_phone (TEXT), joins.supplier_.s_suppkey (INT4)} + => in schema: {(7) joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_nationkey (INT4), joins.supplier_.s_phone (TEXT), joins.supplier_.s_suppkey (INT4)} ======================================================= -Block Id: eb_0000000000000_0000_000008 [ROOT] +Block Id: eb_0000000000000_0000_000010 [ROOT] ======================================================= [Incoming] -[q_0000000000000_0000] 7 => 8 (type=RANGE_SHUFFLE, key=default.nation.n_name (TEXT), joins.part_.p_partkey (INT4), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_name (TEXT), num=32) +[q_0000000000000_0000] 9 => 10 (type=RANGE_SHUFFLE, key=default.nation.n_name (TEXT), joins.part_.p_partkey (INT4), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_name (TEXT), num=32) [Enforcers] - 0: sorted input=eb_0000000000000_0000_000007 + 0: sorted input=eb_0000000000000_0000_000009 SORT(6) => Sort Keys: joins.supplier_.s_acctbal (FLOAT8) (asc),joins.supplier_.s_name (TEXT) (asc),default.nation.n_name (TEXT) (asc),joins.part_.p_partkey (INT4) (asc) - SCAN(21) on eb_0000000000000_0000_000007 + SCAN(27) on eb_0000000000000_0000_000009 => out schema: {(8) default.nation.n_name (TEXT), joins.part_.p_mfgr (TEXT), joins.part_.p_partkey (INT4), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_phone (TEXT)} => in schema: {(8) default.nation.n_name (TEXT), joins.part_.p_mfgr (TEXT), joins.part_.p_partkey (INT4), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_address (TEXT), joins.supplier_.s_comment (TEXT), joins.supplier_.s_name (TEXT), joins.supplier_.s_phone (TEXT)} ======================================================= -Block Id: eb_0000000000000_0000_000009 [TERMINAL] +Block Id: eb_0000000000000_0000_000011 [TERMINAL] ======================================================= diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinWithMultipleJoinQual1.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinWithMultipleJoinQual1.Hash.plan index a661de65dc..d9e413389e 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinWithMultipleJoinQual1.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinWithMultipleJoinQual1.Hash.plan @@ -45,124 +45,92 @@ SORT(10) explain ------------------------------- ------------------------------------------------------------------------------- -Execution Block Graph (TERMINAL - eb_0000000000000_0000_000009) +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000011) ------------------------------------------------------------------------------- -|-eb_0000000000000_0000_000009 - |-eb_0000000000000_0000_000008 - |-eb_0000000000000_0000_000007 - |-eb_0000000000000_0000_000006 - |-eb_0000000000000_0000_000003 +|-eb_0000000000000_0000_000011 + |-eb_0000000000000_0000_000010 + |-eb_0000000000000_0000_000009 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000006 -3: eb_0000000000000_0000_000007 -4: eb_0000000000000_0000_000008 -5: eb_0000000000000_0000_000009 +1: eb_0000000000000_0000_000009 +2: eb_0000000000000_0000_000010 +3: eb_0000000000000_0000_000011 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [LEAF] +Block Id: eb_0000000000000_0000_000009 [LEAF] ======================================================= [Outgoing] -[q_0000000000000_0000] 3 => 7 (type=HASH_SHUFFLE, key=default.ps.ps_suppkey (INT4), default.s.s_nationkey (INT4), num=32) +[q_0000000000000_0000] 9 => 10 (type=RANGE_SHUFFLE, key=default.n.n_name (TEXT), default.p.p_partkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_name (TEXT), num=32) [Enforcers] - 0: type=Broadcast, tables=default.s - 1: type=Broadcast, tables=default.ps - 2: type=Broadcast, tables=default.p + 0: type=Broadcast, tables=default.ps + 1: type=Broadcast, tables=default.p + 2: type=Broadcast, tables=default.s + 3: type=Broadcast, tables=default.n + 4: type=Broadcast, tables=default.r -JOIN(14)(INNER) - => Join Cond: default.p.p_partkey (INT4) = default.ps.ps_partkey (INT4) - => target list: default.p.p_mfgr (TEXT), default.p.p_partkey (INT4), default.ps.ps_suppkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT) - => out schema: {(9) default.p.p_mfgr (TEXT), default.p.p_partkey (INT4), default.ps.ps_suppkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT)} - => in schema: {(10) default.p.p_mfgr (TEXT), default.p.p_partkey (INT4), default.ps.ps_partkey (INT4), default.ps.ps_suppkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT)} - SCAN(7) on default.part as p - => target list: default.p.p_mfgr (TEXT), default.p.p_partkey (INT4) - => out schema: {(2) default.p.p_mfgr (TEXT), default.p.p_partkey (INT4)} - => in schema: {(9) default.p.p_brand (TEXT), default.p.p_comment (TEXT), default.p.p_container (TEXT), default.p.p_mfgr (TEXT), default.p.p_name (TEXT), default.p.p_partkey (INT4), default.p.p_retailprice (FLOAT8), default.p.p_size (INT4), default.p.p_type (TEXT)} - JOIN(13)(INNER) - => Join Cond: default.s.s_suppkey (INT4) = default.ps.ps_suppkey (INT4) - => target list: default.ps.ps_partkey (INT4), default.ps.ps_suppkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT) - => out schema: {(8) default.ps.ps_partkey (INT4), default.ps.ps_suppkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT)} - => in schema: {(9) default.ps.ps_partkey (INT4), default.ps.ps_suppkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT), default.s.s_suppkey (INT4)} - SCAN(5) on default.partsupp as ps - => target list: default.ps.ps_partkey (INT4), default.ps.ps_suppkey (INT4) - => out schema: {(2) default.ps.ps_partkey (INT4), default.ps.ps_suppkey (INT4)} - => in schema: {(5) default.ps.ps_availqty (INT4), default.ps.ps_comment (TEXT), default.ps.ps_partkey (INT4), default.ps.ps_suppkey (INT4), default.ps.ps_supplycost (FLOAT8)} - SCAN(3) on default.supplier as s - => target list: default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT), default.s.s_suppkey (INT4) - => out schema: {(7) default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT), default.s.s_suppkey (INT4)} - => in schema: {(7) default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT), default.s.s_suppkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000006 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 6 => 7 (type=HASH_SHUFFLE, key=default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), num=32) - -[Enforcers] - 0: type=Broadcast, tables=default.n - 1: type=Broadcast, tables=default.r - -JOIN(15)(INNER) - => Join Cond: default.n.n_regionkey (INT4) = default.r.r_regionkey (INT4) - => target list: default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) - => out schema: {(3) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} - => in schema: {(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4)} - SCAN(1) on default.region as r - => target list: default.r.r_regionkey (INT4) - => out schema: {(1) default.r.r_regionkey (INT4)} - => in schema: {(3) default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4)} - SCAN(0) on default.nation as n - => target list: default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) - => out schema: {(3) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} - => in schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000007 [INTERMEDIATE] -======================================================= - -[Incoming] -[q_0000000000000_0000] 3 => 7 (type=HASH_SHUFFLE, key=default.ps.ps_suppkey (INT4), default.s.s_nationkey (INT4), num=32) -[q_0000000000000_0000] 6 => 7 (type=HASH_SHUFFLE, key=default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), num=32) - -[Outgoing] -[q_0000000000000_0000] 7 => 8 (type=RANGE_SHUFFLE, key=default.n.n_name (TEXT), default.p.p_partkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_name (TEXT), num=32) - -SORT(20) +SORT(26) => Sort Keys: default.s.s_acctbal (FLOAT8) (asc),default.s.s_name (TEXT) (asc),default.n.n_name (TEXT) (asc),default.p.p_partkey (INT4) (asc) JOIN(16)(INNER) => Join Cond: default.n.n_regionkey (INT4) = default.ps.ps_suppkey (INT4) AND default.s.s_nationkey (INT4) = default.n.n_nationkey (INT4) => target list: default.n.n_name (TEXT), default.p.p_mfgr (TEXT), default.p.p_partkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_phone (TEXT) => out schema: {(8) default.n.n_name (TEXT), default.p.p_mfgr (TEXT), default.p.p_partkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_phone (TEXT)} => in schema: {(12) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.p.p_mfgr (TEXT), default.p.p_partkey (INT4), default.ps.ps_suppkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT)} - SCAN(19) on eb_0000000000000_0000_000006 + JOIN(15)(INNER) + => Join Cond: default.n.n_regionkey (INT4) = default.r.r_regionkey (INT4) + => target list: default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) => out schema: {(3) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} - => in schema: {(3) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} - SCAN(18) on eb_0000000000000_0000_000003 + => in schema: {(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4)} + SCAN(1) on default.region as r + => target list: default.r.r_regionkey (INT4) + => out schema: {(1) default.r.r_regionkey (INT4)} + => in schema: {(3) default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4)} + SCAN(0) on default.nation as n + => target list: default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) + => out schema: {(3) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + => in schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + JOIN(14)(INNER) + => Join Cond: default.p.p_partkey (INT4) = default.ps.ps_partkey (INT4) + => target list: default.p.p_mfgr (TEXT), default.p.p_partkey (INT4), default.ps.ps_suppkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT) => out schema: {(9) default.p.p_mfgr (TEXT), default.p.p_partkey (INT4), default.ps.ps_suppkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT)} - => in schema: {(9) default.p.p_mfgr (TEXT), default.p.p_partkey (INT4), default.ps.ps_suppkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT)} + => in schema: {(10) default.p.p_mfgr (TEXT), default.p.p_partkey (INT4), default.ps.ps_partkey (INT4), default.ps.ps_suppkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT)} + SCAN(7) on default.part as p + => target list: default.p.p_mfgr (TEXT), default.p.p_partkey (INT4) + => out schema: {(2) default.p.p_mfgr (TEXT), default.p.p_partkey (INT4)} + => in schema: {(9) default.p.p_brand (TEXT), default.p.p_comment (TEXT), default.p.p_container (TEXT), default.p.p_mfgr (TEXT), default.p.p_name (TEXT), default.p.p_partkey (INT4), default.p.p_retailprice (FLOAT8), default.p.p_size (INT4), default.p.p_type (TEXT)} + JOIN(13)(INNER) + => Join Cond: default.s.s_suppkey (INT4) = default.ps.ps_suppkey (INT4) + => target list: default.ps.ps_partkey (INT4), default.ps.ps_suppkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT) + => out schema: {(8) default.ps.ps_partkey (INT4), default.ps.ps_suppkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT)} + => in schema: {(9) default.ps.ps_partkey (INT4), default.ps.ps_suppkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT), default.s.s_suppkey (INT4)} + SCAN(5) on default.partsupp as ps + => target list: default.ps.ps_partkey (INT4), default.ps.ps_suppkey (INT4) + => out schema: {(2) default.ps.ps_partkey (INT4), default.ps.ps_suppkey (INT4)} + => in schema: {(5) default.ps.ps_availqty (INT4), default.ps.ps_comment (TEXT), default.ps.ps_partkey (INT4), default.ps.ps_suppkey (INT4), default.ps.ps_supplycost (FLOAT8)} + SCAN(3) on default.supplier as s + => target list: default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT), default.s.s_suppkey (INT4) + => out schema: {(7) default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT), default.s.s_suppkey (INT4)} + => in schema: {(7) default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT), default.s.s_suppkey (INT4)} ======================================================= -Block Id: eb_0000000000000_0000_000008 [ROOT] +Block Id: eb_0000000000000_0000_000010 [ROOT] ======================================================= [Incoming] -[q_0000000000000_0000] 7 => 8 (type=RANGE_SHUFFLE, key=default.n.n_name (TEXT), default.p.p_partkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_name (TEXT), num=32) +[q_0000000000000_0000] 9 => 10 (type=RANGE_SHUFFLE, key=default.n.n_name (TEXT), default.p.p_partkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_name (TEXT), num=32) [Enforcers] - 0: sorted input=eb_0000000000000_0000_000007 + 0: sorted input=eb_0000000000000_0000_000009 SORT(10) => Sort Keys: default.s.s_acctbal (FLOAT8) (asc),default.s.s_name (TEXT) (asc),default.n.n_name (TEXT) (asc),default.p.p_partkey (INT4) (asc) - SCAN(21) on eb_0000000000000_0000_000007 + SCAN(27) on eb_0000000000000_0000_000009 => out schema: {(8) default.n.n_name (TEXT), default.p.p_mfgr (TEXT), default.p.p_partkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_phone (TEXT)} => in schema: {(8) default.n.n_name (TEXT), default.p.p_mfgr (TEXT), default.p.p_partkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_phone (TEXT)} ======================================================= -Block Id: eb_0000000000000_0000_000009 [TERMINAL] +Block Id: eb_0000000000000_0000_000011 [TERMINAL] ======================================================= diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinWithMultipleJoinQual1.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinWithMultipleJoinQual1.Sort.plan index a661de65dc..d9e413389e 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinWithMultipleJoinQual1.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinWithMultipleJoinQual1.Sort.plan @@ -45,124 +45,92 @@ SORT(10) explain ------------------------------- ------------------------------------------------------------------------------- -Execution Block Graph (TERMINAL - eb_0000000000000_0000_000009) +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000011) ------------------------------------------------------------------------------- -|-eb_0000000000000_0000_000009 - |-eb_0000000000000_0000_000008 - |-eb_0000000000000_0000_000007 - |-eb_0000000000000_0000_000006 - |-eb_0000000000000_0000_000003 +|-eb_0000000000000_0000_000011 + |-eb_0000000000000_0000_000010 + |-eb_0000000000000_0000_000009 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000006 -3: eb_0000000000000_0000_000007 -4: eb_0000000000000_0000_000008 -5: eb_0000000000000_0000_000009 +1: eb_0000000000000_0000_000009 +2: eb_0000000000000_0000_000010 +3: eb_0000000000000_0000_000011 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [LEAF] +Block Id: eb_0000000000000_0000_000009 [LEAF] ======================================================= [Outgoing] -[q_0000000000000_0000] 3 => 7 (type=HASH_SHUFFLE, key=default.ps.ps_suppkey (INT4), default.s.s_nationkey (INT4), num=32) +[q_0000000000000_0000] 9 => 10 (type=RANGE_SHUFFLE, key=default.n.n_name (TEXT), default.p.p_partkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_name (TEXT), num=32) [Enforcers] - 0: type=Broadcast, tables=default.s - 1: type=Broadcast, tables=default.ps - 2: type=Broadcast, tables=default.p + 0: type=Broadcast, tables=default.ps + 1: type=Broadcast, tables=default.p + 2: type=Broadcast, tables=default.s + 3: type=Broadcast, tables=default.n + 4: type=Broadcast, tables=default.r -JOIN(14)(INNER) - => Join Cond: default.p.p_partkey (INT4) = default.ps.ps_partkey (INT4) - => target list: default.p.p_mfgr (TEXT), default.p.p_partkey (INT4), default.ps.ps_suppkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT) - => out schema: {(9) default.p.p_mfgr (TEXT), default.p.p_partkey (INT4), default.ps.ps_suppkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT)} - => in schema: {(10) default.p.p_mfgr (TEXT), default.p.p_partkey (INT4), default.ps.ps_partkey (INT4), default.ps.ps_suppkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT)} - SCAN(7) on default.part as p - => target list: default.p.p_mfgr (TEXT), default.p.p_partkey (INT4) - => out schema: {(2) default.p.p_mfgr (TEXT), default.p.p_partkey (INT4)} - => in schema: {(9) default.p.p_brand (TEXT), default.p.p_comment (TEXT), default.p.p_container (TEXT), default.p.p_mfgr (TEXT), default.p.p_name (TEXT), default.p.p_partkey (INT4), default.p.p_retailprice (FLOAT8), default.p.p_size (INT4), default.p.p_type (TEXT)} - JOIN(13)(INNER) - => Join Cond: default.s.s_suppkey (INT4) = default.ps.ps_suppkey (INT4) - => target list: default.ps.ps_partkey (INT4), default.ps.ps_suppkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT) - => out schema: {(8) default.ps.ps_partkey (INT4), default.ps.ps_suppkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT)} - => in schema: {(9) default.ps.ps_partkey (INT4), default.ps.ps_suppkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT), default.s.s_suppkey (INT4)} - SCAN(5) on default.partsupp as ps - => target list: default.ps.ps_partkey (INT4), default.ps.ps_suppkey (INT4) - => out schema: {(2) default.ps.ps_partkey (INT4), default.ps.ps_suppkey (INT4)} - => in schema: {(5) default.ps.ps_availqty (INT4), default.ps.ps_comment (TEXT), default.ps.ps_partkey (INT4), default.ps.ps_suppkey (INT4), default.ps.ps_supplycost (FLOAT8)} - SCAN(3) on default.supplier as s - => target list: default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT), default.s.s_suppkey (INT4) - => out schema: {(7) default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT), default.s.s_suppkey (INT4)} - => in schema: {(7) default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT), default.s.s_suppkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000006 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 6 => 7 (type=HASH_SHUFFLE, key=default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), num=32) - -[Enforcers] - 0: type=Broadcast, tables=default.n - 1: type=Broadcast, tables=default.r - -JOIN(15)(INNER) - => Join Cond: default.n.n_regionkey (INT4) = default.r.r_regionkey (INT4) - => target list: default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) - => out schema: {(3) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} - => in schema: {(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4)} - SCAN(1) on default.region as r - => target list: default.r.r_regionkey (INT4) - => out schema: {(1) default.r.r_regionkey (INT4)} - => in schema: {(3) default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4)} - SCAN(0) on default.nation as n - => target list: default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) - => out schema: {(3) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} - => in schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000007 [INTERMEDIATE] -======================================================= - -[Incoming] -[q_0000000000000_0000] 3 => 7 (type=HASH_SHUFFLE, key=default.ps.ps_suppkey (INT4), default.s.s_nationkey (INT4), num=32) -[q_0000000000000_0000] 6 => 7 (type=HASH_SHUFFLE, key=default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), num=32) - -[Outgoing] -[q_0000000000000_0000] 7 => 8 (type=RANGE_SHUFFLE, key=default.n.n_name (TEXT), default.p.p_partkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_name (TEXT), num=32) - -SORT(20) +SORT(26) => Sort Keys: default.s.s_acctbal (FLOAT8) (asc),default.s.s_name (TEXT) (asc),default.n.n_name (TEXT) (asc),default.p.p_partkey (INT4) (asc) JOIN(16)(INNER) => Join Cond: default.n.n_regionkey (INT4) = default.ps.ps_suppkey (INT4) AND default.s.s_nationkey (INT4) = default.n.n_nationkey (INT4) => target list: default.n.n_name (TEXT), default.p.p_mfgr (TEXT), default.p.p_partkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_phone (TEXT) => out schema: {(8) default.n.n_name (TEXT), default.p.p_mfgr (TEXT), default.p.p_partkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_phone (TEXT)} => in schema: {(12) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.p.p_mfgr (TEXT), default.p.p_partkey (INT4), default.ps.ps_suppkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT)} - SCAN(19) on eb_0000000000000_0000_000006 + JOIN(15)(INNER) + => Join Cond: default.n.n_regionkey (INT4) = default.r.r_regionkey (INT4) + => target list: default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) => out schema: {(3) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} - => in schema: {(3) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} - SCAN(18) on eb_0000000000000_0000_000003 + => in schema: {(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4)} + SCAN(1) on default.region as r + => target list: default.r.r_regionkey (INT4) + => out schema: {(1) default.r.r_regionkey (INT4)} + => in schema: {(3) default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4)} + SCAN(0) on default.nation as n + => target list: default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) + => out schema: {(3) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + => in schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + JOIN(14)(INNER) + => Join Cond: default.p.p_partkey (INT4) = default.ps.ps_partkey (INT4) + => target list: default.p.p_mfgr (TEXT), default.p.p_partkey (INT4), default.ps.ps_suppkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT) => out schema: {(9) default.p.p_mfgr (TEXT), default.p.p_partkey (INT4), default.ps.ps_suppkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT)} - => in schema: {(9) default.p.p_mfgr (TEXT), default.p.p_partkey (INT4), default.ps.ps_suppkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT)} + => in schema: {(10) default.p.p_mfgr (TEXT), default.p.p_partkey (INT4), default.ps.ps_partkey (INT4), default.ps.ps_suppkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT)} + SCAN(7) on default.part as p + => target list: default.p.p_mfgr (TEXT), default.p.p_partkey (INT4) + => out schema: {(2) default.p.p_mfgr (TEXT), default.p.p_partkey (INT4)} + => in schema: {(9) default.p.p_brand (TEXT), default.p.p_comment (TEXT), default.p.p_container (TEXT), default.p.p_mfgr (TEXT), default.p.p_name (TEXT), default.p.p_partkey (INT4), default.p.p_retailprice (FLOAT8), default.p.p_size (INT4), default.p.p_type (TEXT)} + JOIN(13)(INNER) + => Join Cond: default.s.s_suppkey (INT4) = default.ps.ps_suppkey (INT4) + => target list: default.ps.ps_partkey (INT4), default.ps.ps_suppkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT) + => out schema: {(8) default.ps.ps_partkey (INT4), default.ps.ps_suppkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT)} + => in schema: {(9) default.ps.ps_partkey (INT4), default.ps.ps_suppkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT), default.s.s_suppkey (INT4)} + SCAN(5) on default.partsupp as ps + => target list: default.ps.ps_partkey (INT4), default.ps.ps_suppkey (INT4) + => out schema: {(2) default.ps.ps_partkey (INT4), default.ps.ps_suppkey (INT4)} + => in schema: {(5) default.ps.ps_availqty (INT4), default.ps.ps_comment (TEXT), default.ps.ps_partkey (INT4), default.ps.ps_suppkey (INT4), default.ps.ps_supplycost (FLOAT8)} + SCAN(3) on default.supplier as s + => target list: default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT), default.s.s_suppkey (INT4) + => out schema: {(7) default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT), default.s.s_suppkey (INT4)} + => in schema: {(7) default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT), default.s.s_suppkey (INT4)} ======================================================= -Block Id: eb_0000000000000_0000_000008 [ROOT] +Block Id: eb_0000000000000_0000_000010 [ROOT] ======================================================= [Incoming] -[q_0000000000000_0000] 7 => 8 (type=RANGE_SHUFFLE, key=default.n.n_name (TEXT), default.p.p_partkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_name (TEXT), num=32) +[q_0000000000000_0000] 9 => 10 (type=RANGE_SHUFFLE, key=default.n.n_name (TEXT), default.p.p_partkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_name (TEXT), num=32) [Enforcers] - 0: sorted input=eb_0000000000000_0000_000007 + 0: sorted input=eb_0000000000000_0000_000009 SORT(10) => Sort Keys: default.s.s_acctbal (FLOAT8) (asc),default.s.s_name (TEXT) (asc),default.n.n_name (TEXT) (asc),default.p.p_partkey (INT4) (asc) - SCAN(21) on eb_0000000000000_0000_000007 + SCAN(27) on eb_0000000000000_0000_000009 => out schema: {(8) default.n.n_name (TEXT), default.p.p_mfgr (TEXT), default.p.p_partkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_phone (TEXT)} => in schema: {(8) default.n.n_name (TEXT), default.p.p_mfgr (TEXT), default.p.p_partkey (INT4), default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_phone (TEXT)} ======================================================= -Block Id: eb_0000000000000_0000_000009 [TERMINAL] +Block Id: eb_0000000000000_0000_000011 [TERMINAL] ======================================================= diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinWithOrPredicates.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinWithOrPredicates.Hash.plan index b08ab10d1e..de323b192e 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinWithOrPredicates.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinWithOrPredicates.Hash.plan @@ -48,7 +48,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.n2 1: type=Broadcast, tables=default.n1 -SORT(9) +SORT(11) => Sort Keys: default.n1.n_nationkey (INT4) (asc) SELECTION(2) => Search Cond: default.n1.n_nationkey (INT4) IN (1, 2) OR default.n2.n_nationkey (INT4) IN (2) @@ -82,7 +82,7 @@ PROJECTION(4) => in schema: {(4) default.n1.n_name (TEXT), default.n1.n_nationkey (INT4), default.n2.n_name (TEXT), default.n2.n_nationkey (INT4)} SORT(3) => Sort Keys: default.n1.n_nationkey (INT4) (asc) - SCAN(10) on eb_0000000000000_0000_000003 + SCAN(12) on eb_0000000000000_0000_000003 => out schema: {(4) default.n1.n_name (TEXT), default.n1.n_nationkey (INT4), default.n2.n_name (TEXT), default.n2.n_nationkey (INT4)} => in schema: {(4) default.n1.n_name (TEXT), default.n1.n_nationkey (INT4), default.n2.n_name (TEXT), default.n2.n_nationkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinWithOrPredicates.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinWithOrPredicates.Sort.plan index b08ab10d1e..de323b192e 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinWithOrPredicates.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinWithOrPredicates.Sort.plan @@ -48,7 +48,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.n2 1: type=Broadcast, tables=default.n1 -SORT(9) +SORT(11) => Sort Keys: default.n1.n_nationkey (INT4) (asc) SELECTION(2) => Search Cond: default.n1.n_nationkey (INT4) IN (1, 2) OR default.n2.n_nationkey (INT4) IN (2) @@ -82,7 +82,7 @@ PROJECTION(4) => in schema: {(4) default.n1.n_name (TEXT), default.n1.n_nationkey (INT4), default.n2.n_name (TEXT), default.n2.n_nationkey (INT4)} SORT(3) => Sort Keys: default.n1.n_nationkey (INT4) (asc) - SCAN(10) on eb_0000000000000_0000_000003 + SCAN(12) on eb_0000000000000_0000_000003 => out schema: {(4) default.n1.n_name (TEXT), default.n1.n_nationkey (INT4), default.n2.n_name (TEXT), default.n2.n_nationkey (INT4)} => in schema: {(4) default.n1.n_name (TEXT), default.n1.n_nationkey (INT4), default.n2.n_name (TEXT), default.n2.n_nationkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testNaturalJoin.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testNaturalJoin.Hash.plan index bf9323dc95..e459241fbb 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testNaturalJoin.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testNaturalJoin.Hash.plan @@ -42,7 +42,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.n2 1: type=Broadcast, tables=default.n1 -SORT(8) +SORT(10) => Sort Keys: default.n2.n_name (TEXT) (asc) JOIN(6)(INNER) => Join Cond: default.n1.n_comment (TEXT) = default.n2.n_comment (TEXT) AND default.n1.n_name (TEXT) = default.n2.n_name (TEXT) AND default.n1.n_nationkey (INT4) = default.n2.n_nationkey (INT4) AND default.n1.n_regionkey (INT4) = default.n2.n_regionkey (INT4) @@ -70,7 +70,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.n2.n_name (TEXT) (asc) - SCAN(9) on eb_0000000000000_0000_000003 + SCAN(11) on eb_0000000000000_0000_000003 => out schema: {(2) default.n1.n_name (TEXT), default.n2.n_name (TEXT)} => in schema: {(2) default.n1.n_name (TEXT), default.n2.n_name (TEXT)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testNaturalJoin.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testNaturalJoin.Sort.plan index bf9323dc95..e459241fbb 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testNaturalJoin.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testNaturalJoin.Sort.plan @@ -42,7 +42,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.n2 1: type=Broadcast, tables=default.n1 -SORT(8) +SORT(10) => Sort Keys: default.n2.n_name (TEXT) (asc) JOIN(6)(INNER) => Join Cond: default.n1.n_comment (TEXT) = default.n2.n_comment (TEXT) AND default.n1.n_name (TEXT) = default.n2.n_name (TEXT) AND default.n1.n_nationkey (INT4) = default.n2.n_nationkey (INT4) AND default.n1.n_regionkey (INT4) = default.n2.n_regionkey (INT4) @@ -70,7 +70,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.n2.n_name (TEXT) (asc) - SCAN(9) on eb_0000000000000_0000_000003 + SCAN(11) on eb_0000000000000_0000_000003 => out schema: {(2) default.n1.n_name (TEXT), default.n2.n_name (TEXT)} => in schema: {(2) default.n1.n_name (TEXT), default.n2.n_name (TEXT)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testTPCHQ2Join.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testTPCHQ2Join.Hash.plan index 38d9b38875..d8f2c7b253 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testTPCHQ2Join.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testTPCHQ2Join.Hash.plan @@ -45,124 +45,92 @@ SORT(6) explain ------------------------------- ------------------------------------------------------------------------------- -Execution Block Graph (TERMINAL - eb_0000000000000_0000_000009) +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000011) ------------------------------------------------------------------------------- -|-eb_0000000000000_0000_000009 - |-eb_0000000000000_0000_000008 - |-eb_0000000000000_0000_000007 - |-eb_0000000000000_0000_000006 - |-eb_0000000000000_0000_000003 +|-eb_0000000000000_0000_000011 + |-eb_0000000000000_0000_000010 + |-eb_0000000000000_0000_000009 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000006 -3: eb_0000000000000_0000_000007 -4: eb_0000000000000_0000_000008 -5: eb_0000000000000_0000_000009 +1: eb_0000000000000_0000_000009 +2: eb_0000000000000_0000_000010 +3: eb_0000000000000_0000_000011 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [LEAF] +Block Id: eb_0000000000000_0000_000009 [LEAF] ======================================================= [Outgoing] -[q_0000000000000_0000] 3 => 7 (type=HASH_SHUFFLE, key=default.supplier.s_nationkey (INT4), num=32) +[q_0000000000000_0000] 9 => 10 (type=RANGE_SHUFFLE, key=default.nation.n_name (TEXT), default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_name (TEXT), num=32) [Enforcers] - 0: type=Broadcast, tables=default.supplier - 1: type=Broadcast, tables=default.partsupp - 2: type=Broadcast, tables=default.part + 0: type=Broadcast, tables=default.part + 1: type=Broadcast, tables=default.supplier + 2: type=Broadcast, tables=default.partsupp + 3: type=Broadcast, tables=default.nation + 4: type=Broadcast, tables=default.region -JOIN(14)(INNER) - => Join Cond: default.part.p_partkey (INT4) = default.partsupp.ps_partkey (INT4) - => target list: default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT) - => out schema: {(8) default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT)} - => in schema: {(9) default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.partsupp.ps_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT)} - SCAN(0) on default.part - => target list: default.part.p_mfgr (TEXT), default.part.p_partkey (INT4) - => out schema: {(2) default.part.p_mfgr (TEXT), default.part.p_partkey (INT4)} - => in schema: {(9) default.part.p_brand (TEXT), default.part.p_comment (TEXT), default.part.p_container (TEXT), default.part.p_mfgr (TEXT), default.part.p_name (TEXT), default.part.p_partkey (INT4), default.part.p_retailprice (FLOAT8), default.part.p_size (INT4), default.part.p_type (TEXT)} - JOIN(13)(INNER) - => Join Cond: default.supplier.s_suppkey (INT4) = default.partsupp.ps_suppkey (INT4) - => target list: default.partsupp.ps_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT) - => out schema: {(7) default.partsupp.ps_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT)} - => in schema: {(9) default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} - SCAN(2) on default.partsupp - => target list: default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4) - => out schema: {(2) default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4)} - => in schema: {(5) default.partsupp.ps_availqty (INT4), default.partsupp.ps_comment (TEXT), default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4), default.partsupp.ps_supplycost (FLOAT8)} - SCAN(1) on default.supplier - => target list: default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4) - => out schema: {(7) default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} - => in schema: {(7) default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000006 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 6 => 7 (type=HASH_SHUFFLE, key=default.nation.n_nationkey (INT4), num=32) - -[Enforcers] - 0: type=Broadcast, tables=default.nation - 1: type=Broadcast, tables=default.region - -JOIN(15)(INNER) - => Join Cond: default.nation.n_regionkey (INT4) = default.region.r_regionkey (INT4) - => target list: default.nation.n_name (TEXT), default.nation.n_nationkey (INT4) - => out schema: {(2) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4)} - => in schema: {(4) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4)} - SCAN(4) on default.region - => target list: default.region.r_regionkey (INT4) - => out schema: {(1) default.region.r_regionkey (INT4)} - => in schema: {(3) default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} - SCAN(3) on default.nation - => target list: default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4) - => out schema: {(3) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} - => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000007 [INTERMEDIATE] -======================================================= - -[Incoming] -[q_0000000000000_0000] 3 => 7 (type=HASH_SHUFFLE, key=default.supplier.s_nationkey (INT4), num=32) -[q_0000000000000_0000] 6 => 7 (type=HASH_SHUFFLE, key=default.nation.n_nationkey (INT4), num=32) - -[Outgoing] -[q_0000000000000_0000] 7 => 8 (type=RANGE_SHUFFLE, key=default.nation.n_name (TEXT), default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_name (TEXT), num=32) - -SORT(20) +SORT(26) => Sort Keys: default.supplier.s_acctbal (FLOAT8) (asc),default.supplier.s_name (TEXT) (asc),default.nation.n_name (TEXT) (asc),default.part.p_partkey (INT4) (asc) JOIN(16)(INNER) => Join Cond: default.supplier.s_nationkey (INT4) = default.nation.n_nationkey (INT4) => target list: default.nation.n_name (TEXT), default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_phone (TEXT) => out schema: {(8) default.nation.n_name (TEXT), default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_phone (TEXT)} => in schema: {(10) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT)} - SCAN(19) on eb_0000000000000_0000_000006 + JOIN(15)(INNER) + => Join Cond: default.nation.n_regionkey (INT4) = default.region.r_regionkey (INT4) + => target list: default.nation.n_name (TEXT), default.nation.n_nationkey (INT4) => out schema: {(2) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4)} - => in schema: {(2) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4)} - SCAN(18) on eb_0000000000000_0000_000003 + => in schema: {(4) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4)} + SCAN(4) on default.region + => target list: default.region.r_regionkey (INT4) + => out schema: {(1) default.region.r_regionkey (INT4)} + => in schema: {(3) default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} + SCAN(3) on default.nation + => target list: default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4) + => out schema: {(3) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + JOIN(14)(INNER) + => Join Cond: default.part.p_partkey (INT4) = default.partsupp.ps_partkey (INT4) + => target list: default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT) => out schema: {(8) default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT)} - => in schema: {(8) default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT)} + => in schema: {(9) default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.partsupp.ps_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT)} + SCAN(0) on default.part + => target list: default.part.p_mfgr (TEXT), default.part.p_partkey (INT4) + => out schema: {(2) default.part.p_mfgr (TEXT), default.part.p_partkey (INT4)} + => in schema: {(9) default.part.p_brand (TEXT), default.part.p_comment (TEXT), default.part.p_container (TEXT), default.part.p_mfgr (TEXT), default.part.p_name (TEXT), default.part.p_partkey (INT4), default.part.p_retailprice (FLOAT8), default.part.p_size (INT4), default.part.p_type (TEXT)} + JOIN(13)(INNER) + => Join Cond: default.supplier.s_suppkey (INT4) = default.partsupp.ps_suppkey (INT4) + => target list: default.partsupp.ps_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT) + => out schema: {(7) default.partsupp.ps_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT)} + => in schema: {(9) default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} + SCAN(2) on default.partsupp + => target list: default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4) + => out schema: {(2) default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4)} + => in schema: {(5) default.partsupp.ps_availqty (INT4), default.partsupp.ps_comment (TEXT), default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4), default.partsupp.ps_supplycost (FLOAT8)} + SCAN(1) on default.supplier + => target list: default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4) + => out schema: {(7) default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} + => in schema: {(7) default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} ======================================================= -Block Id: eb_0000000000000_0000_000008 [ROOT] +Block Id: eb_0000000000000_0000_000010 [ROOT] ======================================================= [Incoming] -[q_0000000000000_0000] 7 => 8 (type=RANGE_SHUFFLE, key=default.nation.n_name (TEXT), default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_name (TEXT), num=32) +[q_0000000000000_0000] 9 => 10 (type=RANGE_SHUFFLE, key=default.nation.n_name (TEXT), default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_name (TEXT), num=32) [Enforcers] - 0: sorted input=eb_0000000000000_0000_000007 + 0: sorted input=eb_0000000000000_0000_000009 SORT(6) => Sort Keys: default.supplier.s_acctbal (FLOAT8) (asc),default.supplier.s_name (TEXT) (asc),default.nation.n_name (TEXT) (asc),default.part.p_partkey (INT4) (asc) - SCAN(21) on eb_0000000000000_0000_000007 + SCAN(27) on eb_0000000000000_0000_000009 => out schema: {(8) default.nation.n_name (TEXT), default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_phone (TEXT)} => in schema: {(8) default.nation.n_name (TEXT), default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_phone (TEXT)} ======================================================= -Block Id: eb_0000000000000_0000_000009 [TERMINAL] +Block Id: eb_0000000000000_0000_000011 [TERMINAL] ======================================================= diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testTPCHQ2Join.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testTPCHQ2Join.Sort.plan index 38d9b38875..d8f2c7b253 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testTPCHQ2Join.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testTPCHQ2Join.Sort.plan @@ -45,124 +45,92 @@ SORT(6) explain ------------------------------- ------------------------------------------------------------------------------- -Execution Block Graph (TERMINAL - eb_0000000000000_0000_000009) +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000011) ------------------------------------------------------------------------------- -|-eb_0000000000000_0000_000009 - |-eb_0000000000000_0000_000008 - |-eb_0000000000000_0000_000007 - |-eb_0000000000000_0000_000006 - |-eb_0000000000000_0000_000003 +|-eb_0000000000000_0000_000011 + |-eb_0000000000000_0000_000010 + |-eb_0000000000000_0000_000009 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000006 -3: eb_0000000000000_0000_000007 -4: eb_0000000000000_0000_000008 -5: eb_0000000000000_0000_000009 +1: eb_0000000000000_0000_000009 +2: eb_0000000000000_0000_000010 +3: eb_0000000000000_0000_000011 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [LEAF] +Block Id: eb_0000000000000_0000_000009 [LEAF] ======================================================= [Outgoing] -[q_0000000000000_0000] 3 => 7 (type=HASH_SHUFFLE, key=default.supplier.s_nationkey (INT4), num=32) +[q_0000000000000_0000] 9 => 10 (type=RANGE_SHUFFLE, key=default.nation.n_name (TEXT), default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_name (TEXT), num=32) [Enforcers] - 0: type=Broadcast, tables=default.supplier - 1: type=Broadcast, tables=default.partsupp - 2: type=Broadcast, tables=default.part + 0: type=Broadcast, tables=default.part + 1: type=Broadcast, tables=default.supplier + 2: type=Broadcast, tables=default.partsupp + 3: type=Broadcast, tables=default.nation + 4: type=Broadcast, tables=default.region -JOIN(14)(INNER) - => Join Cond: default.part.p_partkey (INT4) = default.partsupp.ps_partkey (INT4) - => target list: default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT) - => out schema: {(8) default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT)} - => in schema: {(9) default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.partsupp.ps_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT)} - SCAN(0) on default.part - => target list: default.part.p_mfgr (TEXT), default.part.p_partkey (INT4) - => out schema: {(2) default.part.p_mfgr (TEXT), default.part.p_partkey (INT4)} - => in schema: {(9) default.part.p_brand (TEXT), default.part.p_comment (TEXT), default.part.p_container (TEXT), default.part.p_mfgr (TEXT), default.part.p_name (TEXT), default.part.p_partkey (INT4), default.part.p_retailprice (FLOAT8), default.part.p_size (INT4), default.part.p_type (TEXT)} - JOIN(13)(INNER) - => Join Cond: default.supplier.s_suppkey (INT4) = default.partsupp.ps_suppkey (INT4) - => target list: default.partsupp.ps_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT) - => out schema: {(7) default.partsupp.ps_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT)} - => in schema: {(9) default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} - SCAN(2) on default.partsupp - => target list: default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4) - => out schema: {(2) default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4)} - => in schema: {(5) default.partsupp.ps_availqty (INT4), default.partsupp.ps_comment (TEXT), default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4), default.partsupp.ps_supplycost (FLOAT8)} - SCAN(1) on default.supplier - => target list: default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4) - => out schema: {(7) default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} - => in schema: {(7) default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000006 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 6 => 7 (type=HASH_SHUFFLE, key=default.nation.n_nationkey (INT4), num=32) - -[Enforcers] - 0: type=Broadcast, tables=default.nation - 1: type=Broadcast, tables=default.region - -JOIN(15)(INNER) - => Join Cond: default.nation.n_regionkey (INT4) = default.region.r_regionkey (INT4) - => target list: default.nation.n_name (TEXT), default.nation.n_nationkey (INT4) - => out schema: {(2) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4)} - => in schema: {(4) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4)} - SCAN(4) on default.region - => target list: default.region.r_regionkey (INT4) - => out schema: {(1) default.region.r_regionkey (INT4)} - => in schema: {(3) default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} - SCAN(3) on default.nation - => target list: default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4) - => out schema: {(3) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} - => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000007 [INTERMEDIATE] -======================================================= - -[Incoming] -[q_0000000000000_0000] 3 => 7 (type=HASH_SHUFFLE, key=default.supplier.s_nationkey (INT4), num=32) -[q_0000000000000_0000] 6 => 7 (type=HASH_SHUFFLE, key=default.nation.n_nationkey (INT4), num=32) - -[Outgoing] -[q_0000000000000_0000] 7 => 8 (type=RANGE_SHUFFLE, key=default.nation.n_name (TEXT), default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_name (TEXT), num=32) - -SORT(20) +SORT(26) => Sort Keys: default.supplier.s_acctbal (FLOAT8) (asc),default.supplier.s_name (TEXT) (asc),default.nation.n_name (TEXT) (asc),default.part.p_partkey (INT4) (asc) JOIN(16)(INNER) => Join Cond: default.supplier.s_nationkey (INT4) = default.nation.n_nationkey (INT4) => target list: default.nation.n_name (TEXT), default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_phone (TEXT) => out schema: {(8) default.nation.n_name (TEXT), default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_phone (TEXT)} => in schema: {(10) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT)} - SCAN(19) on eb_0000000000000_0000_000006 + JOIN(15)(INNER) + => Join Cond: default.nation.n_regionkey (INT4) = default.region.r_regionkey (INT4) + => target list: default.nation.n_name (TEXT), default.nation.n_nationkey (INT4) => out schema: {(2) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4)} - => in schema: {(2) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4)} - SCAN(18) on eb_0000000000000_0000_000003 + => in schema: {(4) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4)} + SCAN(4) on default.region + => target list: default.region.r_regionkey (INT4) + => out schema: {(1) default.region.r_regionkey (INT4)} + => in schema: {(3) default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} + SCAN(3) on default.nation + => target list: default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4) + => out schema: {(3) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + JOIN(14)(INNER) + => Join Cond: default.part.p_partkey (INT4) = default.partsupp.ps_partkey (INT4) + => target list: default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT) => out schema: {(8) default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT)} - => in schema: {(8) default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT)} + => in schema: {(9) default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.partsupp.ps_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT)} + SCAN(0) on default.part + => target list: default.part.p_mfgr (TEXT), default.part.p_partkey (INT4) + => out schema: {(2) default.part.p_mfgr (TEXT), default.part.p_partkey (INT4)} + => in schema: {(9) default.part.p_brand (TEXT), default.part.p_comment (TEXT), default.part.p_container (TEXT), default.part.p_mfgr (TEXT), default.part.p_name (TEXT), default.part.p_partkey (INT4), default.part.p_retailprice (FLOAT8), default.part.p_size (INT4), default.part.p_type (TEXT)} + JOIN(13)(INNER) + => Join Cond: default.supplier.s_suppkey (INT4) = default.partsupp.ps_suppkey (INT4) + => target list: default.partsupp.ps_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT) + => out schema: {(7) default.partsupp.ps_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT)} + => in schema: {(9) default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} + SCAN(2) on default.partsupp + => target list: default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4) + => out schema: {(2) default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4)} + => in schema: {(5) default.partsupp.ps_availqty (INT4), default.partsupp.ps_comment (TEXT), default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4), default.partsupp.ps_supplycost (FLOAT8)} + SCAN(1) on default.supplier + => target list: default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4) + => out schema: {(7) default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} + => in schema: {(7) default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} ======================================================= -Block Id: eb_0000000000000_0000_000008 [ROOT] +Block Id: eb_0000000000000_0000_000010 [ROOT] ======================================================= [Incoming] -[q_0000000000000_0000] 7 => 8 (type=RANGE_SHUFFLE, key=default.nation.n_name (TEXT), default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_name (TEXT), num=32) +[q_0000000000000_0000] 9 => 10 (type=RANGE_SHUFFLE, key=default.nation.n_name (TEXT), default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_name (TEXT), num=32) [Enforcers] - 0: sorted input=eb_0000000000000_0000_000007 + 0: sorted input=eb_0000000000000_0000_000009 SORT(6) => Sort Keys: default.supplier.s_acctbal (FLOAT8) (asc),default.supplier.s_name (TEXT) (asc),default.nation.n_name (TEXT) (asc),default.part.p_partkey (INT4) (asc) - SCAN(21) on eb_0000000000000_0000_000007 + SCAN(27) on eb_0000000000000_0000_000009 => out schema: {(8) default.nation.n_name (TEXT), default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_phone (TEXT)} => in schema: {(8) default.nation.n_name (TEXT), default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_phone (TEXT)} ======================================================= -Block Id: eb_0000000000000_0000_000009 [TERMINAL] +Block Id: eb_0000000000000_0000_000011 [TERMINAL] ======================================================= diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin1.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin1.Hash.plan index 7ba5577522..6d9a0190f8 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin1.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin1.Hash.plan @@ -42,7 +42,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.region 1: type=Broadcast, tables=default.nation -SORT(9) +SORT(11) => Sort Keys: default.nation.n_name (TEXT) (asc) JOIN(7)(INNER) => Join Cond: default.nation.n_regionkey (INT4) = default.region.r_regionkey (INT4) @@ -70,7 +70,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.nation.n_name (TEXT) (asc) - SCAN(10) on eb_0000000000000_0000_000003 + SCAN(12) on eb_0000000000000_0000_000003 => out schema: {(4) default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} => in schema: {(4) default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin1.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin1.Sort.plan index 7ba5577522..6d9a0190f8 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin1.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin1.Sort.plan @@ -42,7 +42,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.region 1: type=Broadcast, tables=default.nation -SORT(9) +SORT(11) => Sort Keys: default.nation.n_name (TEXT) (asc) JOIN(7)(INNER) => Join Cond: default.nation.n_regionkey (INT4) = default.region.r_regionkey (INT4) @@ -70,7 +70,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.nation.n_name (TEXT) (asc) - SCAN(10) on eb_0000000000000_0000_000003 + SCAN(12) on eb_0000000000000_0000_000003 => out schema: {(4) default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} => in schema: {(4) default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin2.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin2.Hash.plan index 1269493b8c..cd036afc70 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin2.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin2.Hash.plan @@ -42,7 +42,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.region 1: type=Broadcast, tables=default.nation -SORT(9) +SORT(11) => Sort Keys: default.nation.n_name (TEXT) (asc) JOIN(7)(INNER) => Join Cond: default.nation.n_regionkey (INT4) = default.region.r_regionkey (INT4) @@ -70,7 +70,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.nation.n_name (TEXT) (asc) - SCAN(10) on eb_0000000000000_0000_000003 + SCAN(12) on eb_0000000000000_0000_000003 => out schema: {(2) default.nation.n_name (TEXT), default.region.r_name (TEXT)} => in schema: {(2) default.nation.n_name (TEXT), default.region.r_name (TEXT)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin2.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin2.Sort.plan index 1269493b8c..cd036afc70 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin2.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin2.Sort.plan @@ -42,7 +42,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.region 1: type=Broadcast, tables=default.nation -SORT(9) +SORT(11) => Sort Keys: default.nation.n_name (TEXT) (asc) JOIN(7)(INNER) => Join Cond: default.nation.n_regionkey (INT4) = default.region.r_regionkey (INT4) @@ -70,7 +70,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.nation.n_name (TEXT) (asc) - SCAN(10) on eb_0000000000000_0000_000003 + SCAN(12) on eb_0000000000000_0000_000003 => out schema: {(2) default.nation.n_name (TEXT), default.region.r_name (TEXT)} => in schema: {(2) default.nation.n_name (TEXT), default.region.r_name (TEXT)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin3.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin3.Hash.plan index 3f1e25d6da..bebae840b6 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin3.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin3.Hash.plan @@ -42,7 +42,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.region 1: type=Broadcast, tables=default.nation -SORT(9) +SORT(11) => Sort Keys: default.nation.n_name (TEXT) (asc) JOIN(7)(INNER) => Join Cond: default.nation.n_regionkey (INT4) = default.region.r_regionkey (INT4) @@ -70,7 +70,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.nation.n_name (TEXT) (asc) - SCAN(10) on eb_0000000000000_0000_000003 + SCAN(12) on eb_0000000000000_0000_000003 => out schema: {(4) default.nation.n_name (TEXT), default.region.r_name (TEXT), p1 (INT4), p2 (INT4)} => in schema: {(4) default.nation.n_name (TEXT), default.region.r_name (TEXT), p1 (INT4), p2 (INT4)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin3.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin3.Sort.plan index 3f1e25d6da..bebae840b6 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin3.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin3.Sort.plan @@ -42,7 +42,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.region 1: type=Broadcast, tables=default.nation -SORT(9) +SORT(11) => Sort Keys: default.nation.n_name (TEXT) (asc) JOIN(7)(INNER) => Join Cond: default.nation.n_regionkey (INT4) = default.region.r_regionkey (INT4) @@ -70,7 +70,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.nation.n_name (TEXT) (asc) - SCAN(10) on eb_0000000000000_0000_000003 + SCAN(12) on eb_0000000000000_0000_000003 => out schema: {(4) default.nation.n_name (TEXT), default.region.r_name (TEXT), p1 (INT4), p2 (INT4)} => in schema: {(4) default.nation.n_name (TEXT), default.region.r_name (TEXT), p1 (INT4), p2 (INT4)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin4.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin4.Hash.plan index 03a12cf95e..0978d54994 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin4.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin4.Hash.plan @@ -42,7 +42,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.nation 1: type=Broadcast, tables=default.region -SORT(9) +SORT(11) => Sort Keys: default.nation.n_name (TEXT) (asc) JOIN(7)(INNER) => Join Cond: default.nation.n_regionkey (INT4) = default.region.r_regionkey (INT4) @@ -70,7 +70,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.nation.n_name (TEXT) (asc) - SCAN(10) on eb_0000000000000_0000_000003 + SCAN(12) on eb_0000000000000_0000_000003 => out schema: {(3) ?plus (INT4), default.nation.n_name (TEXT), default.region.r_name (TEXT)} => in schema: {(3) ?plus (INT4), default.nation.n_name (TEXT), default.region.r_name (TEXT)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin4.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin4.Sort.plan index 03a12cf95e..0978d54994 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin4.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin4.Sort.plan @@ -42,7 +42,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.nation 1: type=Broadcast, tables=default.region -SORT(9) +SORT(11) => Sort Keys: default.nation.n_name (TEXT) (asc) JOIN(7)(INNER) => Join Cond: default.nation.n_regionkey (INT4) = default.region.r_regionkey (INT4) @@ -70,7 +70,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.nation.n_name (TEXT) (asc) - SCAN(10) on eb_0000000000000_0000_000003 + SCAN(12) on eb_0000000000000_0000_000003 => out schema: {(3) ?plus (INT4), default.nation.n_name (TEXT), default.region.r_name (TEXT)} => in schema: {(3) ?plus (INT4), default.nation.n_name (TEXT), default.region.r_name (TEXT)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin5.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin5.Hash.plan index bdb74667fa..c948e64476 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin5.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin5.Hash.plan @@ -27,32 +27,32 @@ SORT(4) explain ------------------------------- ------------------------------------------------------------------------------- -Execution Block Graph (TERMINAL - eb_0000000000000_0000_000005) +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000007) ------------------------------------------------------------------------------- -|-eb_0000000000000_0000_000005 - |-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 +|-eb_0000000000000_0000_000007 + |-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000005 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 -3: eb_0000000000000_0000_000005 +1: eb_0000000000000_0000_000005 +2: eb_0000000000000_0000_000006 +3: eb_0000000000000_0000_000007 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [LEAF] +Block Id: eb_0000000000000_0000_000005 [LEAF] ======================================================= [Outgoing] -[q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_name (TEXT), num=32) +[q_0000000000000_0000] 5 => 6 (type=RANGE_SHUFFLE, key=default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_name (TEXT), num=32) [Enforcers] 0: type=Broadcast, tables=default.supplier 1: type=Broadcast, tables=default.partsupp 2: type=Broadcast, tables=default.part -SORT(12) +SORT(16) => Sort Keys: default.supplier.s_acctbal (FLOAT8) (asc),default.supplier.s_name (TEXT) (asc),default.part.p_partkey (INT4) (asc) JOIN(10)(INNER) => Join Cond: default.part.p_partkey (INT4) = default.partsupp.ps_partkey (INT4) @@ -78,21 +78,21 @@ SORT(12) => in schema: {(7) default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} ======================================================= -Block Id: eb_0000000000000_0000_000004 [ROOT] +Block Id: eb_0000000000000_0000_000006 [ROOT] ======================================================= [Incoming] -[q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_name (TEXT), num=32) +[q_0000000000000_0000] 5 => 6 (type=RANGE_SHUFFLE, key=default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_name (TEXT), num=32) [Enforcers] - 0: sorted input=eb_0000000000000_0000_000003 + 0: sorted input=eb_0000000000000_0000_000005 SORT(4) => Sort Keys: default.supplier.s_acctbal (FLOAT8) (asc),default.supplier.s_name (TEXT) (asc),default.part.p_partkey (INT4) (asc) - SCAN(13) on eb_0000000000000_0000_000003 + SCAN(17) on eb_0000000000000_0000_000005 => out schema: {(3) default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_name (TEXT)} => in schema: {(3) default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_name (TEXT)} ======================================================= -Block Id: eb_0000000000000_0000_000005 [TERMINAL] +Block Id: eb_0000000000000_0000_000007 [TERMINAL] ======================================================= diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin5.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin5.Sort.plan index bdb74667fa..c948e64476 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin5.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin5.Sort.plan @@ -27,32 +27,32 @@ SORT(4) explain ------------------------------- ------------------------------------------------------------------------------- -Execution Block Graph (TERMINAL - eb_0000000000000_0000_000005) +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000007) ------------------------------------------------------------------------------- -|-eb_0000000000000_0000_000005 - |-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 +|-eb_0000000000000_0000_000007 + |-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000005 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 -3: eb_0000000000000_0000_000005 +1: eb_0000000000000_0000_000005 +2: eb_0000000000000_0000_000006 +3: eb_0000000000000_0000_000007 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [LEAF] +Block Id: eb_0000000000000_0000_000005 [LEAF] ======================================================= [Outgoing] -[q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_name (TEXT), num=32) +[q_0000000000000_0000] 5 => 6 (type=RANGE_SHUFFLE, key=default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_name (TEXT), num=32) [Enforcers] 0: type=Broadcast, tables=default.supplier 1: type=Broadcast, tables=default.partsupp 2: type=Broadcast, tables=default.part -SORT(12) +SORT(16) => Sort Keys: default.supplier.s_acctbal (FLOAT8) (asc),default.supplier.s_name (TEXT) (asc),default.part.p_partkey (INT4) (asc) JOIN(10)(INNER) => Join Cond: default.part.p_partkey (INT4) = default.partsupp.ps_partkey (INT4) @@ -78,21 +78,21 @@ SORT(12) => in schema: {(7) default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} ======================================================= -Block Id: eb_0000000000000_0000_000004 [ROOT] +Block Id: eb_0000000000000_0000_000006 [ROOT] ======================================================= [Incoming] -[q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_name (TEXT), num=32) +[q_0000000000000_0000] 5 => 6 (type=RANGE_SHUFFLE, key=default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_name (TEXT), num=32) [Enforcers] - 0: sorted input=eb_0000000000000_0000_000003 + 0: sorted input=eb_0000000000000_0000_000005 SORT(4) => Sort Keys: default.supplier.s_acctbal (FLOAT8) (asc),default.supplier.s_name (TEXT) (asc),default.part.p_partkey (INT4) (asc) - SCAN(13) on eb_0000000000000_0000_000003 + SCAN(17) on eb_0000000000000_0000_000005 => out schema: {(3) default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_name (TEXT)} => in schema: {(3) default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_name (TEXT)} ======================================================= -Block Id: eb_0000000000000_0000_000005 [TERMINAL] +Block Id: eb_0000000000000_0000_000007 [TERMINAL] ======================================================= diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin6.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin6.Hash.plan index 79e38273ba..08bd65573e 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin6.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin6.Hash.plan @@ -36,33 +36,33 @@ SORT(5) explain ------------------------------- ------------------------------------------------------------------------------- -Execution Block Graph (TERMINAL - eb_0000000000000_0000_000005) +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000009) ------------------------------------------------------------------------------- -|-eb_0000000000000_0000_000005 - |-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 +|-eb_0000000000000_0000_000009 + |-eb_0000000000000_0000_000008 + |-eb_0000000000000_0000_000007 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 -3: eb_0000000000000_0000_000005 +1: eb_0000000000000_0000_000007 +2: eb_0000000000000_0000_000008 +3: eb_0000000000000_0000_000009 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [LEAF] +Block Id: eb_0000000000000_0000_000007 [LEAF] ======================================================= [Outgoing] -[q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.nation.n_name (TEXT), default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_name (TEXT), num=32) +[q_0000000000000_0000] 7 => 8 (type=RANGE_SHUFFLE, key=default.nation.n_name (TEXT), default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_name (TEXT), num=32) [Enforcers] - 0: type=Broadcast, tables=default.supplier - 1: type=Broadcast, tables=default.partsupp - 2: type=Broadcast, tables=default.part + 0: type=Broadcast, tables=default.part + 1: type=Broadcast, tables=default.supplier + 2: type=Broadcast, tables=default.partsupp 3: type=Broadcast, tables=default.nation -SORT(15) +SORT(21) => Sort Keys: default.supplier.s_acctbal (FLOAT8) (asc),default.supplier.s_name (TEXT) (asc),default.part.p_partkey (INT4) (asc),default.nation.n_name (TEXT) (asc) JOIN(13)(INNER) => Join Cond: default.supplier.s_nationkey (INT4) = default.nation.n_nationkey (INT4) @@ -97,21 +97,21 @@ SORT(15) => in schema: {(7) default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} ======================================================= -Block Id: eb_0000000000000_0000_000004 [ROOT] +Block Id: eb_0000000000000_0000_000008 [ROOT] ======================================================= [Incoming] -[q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.nation.n_name (TEXT), default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_name (TEXT), num=32) +[q_0000000000000_0000] 7 => 8 (type=RANGE_SHUFFLE, key=default.nation.n_name (TEXT), default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_name (TEXT), num=32) [Enforcers] - 0: sorted input=eb_0000000000000_0000_000003 + 0: sorted input=eb_0000000000000_0000_000007 SORT(5) => Sort Keys: default.supplier.s_acctbal (FLOAT8) (asc),default.supplier.s_name (TEXT) (asc),default.part.p_partkey (INT4) (asc),default.nation.n_name (TEXT) (asc) - SCAN(16) on eb_0000000000000_0000_000003 + SCAN(22) on eb_0000000000000_0000_000007 => out schema: {(4) default.nation.n_name (TEXT), default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_name (TEXT)} => in schema: {(4) default.nation.n_name (TEXT), default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_name (TEXT)} ======================================================= -Block Id: eb_0000000000000_0000_000005 [TERMINAL] +Block Id: eb_0000000000000_0000_000009 [TERMINAL] ======================================================= diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin6.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin6.Sort.plan index 79e38273ba..08bd65573e 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin6.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin6.Sort.plan @@ -36,33 +36,33 @@ SORT(5) explain ------------------------------- ------------------------------------------------------------------------------- -Execution Block Graph (TERMINAL - eb_0000000000000_0000_000005) +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000009) ------------------------------------------------------------------------------- -|-eb_0000000000000_0000_000005 - |-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 +|-eb_0000000000000_0000_000009 + |-eb_0000000000000_0000_000008 + |-eb_0000000000000_0000_000007 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 -3: eb_0000000000000_0000_000005 +1: eb_0000000000000_0000_000007 +2: eb_0000000000000_0000_000008 +3: eb_0000000000000_0000_000009 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [LEAF] +Block Id: eb_0000000000000_0000_000007 [LEAF] ======================================================= [Outgoing] -[q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.nation.n_name (TEXT), default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_name (TEXT), num=32) +[q_0000000000000_0000] 7 => 8 (type=RANGE_SHUFFLE, key=default.nation.n_name (TEXT), default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_name (TEXT), num=32) [Enforcers] - 0: type=Broadcast, tables=default.supplier - 1: type=Broadcast, tables=default.partsupp - 2: type=Broadcast, tables=default.part + 0: type=Broadcast, tables=default.part + 1: type=Broadcast, tables=default.supplier + 2: type=Broadcast, tables=default.partsupp 3: type=Broadcast, tables=default.nation -SORT(15) +SORT(21) => Sort Keys: default.supplier.s_acctbal (FLOAT8) (asc),default.supplier.s_name (TEXT) (asc),default.part.p_partkey (INT4) (asc),default.nation.n_name (TEXT) (asc) JOIN(13)(INNER) => Join Cond: default.supplier.s_nationkey (INT4) = default.nation.n_nationkey (INT4) @@ -97,21 +97,21 @@ SORT(15) => in schema: {(7) default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} ======================================================= -Block Id: eb_0000000000000_0000_000004 [ROOT] +Block Id: eb_0000000000000_0000_000008 [ROOT] ======================================================= [Incoming] -[q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.nation.n_name (TEXT), default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_name (TEXT), num=32) +[q_0000000000000_0000] 7 => 8 (type=RANGE_SHUFFLE, key=default.nation.n_name (TEXT), default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_name (TEXT), num=32) [Enforcers] - 0: sorted input=eb_0000000000000_0000_000003 + 0: sorted input=eb_0000000000000_0000_000007 SORT(5) => Sort Keys: default.supplier.s_acctbal (FLOAT8) (asc),default.supplier.s_name (TEXT) (asc),default.part.p_partkey (INT4) (asc),default.nation.n_name (TEXT) (asc) - SCAN(16) on eb_0000000000000_0000_000003 + SCAN(22) on eb_0000000000000_0000_000007 => out schema: {(4) default.nation.n_name (TEXT), default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_name (TEXT)} => in schema: {(4) default.nation.n_name (TEXT), default.part.p_partkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_name (TEXT)} ======================================================= -Block Id: eb_0000000000000_0000_000005 [TERMINAL] +Block Id: eb_0000000000000_0000_000009 [TERMINAL] ======================================================= diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery.Hash.plan index 0c134d5453..ab65357321 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery.Hash.plan @@ -36,100 +36,26 @@ Execution Block Graph (TERMINAL - eb_0000000000000_0000_000007) |-eb_0000000000000_0000_000007 |-eb_0000000000000_0000_000006 |-eb_0000000000000_0000_000005 - |-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 - |-eb_0000000000000_0000_000002 - |-eb_0000000000000_0000_000001 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000001 -2: eb_0000000000000_0000_000002 -3: eb_0000000000000_0000_000003 -4: eb_0000000000000_0000_000004 -5: eb_0000000000000_0000_000005 -6: eb_0000000000000_0000_000006 -7: eb_0000000000000_0000_000007 +1: eb_0000000000000_0000_000005 +2: eb_0000000000000_0000_000006 +3: eb_0000000000000_0000_000007 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000001 [LEAF] +Block Id: eb_0000000000000_0000_000005 [LEAF] ======================================================= -[Outgoing] -[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.a.o_orderkey (INT4), num=32) - -TABLE_SUBQUERY(4) as default.a - => Targets: default.a.o_custkey (INT4), default.a.o_orderkey (INT4) - => out schema: {(2) default.a.o_custkey (INT4), default.a.o_orderkey (INT4)} - => in schema: {(2) default.a.o_custkey (INT4), default.a.o_orderkey (INT4)} - PROJECTION(3) - => Targets: default.orders.o_orderkey (INT4), default.orders.o_custkey (INT4) - => out schema: {(2) default.orders.o_custkey (INT4), default.orders.o_orderkey (INT4)} - => in schema: {(2) default.orders.o_custkey (INT4), default.orders.o_orderkey (INT4)} - SCAN(2) on default.orders - => target list: default.orders.o_custkey (INT4), default.orders.o_orderkey (INT4) - => out schema: {(2) default.orders.o_custkey (INT4), default.orders.o_orderkey (INT4)} - => in schema: {(9) default.orders.o_clerk (TEXT), default.orders.o_comment (TEXT), default.orders.o_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4), default.orders.o_orderpriority (TEXT), default.orders.o_orderstatus (TEXT), default.orders.o_shippriority (INT4), default.orders.o_totalprice (FLOAT8)} - -======================================================= -Block Id: eb_0000000000000_0000_000002 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.lineitem.l_orderkey (INT4), num=32) - -SCAN(0) on default.lineitem - => target list: default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4) - => out schema: {(2) default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4)} - => in schema: {(16) default.lineitem.l_comment (TEXT), default.lineitem.l_commitdate (TEXT), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_linenumber (INT4), default.lineitem.l_linestatus (TEXT), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.lineitem.l_quantity (FLOAT8), default.lineitem.l_receiptdate (TEXT), default.lineitem.l_returnflag (TEXT), default.lineitem.l_shipdate (TEXT), default.lineitem.l_shipinstruct (TEXT), default.lineitem.l_shipmode (TEXT), default.lineitem.l_suppkey (INT4), default.lineitem.l_tax (FLOAT8)} - -======================================================= -Block Id: eb_0000000000000_0000_000003 [INTERMEDIATE] -======================================================= - -[Incoming] -[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.a.o_orderkey (INT4), num=32) -[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.lineitem.l_orderkey (INT4), num=32) - -[Outgoing] -[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.lineitem.l_partkey (INT4), num=32) - -JOIN(11)(INNER) - => Join Cond: default.lineitem.l_orderkey (INT4) = default.a.o_orderkey (INT4) - => target list: default.a.o_custkey (INT4), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4) - => out schema: {(3) default.a.o_custkey (INT4), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4)} - => in schema: {(4) default.a.o_custkey (INT4), default.a.o_orderkey (INT4), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4)} - SCAN(15) on eb_0000000000000_0000_000002 - => out schema: {(2) default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4)} - => in schema: {(2) default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4)} - SCAN(14) on eb_0000000000000_0000_000001 - => out schema: {(2) default.a.o_custkey (INT4), default.a.o_orderkey (INT4)} - => in schema: {(2) default.a.o_custkey (INT4), default.a.o_orderkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000004 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.part.p_partkey (INT4), num=32) - -SCAN(1) on default.part - => target list: default.part.p_name (TEXT), default.part.p_partkey (INT4) - => out schema: {(2) default.part.p_name (TEXT), default.part.p_partkey (INT4)} - => in schema: {(9) default.part.p_brand (TEXT), default.part.p_comment (TEXT), default.part.p_container (TEXT), default.part.p_mfgr (TEXT), default.part.p_name (TEXT), default.part.p_partkey (INT4), default.part.p_retailprice (FLOAT8), default.part.p_size (INT4), default.part.p_type (TEXT)} - -======================================================= -Block Id: eb_0000000000000_0000_000005 [INTERMEDIATE] -======================================================= - -[Incoming] -[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.lineitem.l_partkey (INT4), num=32) -[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.part.p_partkey (INT4), num=32) - [Outgoing] [q_0000000000000_0000] 5 => 6 (type=RANGE_SHUFFLE, key=default.a.o_custkey (INT4), default.lineitem.l_orderkey (INT4), default.part.p_name (TEXT), num=32) +[Enforcers] + 0: type=Broadcast, tables=default.orders + 1: type=Broadcast, tables=default.lineitem + 2: type=Broadcast, tables=default.part + SORT(18) => Sort Keys: default.lineitem.l_orderkey (INT4) (asc),default.a.o_custkey (INT4) (asc),default.part.p_name (TEXT) (asc) JOIN(12)(INNER) @@ -137,12 +63,31 @@ SORT(18) => target list: default.a.o_custkey (INT4), default.lineitem.l_orderkey (INT4), default.part.p_name (TEXT) => out schema: {(3) default.a.o_custkey (INT4), default.lineitem.l_orderkey (INT4), default.part.p_name (TEXT)} => in schema: {(5) default.a.o_custkey (INT4), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.part.p_name (TEXT), default.part.p_partkey (INT4)} - SCAN(17) on eb_0000000000000_0000_000004 + SCAN(1) on default.part + => target list: default.part.p_name (TEXT), default.part.p_partkey (INT4) => out schema: {(2) default.part.p_name (TEXT), default.part.p_partkey (INT4)} - => in schema: {(2) default.part.p_name (TEXT), default.part.p_partkey (INT4)} - SCAN(16) on eb_0000000000000_0000_000003 + => in schema: {(9) default.part.p_brand (TEXT), default.part.p_comment (TEXT), default.part.p_container (TEXT), default.part.p_mfgr (TEXT), default.part.p_name (TEXT), default.part.p_partkey (INT4), default.part.p_retailprice (FLOAT8), default.part.p_size (INT4), default.part.p_type (TEXT)} + JOIN(11)(INNER) + => Join Cond: default.lineitem.l_orderkey (INT4) = default.a.o_orderkey (INT4) + => target list: default.a.o_custkey (INT4), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4) => out schema: {(3) default.a.o_custkey (INT4), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4)} - => in schema: {(3) default.a.o_custkey (INT4), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4)} + => in schema: {(4) default.a.o_custkey (INT4), default.a.o_orderkey (INT4), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4)} + SCAN(0) on default.lineitem + => target list: default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4) + => out schema: {(2) default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4)} + => in schema: {(16) default.lineitem.l_comment (TEXT), default.lineitem.l_commitdate (TEXT), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_linenumber (INT4), default.lineitem.l_linestatus (TEXT), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.lineitem.l_quantity (FLOAT8), default.lineitem.l_receiptdate (TEXT), default.lineitem.l_returnflag (TEXT), default.lineitem.l_shipdate (TEXT), default.lineitem.l_shipinstruct (TEXT), default.lineitem.l_shipmode (TEXT), default.lineitem.l_suppkey (INT4), default.lineitem.l_tax (FLOAT8)} + TABLE_SUBQUERY(4) as default.a + => Targets: default.a.o_custkey (INT4), default.a.o_orderkey (INT4) + => out schema: {(2) default.a.o_custkey (INT4), default.a.o_orderkey (INT4)} + => in schema: {(2) default.a.o_custkey (INT4), default.a.o_orderkey (INT4)} + PROJECTION(3) + => Targets: default.orders.o_orderkey (INT4), default.orders.o_custkey (INT4) + => out schema: {(2) default.orders.o_custkey (INT4), default.orders.o_orderkey (INT4)} + => in schema: {(2) default.orders.o_custkey (INT4), default.orders.o_orderkey (INT4)} + SCAN(2) on default.orders + => target list: default.orders.o_custkey (INT4), default.orders.o_orderkey (INT4) + => out schema: {(2) default.orders.o_custkey (INT4), default.orders.o_orderkey (INT4)} + => in schema: {(9) default.orders.o_clerk (TEXT), default.orders.o_comment (TEXT), default.orders.o_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4), default.orders.o_orderpriority (TEXT), default.orders.o_orderstatus (TEXT), default.orders.o_shippriority (INT4), default.orders.o_totalprice (FLOAT8)} ======================================================= Block Id: eb_0000000000000_0000_000006 [ROOT] diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery.Sort.plan index 0c134d5453..ab65357321 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery.Sort.plan @@ -36,100 +36,26 @@ Execution Block Graph (TERMINAL - eb_0000000000000_0000_000007) |-eb_0000000000000_0000_000007 |-eb_0000000000000_0000_000006 |-eb_0000000000000_0000_000005 - |-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 - |-eb_0000000000000_0000_000002 - |-eb_0000000000000_0000_000001 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000001 -2: eb_0000000000000_0000_000002 -3: eb_0000000000000_0000_000003 -4: eb_0000000000000_0000_000004 -5: eb_0000000000000_0000_000005 -6: eb_0000000000000_0000_000006 -7: eb_0000000000000_0000_000007 +1: eb_0000000000000_0000_000005 +2: eb_0000000000000_0000_000006 +3: eb_0000000000000_0000_000007 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000001 [LEAF] +Block Id: eb_0000000000000_0000_000005 [LEAF] ======================================================= -[Outgoing] -[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.a.o_orderkey (INT4), num=32) - -TABLE_SUBQUERY(4) as default.a - => Targets: default.a.o_custkey (INT4), default.a.o_orderkey (INT4) - => out schema: {(2) default.a.o_custkey (INT4), default.a.o_orderkey (INT4)} - => in schema: {(2) default.a.o_custkey (INT4), default.a.o_orderkey (INT4)} - PROJECTION(3) - => Targets: default.orders.o_orderkey (INT4), default.orders.o_custkey (INT4) - => out schema: {(2) default.orders.o_custkey (INT4), default.orders.o_orderkey (INT4)} - => in schema: {(2) default.orders.o_custkey (INT4), default.orders.o_orderkey (INT4)} - SCAN(2) on default.orders - => target list: default.orders.o_custkey (INT4), default.orders.o_orderkey (INT4) - => out schema: {(2) default.orders.o_custkey (INT4), default.orders.o_orderkey (INT4)} - => in schema: {(9) default.orders.o_clerk (TEXT), default.orders.o_comment (TEXT), default.orders.o_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4), default.orders.o_orderpriority (TEXT), default.orders.o_orderstatus (TEXT), default.orders.o_shippriority (INT4), default.orders.o_totalprice (FLOAT8)} - -======================================================= -Block Id: eb_0000000000000_0000_000002 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.lineitem.l_orderkey (INT4), num=32) - -SCAN(0) on default.lineitem - => target list: default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4) - => out schema: {(2) default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4)} - => in schema: {(16) default.lineitem.l_comment (TEXT), default.lineitem.l_commitdate (TEXT), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_linenumber (INT4), default.lineitem.l_linestatus (TEXT), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.lineitem.l_quantity (FLOAT8), default.lineitem.l_receiptdate (TEXT), default.lineitem.l_returnflag (TEXT), default.lineitem.l_shipdate (TEXT), default.lineitem.l_shipinstruct (TEXT), default.lineitem.l_shipmode (TEXT), default.lineitem.l_suppkey (INT4), default.lineitem.l_tax (FLOAT8)} - -======================================================= -Block Id: eb_0000000000000_0000_000003 [INTERMEDIATE] -======================================================= - -[Incoming] -[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.a.o_orderkey (INT4), num=32) -[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.lineitem.l_orderkey (INT4), num=32) - -[Outgoing] -[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.lineitem.l_partkey (INT4), num=32) - -JOIN(11)(INNER) - => Join Cond: default.lineitem.l_orderkey (INT4) = default.a.o_orderkey (INT4) - => target list: default.a.o_custkey (INT4), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4) - => out schema: {(3) default.a.o_custkey (INT4), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4)} - => in schema: {(4) default.a.o_custkey (INT4), default.a.o_orderkey (INT4), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4)} - SCAN(15) on eb_0000000000000_0000_000002 - => out schema: {(2) default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4)} - => in schema: {(2) default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4)} - SCAN(14) on eb_0000000000000_0000_000001 - => out schema: {(2) default.a.o_custkey (INT4), default.a.o_orderkey (INT4)} - => in schema: {(2) default.a.o_custkey (INT4), default.a.o_orderkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000004 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.part.p_partkey (INT4), num=32) - -SCAN(1) on default.part - => target list: default.part.p_name (TEXT), default.part.p_partkey (INT4) - => out schema: {(2) default.part.p_name (TEXT), default.part.p_partkey (INT4)} - => in schema: {(9) default.part.p_brand (TEXT), default.part.p_comment (TEXT), default.part.p_container (TEXT), default.part.p_mfgr (TEXT), default.part.p_name (TEXT), default.part.p_partkey (INT4), default.part.p_retailprice (FLOAT8), default.part.p_size (INT4), default.part.p_type (TEXT)} - -======================================================= -Block Id: eb_0000000000000_0000_000005 [INTERMEDIATE] -======================================================= - -[Incoming] -[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.lineitem.l_partkey (INT4), num=32) -[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.part.p_partkey (INT4), num=32) - [Outgoing] [q_0000000000000_0000] 5 => 6 (type=RANGE_SHUFFLE, key=default.a.o_custkey (INT4), default.lineitem.l_orderkey (INT4), default.part.p_name (TEXT), num=32) +[Enforcers] + 0: type=Broadcast, tables=default.orders + 1: type=Broadcast, tables=default.lineitem + 2: type=Broadcast, tables=default.part + SORT(18) => Sort Keys: default.lineitem.l_orderkey (INT4) (asc),default.a.o_custkey (INT4) (asc),default.part.p_name (TEXT) (asc) JOIN(12)(INNER) @@ -137,12 +63,31 @@ SORT(18) => target list: default.a.o_custkey (INT4), default.lineitem.l_orderkey (INT4), default.part.p_name (TEXT) => out schema: {(3) default.a.o_custkey (INT4), default.lineitem.l_orderkey (INT4), default.part.p_name (TEXT)} => in schema: {(5) default.a.o_custkey (INT4), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.part.p_name (TEXT), default.part.p_partkey (INT4)} - SCAN(17) on eb_0000000000000_0000_000004 + SCAN(1) on default.part + => target list: default.part.p_name (TEXT), default.part.p_partkey (INT4) => out schema: {(2) default.part.p_name (TEXT), default.part.p_partkey (INT4)} - => in schema: {(2) default.part.p_name (TEXT), default.part.p_partkey (INT4)} - SCAN(16) on eb_0000000000000_0000_000003 + => in schema: {(9) default.part.p_brand (TEXT), default.part.p_comment (TEXT), default.part.p_container (TEXT), default.part.p_mfgr (TEXT), default.part.p_name (TEXT), default.part.p_partkey (INT4), default.part.p_retailprice (FLOAT8), default.part.p_size (INT4), default.part.p_type (TEXT)} + JOIN(11)(INNER) + => Join Cond: default.lineitem.l_orderkey (INT4) = default.a.o_orderkey (INT4) + => target list: default.a.o_custkey (INT4), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4) => out schema: {(3) default.a.o_custkey (INT4), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4)} - => in schema: {(3) default.a.o_custkey (INT4), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4)} + => in schema: {(4) default.a.o_custkey (INT4), default.a.o_orderkey (INT4), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4)} + SCAN(0) on default.lineitem + => target list: default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4) + => out schema: {(2) default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4)} + => in schema: {(16) default.lineitem.l_comment (TEXT), default.lineitem.l_commitdate (TEXT), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_linenumber (INT4), default.lineitem.l_linestatus (TEXT), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.lineitem.l_quantity (FLOAT8), default.lineitem.l_receiptdate (TEXT), default.lineitem.l_returnflag (TEXT), default.lineitem.l_shipdate (TEXT), default.lineitem.l_shipinstruct (TEXT), default.lineitem.l_shipmode (TEXT), default.lineitem.l_suppkey (INT4), default.lineitem.l_tax (FLOAT8)} + TABLE_SUBQUERY(4) as default.a + => Targets: default.a.o_custkey (INT4), default.a.o_orderkey (INT4) + => out schema: {(2) default.a.o_custkey (INT4), default.a.o_orderkey (INT4)} + => in schema: {(2) default.a.o_custkey (INT4), default.a.o_orderkey (INT4)} + PROJECTION(3) + => Targets: default.orders.o_orderkey (INT4), default.orders.o_custkey (INT4) + => out schema: {(2) default.orders.o_custkey (INT4), default.orders.o_orderkey (INT4)} + => in schema: {(2) default.orders.o_custkey (INT4), default.orders.o_orderkey (INT4)} + SCAN(2) on default.orders + => target list: default.orders.o_custkey (INT4), default.orders.o_orderkey (INT4) + => out schema: {(2) default.orders.o_custkey (INT4), default.orders.o_orderkey (INT4)} + => in schema: {(9) default.orders.o_clerk (TEXT), default.orders.o_comment (TEXT), default.orders.o_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4), default.orders.o_orderpriority (TEXT), default.orders.o_orderstatus (TEXT), default.orders.o_shippriority (INT4), default.orders.o_totalprice (FLOAT8)} ======================================================= Block Id: eb_0000000000000_0000_000006 [ROOT] diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery2.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery2.Hash.plan index 7cde4c5d61..dd00452aa2 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery2.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery2.Hash.plan @@ -73,22 +73,16 @@ Execution Block Graph (TERMINAL - eb_0000000000000_0000_000012) |-eb_0000000000000_0000_000012 |-eb_0000000000000_0000_000011 |-eb_0000000000000_0000_000010 - |-eb_0000000000000_0000_000009 - |-eb_0000000000000_0000_000008 - |-eb_0000000000000_0000_000007 |-eb_0000000000000_0000_000004 |-eb_0000000000000_0000_000003 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- 1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000007 -3: eb_0000000000000_0000_000008 -4: eb_0000000000000_0000_000004 -5: eb_0000000000000_0000_000009 -6: eb_0000000000000_0000_000010 -7: eb_0000000000000_0000_000011 -8: eb_0000000000000_0000_000012 +2: eb_0000000000000_0000_000004 +3: eb_0000000000000_0000_000010 +4: eb_0000000000000_0000_000011 +5: eb_0000000000000_0000_000012 ------------------------------------------------------------------------------- ======================================================= @@ -102,7 +96,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.a 1: type=Broadcast, tables=default.part -GROUP_BY(25)(l_orderkey) +GROUP_BY(27)(l_orderkey) => exprs: (avg(default.e.l_quantity (FLOAT8))) => target list: default.e.l_orderkey (INT4), ?avg_11 (PROTOBUF) => out schema:{(2) default.e.l_orderkey (INT4), ?avg_11 (PROTOBUF)} @@ -152,91 +146,26 @@ TABLE_SUBQUERY(14) as default.f => target list: default.e.l_orderkey (INT4), avg_quantity (FLOAT8) => out schema:{(2) avg_quantity (FLOAT8), default.e.l_orderkey (INT4)} => in schema:{(2) default.e.l_orderkey (INT4), ?avg_11 (PROTOBUF)} - SCAN(26) on eb_0000000000000_0000_000003 + SCAN(28) on eb_0000000000000_0000_000003 => out schema: {(2) default.e.l_orderkey (INT4), ?avg_11 (PROTOBUF)} => in schema: {(2) default.e.l_orderkey (INT4), ?avg_11 (PROTOBUF)} -======================================================= -Block Id: eb_0000000000000_0000_000007 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 7 => 9 (type=HASH_SHUFFLE, key=default.b.l_orderkey (INT4), num=32) - -[Enforcers] - 0: type=Broadcast, tables=default.d - 1: type=Broadcast, tables=default.part - -TABLE_SUBQUERY(4) as default.b - => Targets: default.b.l_quantity (FLOAT8), default.b.l_orderkey (INT4) - => out schema: {(2) default.b.l_quantity (FLOAT8), default.b.l_orderkey (INT4)} - => in schema: {(2) default.b.l_orderkey (INT4), default.b.l_quantity (FLOAT8)} - PROJECTION(10) - => Targets: default.d.l_orderkey (INT4), default.d.l_quantity (FLOAT8) - => out schema: {(2) default.d.l_orderkey (INT4), default.d.l_quantity (FLOAT8)} - => in schema: {(2) default.d.l_orderkey (INT4), default.d.l_quantity (FLOAT8)} - JOIN(20)(INNER) - => Join Cond: default.d.l_partkey (INT4) = default.part.p_partkey (INT4) - => target list: default.d.l_orderkey (INT4), default.d.l_quantity (FLOAT8) - => out schema: {(2) default.d.l_orderkey (INT4), default.d.l_quantity (FLOAT8)} - => in schema: {(4) default.d.l_orderkey (INT4), default.d.l_partkey (INT4), default.d.l_quantity (FLOAT8), default.part.p_partkey (INT4)} - SCAN(8) on default.part - => target list: default.part.p_partkey (INT4) - => out schema: {(1) default.part.p_partkey (INT4)} - => in schema: {(9) default.part.p_brand (TEXT), default.part.p_comment (TEXT), default.part.p_container (TEXT), default.part.p_mfgr (TEXT), default.part.p_name (TEXT), default.part.p_partkey (INT4), default.part.p_retailprice (FLOAT8), default.part.p_size (INT4), default.part.p_type (TEXT)} - SCAN(7) on default.lineitem as d - => target list: default.d.l_orderkey (INT4), default.d.l_partkey (INT4), default.d.l_quantity (FLOAT8) - => out schema: {(3) default.d.l_orderkey (INT4), default.d.l_partkey (INT4), default.d.l_quantity (FLOAT8)} - => in schema: {(16) default.d.l_comment (TEXT), default.d.l_commitdate (TEXT), default.d.l_discount (FLOAT8), default.d.l_extendedprice (FLOAT8), default.d.l_linenumber (INT4), default.d.l_linestatus (TEXT), default.d.l_orderkey (INT4), default.d.l_partkey (INT4), default.d.l_quantity (FLOAT8), default.d.l_receiptdate (TEXT), default.d.l_returnflag (TEXT), default.d.l_shipdate (TEXT), default.d.l_shipinstruct (TEXT), default.d.l_shipmode (TEXT), default.d.l_suppkey (INT4), default.d.l_tax (FLOAT8)} - -======================================================= -Block Id: eb_0000000000000_0000_000008 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 8 => 9 (type=HASH_SHUFFLE, key=default.c.o_orderkey (INT4), num=32) - -SCAN(5) on default.orders as c - => filter: default.c.o_orderkey (INT4) > 0 - => target list: default.c.o_orderkey (INT4) - => out schema: {(1) default.c.o_orderkey (INT4)} - => in schema: {(9) default.c.o_clerk (TEXT), default.c.o_comment (TEXT), default.c.o_custkey (INT4), default.c.o_orderdate (TEXT), default.c.o_orderkey (INT4), default.c.o_orderpriority (TEXT), default.c.o_orderstatus (TEXT), default.c.o_shippriority (INT4), default.c.o_totalprice (FLOAT8)} - -======================================================= -Block Id: eb_0000000000000_0000_000009 [INTERMEDIATE] -======================================================= - -[Incoming] -[q_0000000000000_0000] 7 => 9 (type=HASH_SHUFFLE, key=default.b.l_orderkey (INT4), num=32) -[q_0000000000000_0000] 8 => 9 (type=HASH_SHUFFLE, key=default.c.o_orderkey (INT4), num=32) - -[Outgoing] -[q_0000000000000_0000] 9 => 10 (type=HASH_SHUFFLE, key=default.c.o_orderkey (INT4), num=32) - -JOIN(22)(INNER) - => Join Cond: default.c.o_orderkey (INT4) = default.b.l_orderkey (INT4) - => target list: default.b.l_quantity (FLOAT8), default.c.o_orderkey (INT4) - => out schema: {(2) default.b.l_quantity (FLOAT8), default.c.o_orderkey (INT4)} - => in schema: {(3) default.b.l_orderkey (INT4), default.b.l_quantity (FLOAT8), default.c.o_orderkey (INT4)} - SCAN(28) on eb_0000000000000_0000_000008 - => out schema: {(1) default.c.o_orderkey (INT4)} - => in schema: {(1) default.c.o_orderkey (INT4)} - SCAN(27) on eb_0000000000000_0000_000007 - => out schema: {(2) default.b.l_quantity (FLOAT8), default.b.l_orderkey (INT4)} - => in schema: {(2) default.b.l_quantity (FLOAT8), default.b.l_orderkey (INT4)} - ======================================================= Block Id: eb_0000000000000_0000_000010 [INTERMEDIATE] ======================================================= [Incoming] [q_0000000000000_0000] 4 => 10 (type=HASH_SHUFFLE, key=default.f.l_orderkey (INT4), num=32) -[q_0000000000000_0000] 9 => 10 (type=HASH_SHUFFLE, key=default.c.o_orderkey (INT4), num=32) [Outgoing] [q_0000000000000_0000] 10 => 11 (type=HASH_SHUFFLE, key=, num=1) -GROUP_BY(31)() +[Enforcers] + 0: type=Broadcast, tables=default.part + 1: type=Broadcast, tables=default.c + 2: type=Broadcast, tables=default.d + +GROUP_BY(35)() => exprs: (sum(default.b.l_quantity (FLOAT8))) => target list: ?sum_12 (FLOAT8) => out schema:{(1) ?sum_12 (FLOAT8)} @@ -248,10 +177,38 @@ GROUP_BY(31)() => target list: default.b.l_quantity (FLOAT8), default.f.avg_quantity (FLOAT8) => out schema: {(2) default.b.l_quantity (FLOAT8), default.f.avg_quantity (FLOAT8)} => in schema: {(4) default.b.l_quantity (FLOAT8), default.c.o_orderkey (INT4), default.f.avg_quantity (FLOAT8), default.f.l_orderkey (INT4)} - SCAN(30) on eb_0000000000000_0000_000009 + JOIN(22)(INNER) + => Join Cond: default.c.o_orderkey (INT4) = default.b.l_orderkey (INT4) + => target list: default.b.l_quantity (FLOAT8), default.c.o_orderkey (INT4) => out schema: {(2) default.b.l_quantity (FLOAT8), default.c.o_orderkey (INT4)} - => in schema: {(2) default.b.l_quantity (FLOAT8), default.c.o_orderkey (INT4)} - SCAN(29) on eb_0000000000000_0000_000004 + => in schema: {(3) default.b.l_orderkey (INT4), default.b.l_quantity (FLOAT8), default.c.o_orderkey (INT4)} + SCAN(5) on default.orders as c + => filter: default.c.o_orderkey (INT4) > 0 + => target list: default.c.o_orderkey (INT4) + => out schema: {(1) default.c.o_orderkey (INT4)} + => in schema: {(9) default.c.o_clerk (TEXT), default.c.o_comment (TEXT), default.c.o_custkey (INT4), default.c.o_orderdate (TEXT), default.c.o_orderkey (INT4), default.c.o_orderpriority (TEXT), default.c.o_orderstatus (TEXT), default.c.o_shippriority (INT4), default.c.o_totalprice (FLOAT8)} + TABLE_SUBQUERY(4) as default.b + => Targets: default.b.l_quantity (FLOAT8), default.b.l_orderkey (INT4) + => out schema: {(2) default.b.l_quantity (FLOAT8), default.b.l_orderkey (INT4)} + => in schema: {(2) default.b.l_orderkey (INT4), default.b.l_quantity (FLOAT8)} + PROJECTION(10) + => Targets: default.d.l_orderkey (INT4), default.d.l_quantity (FLOAT8) + => out schema: {(2) default.d.l_orderkey (INT4), default.d.l_quantity (FLOAT8)} + => in schema: {(2) default.d.l_orderkey (INT4), default.d.l_quantity (FLOAT8)} + JOIN(20)(INNER) + => Join Cond: default.d.l_partkey (INT4) = default.part.p_partkey (INT4) + => target list: default.d.l_orderkey (INT4), default.d.l_quantity (FLOAT8) + => out schema: {(2) default.d.l_orderkey (INT4), default.d.l_quantity (FLOAT8)} + => in schema: {(4) default.d.l_orderkey (INT4), default.d.l_partkey (INT4), default.d.l_quantity (FLOAT8), default.part.p_partkey (INT4)} + SCAN(8) on default.part + => target list: default.part.p_partkey (INT4) + => out schema: {(1) default.part.p_partkey (INT4)} + => in schema: {(9) default.part.p_brand (TEXT), default.part.p_comment (TEXT), default.part.p_container (TEXT), default.part.p_mfgr (TEXT), default.part.p_name (TEXT), default.part.p_partkey (INT4), default.part.p_retailprice (FLOAT8), default.part.p_size (INT4), default.part.p_type (TEXT)} + SCAN(7) on default.lineitem as d + => target list: default.d.l_orderkey (INT4), default.d.l_partkey (INT4), default.d.l_quantity (FLOAT8) + => out schema: {(3) default.d.l_orderkey (INT4), default.d.l_partkey (INT4), default.d.l_quantity (FLOAT8)} + => in schema: {(16) default.d.l_comment (TEXT), default.d.l_commitdate (TEXT), default.d.l_discount (FLOAT8), default.d.l_extendedprice (FLOAT8), default.d.l_linenumber (INT4), default.d.l_linestatus (TEXT), default.d.l_orderkey (INT4), default.d.l_partkey (INT4), default.d.l_quantity (FLOAT8), default.d.l_receiptdate (TEXT), default.d.l_returnflag (TEXT), default.d.l_shipdate (TEXT), default.d.l_shipinstruct (TEXT), default.d.l_shipmode (TEXT), default.d.l_suppkey (INT4), default.d.l_tax (FLOAT8)} + SCAN(33) on eb_0000000000000_0000_000004 => out schema: {(2) default.f.avg_quantity (FLOAT8), default.f.l_orderkey (INT4)} => in schema: {(2) default.f.avg_quantity (FLOAT8), default.f.l_orderkey (INT4)} @@ -267,7 +224,7 @@ GROUP_BY(18)() => target list: ?sum (FLOAT8) => out schema:{(1) ?sum (FLOAT8)} => in schema:{(1) ?sum_12 (FLOAT8)} - SCAN(32) on eb_0000000000000_0000_000010 + SCAN(36) on eb_0000000000000_0000_000010 => out schema: {(1) ?sum_12 (FLOAT8)} => in schema: {(1) ?sum_12 (FLOAT8)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery2.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery2.Sort.plan index 7cde4c5d61..dd00452aa2 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery2.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery2.Sort.plan @@ -73,22 +73,16 @@ Execution Block Graph (TERMINAL - eb_0000000000000_0000_000012) |-eb_0000000000000_0000_000012 |-eb_0000000000000_0000_000011 |-eb_0000000000000_0000_000010 - |-eb_0000000000000_0000_000009 - |-eb_0000000000000_0000_000008 - |-eb_0000000000000_0000_000007 |-eb_0000000000000_0000_000004 |-eb_0000000000000_0000_000003 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- 1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000007 -3: eb_0000000000000_0000_000008 -4: eb_0000000000000_0000_000004 -5: eb_0000000000000_0000_000009 -6: eb_0000000000000_0000_000010 -7: eb_0000000000000_0000_000011 -8: eb_0000000000000_0000_000012 +2: eb_0000000000000_0000_000004 +3: eb_0000000000000_0000_000010 +4: eb_0000000000000_0000_000011 +5: eb_0000000000000_0000_000012 ------------------------------------------------------------------------------- ======================================================= @@ -102,7 +96,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] 0: type=Broadcast, tables=default.a 1: type=Broadcast, tables=default.part -GROUP_BY(25)(l_orderkey) +GROUP_BY(27)(l_orderkey) => exprs: (avg(default.e.l_quantity (FLOAT8))) => target list: default.e.l_orderkey (INT4), ?avg_11 (PROTOBUF) => out schema:{(2) default.e.l_orderkey (INT4), ?avg_11 (PROTOBUF)} @@ -152,91 +146,26 @@ TABLE_SUBQUERY(14) as default.f => target list: default.e.l_orderkey (INT4), avg_quantity (FLOAT8) => out schema:{(2) avg_quantity (FLOAT8), default.e.l_orderkey (INT4)} => in schema:{(2) default.e.l_orderkey (INT4), ?avg_11 (PROTOBUF)} - SCAN(26) on eb_0000000000000_0000_000003 + SCAN(28) on eb_0000000000000_0000_000003 => out schema: {(2) default.e.l_orderkey (INT4), ?avg_11 (PROTOBUF)} => in schema: {(2) default.e.l_orderkey (INT4), ?avg_11 (PROTOBUF)} -======================================================= -Block Id: eb_0000000000000_0000_000007 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 7 => 9 (type=HASH_SHUFFLE, key=default.b.l_orderkey (INT4), num=32) - -[Enforcers] - 0: type=Broadcast, tables=default.d - 1: type=Broadcast, tables=default.part - -TABLE_SUBQUERY(4) as default.b - => Targets: default.b.l_quantity (FLOAT8), default.b.l_orderkey (INT4) - => out schema: {(2) default.b.l_quantity (FLOAT8), default.b.l_orderkey (INT4)} - => in schema: {(2) default.b.l_orderkey (INT4), default.b.l_quantity (FLOAT8)} - PROJECTION(10) - => Targets: default.d.l_orderkey (INT4), default.d.l_quantity (FLOAT8) - => out schema: {(2) default.d.l_orderkey (INT4), default.d.l_quantity (FLOAT8)} - => in schema: {(2) default.d.l_orderkey (INT4), default.d.l_quantity (FLOAT8)} - JOIN(20)(INNER) - => Join Cond: default.d.l_partkey (INT4) = default.part.p_partkey (INT4) - => target list: default.d.l_orderkey (INT4), default.d.l_quantity (FLOAT8) - => out schema: {(2) default.d.l_orderkey (INT4), default.d.l_quantity (FLOAT8)} - => in schema: {(4) default.d.l_orderkey (INT4), default.d.l_partkey (INT4), default.d.l_quantity (FLOAT8), default.part.p_partkey (INT4)} - SCAN(8) on default.part - => target list: default.part.p_partkey (INT4) - => out schema: {(1) default.part.p_partkey (INT4)} - => in schema: {(9) default.part.p_brand (TEXT), default.part.p_comment (TEXT), default.part.p_container (TEXT), default.part.p_mfgr (TEXT), default.part.p_name (TEXT), default.part.p_partkey (INT4), default.part.p_retailprice (FLOAT8), default.part.p_size (INT4), default.part.p_type (TEXT)} - SCAN(7) on default.lineitem as d - => target list: default.d.l_orderkey (INT4), default.d.l_partkey (INT4), default.d.l_quantity (FLOAT8) - => out schema: {(3) default.d.l_orderkey (INT4), default.d.l_partkey (INT4), default.d.l_quantity (FLOAT8)} - => in schema: {(16) default.d.l_comment (TEXT), default.d.l_commitdate (TEXT), default.d.l_discount (FLOAT8), default.d.l_extendedprice (FLOAT8), default.d.l_linenumber (INT4), default.d.l_linestatus (TEXT), default.d.l_orderkey (INT4), default.d.l_partkey (INT4), default.d.l_quantity (FLOAT8), default.d.l_receiptdate (TEXT), default.d.l_returnflag (TEXT), default.d.l_shipdate (TEXT), default.d.l_shipinstruct (TEXT), default.d.l_shipmode (TEXT), default.d.l_suppkey (INT4), default.d.l_tax (FLOAT8)} - -======================================================= -Block Id: eb_0000000000000_0000_000008 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 8 => 9 (type=HASH_SHUFFLE, key=default.c.o_orderkey (INT4), num=32) - -SCAN(5) on default.orders as c - => filter: default.c.o_orderkey (INT4) > 0 - => target list: default.c.o_orderkey (INT4) - => out schema: {(1) default.c.o_orderkey (INT4)} - => in schema: {(9) default.c.o_clerk (TEXT), default.c.o_comment (TEXT), default.c.o_custkey (INT4), default.c.o_orderdate (TEXT), default.c.o_orderkey (INT4), default.c.o_orderpriority (TEXT), default.c.o_orderstatus (TEXT), default.c.o_shippriority (INT4), default.c.o_totalprice (FLOAT8)} - -======================================================= -Block Id: eb_0000000000000_0000_000009 [INTERMEDIATE] -======================================================= - -[Incoming] -[q_0000000000000_0000] 7 => 9 (type=HASH_SHUFFLE, key=default.b.l_orderkey (INT4), num=32) -[q_0000000000000_0000] 8 => 9 (type=HASH_SHUFFLE, key=default.c.o_orderkey (INT4), num=32) - -[Outgoing] -[q_0000000000000_0000] 9 => 10 (type=HASH_SHUFFLE, key=default.c.o_orderkey (INT4), num=32) - -JOIN(22)(INNER) - => Join Cond: default.c.o_orderkey (INT4) = default.b.l_orderkey (INT4) - => target list: default.b.l_quantity (FLOAT8), default.c.o_orderkey (INT4) - => out schema: {(2) default.b.l_quantity (FLOAT8), default.c.o_orderkey (INT4)} - => in schema: {(3) default.b.l_orderkey (INT4), default.b.l_quantity (FLOAT8), default.c.o_orderkey (INT4)} - SCAN(28) on eb_0000000000000_0000_000008 - => out schema: {(1) default.c.o_orderkey (INT4)} - => in schema: {(1) default.c.o_orderkey (INT4)} - SCAN(27) on eb_0000000000000_0000_000007 - => out schema: {(2) default.b.l_quantity (FLOAT8), default.b.l_orderkey (INT4)} - => in schema: {(2) default.b.l_quantity (FLOAT8), default.b.l_orderkey (INT4)} - ======================================================= Block Id: eb_0000000000000_0000_000010 [INTERMEDIATE] ======================================================= [Incoming] [q_0000000000000_0000] 4 => 10 (type=HASH_SHUFFLE, key=default.f.l_orderkey (INT4), num=32) -[q_0000000000000_0000] 9 => 10 (type=HASH_SHUFFLE, key=default.c.o_orderkey (INT4), num=32) [Outgoing] [q_0000000000000_0000] 10 => 11 (type=HASH_SHUFFLE, key=, num=1) -GROUP_BY(31)() +[Enforcers] + 0: type=Broadcast, tables=default.part + 1: type=Broadcast, tables=default.c + 2: type=Broadcast, tables=default.d + +GROUP_BY(35)() => exprs: (sum(default.b.l_quantity (FLOAT8))) => target list: ?sum_12 (FLOAT8) => out schema:{(1) ?sum_12 (FLOAT8)} @@ -248,10 +177,38 @@ GROUP_BY(31)() => target list: default.b.l_quantity (FLOAT8), default.f.avg_quantity (FLOAT8) => out schema: {(2) default.b.l_quantity (FLOAT8), default.f.avg_quantity (FLOAT8)} => in schema: {(4) default.b.l_quantity (FLOAT8), default.c.o_orderkey (INT4), default.f.avg_quantity (FLOAT8), default.f.l_orderkey (INT4)} - SCAN(30) on eb_0000000000000_0000_000009 + JOIN(22)(INNER) + => Join Cond: default.c.o_orderkey (INT4) = default.b.l_orderkey (INT4) + => target list: default.b.l_quantity (FLOAT8), default.c.o_orderkey (INT4) => out schema: {(2) default.b.l_quantity (FLOAT8), default.c.o_orderkey (INT4)} - => in schema: {(2) default.b.l_quantity (FLOAT8), default.c.o_orderkey (INT4)} - SCAN(29) on eb_0000000000000_0000_000004 + => in schema: {(3) default.b.l_orderkey (INT4), default.b.l_quantity (FLOAT8), default.c.o_orderkey (INT4)} + SCAN(5) on default.orders as c + => filter: default.c.o_orderkey (INT4) > 0 + => target list: default.c.o_orderkey (INT4) + => out schema: {(1) default.c.o_orderkey (INT4)} + => in schema: {(9) default.c.o_clerk (TEXT), default.c.o_comment (TEXT), default.c.o_custkey (INT4), default.c.o_orderdate (TEXT), default.c.o_orderkey (INT4), default.c.o_orderpriority (TEXT), default.c.o_orderstatus (TEXT), default.c.o_shippriority (INT4), default.c.o_totalprice (FLOAT8)} + TABLE_SUBQUERY(4) as default.b + => Targets: default.b.l_quantity (FLOAT8), default.b.l_orderkey (INT4) + => out schema: {(2) default.b.l_quantity (FLOAT8), default.b.l_orderkey (INT4)} + => in schema: {(2) default.b.l_orderkey (INT4), default.b.l_quantity (FLOAT8)} + PROJECTION(10) + => Targets: default.d.l_orderkey (INT4), default.d.l_quantity (FLOAT8) + => out schema: {(2) default.d.l_orderkey (INT4), default.d.l_quantity (FLOAT8)} + => in schema: {(2) default.d.l_orderkey (INT4), default.d.l_quantity (FLOAT8)} + JOIN(20)(INNER) + => Join Cond: default.d.l_partkey (INT4) = default.part.p_partkey (INT4) + => target list: default.d.l_orderkey (INT4), default.d.l_quantity (FLOAT8) + => out schema: {(2) default.d.l_orderkey (INT4), default.d.l_quantity (FLOAT8)} + => in schema: {(4) default.d.l_orderkey (INT4), default.d.l_partkey (INT4), default.d.l_quantity (FLOAT8), default.part.p_partkey (INT4)} + SCAN(8) on default.part + => target list: default.part.p_partkey (INT4) + => out schema: {(1) default.part.p_partkey (INT4)} + => in schema: {(9) default.part.p_brand (TEXT), default.part.p_comment (TEXT), default.part.p_container (TEXT), default.part.p_mfgr (TEXT), default.part.p_name (TEXT), default.part.p_partkey (INT4), default.part.p_retailprice (FLOAT8), default.part.p_size (INT4), default.part.p_type (TEXT)} + SCAN(7) on default.lineitem as d + => target list: default.d.l_orderkey (INT4), default.d.l_partkey (INT4), default.d.l_quantity (FLOAT8) + => out schema: {(3) default.d.l_orderkey (INT4), default.d.l_partkey (INT4), default.d.l_quantity (FLOAT8)} + => in schema: {(16) default.d.l_comment (TEXT), default.d.l_commitdate (TEXT), default.d.l_discount (FLOAT8), default.d.l_extendedprice (FLOAT8), default.d.l_linenumber (INT4), default.d.l_linestatus (TEXT), default.d.l_orderkey (INT4), default.d.l_partkey (INT4), default.d.l_quantity (FLOAT8), default.d.l_receiptdate (TEXT), default.d.l_returnflag (TEXT), default.d.l_shipdate (TEXT), default.d.l_shipinstruct (TEXT), default.d.l_shipmode (TEXT), default.d.l_suppkey (INT4), default.d.l_tax (FLOAT8)} + SCAN(33) on eb_0000000000000_0000_000004 => out schema: {(2) default.f.avg_quantity (FLOAT8), default.f.l_orderkey (INT4)} => in schema: {(2) default.f.avg_quantity (FLOAT8), default.f.l_orderkey (INT4)} @@ -267,7 +224,7 @@ GROUP_BY(18)() => target list: ?sum (FLOAT8) => out schema:{(1) ?sum (FLOAT8)} => in schema:{(1) ?sum_12 (FLOAT8)} - SCAN(32) on eb_0000000000000_0000_000010 + SCAN(36) on eb_0000000000000_0000_000010 => out schema: {(1) ?sum_12 (FLOAT8)} => in schema: {(1) ?sum_12 (FLOAT8)} diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition5.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition5.Hash.plan index 2325ed7ebd..406e704cae 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition5.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition5.Hash.plan @@ -27,61 +27,25 @@ Execution Block Graph (TERMINAL - eb_0000000000000_0000_000005) |-eb_0000000000000_0000_000005 |-eb_0000000000000_0000_000004 |-eb_0000000000000_0000_000003 - |-eb_0000000000000_0000_000002 - |-eb_0000000000000_0000_000001 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000001 -2: eb_0000000000000_0000_000002 -3: eb_0000000000000_0000_000003 -4: eb_0000000000000_0000_000004 -5: eb_0000000000000_0000_000005 +1: eb_0000000000000_0000_000003 +2: eb_0000000000000_0000_000004 +3: eb_0000000000000_0000_000005 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000001 [LEAF] +Block Id: eb_0000000000000_0000_000003 [LEAF] ======================================================= -[Outgoing] -[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=name2 (TEXT), num=32) - -TABLE_SUBQUERY(3) as default.n2 - => Targets: substr(default.n2.n_name (TEXT),1,4) as name2 - => out schema: {(1) name2 (TEXT)} - => in schema: {(4) default.n2.n_comment (TEXT), default.n2.n_name (TEXT), default.n2.n_nationkey (INT4), default.n2.n_regionkey (INT4)} - PROJECTION(2) - => Targets: default.nation.n_nationkey (INT4), default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.nation.n_comment (TEXT) - => out schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} - => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} - SCAN(1) on default.nation - => target list: default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4) - => out schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} - => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000002 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=name1 (TEXT), num=32) - -SCAN(0) on default.nation as n1 - => target list: default.n1.n_nationkey (INT4), substr(default.n1.n_name (TEXT),1,4) as name1 - => out schema: {(2) default.n1.n_nationkey (INT4), name1 (TEXT)} - => in schema: {(4) default.n1.n_comment (TEXT), default.n1.n_name (TEXT), default.n1.n_nationkey (INT4), default.n1.n_regionkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000003 [INTERMEDIATE] -======================================================= - -[Incoming] -[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=name2 (TEXT), num=32) -[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=name1 (TEXT), num=32) - [Outgoing] [q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.n1.n_nationkey (INT4), num=32) +[Enforcers] + 0: type=Broadcast, tables=default.nation + 1: type=Broadcast, tables=default.n1 + SORT(12) => Sort Keys: default.n1.n_nationkey (INT4) (asc) JOIN(8)(INNER) @@ -89,12 +53,22 @@ SORT(12) => target list: default.n1.n_nationkey (INT4), name1 (TEXT), name2 (TEXT) => out schema: {(3) default.n1.n_nationkey (INT4), name1 (TEXT), name2 (TEXT)} => in schema: {(3) default.n1.n_nationkey (INT4), name1 (TEXT), name2 (TEXT)} - SCAN(11) on eb_0000000000000_0000_000002 + SCAN(0) on default.nation as n1 + => target list: default.n1.n_nationkey (INT4), substr(default.n1.n_name (TEXT),1,4) as name1 => out schema: {(2) default.n1.n_nationkey (INT4), name1 (TEXT)} - => in schema: {(2) default.n1.n_nationkey (INT4), name1 (TEXT)} - SCAN(10) on eb_0000000000000_0000_000001 + => in schema: {(4) default.n1.n_comment (TEXT), default.n1.n_name (TEXT), default.n1.n_nationkey (INT4), default.n1.n_regionkey (INT4)} + TABLE_SUBQUERY(3) as default.n2 + => Targets: substr(default.n2.n_name (TEXT),1,4) as name2 => out schema: {(1) name2 (TEXT)} - => in schema: {(1) name2 (TEXT)} + => in schema: {(4) default.n2.n_comment (TEXT), default.n2.n_name (TEXT), default.n2.n_nationkey (INT4), default.n2.n_regionkey (INT4)} + PROJECTION(2) + => Targets: default.nation.n_nationkey (INT4), default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.nation.n_comment (TEXT) + => out schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + SCAN(1) on default.nation + => target list: default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4) + => out schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} ======================================================= Block Id: eb_0000000000000_0000_000004 [ROOT] diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition5.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition5.Sort.plan index 2325ed7ebd..406e704cae 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition5.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition5.Sort.plan @@ -27,61 +27,25 @@ Execution Block Graph (TERMINAL - eb_0000000000000_0000_000005) |-eb_0000000000000_0000_000005 |-eb_0000000000000_0000_000004 |-eb_0000000000000_0000_000003 - |-eb_0000000000000_0000_000002 - |-eb_0000000000000_0000_000001 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000001 -2: eb_0000000000000_0000_000002 -3: eb_0000000000000_0000_000003 -4: eb_0000000000000_0000_000004 -5: eb_0000000000000_0000_000005 +1: eb_0000000000000_0000_000003 +2: eb_0000000000000_0000_000004 +3: eb_0000000000000_0000_000005 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000001 [LEAF] +Block Id: eb_0000000000000_0000_000003 [LEAF] ======================================================= -[Outgoing] -[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=name2 (TEXT), num=32) - -TABLE_SUBQUERY(3) as default.n2 - => Targets: substr(default.n2.n_name (TEXT),1,4) as name2 - => out schema: {(1) name2 (TEXT)} - => in schema: {(4) default.n2.n_comment (TEXT), default.n2.n_name (TEXT), default.n2.n_nationkey (INT4), default.n2.n_regionkey (INT4)} - PROJECTION(2) - => Targets: default.nation.n_nationkey (INT4), default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.nation.n_comment (TEXT) - => out schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} - => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} - SCAN(1) on default.nation - => target list: default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4) - => out schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} - => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000002 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=name1 (TEXT), num=32) - -SCAN(0) on default.nation as n1 - => target list: default.n1.n_nationkey (INT4), substr(default.n1.n_name (TEXT),1,4) as name1 - => out schema: {(2) default.n1.n_nationkey (INT4), name1 (TEXT)} - => in schema: {(4) default.n1.n_comment (TEXT), default.n1.n_name (TEXT), default.n1.n_nationkey (INT4), default.n1.n_regionkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000003 [INTERMEDIATE] -======================================================= - -[Incoming] -[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=name2 (TEXT), num=32) -[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=name1 (TEXT), num=32) - [Outgoing] [q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.n1.n_nationkey (INT4), num=32) +[Enforcers] + 0: type=Broadcast, tables=default.nation + 1: type=Broadcast, tables=default.n1 + SORT(12) => Sort Keys: default.n1.n_nationkey (INT4) (asc) JOIN(8)(INNER) @@ -89,12 +53,22 @@ SORT(12) => target list: default.n1.n_nationkey (INT4), name1 (TEXT), name2 (TEXT) => out schema: {(3) default.n1.n_nationkey (INT4), name1 (TEXT), name2 (TEXT)} => in schema: {(3) default.n1.n_nationkey (INT4), name1 (TEXT), name2 (TEXT)} - SCAN(11) on eb_0000000000000_0000_000002 + SCAN(0) on default.nation as n1 + => target list: default.n1.n_nationkey (INT4), substr(default.n1.n_name (TEXT),1,4) as name1 => out schema: {(2) default.n1.n_nationkey (INT4), name1 (TEXT)} - => in schema: {(2) default.n1.n_nationkey (INT4), name1 (TEXT)} - SCAN(10) on eb_0000000000000_0000_000001 + => in schema: {(4) default.n1.n_comment (TEXT), default.n1.n_name (TEXT), default.n1.n_nationkey (INT4), default.n1.n_regionkey (INT4)} + TABLE_SUBQUERY(3) as default.n2 + => Targets: substr(default.n2.n_name (TEXT),1,4) as name2 => out schema: {(1) name2 (TEXT)} - => in schema: {(1) name2 (TEXT)} + => in schema: {(4) default.n2.n_comment (TEXT), default.n2.n_name (TEXT), default.n2.n_nationkey (INT4), default.n2.n_regionkey (INT4)} + PROJECTION(2) + => Targets: default.nation.n_nationkey (INT4), default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.nation.n_comment (TEXT) + => out schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + SCAN(1) on default.nation + => target list: default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4) + => out schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} ======================================================= Block Id: eb_0000000000000_0000_000004 [ROOT] diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition6.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition6.Hash.plan index 15a4a6b051..2a6177e9f4 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition6.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition6.Hash.plan @@ -40,89 +40,54 @@ Execution Block Graph (TERMINAL - eb_0000000000000_0000_000006) |-eb_0000000000000_0000_000006 |-eb_0000000000000_0000_000005 |-eb_0000000000000_0000_000003 - |-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000002 - |-eb_0000000000000_0000_000001 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000001 -2: eb_0000000000000_0000_000002 -3: eb_0000000000000_0000_000004 -4: eb_0000000000000_0000_000003 -5: eb_0000000000000_0000_000005 -6: eb_0000000000000_0000_000006 +1: eb_0000000000000_0000_000003 +2: eb_0000000000000_0000_000005 +3: eb_0000000000000_0000_000006 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000001 [LEAF] +Block Id: eb_0000000000000_0000_000003 [LEAF] ======================================================= -[Outgoing] -[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=name2 (TEXT), num=32) - -TABLE_SUBQUERY(13) as default.n2 - => Targets: substr(default.n2.n_name (TEXT),1,4) as name2 - => out schema: {(1) name2 (TEXT)} - => in schema: {(4) default.n2.n_comment (TEXT), default.n2.n_name (TEXT), default.n2.n_nationkey (INT4), default.n2.n_regionkey (INT4)} - SCAN(1) on default.nation - => target list: default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4) - => out schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} - => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000002 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=name2 (TEXT), num=32) - -TABLE_SUBQUERY(14) as default.n2 - => Targets: substr(default.n2.n_name (TEXT),1,4) as name2 - => out schema: {(1) name2 (TEXT)} - => in schema: {(4) default.n2.n_comment (TEXT), default.n2.n_name (TEXT), default.n2.n_nationkey (INT4), default.n2.n_regionkey (INT4)} - SCAN(3) on default.nation - => target list: default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4) - => out schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} - => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000004 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 4 => 3 (type=HASH_SHUFFLE, key=name1 (TEXT), num=32) - -SCAN(0) on default.nation as n1 - => target list: default.n1.n_nationkey (INT4), substr(default.n1.n_name (TEXT),1,4) as name1 - => out schema: {(2) default.n1.n_nationkey (INT4), name1 (TEXT)} - => in schema: {(4) default.n1.n_comment (TEXT), default.n1.n_name (TEXT), default.n1.n_nationkey (INT4), default.n1.n_regionkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000003 [INTERMEDIATE] -======================================================= - -[Incoming] -[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=name2 (TEXT), num=32), union delegated scan: eb_0000000000000_0000_000001 -[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=name2 (TEXT), num=32), union delegated scan: eb_0000000000000_0000_000001 -[q_0000000000000_0000] 4 => 3 (type=HASH_SHUFFLE, key=name1 (TEXT), num=32) - [Outgoing] [q_0000000000000_0000] 3 => 5 (type=RANGE_SHUFFLE, key=default.n1.n_nationkey (INT4), num=32) +[Enforcers] + 0: type=Broadcast, tables=default.nation + 1: type=Broadcast, tables=default.nation + 2: type=Broadcast, tables=default.n1 + SORT(18) => Sort Keys: default.n1.n_nationkey (INT4) (asc) JOIN(11)(INNER) => Join Cond: name1 (TEXT) = name2 (TEXT) => target list: default.n1.n_nationkey (INT4), name1 (TEXT), name2 (TEXT) => out schema: {(3) default.n1.n_nationkey (INT4), name1 (TEXT), name2 (TEXT)} - => in schema: {(3) default.n1.n_nationkey (INT4), name1 (TEXT), name2 (TEXT)} - SCAN(17) on eb_0000000000000_0000_000004 + => in schema: {(3) default.n2.name2 (TEXT), default.n1.n_nationkey (INT4), name1 (TEXT)} + SCAN(0) on default.nation as n1 + => target list: default.n1.n_nationkey (INT4), substr(default.n1.n_name (TEXT),1,4) as name1 => out schema: {(2) default.n1.n_nationkey (INT4), name1 (TEXT)} - => in schema: {(2) default.n1.n_nationkey (INT4), name1 (TEXT)} - SCAN(15) on eb_0000000000000_0000_000001 - => out schema: {(1) default.n2.name2 (TEXT)} - => in schema: {(1) default.n2.name2 (TEXT)} + => in schema: {(4) default.n1.n_comment (TEXT), default.n1.n_name (TEXT), default.n1.n_nationkey (INT4), default.n1.n_regionkey (INT4)} + UNION(21) + TABLE_SUBQUERY(14) as default.n2 + => Targets: substr(default.n2.n_name (TEXT),1,4) as name2 + => out schema: {(1) name2 (TEXT)} + => in schema: {(4) default.n2.n_comment (TEXT), default.n2.n_name (TEXT), default.n2.n_nationkey (INT4), default.n2.n_regionkey (INT4)} + SCAN(3) on default.nation + => target list: default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4) + => out schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + TABLE_SUBQUERY(13) as default.n2 + => Targets: substr(default.n2.n_name (TEXT),1,4) as name2 + => out schema: {(1) name2 (TEXT)} + => in schema: {(4) default.n2.n_comment (TEXT), default.n2.n_name (TEXT), default.n2.n_nationkey (INT4), default.n2.n_regionkey (INT4)} + SCAN(1) on default.nation + => target list: default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4) + => out schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} ======================================================= Block Id: eb_0000000000000_0000_000005 [ROOT] diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition6.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition6.Sort.plan index 15a4a6b051..2a6177e9f4 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition6.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition6.Sort.plan @@ -40,89 +40,54 @@ Execution Block Graph (TERMINAL - eb_0000000000000_0000_000006) |-eb_0000000000000_0000_000006 |-eb_0000000000000_0000_000005 |-eb_0000000000000_0000_000003 - |-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000002 - |-eb_0000000000000_0000_000001 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000001 -2: eb_0000000000000_0000_000002 -3: eb_0000000000000_0000_000004 -4: eb_0000000000000_0000_000003 -5: eb_0000000000000_0000_000005 -6: eb_0000000000000_0000_000006 +1: eb_0000000000000_0000_000003 +2: eb_0000000000000_0000_000005 +3: eb_0000000000000_0000_000006 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000001 [LEAF] +Block Id: eb_0000000000000_0000_000003 [LEAF] ======================================================= -[Outgoing] -[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=name2 (TEXT), num=32) - -TABLE_SUBQUERY(13) as default.n2 - => Targets: substr(default.n2.n_name (TEXT),1,4) as name2 - => out schema: {(1) name2 (TEXT)} - => in schema: {(4) default.n2.n_comment (TEXT), default.n2.n_name (TEXT), default.n2.n_nationkey (INT4), default.n2.n_regionkey (INT4)} - SCAN(1) on default.nation - => target list: default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4) - => out schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} - => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000002 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=name2 (TEXT), num=32) - -TABLE_SUBQUERY(14) as default.n2 - => Targets: substr(default.n2.n_name (TEXT),1,4) as name2 - => out schema: {(1) name2 (TEXT)} - => in schema: {(4) default.n2.n_comment (TEXT), default.n2.n_name (TEXT), default.n2.n_nationkey (INT4), default.n2.n_regionkey (INT4)} - SCAN(3) on default.nation - => target list: default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4) - => out schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} - => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000004 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 4 => 3 (type=HASH_SHUFFLE, key=name1 (TEXT), num=32) - -SCAN(0) on default.nation as n1 - => target list: default.n1.n_nationkey (INT4), substr(default.n1.n_name (TEXT),1,4) as name1 - => out schema: {(2) default.n1.n_nationkey (INT4), name1 (TEXT)} - => in schema: {(4) default.n1.n_comment (TEXT), default.n1.n_name (TEXT), default.n1.n_nationkey (INT4), default.n1.n_regionkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000003 [INTERMEDIATE] -======================================================= - -[Incoming] -[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=name2 (TEXT), num=32), union delegated scan: eb_0000000000000_0000_000001 -[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=name2 (TEXT), num=32), union delegated scan: eb_0000000000000_0000_000001 -[q_0000000000000_0000] 4 => 3 (type=HASH_SHUFFLE, key=name1 (TEXT), num=32) - [Outgoing] [q_0000000000000_0000] 3 => 5 (type=RANGE_SHUFFLE, key=default.n1.n_nationkey (INT4), num=32) +[Enforcers] + 0: type=Broadcast, tables=default.nation + 1: type=Broadcast, tables=default.nation + 2: type=Broadcast, tables=default.n1 + SORT(18) => Sort Keys: default.n1.n_nationkey (INT4) (asc) JOIN(11)(INNER) => Join Cond: name1 (TEXT) = name2 (TEXT) => target list: default.n1.n_nationkey (INT4), name1 (TEXT), name2 (TEXT) => out schema: {(3) default.n1.n_nationkey (INT4), name1 (TEXT), name2 (TEXT)} - => in schema: {(3) default.n1.n_nationkey (INT4), name1 (TEXT), name2 (TEXT)} - SCAN(17) on eb_0000000000000_0000_000004 + => in schema: {(3) default.n2.name2 (TEXT), default.n1.n_nationkey (INT4), name1 (TEXT)} + SCAN(0) on default.nation as n1 + => target list: default.n1.n_nationkey (INT4), substr(default.n1.n_name (TEXT),1,4) as name1 => out schema: {(2) default.n1.n_nationkey (INT4), name1 (TEXT)} - => in schema: {(2) default.n1.n_nationkey (INT4), name1 (TEXT)} - SCAN(15) on eb_0000000000000_0000_000001 - => out schema: {(1) default.n2.name2 (TEXT)} - => in schema: {(1) default.n2.name2 (TEXT)} + => in schema: {(4) default.n1.n_comment (TEXT), default.n1.n_name (TEXT), default.n1.n_nationkey (INT4), default.n1.n_regionkey (INT4)} + UNION(21) + TABLE_SUBQUERY(14) as default.n2 + => Targets: substr(default.n2.n_name (TEXT),1,4) as name2 + => out schema: {(1) name2 (TEXT)} + => in schema: {(4) default.n2.n_comment (TEXT), default.n2.n_name (TEXT), default.n2.n_nationkey (INT4), default.n2.n_regionkey (INT4)} + SCAN(3) on default.nation + => target list: default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4) + => out schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + TABLE_SUBQUERY(13) as default.n2 + => Targets: substr(default.n2.n_name (TEXT),1,4) as name2 + => out schema: {(1) name2 (TEXT)} + => in schema: {(4) default.n2.n_comment (TEXT), default.n2.n_name (TEXT), default.n2.n_nationkey (INT4), default.n2.n_regionkey (INT4)} + SCAN(1) on default.nation + => target list: default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4) + => out schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} ======================================================= Block Id: eb_0000000000000_0000_000005 [ROOT] diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition7.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition7.Hash.plan index cd716a4a32..e093471766 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition7.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition7.Hash.plan @@ -40,89 +40,54 @@ Execution Block Graph (TERMINAL - eb_0000000000000_0000_000006) |-eb_0000000000000_0000_000006 |-eb_0000000000000_0000_000005 |-eb_0000000000000_0000_000003 - |-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000002 - |-eb_0000000000000_0000_000001 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000001 -2: eb_0000000000000_0000_000002 -3: eb_0000000000000_0000_000004 -4: eb_0000000000000_0000_000003 -5: eb_0000000000000_0000_000005 -6: eb_0000000000000_0000_000006 +1: eb_0000000000000_0000_000003 +2: eb_0000000000000_0000_000005 +3: eb_0000000000000_0000_000006 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000001 [LEAF] +Block Id: eb_0000000000000_0000_000003 [LEAF] ======================================================= -[Outgoing] -[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=?substr_2 (TEXT), num=32) - -TABLE_SUBQUERY(13) as default.n2 - => Targets: default.n2.n_name (TEXT), substr(default.n2.n_name (TEXT),1,4) as ?substr_2 - => out schema: {(2) default.n2.n_name (TEXT), ?substr_2 (TEXT)} - => in schema: {(4) default.n2.n_comment (TEXT), default.n2.n_name (TEXT), default.n2.n_nationkey (INT4), default.n2.n_regionkey (INT4)} - SCAN(1) on default.nation - => target list: default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4) - => out schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} - => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000002 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=?substr_2 (TEXT), num=32) - -TABLE_SUBQUERY(14) as default.n2 - => Targets: default.n2.n_name (TEXT), substr(default.n2.n_name (TEXT),1,4) as ?substr_2 - => out schema: {(2) default.n2.n_name (TEXT), ?substr_2 (TEXT)} - => in schema: {(4) default.n2.n_comment (TEXT), default.n2.n_name (TEXT), default.n2.n_nationkey (INT4), default.n2.n_regionkey (INT4)} - SCAN(3) on default.nation - => target list: default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4) - => out schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} - => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000004 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 4 => 3 (type=HASH_SHUFFLE, key=?substr_1 (TEXT), num=32) - -SCAN(0) on default.nation as n1 - => target list: default.n1.n_name (TEXT), default.n1.n_nationkey (INT4), substr(default.n1.n_name (TEXT),1,4) as ?substr_1 - => out schema: {(3) ?substr_1 (TEXT), default.n1.n_name (TEXT), default.n1.n_nationkey (INT4)} - => in schema: {(4) default.n1.n_comment (TEXT), default.n1.n_name (TEXT), default.n1.n_nationkey (INT4), default.n1.n_regionkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000003 [INTERMEDIATE] -======================================================= - -[Incoming] -[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=?substr_2 (TEXT), num=32), union delegated scan: eb_0000000000000_0000_000001 -[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=?substr_2 (TEXT), num=32), union delegated scan: eb_0000000000000_0000_000001 -[q_0000000000000_0000] 4 => 3 (type=HASH_SHUFFLE, key=?substr_1 (TEXT), num=32) - [Outgoing] [q_0000000000000_0000] 3 => 5 (type=RANGE_SHUFFLE, key=default.n1.n_nationkey (INT4), default.n2.n_name (TEXT), num=32) +[Enforcers] + 0: type=Broadcast, tables=default.nation + 1: type=Broadcast, tables=default.nation + 2: type=Broadcast, tables=default.n1 + SORT(18) => Sort Keys: default.n1.n_nationkey (INT4) (asc),default.n2.n_name (TEXT) (asc) JOIN(11)(INNER) => Join Cond: ?substr_1 (TEXT) = ?substr_2 (TEXT) => target list: default.n1.n_name (TEXT), default.n1.n_nationkey (INT4), default.n2.n_name (TEXT) => out schema: {(3) default.n1.n_name (TEXT), default.n1.n_nationkey (INT4), default.n2.n_name (TEXT)} - => in schema: {(5) ?substr_1 (TEXT), ?substr_2 (TEXT), default.n1.n_name (TEXT), default.n1.n_nationkey (INT4), default.n2.n_name (TEXT)} - SCAN(17) on eb_0000000000000_0000_000004 + => in schema: {(5) default.n2.n_name (TEXT), default.n2.?substr_2 (TEXT), ?substr_1 (TEXT), default.n1.n_name (TEXT), default.n1.n_nationkey (INT4)} + SCAN(0) on default.nation as n1 + => target list: default.n1.n_name (TEXT), default.n1.n_nationkey (INT4), substr(default.n1.n_name (TEXT),1,4) as ?substr_1 => out schema: {(3) ?substr_1 (TEXT), default.n1.n_name (TEXT), default.n1.n_nationkey (INT4)} - => in schema: {(3) ?substr_1 (TEXT), default.n1.n_name (TEXT), default.n1.n_nationkey (INT4)} - SCAN(15) on eb_0000000000000_0000_000001 - => out schema: {(2) default.n2.n_name (TEXT), default.n2.?substr_2 (TEXT)} - => in schema: {(2) default.n2.n_name (TEXT), default.n2.?substr_2 (TEXT)} + => in schema: {(4) default.n1.n_comment (TEXT), default.n1.n_name (TEXT), default.n1.n_nationkey (INT4), default.n1.n_regionkey (INT4)} + UNION(21) + TABLE_SUBQUERY(14) as default.n2 + => Targets: default.n2.n_name (TEXT), substr(default.n2.n_name (TEXT),1,4) as ?substr_2 + => out schema: {(2) default.n2.n_name (TEXT), ?substr_2 (TEXT)} + => in schema: {(4) default.n2.n_comment (TEXT), default.n2.n_name (TEXT), default.n2.n_nationkey (INT4), default.n2.n_regionkey (INT4)} + SCAN(3) on default.nation + => target list: default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4) + => out schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + TABLE_SUBQUERY(13) as default.n2 + => Targets: default.n2.n_name (TEXT), substr(default.n2.n_name (TEXT),1,4) as ?substr_2 + => out schema: {(2) default.n2.n_name (TEXT), ?substr_2 (TEXT)} + => in schema: {(4) default.n2.n_comment (TEXT), default.n2.n_name (TEXT), default.n2.n_nationkey (INT4), default.n2.n_regionkey (INT4)} + SCAN(1) on default.nation + => target list: default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4) + => out schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} ======================================================= Block Id: eb_0000000000000_0000_000005 [ROOT] diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition7.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition7.Sort.plan index cd716a4a32..e093471766 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition7.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition7.Sort.plan @@ -40,89 +40,54 @@ Execution Block Graph (TERMINAL - eb_0000000000000_0000_000006) |-eb_0000000000000_0000_000006 |-eb_0000000000000_0000_000005 |-eb_0000000000000_0000_000003 - |-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000002 - |-eb_0000000000000_0000_000001 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000001 -2: eb_0000000000000_0000_000002 -3: eb_0000000000000_0000_000004 -4: eb_0000000000000_0000_000003 -5: eb_0000000000000_0000_000005 -6: eb_0000000000000_0000_000006 +1: eb_0000000000000_0000_000003 +2: eb_0000000000000_0000_000005 +3: eb_0000000000000_0000_000006 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000001 [LEAF] +Block Id: eb_0000000000000_0000_000003 [LEAF] ======================================================= -[Outgoing] -[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=?substr_2 (TEXT), num=32) - -TABLE_SUBQUERY(13) as default.n2 - => Targets: default.n2.n_name (TEXT), substr(default.n2.n_name (TEXT),1,4) as ?substr_2 - => out schema: {(2) default.n2.n_name (TEXT), ?substr_2 (TEXT)} - => in schema: {(4) default.n2.n_comment (TEXT), default.n2.n_name (TEXT), default.n2.n_nationkey (INT4), default.n2.n_regionkey (INT4)} - SCAN(1) on default.nation - => target list: default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4) - => out schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} - => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000002 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=?substr_2 (TEXT), num=32) - -TABLE_SUBQUERY(14) as default.n2 - => Targets: default.n2.n_name (TEXT), substr(default.n2.n_name (TEXT),1,4) as ?substr_2 - => out schema: {(2) default.n2.n_name (TEXT), ?substr_2 (TEXT)} - => in schema: {(4) default.n2.n_comment (TEXT), default.n2.n_name (TEXT), default.n2.n_nationkey (INT4), default.n2.n_regionkey (INT4)} - SCAN(3) on default.nation - => target list: default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4) - => out schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} - => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000004 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 4 => 3 (type=HASH_SHUFFLE, key=?substr_1 (TEXT), num=32) - -SCAN(0) on default.nation as n1 - => target list: default.n1.n_name (TEXT), default.n1.n_nationkey (INT4), substr(default.n1.n_name (TEXT),1,4) as ?substr_1 - => out schema: {(3) ?substr_1 (TEXT), default.n1.n_name (TEXT), default.n1.n_nationkey (INT4)} - => in schema: {(4) default.n1.n_comment (TEXT), default.n1.n_name (TEXT), default.n1.n_nationkey (INT4), default.n1.n_regionkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000003 [INTERMEDIATE] -======================================================= - -[Incoming] -[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=?substr_2 (TEXT), num=32), union delegated scan: eb_0000000000000_0000_000001 -[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=?substr_2 (TEXT), num=32), union delegated scan: eb_0000000000000_0000_000001 -[q_0000000000000_0000] 4 => 3 (type=HASH_SHUFFLE, key=?substr_1 (TEXT), num=32) - [Outgoing] [q_0000000000000_0000] 3 => 5 (type=RANGE_SHUFFLE, key=default.n1.n_nationkey (INT4), default.n2.n_name (TEXT), num=32) +[Enforcers] + 0: type=Broadcast, tables=default.nation + 1: type=Broadcast, tables=default.nation + 2: type=Broadcast, tables=default.n1 + SORT(18) => Sort Keys: default.n1.n_nationkey (INT4) (asc),default.n2.n_name (TEXT) (asc) JOIN(11)(INNER) => Join Cond: ?substr_1 (TEXT) = ?substr_2 (TEXT) => target list: default.n1.n_name (TEXT), default.n1.n_nationkey (INT4), default.n2.n_name (TEXT) => out schema: {(3) default.n1.n_name (TEXT), default.n1.n_nationkey (INT4), default.n2.n_name (TEXT)} - => in schema: {(5) ?substr_1 (TEXT), ?substr_2 (TEXT), default.n1.n_name (TEXT), default.n1.n_nationkey (INT4), default.n2.n_name (TEXT)} - SCAN(17) on eb_0000000000000_0000_000004 + => in schema: {(5) default.n2.n_name (TEXT), default.n2.?substr_2 (TEXT), ?substr_1 (TEXT), default.n1.n_name (TEXT), default.n1.n_nationkey (INT4)} + SCAN(0) on default.nation as n1 + => target list: default.n1.n_name (TEXT), default.n1.n_nationkey (INT4), substr(default.n1.n_name (TEXT),1,4) as ?substr_1 => out schema: {(3) ?substr_1 (TEXT), default.n1.n_name (TEXT), default.n1.n_nationkey (INT4)} - => in schema: {(3) ?substr_1 (TEXT), default.n1.n_name (TEXT), default.n1.n_nationkey (INT4)} - SCAN(15) on eb_0000000000000_0000_000001 - => out schema: {(2) default.n2.n_name (TEXT), default.n2.?substr_2 (TEXT)} - => in schema: {(2) default.n2.n_name (TEXT), default.n2.?substr_2 (TEXT)} + => in schema: {(4) default.n1.n_comment (TEXT), default.n1.n_name (TEXT), default.n1.n_nationkey (INT4), default.n1.n_regionkey (INT4)} + UNION(21) + TABLE_SUBQUERY(14) as default.n2 + => Targets: default.n2.n_name (TEXT), substr(default.n2.n_name (TEXT),1,4) as ?substr_2 + => out schema: {(2) default.n2.n_name (TEXT), ?substr_2 (TEXT)} + => in schema: {(4) default.n2.n_comment (TEXT), default.n2.n_name (TEXT), default.n2.n_nationkey (INT4), default.n2.n_regionkey (INT4)} + SCAN(3) on default.nation + => target list: default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4) + => out schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + TABLE_SUBQUERY(13) as default.n2 + => Targets: default.n2.n_name (TEXT), substr(default.n2.n_name (TEXT),1,4) as ?substr_2 + => out schema: {(2) default.n2.n_name (TEXT), ?substr_2 (TEXT)} + => in schema: {(4) default.n2.n_comment (TEXT), default.n2.n_name (TEXT), default.n2.n_nationkey (INT4), default.n2.n_regionkey (INT4)} + SCAN(1) on default.nation + => target list: default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4) + => out schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} ======================================================= Block Id: eb_0000000000000_0000_000005 [ROOT] diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual2.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual2.Hash.plan index 9f790f3a3b..b647b53fc0 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual2.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual2.Hash.plan @@ -34,82 +34,53 @@ Execution Block Graph (TERMINAL - eb_0000000000000_0000_000006) ------------------------------------------------------------------------------- |-eb_0000000000000_0000_000006 |-eb_0000000000000_0000_000005 - |-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 -3: eb_0000000000000_0000_000005 -4: eb_0000000000000_0000_000006 +1: eb_0000000000000_0000_000005 +2: eb_0000000000000_0000_000006 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [LEAF] +Block Id: eb_0000000000000_0000_000005 [ROOT] ======================================================= -[Outgoing] -[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.t.n_nationkey (INT4), num=32) - [Enforcers] 0: type=Broadcast, tables=default.n 1: type=Broadcast, tables=default.r - -TABLE_SUBQUERY(4) as default.t - => Targets: default.t.n_nationkey (INT4), default.t.name (TEXT), default.t.n_regionkey (INT4), default.t.n_comment (TEXT) - => out schema: {(4) default.t.n_nationkey (INT4), default.t.name (TEXT), default.t.n_regionkey (INT4), default.t.n_comment (TEXT)} - => in schema: {(4) default.t.n_comment (TEXT), default.t.n_nationkey (INT4), default.t.n_regionkey (INT4), default.t.name (TEXT)} - PROJECTION(3) - => Targets: default.n.n_nationkey (INT4), name (TEXT), default.n.n_regionkey (INT4), default.n.n_comment (TEXT) - => out schema: {(4) default.n.n_comment (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), name (TEXT)} - => in schema: {(4) default.n.n_comment (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), name (TEXT)} - JOIN(10)(INNER) - => Join Cond: default.n.n_regionkey (INT4) = default.r.r_regionkey (INT4) - => target list: default.n.n_comment (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), name (TEXT) - => out schema: {(4) default.n.n_comment (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), name (TEXT)} - => in schema: {(5) default.n.n_comment (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4), name (TEXT)} - SCAN(1) on default.region as r - => target list: default.r.r_regionkey (INT4) - => out schema: {(1) default.r.r_regionkey (INT4)} - => in schema: {(3) default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4)} - SCAN(0) on default.nation as n - => filter: default.n.n_name (TEXT) = MOROCCO - => target list: default.n.n_comment (TEXT), default.n.n_name (TEXT) as name, default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) - => out schema: {(4) default.n.n_comment (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), name (TEXT)} - => in schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000004 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.s.s_nationkey (INT4), num=32) - -SCAN(5) on default.supplier as s - => target list: default.s.s_nationkey (INT4) - => out schema: {(1) default.s.s_nationkey (INT4)} - => in schema: {(7) default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT), default.s.s_suppkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000005 [ROOT] -======================================================= - -[Incoming] -[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.t.n_nationkey (INT4), num=32) -[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.s.s_nationkey (INT4), num=32) + 2: type=Broadcast, tables=default.s JOIN(11)(INNER) => Join Cond: default.s.s_nationkey (INT4) = default.t.n_nationkey (INT4) => target list: default.t.n_comment (TEXT), default.t.n_nationkey (INT4), default.t.n_regionkey (INT4), default.t.name (TEXT) => out schema: {(4) default.t.n_comment (TEXT), default.t.n_nationkey (INT4), default.t.n_regionkey (INT4), default.t.name (TEXT)} => in schema: {(5) default.s.s_nationkey (INT4), default.t.n_comment (TEXT), default.t.n_nationkey (INT4), default.t.n_regionkey (INT4), default.t.name (TEXT)} - SCAN(14) on eb_0000000000000_0000_000004 + SCAN(5) on default.supplier as s + => target list: default.s.s_nationkey (INT4) => out schema: {(1) default.s.s_nationkey (INT4)} - => in schema: {(1) default.s.s_nationkey (INT4)} - SCAN(13) on eb_0000000000000_0000_000003 + => in schema: {(7) default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT), default.s.s_suppkey (INT4)} + TABLE_SUBQUERY(4) as default.t + => Targets: default.t.n_nationkey (INT4), default.t.name (TEXT), default.t.n_regionkey (INT4), default.t.n_comment (TEXT) => out schema: {(4) default.t.n_nationkey (INT4), default.t.name (TEXT), default.t.n_regionkey (INT4), default.t.n_comment (TEXT)} - => in schema: {(4) default.t.n_nationkey (INT4), default.t.name (TEXT), default.t.n_regionkey (INT4), default.t.n_comment (TEXT)} + => in schema: {(4) default.t.n_comment (TEXT), default.t.n_nationkey (INT4), default.t.n_regionkey (INT4), default.t.name (TEXT)} + PROJECTION(3) + => Targets: default.n.n_nationkey (INT4), name (TEXT), default.n.n_regionkey (INT4), default.n.n_comment (TEXT) + => out schema: {(4) default.n.n_comment (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), name (TEXT)} + => in schema: {(4) default.n.n_comment (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), name (TEXT)} + JOIN(10)(INNER) + => Join Cond: default.n.n_regionkey (INT4) = default.r.r_regionkey (INT4) + => target list: default.n.n_comment (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), name (TEXT) + => out schema: {(4) default.n.n_comment (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), name (TEXT)} + => in schema: {(5) default.n.n_comment (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4), name (TEXT)} + SCAN(1) on default.region as r + => target list: default.r.r_regionkey (INT4) + => out schema: {(1) default.r.r_regionkey (INT4)} + => in schema: {(3) default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4)} + SCAN(0) on default.nation as n + => filter: default.n.n_name (TEXT) = MOROCCO + => target list: default.n.n_comment (TEXT), default.n.n_name (TEXT) as name, default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) + => out schema: {(4) default.n.n_comment (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), name (TEXT)} + => in schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} ======================================================= Block Id: eb_0000000000000_0000_000006 [TERMINAL] diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual2.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual2.Sort.plan index 9f790f3a3b..b647b53fc0 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual2.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual2.Sort.plan @@ -34,82 +34,53 @@ Execution Block Graph (TERMINAL - eb_0000000000000_0000_000006) ------------------------------------------------------------------------------- |-eb_0000000000000_0000_000006 |-eb_0000000000000_0000_000005 - |-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 -3: eb_0000000000000_0000_000005 -4: eb_0000000000000_0000_000006 +1: eb_0000000000000_0000_000005 +2: eb_0000000000000_0000_000006 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [LEAF] +Block Id: eb_0000000000000_0000_000005 [ROOT] ======================================================= -[Outgoing] -[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.t.n_nationkey (INT4), num=32) - [Enforcers] 0: type=Broadcast, tables=default.n 1: type=Broadcast, tables=default.r - -TABLE_SUBQUERY(4) as default.t - => Targets: default.t.n_nationkey (INT4), default.t.name (TEXT), default.t.n_regionkey (INT4), default.t.n_comment (TEXT) - => out schema: {(4) default.t.n_nationkey (INT4), default.t.name (TEXT), default.t.n_regionkey (INT4), default.t.n_comment (TEXT)} - => in schema: {(4) default.t.n_comment (TEXT), default.t.n_nationkey (INT4), default.t.n_regionkey (INT4), default.t.name (TEXT)} - PROJECTION(3) - => Targets: default.n.n_nationkey (INT4), name (TEXT), default.n.n_regionkey (INT4), default.n.n_comment (TEXT) - => out schema: {(4) default.n.n_comment (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), name (TEXT)} - => in schema: {(4) default.n.n_comment (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), name (TEXT)} - JOIN(10)(INNER) - => Join Cond: default.n.n_regionkey (INT4) = default.r.r_regionkey (INT4) - => target list: default.n.n_comment (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), name (TEXT) - => out schema: {(4) default.n.n_comment (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), name (TEXT)} - => in schema: {(5) default.n.n_comment (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4), name (TEXT)} - SCAN(1) on default.region as r - => target list: default.r.r_regionkey (INT4) - => out schema: {(1) default.r.r_regionkey (INT4)} - => in schema: {(3) default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4)} - SCAN(0) on default.nation as n - => filter: default.n.n_name (TEXT) = MOROCCO - => target list: default.n.n_comment (TEXT), default.n.n_name (TEXT) as name, default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) - => out schema: {(4) default.n.n_comment (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), name (TEXT)} - => in schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000004 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.s.s_nationkey (INT4), num=32) - -SCAN(5) on default.supplier as s - => target list: default.s.s_nationkey (INT4) - => out schema: {(1) default.s.s_nationkey (INT4)} - => in schema: {(7) default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT), default.s.s_suppkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000005 [ROOT] -======================================================= - -[Incoming] -[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.t.n_nationkey (INT4), num=32) -[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.s.s_nationkey (INT4), num=32) + 2: type=Broadcast, tables=default.s JOIN(11)(INNER) => Join Cond: default.s.s_nationkey (INT4) = default.t.n_nationkey (INT4) => target list: default.t.n_comment (TEXT), default.t.n_nationkey (INT4), default.t.n_regionkey (INT4), default.t.name (TEXT) => out schema: {(4) default.t.n_comment (TEXT), default.t.n_nationkey (INT4), default.t.n_regionkey (INT4), default.t.name (TEXT)} => in schema: {(5) default.s.s_nationkey (INT4), default.t.n_comment (TEXT), default.t.n_nationkey (INT4), default.t.n_regionkey (INT4), default.t.name (TEXT)} - SCAN(14) on eb_0000000000000_0000_000004 + SCAN(5) on default.supplier as s + => target list: default.s.s_nationkey (INT4) => out schema: {(1) default.s.s_nationkey (INT4)} - => in schema: {(1) default.s.s_nationkey (INT4)} - SCAN(13) on eb_0000000000000_0000_000003 + => in schema: {(7) default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT), default.s.s_suppkey (INT4)} + TABLE_SUBQUERY(4) as default.t + => Targets: default.t.n_nationkey (INT4), default.t.name (TEXT), default.t.n_regionkey (INT4), default.t.n_comment (TEXT) => out schema: {(4) default.t.n_nationkey (INT4), default.t.name (TEXT), default.t.n_regionkey (INT4), default.t.n_comment (TEXT)} - => in schema: {(4) default.t.n_nationkey (INT4), default.t.name (TEXT), default.t.n_regionkey (INT4), default.t.n_comment (TEXT)} + => in schema: {(4) default.t.n_comment (TEXT), default.t.n_nationkey (INT4), default.t.n_regionkey (INT4), default.t.name (TEXT)} + PROJECTION(3) + => Targets: default.n.n_nationkey (INT4), name (TEXT), default.n.n_regionkey (INT4), default.n.n_comment (TEXT) + => out schema: {(4) default.n.n_comment (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), name (TEXT)} + => in schema: {(4) default.n.n_comment (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), name (TEXT)} + JOIN(10)(INNER) + => Join Cond: default.n.n_regionkey (INT4) = default.r.r_regionkey (INT4) + => target list: default.n.n_comment (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), name (TEXT) + => out schema: {(4) default.n.n_comment (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), name (TEXT)} + => in schema: {(5) default.n.n_comment (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4), name (TEXT)} + SCAN(1) on default.region as r + => target list: default.r.r_regionkey (INT4) + => out schema: {(1) default.r.r_regionkey (INT4)} + => in schema: {(3) default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4)} + SCAN(0) on default.nation as n + => filter: default.n.n_name (TEXT) = MOROCCO + => target list: default.n.n_comment (TEXT), default.n.n_name (TEXT) as name, default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) + => out schema: {(4) default.n.n_comment (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), name (TEXT)} + => in schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} ======================================================= Block Id: eb_0000000000000_0000_000006 [TERMINAL] diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual3.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual3.Hash.plan index 88c814d275..4356e9c135 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual3.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual3.Hash.plan @@ -43,95 +43,63 @@ Execution Block Graph (TERMINAL - eb_0000000000000_0000_000008) ------------------------------------------------------------------------------- |-eb_0000000000000_0000_000008 |-eb_0000000000000_0000_000007 - |-eb_0000000000000_0000_000006 - |-eb_0000000000000_0000_000003 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000006 -3: eb_0000000000000_0000_000007 -4: eb_0000000000000_0000_000008 +1: eb_0000000000000_0000_000007 +2: eb_0000000000000_0000_000008 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [LEAF] +Block Id: eb_0000000000000_0000_000007 [ROOT] ======================================================= -[Outgoing] -[q_0000000000000_0000] 3 => 7 (type=HASH_SHUFFLE, key=default.t.n_nationkey (INT4), num=32) - [Enforcers] 0: type=Broadcast, tables=default.n 1: type=Broadcast, tables=default.r - -TABLE_SUBQUERY(4) as default.t - => Targets: default.t.n_nationkey (INT4), default.t.n_name (TEXT), default.t.n_regionkey (INT4), default.t.n_comment (TEXT) - => out schema: {(4) default.t.n_nationkey (INT4), default.t.n_name (TEXT), default.t.n_regionkey (INT4), default.t.n_comment (TEXT)} - => in schema: {(4) default.t.n_comment (TEXT), default.t.n_name (TEXT), default.t.n_nationkey (INT4), default.t.n_regionkey (INT4)} - PROJECTION(3) - => Targets: default.n.n_nationkey (INT4), default.n.n_name (TEXT), default.n.n_regionkey (INT4), default.n.n_comment (TEXT) - => out schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} - => in schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} - JOIN(12)(INNER) - => Join Cond: default.n.n_regionkey (INT4) = default.r.r_regionkey (INT4) - => target list: default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) - => out schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} - => in schema: {(5) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4)} - SCAN(1) on default.region as r - => target list: default.r.r_regionkey (INT4) - => out schema: {(1) default.r.r_regionkey (INT4)} - => in schema: {(3) default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4)} - SCAN(0) on default.nation as n - => filter: default.n.n_name (TEXT) IN (ARGENTINA, ETHIOPIA, MOROCCO) - => target list: default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) - => out schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} - => in schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000006 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 6 => 7 (type=HASH_SHUFFLE, key=default.s.s_nationkey (INT4), num=32) - -[Enforcers] - 0: type=Broadcast, tables=default.s - 1: type=Broadcast, tables=default.ps - -JOIN(13)(INNER) - => Join Cond: default.s.s_suppkey (INT4) = default.ps.ps_suppkey (INT4) - => target list: default.ps.ps_availqty (INT4), default.s.s_nationkey (INT4), default.s.s_suppkey (INT4) - => out schema: {(3) default.ps.ps_availqty (INT4), default.s.s_nationkey (INT4), default.s.s_suppkey (INT4)} - => in schema: {(4) default.ps.ps_availqty (INT4), default.ps.ps_suppkey (INT4), default.s.s_nationkey (INT4), default.s.s_suppkey (INT4)} - SCAN(7) on default.partsupp as ps - => target list: default.ps.ps_availqty (INT4), default.ps.ps_suppkey (INT4) - => out schema: {(2) default.ps.ps_availqty (INT4), default.ps.ps_suppkey (INT4)} - => in schema: {(5) default.ps.ps_availqty (INT4), default.ps.ps_comment (TEXT), default.ps.ps_partkey (INT4), default.ps.ps_suppkey (INT4), default.ps.ps_supplycost (FLOAT8)} - SCAN(5) on default.supplier as s - => target list: default.s.s_nationkey (INT4), default.s.s_suppkey (INT4) - => out schema: {(2) default.s.s_nationkey (INT4), default.s.s_suppkey (INT4)} - => in schema: {(7) default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT), default.s.s_suppkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000007 [ROOT] -======================================================= - -[Incoming] -[q_0000000000000_0000] 3 => 7 (type=HASH_SHUFFLE, key=default.t.n_nationkey (INT4), num=32) -[q_0000000000000_0000] 6 => 7 (type=HASH_SHUFFLE, key=default.s.s_nationkey (INT4), num=32) + 2: type=Broadcast, tables=default.ps + 3: type=Broadcast, tables=default.s JOIN(14)(INNER) => Join Cond: default.s.s_nationkey (INT4) = default.t.n_nationkey (INT4) => target list: default.ps.ps_availqty (INT4), default.s.s_suppkey (INT4), default.t.n_comment (TEXT), default.t.n_name (TEXT), default.t.n_nationkey (INT4), default.t.n_regionkey (INT4) => out schema: {(6) default.ps.ps_availqty (INT4), default.s.s_suppkey (INT4), default.t.n_comment (TEXT), default.t.n_name (TEXT), default.t.n_nationkey (INT4), default.t.n_regionkey (INT4)} => in schema: {(7) default.ps.ps_availqty (INT4), default.s.s_nationkey (INT4), default.s.s_suppkey (INT4), default.t.n_comment (TEXT), default.t.n_name (TEXT), default.t.n_nationkey (INT4), default.t.n_regionkey (INT4)} - SCAN(17) on eb_0000000000000_0000_000006 + JOIN(13)(INNER) + => Join Cond: default.s.s_suppkey (INT4) = default.ps.ps_suppkey (INT4) + => target list: default.ps.ps_availqty (INT4), default.s.s_nationkey (INT4), default.s.s_suppkey (INT4) => out schema: {(3) default.ps.ps_availqty (INT4), default.s.s_nationkey (INT4), default.s.s_suppkey (INT4)} - => in schema: {(3) default.ps.ps_availqty (INT4), default.s.s_nationkey (INT4), default.s.s_suppkey (INT4)} - SCAN(16) on eb_0000000000000_0000_000003 + => in schema: {(4) default.ps.ps_availqty (INT4), default.ps.ps_suppkey (INT4), default.s.s_nationkey (INT4), default.s.s_suppkey (INT4)} + SCAN(7) on default.partsupp as ps + => target list: default.ps.ps_availqty (INT4), default.ps.ps_suppkey (INT4) + => out schema: {(2) default.ps.ps_availqty (INT4), default.ps.ps_suppkey (INT4)} + => in schema: {(5) default.ps.ps_availqty (INT4), default.ps.ps_comment (TEXT), default.ps.ps_partkey (INT4), default.ps.ps_suppkey (INT4), default.ps.ps_supplycost (FLOAT8)} + SCAN(5) on default.supplier as s + => target list: default.s.s_nationkey (INT4), default.s.s_suppkey (INT4) + => out schema: {(2) default.s.s_nationkey (INT4), default.s.s_suppkey (INT4)} + => in schema: {(7) default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT), default.s.s_suppkey (INT4)} + TABLE_SUBQUERY(4) as default.t + => Targets: default.t.n_nationkey (INT4), default.t.n_name (TEXT), default.t.n_regionkey (INT4), default.t.n_comment (TEXT) => out schema: {(4) default.t.n_nationkey (INT4), default.t.n_name (TEXT), default.t.n_regionkey (INT4), default.t.n_comment (TEXT)} - => in schema: {(4) default.t.n_nationkey (INT4), default.t.n_name (TEXT), default.t.n_regionkey (INT4), default.t.n_comment (TEXT)} + => in schema: {(4) default.t.n_comment (TEXT), default.t.n_name (TEXT), default.t.n_nationkey (INT4), default.t.n_regionkey (INT4)} + PROJECTION(3) + => Targets: default.n.n_nationkey (INT4), default.n.n_name (TEXT), default.n.n_regionkey (INT4), default.n.n_comment (TEXT) + => out schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + => in schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + JOIN(12)(INNER) + => Join Cond: default.n.n_regionkey (INT4) = default.r.r_regionkey (INT4) + => target list: default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) + => out schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + => in schema: {(5) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4)} + SCAN(1) on default.region as r + => target list: default.r.r_regionkey (INT4) + => out schema: {(1) default.r.r_regionkey (INT4)} + => in schema: {(3) default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4)} + SCAN(0) on default.nation as n + => filter: default.n.n_name (TEXT) IN (ARGENTINA, ETHIOPIA, MOROCCO) + => target list: default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) + => out schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + => in schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} ======================================================= Block Id: eb_0000000000000_0000_000008 [TERMINAL] diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual3.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual3.Sort.plan index 88c814d275..4356e9c135 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual3.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual3.Sort.plan @@ -43,95 +43,63 @@ Execution Block Graph (TERMINAL - eb_0000000000000_0000_000008) ------------------------------------------------------------------------------- |-eb_0000000000000_0000_000008 |-eb_0000000000000_0000_000007 - |-eb_0000000000000_0000_000006 - |-eb_0000000000000_0000_000003 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000006 -3: eb_0000000000000_0000_000007 -4: eb_0000000000000_0000_000008 +1: eb_0000000000000_0000_000007 +2: eb_0000000000000_0000_000008 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [LEAF] +Block Id: eb_0000000000000_0000_000007 [ROOT] ======================================================= -[Outgoing] -[q_0000000000000_0000] 3 => 7 (type=HASH_SHUFFLE, key=default.t.n_nationkey (INT4), num=32) - [Enforcers] 0: type=Broadcast, tables=default.n 1: type=Broadcast, tables=default.r - -TABLE_SUBQUERY(4) as default.t - => Targets: default.t.n_nationkey (INT4), default.t.n_name (TEXT), default.t.n_regionkey (INT4), default.t.n_comment (TEXT) - => out schema: {(4) default.t.n_nationkey (INT4), default.t.n_name (TEXT), default.t.n_regionkey (INT4), default.t.n_comment (TEXT)} - => in schema: {(4) default.t.n_comment (TEXT), default.t.n_name (TEXT), default.t.n_nationkey (INT4), default.t.n_regionkey (INT4)} - PROJECTION(3) - => Targets: default.n.n_nationkey (INT4), default.n.n_name (TEXT), default.n.n_regionkey (INT4), default.n.n_comment (TEXT) - => out schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} - => in schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} - JOIN(12)(INNER) - => Join Cond: default.n.n_regionkey (INT4) = default.r.r_regionkey (INT4) - => target list: default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) - => out schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} - => in schema: {(5) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4)} - SCAN(1) on default.region as r - => target list: default.r.r_regionkey (INT4) - => out schema: {(1) default.r.r_regionkey (INT4)} - => in schema: {(3) default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4)} - SCAN(0) on default.nation as n - => filter: default.n.n_name (TEXT) IN (ARGENTINA, ETHIOPIA, MOROCCO) - => target list: default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) - => out schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} - => in schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000006 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 6 => 7 (type=HASH_SHUFFLE, key=default.s.s_nationkey (INT4), num=32) - -[Enforcers] - 0: type=Broadcast, tables=default.s - 1: type=Broadcast, tables=default.ps - -JOIN(13)(INNER) - => Join Cond: default.s.s_suppkey (INT4) = default.ps.ps_suppkey (INT4) - => target list: default.ps.ps_availqty (INT4), default.s.s_nationkey (INT4), default.s.s_suppkey (INT4) - => out schema: {(3) default.ps.ps_availqty (INT4), default.s.s_nationkey (INT4), default.s.s_suppkey (INT4)} - => in schema: {(4) default.ps.ps_availqty (INT4), default.ps.ps_suppkey (INT4), default.s.s_nationkey (INT4), default.s.s_suppkey (INT4)} - SCAN(7) on default.partsupp as ps - => target list: default.ps.ps_availqty (INT4), default.ps.ps_suppkey (INT4) - => out schema: {(2) default.ps.ps_availqty (INT4), default.ps.ps_suppkey (INT4)} - => in schema: {(5) default.ps.ps_availqty (INT4), default.ps.ps_comment (TEXT), default.ps.ps_partkey (INT4), default.ps.ps_suppkey (INT4), default.ps.ps_supplycost (FLOAT8)} - SCAN(5) on default.supplier as s - => target list: default.s.s_nationkey (INT4), default.s.s_suppkey (INT4) - => out schema: {(2) default.s.s_nationkey (INT4), default.s.s_suppkey (INT4)} - => in schema: {(7) default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT), default.s.s_suppkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000007 [ROOT] -======================================================= - -[Incoming] -[q_0000000000000_0000] 3 => 7 (type=HASH_SHUFFLE, key=default.t.n_nationkey (INT4), num=32) -[q_0000000000000_0000] 6 => 7 (type=HASH_SHUFFLE, key=default.s.s_nationkey (INT4), num=32) + 2: type=Broadcast, tables=default.ps + 3: type=Broadcast, tables=default.s JOIN(14)(INNER) => Join Cond: default.s.s_nationkey (INT4) = default.t.n_nationkey (INT4) => target list: default.ps.ps_availqty (INT4), default.s.s_suppkey (INT4), default.t.n_comment (TEXT), default.t.n_name (TEXT), default.t.n_nationkey (INT4), default.t.n_regionkey (INT4) => out schema: {(6) default.ps.ps_availqty (INT4), default.s.s_suppkey (INT4), default.t.n_comment (TEXT), default.t.n_name (TEXT), default.t.n_nationkey (INT4), default.t.n_regionkey (INT4)} => in schema: {(7) default.ps.ps_availqty (INT4), default.s.s_nationkey (INT4), default.s.s_suppkey (INT4), default.t.n_comment (TEXT), default.t.n_name (TEXT), default.t.n_nationkey (INT4), default.t.n_regionkey (INT4)} - SCAN(17) on eb_0000000000000_0000_000006 + JOIN(13)(INNER) + => Join Cond: default.s.s_suppkey (INT4) = default.ps.ps_suppkey (INT4) + => target list: default.ps.ps_availqty (INT4), default.s.s_nationkey (INT4), default.s.s_suppkey (INT4) => out schema: {(3) default.ps.ps_availqty (INT4), default.s.s_nationkey (INT4), default.s.s_suppkey (INT4)} - => in schema: {(3) default.ps.ps_availqty (INT4), default.s.s_nationkey (INT4), default.s.s_suppkey (INT4)} - SCAN(16) on eb_0000000000000_0000_000003 + => in schema: {(4) default.ps.ps_availqty (INT4), default.ps.ps_suppkey (INT4), default.s.s_nationkey (INT4), default.s.s_suppkey (INT4)} + SCAN(7) on default.partsupp as ps + => target list: default.ps.ps_availqty (INT4), default.ps.ps_suppkey (INT4) + => out schema: {(2) default.ps.ps_availqty (INT4), default.ps.ps_suppkey (INT4)} + => in schema: {(5) default.ps.ps_availqty (INT4), default.ps.ps_comment (TEXT), default.ps.ps_partkey (INT4), default.ps.ps_suppkey (INT4), default.ps.ps_supplycost (FLOAT8)} + SCAN(5) on default.supplier as s + => target list: default.s.s_nationkey (INT4), default.s.s_suppkey (INT4) + => out schema: {(2) default.s.s_nationkey (INT4), default.s.s_suppkey (INT4)} + => in schema: {(7) default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT), default.s.s_suppkey (INT4)} + TABLE_SUBQUERY(4) as default.t + => Targets: default.t.n_nationkey (INT4), default.t.n_name (TEXT), default.t.n_regionkey (INT4), default.t.n_comment (TEXT) => out schema: {(4) default.t.n_nationkey (INT4), default.t.n_name (TEXT), default.t.n_regionkey (INT4), default.t.n_comment (TEXT)} - => in schema: {(4) default.t.n_nationkey (INT4), default.t.n_name (TEXT), default.t.n_regionkey (INT4), default.t.n_comment (TEXT)} + => in schema: {(4) default.t.n_comment (TEXT), default.t.n_name (TEXT), default.t.n_nationkey (INT4), default.t.n_regionkey (INT4)} + PROJECTION(3) + => Targets: default.n.n_nationkey (INT4), default.n.n_name (TEXT), default.n.n_regionkey (INT4), default.n.n_comment (TEXT) + => out schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + => in schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + JOIN(12)(INNER) + => Join Cond: default.n.n_regionkey (INT4) = default.r.r_regionkey (INT4) + => target list: default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) + => out schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + => in schema: {(5) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4)} + SCAN(1) on default.region as r + => target list: default.r.r_regionkey (INT4) + => out schema: {(1) default.r.r_regionkey (INT4)} + => in schema: {(3) default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4)} + SCAN(0) on default.nation as n + => filter: default.n.n_name (TEXT) IN (ARGENTINA, ETHIOPIA, MOROCCO) + => target list: default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) + => out schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + => in schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} ======================================================= Block Id: eb_0000000000000_0000_000008 [TERMINAL] diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual4.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual4.Hash.plan index 15be61090a..a44067d673 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual4.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual4.Hash.plan @@ -45,83 +45,22 @@ Execution Block Graph (TERMINAL - eb_0000000000000_0000_000008) ------------------------------------------------------------------------------- |-eb_0000000000000_0000_000008 |-eb_0000000000000_0000_000007 - |-eb_0000000000000_0000_000006 - |-eb_0000000000000_0000_000003 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000006 -3: eb_0000000000000_0000_000007 -4: eb_0000000000000_0000_000008 +1: eb_0000000000000_0000_000007 +2: eb_0000000000000_0000_000008 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [LEAF] +Block Id: eb_0000000000000_0000_000007 [ROOT] ======================================================= -[Outgoing] -[q_0000000000000_0000] 3 => 7 (type=HASH_SHUFFLE, key=default.t.n_nationkey (INT4), num=32) - [Enforcers] 0: type=Broadcast, tables=default.n 1: type=Broadcast, tables=default.r - -TABLE_SUBQUERY(4) as default.t - => Targets: default.t.n_nationkey (INT4), default.t.n_name (TEXT), default.t.n_regionkey (INT4), default.t.n_comment (TEXT) - => out schema: {(4) default.t.n_nationkey (INT4), default.t.n_name (TEXT), default.t.n_regionkey (INT4), default.t.n_comment (TEXT)} - => in schema: {(4) default.t.n_comment (TEXT), default.t.n_name (TEXT), default.t.n_nationkey (INT4), default.t.n_regionkey (INT4)} - PROJECTION(3) - => Targets: default.n.n_nationkey (INT4), default.n.n_name (TEXT), default.n.n_regionkey (INT4), default.n.n_comment (TEXT) - => out schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} - => in schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} - JOIN(12)(INNER) - => Join Cond: default.n.n_regionkey (INT4) = default.r.r_regionkey (INT4) - => target list: default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) - => out schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} - => in schema: {(5) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4)} - SCAN(1) on default.region as r - => target list: default.r.r_regionkey (INT4) - => out schema: {(1) default.r.r_regionkey (INT4)} - => in schema: {(3) default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4)} - SCAN(0) on default.nation as n - => filter: default.n.n_name (TEXT) IN (ARGENTINA, ETHIOPIA, MOROCCO) - => target list: default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) - => out schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} - => in schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000006 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 6 => 7 (type=HASH_SHUFFLE, key=default.s.s_nationkey (INT4), num=32) - -[Enforcers] - 0: type=Broadcast, tables=default.s - 1: type=Broadcast, tables=default.ps - -JOIN(13)(INNER) - => Join Cond: default.s.s_suppkey (INT4) = default.ps.ps_suppkey (INT4) - => target list: default.ps.ps_availqty (INT4), default.s.s_nationkey (INT4), default.s.s_suppkey (INT4) - => out schema: {(3) default.ps.ps_availqty (INT4), default.s.s_nationkey (INT4), default.s.s_suppkey (INT4)} - => in schema: {(4) default.ps.ps_availqty (INT4), default.ps.ps_suppkey (INT4), default.s.s_nationkey (INT4), default.s.s_suppkey (INT4)} - SCAN(7) on default.partsupp as ps - => target list: default.ps.ps_availqty (INT4), default.ps.ps_suppkey (INT4) - => out schema: {(2) default.ps.ps_availqty (INT4), default.ps.ps_suppkey (INT4)} - => in schema: {(5) default.ps.ps_availqty (INT4), default.ps.ps_comment (TEXT), default.ps.ps_partkey (INT4), default.ps.ps_suppkey (INT4), default.ps.ps_supplycost (FLOAT8)} - SCAN(5) on default.supplier as s - => target list: default.s.s_nationkey (INT4), default.s.s_suppkey (INT4) - => out schema: {(2) default.s.s_nationkey (INT4), default.s.s_suppkey (INT4)} - => in schema: {(7) default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT), default.s.s_suppkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000007 [ROOT] -======================================================= - -[Incoming] -[q_0000000000000_0000] 3 => 7 (type=HASH_SHUFFLE, key=default.t.n_nationkey (INT4), num=32) -[q_0000000000000_0000] 6 => 7 (type=HASH_SHUFFLE, key=default.s.s_nationkey (INT4), num=32) + 2: type=Broadcast, tables=default.ps + 3: type=Broadcast, tables=default.s SELECTION(9) => Search Cond: default.t.n_nationkey (INT4) > default.s.s_suppkey (INT4) @@ -130,12 +69,41 @@ SELECTION(9) => target list: default.ps.ps_availqty (INT4), default.s.s_suppkey (INT4), default.t.n_comment (TEXT), default.t.n_name (TEXT), default.t.n_nationkey (INT4), default.t.n_regionkey (INT4) => out schema: {(6) default.ps.ps_availqty (INT4), default.s.s_suppkey (INT4), default.t.n_comment (TEXT), default.t.n_name (TEXT), default.t.n_nationkey (INT4), default.t.n_regionkey (INT4)} => in schema: {(7) default.ps.ps_availqty (INT4), default.s.s_nationkey (INT4), default.s.s_suppkey (INT4), default.t.n_comment (TEXT), default.t.n_name (TEXT), default.t.n_nationkey (INT4), default.t.n_regionkey (INT4)} - SCAN(17) on eb_0000000000000_0000_000006 + JOIN(13)(INNER) + => Join Cond: default.s.s_suppkey (INT4) = default.ps.ps_suppkey (INT4) + => target list: default.ps.ps_availqty (INT4), default.s.s_nationkey (INT4), default.s.s_suppkey (INT4) => out schema: {(3) default.ps.ps_availqty (INT4), default.s.s_nationkey (INT4), default.s.s_suppkey (INT4)} - => in schema: {(3) default.ps.ps_availqty (INT4), default.s.s_nationkey (INT4), default.s.s_suppkey (INT4)} - SCAN(16) on eb_0000000000000_0000_000003 + => in schema: {(4) default.ps.ps_availqty (INT4), default.ps.ps_suppkey (INT4), default.s.s_nationkey (INT4), default.s.s_suppkey (INT4)} + SCAN(7) on default.partsupp as ps + => target list: default.ps.ps_availqty (INT4), default.ps.ps_suppkey (INT4) + => out schema: {(2) default.ps.ps_availqty (INT4), default.ps.ps_suppkey (INT4)} + => in schema: {(5) default.ps.ps_availqty (INT4), default.ps.ps_comment (TEXT), default.ps.ps_partkey (INT4), default.ps.ps_suppkey (INT4), default.ps.ps_supplycost (FLOAT8)} + SCAN(5) on default.supplier as s + => target list: default.s.s_nationkey (INT4), default.s.s_suppkey (INT4) + => out schema: {(2) default.s.s_nationkey (INT4), default.s.s_suppkey (INT4)} + => in schema: {(7) default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT), default.s.s_suppkey (INT4)} + TABLE_SUBQUERY(4) as default.t + => Targets: default.t.n_nationkey (INT4), default.t.n_name (TEXT), default.t.n_regionkey (INT4), default.t.n_comment (TEXT) => out schema: {(4) default.t.n_nationkey (INT4), default.t.n_name (TEXT), default.t.n_regionkey (INT4), default.t.n_comment (TEXT)} - => in schema: {(4) default.t.n_nationkey (INT4), default.t.n_name (TEXT), default.t.n_regionkey (INT4), default.t.n_comment (TEXT)} + => in schema: {(4) default.t.n_comment (TEXT), default.t.n_name (TEXT), default.t.n_nationkey (INT4), default.t.n_regionkey (INT4)} + PROJECTION(3) + => Targets: default.n.n_nationkey (INT4), default.n.n_name (TEXT), default.n.n_regionkey (INT4), default.n.n_comment (TEXT) + => out schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + => in schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + JOIN(12)(INNER) + => Join Cond: default.n.n_regionkey (INT4) = default.r.r_regionkey (INT4) + => target list: default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) + => out schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + => in schema: {(5) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4)} + SCAN(1) on default.region as r + => target list: default.r.r_regionkey (INT4) + => out schema: {(1) default.r.r_regionkey (INT4)} + => in schema: {(3) default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4)} + SCAN(0) on default.nation as n + => filter: default.n.n_name (TEXT) IN (ARGENTINA, ETHIOPIA, MOROCCO) + => target list: default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) + => out schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + => in schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} ======================================================= Block Id: eb_0000000000000_0000_000008 [TERMINAL] diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual4.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual4.Sort.plan index 15be61090a..a44067d673 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual4.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual4.Sort.plan @@ -45,83 +45,22 @@ Execution Block Graph (TERMINAL - eb_0000000000000_0000_000008) ------------------------------------------------------------------------------- |-eb_0000000000000_0000_000008 |-eb_0000000000000_0000_000007 - |-eb_0000000000000_0000_000006 - |-eb_0000000000000_0000_000003 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000006 -3: eb_0000000000000_0000_000007 -4: eb_0000000000000_0000_000008 +1: eb_0000000000000_0000_000007 +2: eb_0000000000000_0000_000008 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [LEAF] +Block Id: eb_0000000000000_0000_000007 [ROOT] ======================================================= -[Outgoing] -[q_0000000000000_0000] 3 => 7 (type=HASH_SHUFFLE, key=default.t.n_nationkey (INT4), num=32) - [Enforcers] 0: type=Broadcast, tables=default.n 1: type=Broadcast, tables=default.r - -TABLE_SUBQUERY(4) as default.t - => Targets: default.t.n_nationkey (INT4), default.t.n_name (TEXT), default.t.n_regionkey (INT4), default.t.n_comment (TEXT) - => out schema: {(4) default.t.n_nationkey (INT4), default.t.n_name (TEXT), default.t.n_regionkey (INT4), default.t.n_comment (TEXT)} - => in schema: {(4) default.t.n_comment (TEXT), default.t.n_name (TEXT), default.t.n_nationkey (INT4), default.t.n_regionkey (INT4)} - PROJECTION(3) - => Targets: default.n.n_nationkey (INT4), default.n.n_name (TEXT), default.n.n_regionkey (INT4), default.n.n_comment (TEXT) - => out schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} - => in schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} - JOIN(12)(INNER) - => Join Cond: default.n.n_regionkey (INT4) = default.r.r_regionkey (INT4) - => target list: default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) - => out schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} - => in schema: {(5) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4)} - SCAN(1) on default.region as r - => target list: default.r.r_regionkey (INT4) - => out schema: {(1) default.r.r_regionkey (INT4)} - => in schema: {(3) default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4)} - SCAN(0) on default.nation as n - => filter: default.n.n_name (TEXT) IN (ARGENTINA, ETHIOPIA, MOROCCO) - => target list: default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) - => out schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} - => in schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000006 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 6 => 7 (type=HASH_SHUFFLE, key=default.s.s_nationkey (INT4), num=32) - -[Enforcers] - 0: type=Broadcast, tables=default.s - 1: type=Broadcast, tables=default.ps - -JOIN(13)(INNER) - => Join Cond: default.s.s_suppkey (INT4) = default.ps.ps_suppkey (INT4) - => target list: default.ps.ps_availqty (INT4), default.s.s_nationkey (INT4), default.s.s_suppkey (INT4) - => out schema: {(3) default.ps.ps_availqty (INT4), default.s.s_nationkey (INT4), default.s.s_suppkey (INT4)} - => in schema: {(4) default.ps.ps_availqty (INT4), default.ps.ps_suppkey (INT4), default.s.s_nationkey (INT4), default.s.s_suppkey (INT4)} - SCAN(7) on default.partsupp as ps - => target list: default.ps.ps_availqty (INT4), default.ps.ps_suppkey (INT4) - => out schema: {(2) default.ps.ps_availqty (INT4), default.ps.ps_suppkey (INT4)} - => in schema: {(5) default.ps.ps_availqty (INT4), default.ps.ps_comment (TEXT), default.ps.ps_partkey (INT4), default.ps.ps_suppkey (INT4), default.ps.ps_supplycost (FLOAT8)} - SCAN(5) on default.supplier as s - => target list: default.s.s_nationkey (INT4), default.s.s_suppkey (INT4) - => out schema: {(2) default.s.s_nationkey (INT4), default.s.s_suppkey (INT4)} - => in schema: {(7) default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT), default.s.s_suppkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000007 [ROOT] -======================================================= - -[Incoming] -[q_0000000000000_0000] 3 => 7 (type=HASH_SHUFFLE, key=default.t.n_nationkey (INT4), num=32) -[q_0000000000000_0000] 6 => 7 (type=HASH_SHUFFLE, key=default.s.s_nationkey (INT4), num=32) + 2: type=Broadcast, tables=default.ps + 3: type=Broadcast, tables=default.s SELECTION(9) => Search Cond: default.t.n_nationkey (INT4) > default.s.s_suppkey (INT4) @@ -130,12 +69,41 @@ SELECTION(9) => target list: default.ps.ps_availqty (INT4), default.s.s_suppkey (INT4), default.t.n_comment (TEXT), default.t.n_name (TEXT), default.t.n_nationkey (INT4), default.t.n_regionkey (INT4) => out schema: {(6) default.ps.ps_availqty (INT4), default.s.s_suppkey (INT4), default.t.n_comment (TEXT), default.t.n_name (TEXT), default.t.n_nationkey (INT4), default.t.n_regionkey (INT4)} => in schema: {(7) default.ps.ps_availqty (INT4), default.s.s_nationkey (INT4), default.s.s_suppkey (INT4), default.t.n_comment (TEXT), default.t.n_name (TEXT), default.t.n_nationkey (INT4), default.t.n_regionkey (INT4)} - SCAN(17) on eb_0000000000000_0000_000006 + JOIN(13)(INNER) + => Join Cond: default.s.s_suppkey (INT4) = default.ps.ps_suppkey (INT4) + => target list: default.ps.ps_availqty (INT4), default.s.s_nationkey (INT4), default.s.s_suppkey (INT4) => out schema: {(3) default.ps.ps_availqty (INT4), default.s.s_nationkey (INT4), default.s.s_suppkey (INT4)} - => in schema: {(3) default.ps.ps_availqty (INT4), default.s.s_nationkey (INT4), default.s.s_suppkey (INT4)} - SCAN(16) on eb_0000000000000_0000_000003 + => in schema: {(4) default.ps.ps_availqty (INT4), default.ps.ps_suppkey (INT4), default.s.s_nationkey (INT4), default.s.s_suppkey (INT4)} + SCAN(7) on default.partsupp as ps + => target list: default.ps.ps_availqty (INT4), default.ps.ps_suppkey (INT4) + => out schema: {(2) default.ps.ps_availqty (INT4), default.ps.ps_suppkey (INT4)} + => in schema: {(5) default.ps.ps_availqty (INT4), default.ps.ps_comment (TEXT), default.ps.ps_partkey (INT4), default.ps.ps_suppkey (INT4), default.ps.ps_supplycost (FLOAT8)} + SCAN(5) on default.supplier as s + => target list: default.s.s_nationkey (INT4), default.s.s_suppkey (INT4) + => out schema: {(2) default.s.s_nationkey (INT4), default.s.s_suppkey (INT4)} + => in schema: {(7) default.s.s_acctbal (FLOAT8), default.s.s_address (TEXT), default.s.s_comment (TEXT), default.s.s_name (TEXT), default.s.s_nationkey (INT4), default.s.s_phone (TEXT), default.s.s_suppkey (INT4)} + TABLE_SUBQUERY(4) as default.t + => Targets: default.t.n_nationkey (INT4), default.t.n_name (TEXT), default.t.n_regionkey (INT4), default.t.n_comment (TEXT) => out schema: {(4) default.t.n_nationkey (INT4), default.t.n_name (TEXT), default.t.n_regionkey (INT4), default.t.n_comment (TEXT)} - => in schema: {(4) default.t.n_nationkey (INT4), default.t.n_name (TEXT), default.t.n_regionkey (INT4), default.t.n_comment (TEXT)} + => in schema: {(4) default.t.n_comment (TEXT), default.t.n_name (TEXT), default.t.n_nationkey (INT4), default.t.n_regionkey (INT4)} + PROJECTION(3) + => Targets: default.n.n_nationkey (INT4), default.n.n_name (TEXT), default.n.n_regionkey (INT4), default.n.n_comment (TEXT) + => out schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + => in schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + JOIN(12)(INNER) + => Join Cond: default.n.n_regionkey (INT4) = default.r.r_regionkey (INT4) + => target list: default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) + => out schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + => in schema: {(5) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4)} + SCAN(1) on default.region as r + => target list: default.r.r_regionkey (INT4) + => out schema: {(1) default.r.r_regionkey (INT4)} + => in schema: {(3) default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4)} + SCAN(0) on default.nation as n + => filter: default.n.n_name (TEXT) IN (ARGENTINA, ETHIOPIA, MOROCCO) + => target list: default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) + => out schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + => in schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} ======================================================= Block Id: eb_0000000000000_0000_000008 [TERMINAL] From af3a1843377befef6c280b26180c5472076eb382 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 22 May 2015 23:28:12 +0900 Subject: [PATCH 30/37] Fix outer join tests except for full outer join --- .../rewriter/rules/BroadcastJoinRule.java | 5 +- .../testBroadcastSubquery2.Hash.plan | 58 ++++------ .../testBroadcastSubquery2.Sort.plan | 58 ++++------ .../testComplexJoinsWithCaseWhen.Hash.plan | 34 +++--- .../testComplexJoinsWithCaseWhen.Sort.plan | 34 +++--- .../testComplexJoinsWithCaseWhen2.Hash.plan | 91 +++++---------- .../testComplexJoinsWithCaseWhen2.Sort.plan | 91 +++++---------- .../testInnerAndOuterWithEmpty.1.Hash.plan | 18 +-- .../testInnerAndOuterWithEmpty.1.Sort.plan | 18 +-- .../testJoinWithMultipleJoinTypes.Hash.plan | 108 +++++------------- .../testJoinWithMultipleJoinTypes.Sort.plan | 108 +++++------------- ...stJoinFilterOfRowPreservedTable1.Hash.plan | 4 +- ...stJoinFilterOfRowPreservedTable1.Sort.plan | 4 +- .../testLeftOuterJoin1.Hash.plan | 4 +- .../testLeftOuterJoin1.Sort.plan | 4 +- .../testLeftOuterJoin2.Hash.plan | 14 +-- .../testLeftOuterJoin2.Sort.plan | 14 +-- .../testLeftOuterJoin3.Hash.plan | 22 ++-- .../testLeftOuterJoin3.Sort.plan | 22 ++-- ...uterJoinPredicationCaseByCase1.1.Hash.plan | 14 +-- ...uterJoinPredicationCaseByCase1.1.Sort.plan | 14 +-- ...uterJoinPredicationCaseByCase2.1.Hash.plan | 18 +-- ...uterJoinPredicationCaseByCase2.1.Sort.plan | 18 +-- ...erJoinPredicationCaseByCase2_1.1.Hash.plan | 20 ++-- ...erJoinPredicationCaseByCase2_1.1.Sort.plan | 20 ++-- ...uterJoinPredicationCaseByCase3.1.Hash.plan | 14 +-- ...uterJoinPredicationCaseByCase3.1.Sort.plan | 14 +-- ...uterJoinPredicationCaseByCase4.1.Hash.plan | 14 +-- ...uterJoinPredicationCaseByCase4.1.Sort.plan | 14 +-- ...uterJoinPredicationCaseByCase5.1.Hash.plan | 14 +-- ...uterJoinPredicationCaseByCase5.1.Sort.plan | 14 +-- ...uterJoinPredicationCaseByCase6.1.Hash.plan | 14 +-- ...uterJoinPredicationCaseByCase6.1.Sort.plan | 14 +-- ...stLeftOuterJoinWithConstantExpr1.Hash.plan | 4 +- ...stLeftOuterJoinWithConstantExpr1.Sort.plan | 4 +- ...testLeftOuterJoinWithEmptyTable1.Hash.plan | 4 +- ...testLeftOuterJoinWithEmptyTable1.Sort.plan | 4 +- ...testLeftOuterJoinWithEmptyTable2.Hash.plan | 8 +- ...testLeftOuterJoinWithEmptyTable2.Sort.plan | 8 +- ...testLeftOuterJoinWithEmptyTable3.Hash.plan | 12 +- ...testLeftOuterJoinWithEmptyTable3.Sort.plan | 12 +- ...testLeftOuterJoinWithEmptyTable4.Hash.plan | 8 +- ...testLeftOuterJoinWithEmptyTable4.Sort.plan | 8 +- ...testLeftOuterJoinWithEmptyTable5.Hash.plan | 8 +- ...testLeftOuterJoinWithEmptyTable5.Sort.plan | 8 +- .../testLeftOuterJoinWithNull1.Hash.plan | 4 +- .../testLeftOuterJoinWithNull1.Sort.plan | 4 +- .../testLeftOuterJoinWithNull2.Hash.plan | 4 +- .../testLeftOuterJoinWithNull2.Sort.plan | 4 +- .../testLeftOuterJoinWithNull3.Hash.plan | 4 +- .../testLeftOuterJoinWithNull3.Sort.plan | 4 +- ...BroadcastDataFileWithZeroLength2.Hash.plan | 14 +-- ...BroadcastDataFileWithZeroLength2.Sort.plan | 14 +-- .../testOuterJoinAndCaseWhen1.Hash.plan | 30 ++--- .../testOuterJoinAndCaseWhen1.Sort.plan | 30 ++--- .../testRightOuterJoin1.Hash.plan | 4 +- .../testRightOuterJoin1.Sort.plan | 4 +- ...estRightOuterJoinWithEmptyTable1.Hash.plan | 4 +- ...estRightOuterJoinWithEmptyTable1.Sort.plan | 4 +- ...stLeftOuterJoinWithConstantExpr2.Hash.plan | 69 ++++------- ...stLeftOuterJoinWithConstantExpr2.Sort.plan | 69 ++++------- ...stLeftOuterJoinWithConstantExpr3.Hash.plan | 3 + ...stLeftOuterJoinWithConstantExpr3.Sort.plan | 3 + ...eftOuterJoinWithEmptySubquery1.1.Hash.plan | 96 ++++++---------- ...eftOuterJoinWithEmptySubquery1.1.Sort.plan | 96 ++++++---------- ...eftOuterJoinWithEmptySubquery2.1.Hash.plan | 89 +++++---------- ...eftOuterJoinWithEmptySubquery2.1.Sort.plan | 89 +++++---------- 67 files changed, 645 insertions(+), 1012 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java index 720fe66929..8b5ba50441 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java @@ -135,8 +135,9 @@ private void rewrite(MasterPlan plan, ExecutionBlock current) throws PlanningExc } // TODO: check all inputs are marked as broadcast - for (ExecutionBlock child : childs) { - if (child.hasBroadcastRelation()) { + if (broadcastCandidates.size() > 0) { + for (ExecutionBlock child : childs) { +// if (child.hasBroadcastRelation()) { List unionScans = TUtil.newList(); ExecutionBlockId representativeId = null; if (unionScanMap.containsKey(child.getId())) { diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery2.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery2.Hash.plan index dd00452aa2..dd370aa2be 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery2.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery2.Hash.plan @@ -73,16 +73,14 @@ Execution Block Graph (TERMINAL - eb_0000000000000_0000_000012) |-eb_0000000000000_0000_000012 |-eb_0000000000000_0000_000011 |-eb_0000000000000_0000_000010 - |-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 + |-eb_0000000000000_0000_000003 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- 1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 -3: eb_0000000000000_0000_000010 -4: eb_0000000000000_0000_000011 -5: eb_0000000000000_0000_000012 +2: eb_0000000000000_0000_000010 +3: eb_0000000000000_0000_000011 +4: eb_0000000000000_0000_000012 ------------------------------------------------------------------------------- ======================================================= @@ -90,7 +88,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] ======================================================= [Outgoing] -[q_0000000000000_0000] 3 => 4 (type=HASH_SHUFFLE, key=default.e.l_orderkey (INT4), num=32) +[q_0000000000000_0000] 3 => 10 (type=HASH_SHUFFLE) [Enforcers] 0: type=Broadcast, tables=default.a @@ -123,39 +121,12 @@ GROUP_BY(27)(l_orderkey) => out schema: {(3) default.a.l_orderkey (INT4), default.a.l_partkey (INT4), default.a.l_quantity (FLOAT8)} => in schema: {(16) default.a.l_comment (TEXT), default.a.l_commitdate (TEXT), default.a.l_discount (FLOAT8), default.a.l_extendedprice (FLOAT8), default.a.l_linenumber (INT4), default.a.l_linestatus (TEXT), default.a.l_orderkey (INT4), default.a.l_partkey (INT4), default.a.l_quantity (FLOAT8), default.a.l_receiptdate (TEXT), default.a.l_returnflag (TEXT), default.a.l_shipdate (TEXT), default.a.l_shipinstruct (TEXT), default.a.l_shipmode (TEXT), default.a.l_suppkey (INT4), default.a.l_tax (FLOAT8)} -======================================================= -Block Id: eb_0000000000000_0000_000004 [INTERMEDIATE] -======================================================= - -[Incoming] -[q_0000000000000_0000] 3 => 4 (type=HASH_SHUFFLE, key=default.e.l_orderkey (INT4), num=32) - -[Outgoing] -[q_0000000000000_0000] 4 => 10 (type=HASH_SHUFFLE, key=default.f.l_orderkey (INT4), num=32) - -TABLE_SUBQUERY(14) as default.f - => Targets: default.f.avg_quantity (FLOAT8), default.f.l_orderkey (INT4) - => out schema: {(2) default.f.avg_quantity (FLOAT8), default.f.l_orderkey (INT4)} - => in schema: {(2) default.f.avg_quantity (FLOAT8), default.f.l_orderkey (INT4)} - PROJECTION(13) - => Targets: default.e.l_orderkey (INT4), avg_quantity (FLOAT8) - => out schema: {(2) avg_quantity (FLOAT8), default.e.l_orderkey (INT4)} - => in schema: {(2) avg_quantity (FLOAT8), default.e.l_orderkey (INT4)} - GROUP_BY(12)(l_orderkey) - => exprs: (avg(?avg_11 (PROTOBUF))) - => target list: default.e.l_orderkey (INT4), avg_quantity (FLOAT8) - => out schema:{(2) avg_quantity (FLOAT8), default.e.l_orderkey (INT4)} - => in schema:{(2) default.e.l_orderkey (INT4), ?avg_11 (PROTOBUF)} - SCAN(28) on eb_0000000000000_0000_000003 - => out schema: {(2) default.e.l_orderkey (INT4), ?avg_11 (PROTOBUF)} - => in schema: {(2) default.e.l_orderkey (INT4), ?avg_11 (PROTOBUF)} - ======================================================= Block Id: eb_0000000000000_0000_000010 [INTERMEDIATE] ======================================================= [Incoming] -[q_0000000000000_0000] 4 => 10 (type=HASH_SHUFFLE, key=default.f.l_orderkey (INT4), num=32) +[q_0000000000000_0000] 3 => 10 (type=HASH_SHUFFLE) [Outgoing] [q_0000000000000_0000] 10 => 11 (type=HASH_SHUFFLE, key=, num=1) @@ -208,9 +179,22 @@ GROUP_BY(35)() => target list: default.d.l_orderkey (INT4), default.d.l_partkey (INT4), default.d.l_quantity (FLOAT8) => out schema: {(3) default.d.l_orderkey (INT4), default.d.l_partkey (INT4), default.d.l_quantity (FLOAT8)} => in schema: {(16) default.d.l_comment (TEXT), default.d.l_commitdate (TEXT), default.d.l_discount (FLOAT8), default.d.l_extendedprice (FLOAT8), default.d.l_linenumber (INT4), default.d.l_linestatus (TEXT), default.d.l_orderkey (INT4), default.d.l_partkey (INT4), default.d.l_quantity (FLOAT8), default.d.l_receiptdate (TEXT), default.d.l_returnflag (TEXT), default.d.l_shipdate (TEXT), default.d.l_shipinstruct (TEXT), default.d.l_shipmode (TEXT), default.d.l_suppkey (INT4), default.d.l_tax (FLOAT8)} - SCAN(33) on eb_0000000000000_0000_000004 + TABLE_SUBQUERY(14) as default.f + => Targets: default.f.avg_quantity (FLOAT8), default.f.l_orderkey (INT4) => out schema: {(2) default.f.avg_quantity (FLOAT8), default.f.l_orderkey (INT4)} - => in schema: {(2) default.f.avg_quantity (FLOAT8), default.f.l_orderkey (INT4)} + => in schema: {(2) default.f.avg_quantity (FLOAT8), default.f.l_orderkey (INT4)} + PROJECTION(13) + => Targets: default.e.l_orderkey (INT4), avg_quantity (FLOAT8) + => out schema: {(2) avg_quantity (FLOAT8), default.e.l_orderkey (INT4)} + => in schema: {(2) avg_quantity (FLOAT8), default.e.l_orderkey (INT4)} + GROUP_BY(12)(l_orderkey) + => exprs: (avg(?avg_11 (PROTOBUF))) + => target list: default.e.l_orderkey (INT4), avg_quantity (FLOAT8) + => out schema:{(2) avg_quantity (FLOAT8), default.e.l_orderkey (INT4)} + => in schema:{(2) default.e.l_orderkey (INT4), ?avg_11 (PROTOBUF)} + SCAN(28) on eb_0000000000000_0000_000003 + => out schema: {(2) default.e.l_orderkey (INT4), ?avg_11 (PROTOBUF)} + => in schema: {(2) default.e.l_orderkey (INT4), ?avg_11 (PROTOBUF)} ======================================================= Block Id: eb_0000000000000_0000_000011 [ROOT] diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery2.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery2.Sort.plan index dd00452aa2..dd370aa2be 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery2.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery2.Sort.plan @@ -73,16 +73,14 @@ Execution Block Graph (TERMINAL - eb_0000000000000_0000_000012) |-eb_0000000000000_0000_000012 |-eb_0000000000000_0000_000011 |-eb_0000000000000_0000_000010 - |-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 + |-eb_0000000000000_0000_000003 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- 1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 -3: eb_0000000000000_0000_000010 -4: eb_0000000000000_0000_000011 -5: eb_0000000000000_0000_000012 +2: eb_0000000000000_0000_000010 +3: eb_0000000000000_0000_000011 +4: eb_0000000000000_0000_000012 ------------------------------------------------------------------------------- ======================================================= @@ -90,7 +88,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] ======================================================= [Outgoing] -[q_0000000000000_0000] 3 => 4 (type=HASH_SHUFFLE, key=default.e.l_orderkey (INT4), num=32) +[q_0000000000000_0000] 3 => 10 (type=HASH_SHUFFLE) [Enforcers] 0: type=Broadcast, tables=default.a @@ -123,39 +121,12 @@ GROUP_BY(27)(l_orderkey) => out schema: {(3) default.a.l_orderkey (INT4), default.a.l_partkey (INT4), default.a.l_quantity (FLOAT8)} => in schema: {(16) default.a.l_comment (TEXT), default.a.l_commitdate (TEXT), default.a.l_discount (FLOAT8), default.a.l_extendedprice (FLOAT8), default.a.l_linenumber (INT4), default.a.l_linestatus (TEXT), default.a.l_orderkey (INT4), default.a.l_partkey (INT4), default.a.l_quantity (FLOAT8), default.a.l_receiptdate (TEXT), default.a.l_returnflag (TEXT), default.a.l_shipdate (TEXT), default.a.l_shipinstruct (TEXT), default.a.l_shipmode (TEXT), default.a.l_suppkey (INT4), default.a.l_tax (FLOAT8)} -======================================================= -Block Id: eb_0000000000000_0000_000004 [INTERMEDIATE] -======================================================= - -[Incoming] -[q_0000000000000_0000] 3 => 4 (type=HASH_SHUFFLE, key=default.e.l_orderkey (INT4), num=32) - -[Outgoing] -[q_0000000000000_0000] 4 => 10 (type=HASH_SHUFFLE, key=default.f.l_orderkey (INT4), num=32) - -TABLE_SUBQUERY(14) as default.f - => Targets: default.f.avg_quantity (FLOAT8), default.f.l_orderkey (INT4) - => out schema: {(2) default.f.avg_quantity (FLOAT8), default.f.l_orderkey (INT4)} - => in schema: {(2) default.f.avg_quantity (FLOAT8), default.f.l_orderkey (INT4)} - PROJECTION(13) - => Targets: default.e.l_orderkey (INT4), avg_quantity (FLOAT8) - => out schema: {(2) avg_quantity (FLOAT8), default.e.l_orderkey (INT4)} - => in schema: {(2) avg_quantity (FLOAT8), default.e.l_orderkey (INT4)} - GROUP_BY(12)(l_orderkey) - => exprs: (avg(?avg_11 (PROTOBUF))) - => target list: default.e.l_orderkey (INT4), avg_quantity (FLOAT8) - => out schema:{(2) avg_quantity (FLOAT8), default.e.l_orderkey (INT4)} - => in schema:{(2) default.e.l_orderkey (INT4), ?avg_11 (PROTOBUF)} - SCAN(28) on eb_0000000000000_0000_000003 - => out schema: {(2) default.e.l_orderkey (INT4), ?avg_11 (PROTOBUF)} - => in schema: {(2) default.e.l_orderkey (INT4), ?avg_11 (PROTOBUF)} - ======================================================= Block Id: eb_0000000000000_0000_000010 [INTERMEDIATE] ======================================================= [Incoming] -[q_0000000000000_0000] 4 => 10 (type=HASH_SHUFFLE, key=default.f.l_orderkey (INT4), num=32) +[q_0000000000000_0000] 3 => 10 (type=HASH_SHUFFLE) [Outgoing] [q_0000000000000_0000] 10 => 11 (type=HASH_SHUFFLE, key=, num=1) @@ -208,9 +179,22 @@ GROUP_BY(35)() => target list: default.d.l_orderkey (INT4), default.d.l_partkey (INT4), default.d.l_quantity (FLOAT8) => out schema: {(3) default.d.l_orderkey (INT4), default.d.l_partkey (INT4), default.d.l_quantity (FLOAT8)} => in schema: {(16) default.d.l_comment (TEXT), default.d.l_commitdate (TEXT), default.d.l_discount (FLOAT8), default.d.l_extendedprice (FLOAT8), default.d.l_linenumber (INT4), default.d.l_linestatus (TEXT), default.d.l_orderkey (INT4), default.d.l_partkey (INT4), default.d.l_quantity (FLOAT8), default.d.l_receiptdate (TEXT), default.d.l_returnflag (TEXT), default.d.l_shipdate (TEXT), default.d.l_shipinstruct (TEXT), default.d.l_shipmode (TEXT), default.d.l_suppkey (INT4), default.d.l_tax (FLOAT8)} - SCAN(33) on eb_0000000000000_0000_000004 + TABLE_SUBQUERY(14) as default.f + => Targets: default.f.avg_quantity (FLOAT8), default.f.l_orderkey (INT4) => out schema: {(2) default.f.avg_quantity (FLOAT8), default.f.l_orderkey (INT4)} - => in schema: {(2) default.f.avg_quantity (FLOAT8), default.f.l_orderkey (INT4)} + => in schema: {(2) default.f.avg_quantity (FLOAT8), default.f.l_orderkey (INT4)} + PROJECTION(13) + => Targets: default.e.l_orderkey (INT4), avg_quantity (FLOAT8) + => out schema: {(2) avg_quantity (FLOAT8), default.e.l_orderkey (INT4)} + => in schema: {(2) avg_quantity (FLOAT8), default.e.l_orderkey (INT4)} + GROUP_BY(12)(l_orderkey) + => exprs: (avg(?avg_11 (PROTOBUF))) + => target list: default.e.l_orderkey (INT4), avg_quantity (FLOAT8) + => out schema:{(2) avg_quantity (FLOAT8), default.e.l_orderkey (INT4)} + => in schema:{(2) default.e.l_orderkey (INT4), ?avg_11 (PROTOBUF)} + SCAN(28) on eb_0000000000000_0000_000003 + => out schema: {(2) default.e.l_orderkey (INT4), ?avg_11 (PROTOBUF)} + => in schema: {(2) default.e.l_orderkey (INT4), ?avg_11 (PROTOBUF)} ======================================================= Block Id: eb_0000000000000_0000_000011 [ROOT] diff --git a/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testComplexJoinsWithCaseWhen.Hash.plan b/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testComplexJoinsWithCaseWhen.Hash.plan index 668ddabb58..a56bba9e7a 100644 --- a/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testComplexJoinsWithCaseWhen.Hash.plan +++ b/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testComplexJoinsWithCaseWhen.Hash.plan @@ -27,32 +27,32 @@ SORT(5) explain ------------------------------- ------------------------------------------------------------------------------- -Execution Block Graph (TERMINAL - eb_0000000000000_0000_000005) +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000007) ------------------------------------------------------------------------------- -|-eb_0000000000000_0000_000005 - |-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 +|-eb_0000000000000_0000_000007 + |-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000005 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 -3: eb_0000000000000_0000_000005 +1: eb_0000000000000_0000_000005 +2: eb_0000000000000_0000_000006 +3: eb_0000000000000_0000_000007 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [LEAF] +Block Id: eb_0000000000000_0000_000005 [LEAF] ======================================================= [Outgoing] -[q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.region.r_name (TEXT), s1 (TEXT), num=32) +[q_0000000000000_0000] 5 => 6 (type=RANGE_SHUFFLE, key=default.region.r_name (TEXT), s1 (TEXT), num=32) [Enforcers] - 0: type=Broadcast, tables=default.region - 1: type=Broadcast, tables=default.nation + 0: type=Broadcast, tables=default.nation + 1: type=Broadcast, tables=default.region 2: type=Broadcast, tables=default.supplier -SORT(11) +SORT(15) => Sort Keys: default.region.r_name (TEXT) (asc),s1 (TEXT) (asc) JOIN(9)(LEFT_OUTER) => Join Cond: default.supplier.s_nationkey (INT4) = default.nation.n_nationkey (INT4) @@ -78,21 +78,21 @@ SORT(11) => in schema: {(3) default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} ======================================================= -Block Id: eb_0000000000000_0000_000004 [ROOT] +Block Id: eb_0000000000000_0000_000006 [ROOT] ======================================================= [Incoming] -[q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.region.r_name (TEXT), s1 (TEXT), num=32) +[q_0000000000000_0000] 5 => 6 (type=RANGE_SHUFFLE, key=default.region.r_name (TEXT), s1 (TEXT), num=32) [Enforcers] - 0: sorted input=eb_0000000000000_0000_000003 + 0: sorted input=eb_0000000000000_0000_000005 SORT(5) => Sort Keys: default.region.r_name (TEXT) (asc),s1 (TEXT) (asc) - SCAN(12) on eb_0000000000000_0000_000003 + SCAN(16) on eb_0000000000000_0000_000005 => out schema: {(2) default.region.r_name (TEXT), s1 (TEXT)} => in schema: {(2) default.region.r_name (TEXT), s1 (TEXT)} ======================================================= -Block Id: eb_0000000000000_0000_000005 [TERMINAL] +Block Id: eb_0000000000000_0000_000007 [TERMINAL] ======================================================= diff --git a/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testComplexJoinsWithCaseWhen.Sort.plan b/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testComplexJoinsWithCaseWhen.Sort.plan index 668ddabb58..a56bba9e7a 100644 --- a/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testComplexJoinsWithCaseWhen.Sort.plan +++ b/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testComplexJoinsWithCaseWhen.Sort.plan @@ -27,32 +27,32 @@ SORT(5) explain ------------------------------- ------------------------------------------------------------------------------- -Execution Block Graph (TERMINAL - eb_0000000000000_0000_000005) +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000007) ------------------------------------------------------------------------------- -|-eb_0000000000000_0000_000005 - |-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 +|-eb_0000000000000_0000_000007 + |-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000005 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 -3: eb_0000000000000_0000_000005 +1: eb_0000000000000_0000_000005 +2: eb_0000000000000_0000_000006 +3: eb_0000000000000_0000_000007 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [LEAF] +Block Id: eb_0000000000000_0000_000005 [LEAF] ======================================================= [Outgoing] -[q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.region.r_name (TEXT), s1 (TEXT), num=32) +[q_0000000000000_0000] 5 => 6 (type=RANGE_SHUFFLE, key=default.region.r_name (TEXT), s1 (TEXT), num=32) [Enforcers] - 0: type=Broadcast, tables=default.region - 1: type=Broadcast, tables=default.nation + 0: type=Broadcast, tables=default.nation + 1: type=Broadcast, tables=default.region 2: type=Broadcast, tables=default.supplier -SORT(11) +SORT(15) => Sort Keys: default.region.r_name (TEXT) (asc),s1 (TEXT) (asc) JOIN(9)(LEFT_OUTER) => Join Cond: default.supplier.s_nationkey (INT4) = default.nation.n_nationkey (INT4) @@ -78,21 +78,21 @@ SORT(11) => in schema: {(3) default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} ======================================================= -Block Id: eb_0000000000000_0000_000004 [ROOT] +Block Id: eb_0000000000000_0000_000006 [ROOT] ======================================================= [Incoming] -[q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.region.r_name (TEXT), s1 (TEXT), num=32) +[q_0000000000000_0000] 5 => 6 (type=RANGE_SHUFFLE, key=default.region.r_name (TEXT), s1 (TEXT), num=32) [Enforcers] - 0: sorted input=eb_0000000000000_0000_000003 + 0: sorted input=eb_0000000000000_0000_000005 SORT(5) => Sort Keys: default.region.r_name (TEXT) (asc),s1 (TEXT) (asc) - SCAN(12) on eb_0000000000000_0000_000003 + SCAN(16) on eb_0000000000000_0000_000005 => out schema: {(2) default.region.r_name (TEXT), s1 (TEXT)} => in schema: {(2) default.region.r_name (TEXT), s1 (TEXT)} ======================================================= -Block Id: eb_0000000000000_0000_000005 [TERMINAL] +Block Id: eb_0000000000000_0000_000007 [TERMINAL] ======================================================= diff --git a/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testComplexJoinsWithCaseWhen2.Hash.plan b/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testComplexJoinsWithCaseWhen2.Hash.plan index 483457a6b6..a719990d2d 100644 --- a/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testComplexJoinsWithCaseWhen2.Hash.plan +++ b/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testComplexJoinsWithCaseWhen2.Hash.plan @@ -36,86 +36,57 @@ Execution Block Graph (TERMINAL - eb_0000000000000_0000_000007) |-eb_0000000000000_0000_000007 |-eb_0000000000000_0000_000006 |-eb_0000000000000_0000_000005 - |-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 -3: eb_0000000000000_0000_000005 -4: eb_0000000000000_0000_000006 -5: eb_0000000000000_0000_000007 +1: eb_0000000000000_0000_000005 +2: eb_0000000000000_0000_000006 +3: eb_0000000000000_0000_000007 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [LEAF] +Block Id: eb_0000000000000_0000_000005 [LEAF] ======================================================= [Outgoing] -[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.t.n_regionkey (INT4), num=32) +[q_0000000000000_0000] 5 => 6 (type=RANGE_SHUFFLE, key=default.region.r_name (TEXT), s1 (TEXT), num=32) [Enforcers] 0: type=Broadcast, tables=default.supplier + 1: type=Broadcast, tables=default.region -TABLE_SUBQUERY(5) as default.t - => Targets: CASE WHEN default.t.s_name (TEXT) IS NULL THEN N/O ELSE default.t.s_name (TEXT) END as s1, default.t.n_regionkey (INT4) - => out schema: {(2) s1 (TEXT), default.t.n_regionkey (INT4)} - => in schema: {(11) default.t.n_comment (TEXT), default.t.n_name (TEXT), default.t.n_nationkey (INT4), default.t.n_regionkey (INT4), default.t.s_acctbal (FLOAT8), default.t.s_address (TEXT), default.t.s_comment (TEXT), default.t.s_name (TEXT), default.t.s_nationkey (INT4), default.t.s_phone (TEXT), default.t.s_suppkey (INT4)} - PROJECTION(4) - => Targets: default.nation.n_nationkey (INT4), default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.nation.n_comment (TEXT), default.supplier.s_suppkey (INT4), default.supplier.s_name (TEXT), default.supplier.s_address (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_acctbal (FLOAT8), default.supplier.s_comment (TEXT) - => out schema: {(11) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} - => in schema: {(11) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} - JOIN(10)(LEFT_OUTER) - => Join Cond: default.supplier.s_nationkey (INT4) = default.nation.n_nationkey (INT4) - => target list: default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4) - => out schema: {(11) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} - => in schema: {(11) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} - SCAN(2) on default.supplier - => target list: default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4) - => out schema: {(7) default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} - => in schema: {(7) default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} - SCAN(1) on default.nation - => target list: default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4) - => out schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} - => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000004 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.region.r_regionkey (INT4), num=32) - -SCAN(0) on default.region - => target list: default.region.r_name (TEXT), default.region.r_regionkey (INT4) - => out schema: {(2) default.region.r_name (TEXT), default.region.r_regionkey (INT4)} - => in schema: {(3) default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000005 [INTERMEDIATE] -======================================================= - -[Incoming] -[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.t.n_regionkey (INT4), num=32) -[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.region.r_regionkey (INT4), num=32) - -[Outgoing] -[q_0000000000000_0000] 5 => 6 (type=RANGE_SHUFFLE, key=default.region.r_name (TEXT), s1 (TEXT), num=32) - -SORT(15) +SORT(17) => Sort Keys: default.region.r_name (TEXT) (asc),s1 (TEXT) (asc) JOIN(11)(INNER) => Join Cond: default.t.n_regionkey (INT4) = default.region.r_regionkey (INT4) => target list: default.region.r_name (TEXT), s1 (TEXT) => out schema: {(2) default.region.r_name (TEXT), s1 (TEXT)} => in schema: {(4) default.region.r_name (TEXT), default.region.r_regionkey (INT4), default.t.n_regionkey (INT4), s1 (TEXT)} - SCAN(14) on eb_0000000000000_0000_000004 + SCAN(0) on default.region + => target list: default.region.r_name (TEXT), default.region.r_regionkey (INT4) => out schema: {(2) default.region.r_name (TEXT), default.region.r_regionkey (INT4)} - => in schema: {(2) default.region.r_name (TEXT), default.region.r_regionkey (INT4)} - SCAN(13) on eb_0000000000000_0000_000003 + => in schema: {(3) default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} + TABLE_SUBQUERY(5) as default.t + => Targets: CASE WHEN default.t.s_name (TEXT) IS NULL THEN N/O ELSE default.t.s_name (TEXT) END as s1, default.t.n_regionkey (INT4) => out schema: {(2) s1 (TEXT), default.t.n_regionkey (INT4)} - => in schema: {(2) s1 (TEXT), default.t.n_regionkey (INT4)} + => in schema: {(11) default.t.n_comment (TEXT), default.t.n_name (TEXT), default.t.n_nationkey (INT4), default.t.n_regionkey (INT4), default.t.s_acctbal (FLOAT8), default.t.s_address (TEXT), default.t.s_comment (TEXT), default.t.s_name (TEXT), default.t.s_nationkey (INT4), default.t.s_phone (TEXT), default.t.s_suppkey (INT4)} + PROJECTION(4) + => Targets: default.nation.n_nationkey (INT4), default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.nation.n_comment (TEXT), default.supplier.s_suppkey (INT4), default.supplier.s_name (TEXT), default.supplier.s_address (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_acctbal (FLOAT8), default.supplier.s_comment (TEXT) + => out schema: {(11) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} + => in schema: {(11) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} + JOIN(10)(LEFT_OUTER) + => Join Cond: default.supplier.s_nationkey (INT4) = default.nation.n_nationkey (INT4) + => target list: default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4) + => out schema: {(11) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} + => in schema: {(11) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} + SCAN(2) on default.supplier + => target list: default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4) + => out schema: {(7) default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} + => in schema: {(7) default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} + SCAN(1) on default.nation + => target list: default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4) + => out schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} ======================================================= Block Id: eb_0000000000000_0000_000006 [ROOT] @@ -129,7 +100,7 @@ Block Id: eb_0000000000000_0000_000006 [ROOT] SORT(7) => Sort Keys: default.region.r_name (TEXT) (asc),s1 (TEXT) (asc) - SCAN(16) on eb_0000000000000_0000_000005 + SCAN(18) on eb_0000000000000_0000_000005 => out schema: {(2) default.region.r_name (TEXT), s1 (TEXT)} => in schema: {(2) default.region.r_name (TEXT), s1 (TEXT)} diff --git a/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testComplexJoinsWithCaseWhen2.Sort.plan b/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testComplexJoinsWithCaseWhen2.Sort.plan index 483457a6b6..a719990d2d 100644 --- a/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testComplexJoinsWithCaseWhen2.Sort.plan +++ b/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testComplexJoinsWithCaseWhen2.Sort.plan @@ -36,86 +36,57 @@ Execution Block Graph (TERMINAL - eb_0000000000000_0000_000007) |-eb_0000000000000_0000_000007 |-eb_0000000000000_0000_000006 |-eb_0000000000000_0000_000005 - |-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 -3: eb_0000000000000_0000_000005 -4: eb_0000000000000_0000_000006 -5: eb_0000000000000_0000_000007 +1: eb_0000000000000_0000_000005 +2: eb_0000000000000_0000_000006 +3: eb_0000000000000_0000_000007 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [LEAF] +Block Id: eb_0000000000000_0000_000005 [LEAF] ======================================================= [Outgoing] -[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.t.n_regionkey (INT4), num=32) +[q_0000000000000_0000] 5 => 6 (type=RANGE_SHUFFLE, key=default.region.r_name (TEXT), s1 (TEXT), num=32) [Enforcers] 0: type=Broadcast, tables=default.supplier + 1: type=Broadcast, tables=default.region -TABLE_SUBQUERY(5) as default.t - => Targets: CASE WHEN default.t.s_name (TEXT) IS NULL THEN N/O ELSE default.t.s_name (TEXT) END as s1, default.t.n_regionkey (INT4) - => out schema: {(2) s1 (TEXT), default.t.n_regionkey (INT4)} - => in schema: {(11) default.t.n_comment (TEXT), default.t.n_name (TEXT), default.t.n_nationkey (INT4), default.t.n_regionkey (INT4), default.t.s_acctbal (FLOAT8), default.t.s_address (TEXT), default.t.s_comment (TEXT), default.t.s_name (TEXT), default.t.s_nationkey (INT4), default.t.s_phone (TEXT), default.t.s_suppkey (INT4)} - PROJECTION(4) - => Targets: default.nation.n_nationkey (INT4), default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.nation.n_comment (TEXT), default.supplier.s_suppkey (INT4), default.supplier.s_name (TEXT), default.supplier.s_address (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_acctbal (FLOAT8), default.supplier.s_comment (TEXT) - => out schema: {(11) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} - => in schema: {(11) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} - JOIN(10)(LEFT_OUTER) - => Join Cond: default.supplier.s_nationkey (INT4) = default.nation.n_nationkey (INT4) - => target list: default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4) - => out schema: {(11) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} - => in schema: {(11) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} - SCAN(2) on default.supplier - => target list: default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4) - => out schema: {(7) default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} - => in schema: {(7) default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} - SCAN(1) on default.nation - => target list: default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4) - => out schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} - => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000004 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.region.r_regionkey (INT4), num=32) - -SCAN(0) on default.region - => target list: default.region.r_name (TEXT), default.region.r_regionkey (INT4) - => out schema: {(2) default.region.r_name (TEXT), default.region.r_regionkey (INT4)} - => in schema: {(3) default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000005 [INTERMEDIATE] -======================================================= - -[Incoming] -[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.t.n_regionkey (INT4), num=32) -[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.region.r_regionkey (INT4), num=32) - -[Outgoing] -[q_0000000000000_0000] 5 => 6 (type=RANGE_SHUFFLE, key=default.region.r_name (TEXT), s1 (TEXT), num=32) - -SORT(15) +SORT(17) => Sort Keys: default.region.r_name (TEXT) (asc),s1 (TEXT) (asc) JOIN(11)(INNER) => Join Cond: default.t.n_regionkey (INT4) = default.region.r_regionkey (INT4) => target list: default.region.r_name (TEXT), s1 (TEXT) => out schema: {(2) default.region.r_name (TEXT), s1 (TEXT)} => in schema: {(4) default.region.r_name (TEXT), default.region.r_regionkey (INT4), default.t.n_regionkey (INT4), s1 (TEXT)} - SCAN(14) on eb_0000000000000_0000_000004 + SCAN(0) on default.region + => target list: default.region.r_name (TEXT), default.region.r_regionkey (INT4) => out schema: {(2) default.region.r_name (TEXT), default.region.r_regionkey (INT4)} - => in schema: {(2) default.region.r_name (TEXT), default.region.r_regionkey (INT4)} - SCAN(13) on eb_0000000000000_0000_000003 + => in schema: {(3) default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} + TABLE_SUBQUERY(5) as default.t + => Targets: CASE WHEN default.t.s_name (TEXT) IS NULL THEN N/O ELSE default.t.s_name (TEXT) END as s1, default.t.n_regionkey (INT4) => out schema: {(2) s1 (TEXT), default.t.n_regionkey (INT4)} - => in schema: {(2) s1 (TEXT), default.t.n_regionkey (INT4)} + => in schema: {(11) default.t.n_comment (TEXT), default.t.n_name (TEXT), default.t.n_nationkey (INT4), default.t.n_regionkey (INT4), default.t.s_acctbal (FLOAT8), default.t.s_address (TEXT), default.t.s_comment (TEXT), default.t.s_name (TEXT), default.t.s_nationkey (INT4), default.t.s_phone (TEXT), default.t.s_suppkey (INT4)} + PROJECTION(4) + => Targets: default.nation.n_nationkey (INT4), default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.nation.n_comment (TEXT), default.supplier.s_suppkey (INT4), default.supplier.s_name (TEXT), default.supplier.s_address (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_acctbal (FLOAT8), default.supplier.s_comment (TEXT) + => out schema: {(11) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} + => in schema: {(11) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} + JOIN(10)(LEFT_OUTER) + => Join Cond: default.supplier.s_nationkey (INT4) = default.nation.n_nationkey (INT4) + => target list: default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4) + => out schema: {(11) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} + => in schema: {(11) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} + SCAN(2) on default.supplier + => target list: default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4) + => out schema: {(7) default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} + => in schema: {(7) default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} + SCAN(1) on default.nation + => target list: default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4) + => out schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} ======================================================= Block Id: eb_0000000000000_0000_000006 [ROOT] @@ -129,7 +100,7 @@ Block Id: eb_0000000000000_0000_000006 [ROOT] SORT(7) => Sort Keys: default.region.r_name (TEXT) (asc),s1 (TEXT) (asc) - SCAN(16) on eb_0000000000000_0000_000005 + SCAN(18) on eb_0000000000000_0000_000005 => out schema: {(2) default.region.r_name (TEXT), s1 (TEXT)} => in schema: {(2) default.region.r_name (TEXT), s1 (TEXT)} diff --git a/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testInnerAndOuterWithEmpty.1.Hash.plan b/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testInnerAndOuterWithEmpty.1.Hash.plan index 9721e73e7a..e6ee3dbf61 100644 --- a/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testInnerAndOuterWithEmpty.1.Hash.plan +++ b/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testInnerAndOuterWithEmpty.1.Hash.plan @@ -26,24 +26,24 @@ JOIN(8)(LEFT_OUTER) explain ------------------------------- ------------------------------------------------------------------------------- -Execution Block Graph (TERMINAL - eb_0000000000000_0000_000004) +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000006) ------------------------------------------------------------------------------- -|-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 +|-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000005 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 +1: eb_0000000000000_0000_000005 +2: eb_0000000000000_0000_000006 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [ROOT] +Block Id: eb_0000000000000_0000_000005 [ROOT] ======================================================= [Enforcers] - 0: type=Broadcast, tables=default.b - 1: type=Broadcast, tables=default.a + 0: type=Broadcast, tables=default.a + 1: type=Broadcast, tables=default.b 2: type=Broadcast, tables=default.c JOIN(8)(LEFT_OUTER) @@ -71,5 +71,5 @@ JOIN(8)(LEFT_OUTER) => in schema: {(9) default.b.o_clerk (TEXT), default.b.o_comment (TEXT), default.b.o_custkey (INT4), default.b.o_orderdate (TEXT), default.b.o_orderkey (INT4), default.b.o_orderpriority (TEXT), default.b.o_orderstatus (TEXT), default.b.o_shippriority (INT4), default.b.o_totalprice (FLOAT8)} ======================================================= -Block Id: eb_0000000000000_0000_000004 [TERMINAL] +Block Id: eb_0000000000000_0000_000006 [TERMINAL] ======================================================= diff --git a/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testInnerAndOuterWithEmpty.1.Sort.plan b/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testInnerAndOuterWithEmpty.1.Sort.plan index 9721e73e7a..e6ee3dbf61 100644 --- a/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testInnerAndOuterWithEmpty.1.Sort.plan +++ b/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testInnerAndOuterWithEmpty.1.Sort.plan @@ -26,24 +26,24 @@ JOIN(8)(LEFT_OUTER) explain ------------------------------- ------------------------------------------------------------------------------- -Execution Block Graph (TERMINAL - eb_0000000000000_0000_000004) +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000006) ------------------------------------------------------------------------------- -|-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 +|-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000005 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 +1: eb_0000000000000_0000_000005 +2: eb_0000000000000_0000_000006 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [ROOT] +Block Id: eb_0000000000000_0000_000005 [ROOT] ======================================================= [Enforcers] - 0: type=Broadcast, tables=default.b - 1: type=Broadcast, tables=default.a + 0: type=Broadcast, tables=default.a + 1: type=Broadcast, tables=default.b 2: type=Broadcast, tables=default.c JOIN(8)(LEFT_OUTER) @@ -71,5 +71,5 @@ JOIN(8)(LEFT_OUTER) => in schema: {(9) default.b.o_clerk (TEXT), default.b.o_comment (TEXT), default.b.o_custkey (INT4), default.b.o_orderdate (TEXT), default.b.o_orderkey (INT4), default.b.o_orderpriority (TEXT), default.b.o_orderstatus (TEXT), default.b.o_shippriority (INT4), default.b.o_totalprice (FLOAT8)} ======================================================= -Block Id: eb_0000000000000_0000_000004 [TERMINAL] +Block Id: eb_0000000000000_0000_000006 [TERMINAL] ======================================================= diff --git a/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testJoinWithMultipleJoinTypes.Hash.plan b/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testJoinWithMultipleJoinTypes.Hash.plan index b2021fc76b..2c990f8ea3 100644 --- a/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testJoinWithMultipleJoinTypes.Hash.plan +++ b/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testJoinWithMultipleJoinTypes.Hash.plan @@ -33,107 +33,51 @@ Execution Block Graph (TERMINAL - eb_0000000000000_0000_000006) ------------------------------------------------------------------------------- |-eb_0000000000000_0000_000006 |-eb_0000000000000_0000_000005 - |-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 - |-eb_0000000000000_0000_000002 - |-eb_0000000000000_0000_000001 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000001 -2: eb_0000000000000_0000_000002 -3: eb_0000000000000_0000_000003 -4: eb_0000000000000_0000_000004 -5: eb_0000000000000_0000_000005 -6: eb_0000000000000_0000_000006 +1: eb_0000000000000_0000_000005 +2: eb_0000000000000_0000_000006 ------------------------------------------------------------------------------- -======================================================= -Block Id: eb_0000000000000_0000_000001 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.c.c_custkey (INT4), num=32) - -SCAN(0) on default.customer as c - => target list: default.c.c_acctbal (FLOAT8), default.c.c_address (TEXT), default.c.c_comment (TEXT), default.c.c_custkey (INT4), default.c.c_mktsegment (TEXT), default.c.c_name (TEXT), default.c.c_nationkey (INT4), default.c.c_phone (TEXT) - => out schema: {(8) default.c.c_acctbal (FLOAT8), default.c.c_address (TEXT), default.c.c_comment (TEXT), default.c.c_custkey (INT4), default.c.c_mktsegment (TEXT), default.c.c_name (TEXT), default.c.c_nationkey (INT4), default.c.c_phone (TEXT)} - => in schema: {(8) default.c.c_acctbal (FLOAT8), default.c.c_address (TEXT), default.c.c_comment (TEXT), default.c.c_custkey (INT4), default.c.c_mktsegment (TEXT), default.c.c_name (TEXT), default.c.c_nationkey (INT4), default.c.c_phone (TEXT)} - -======================================================= -Block Id: eb_0000000000000_0000_000002 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.n.n_nationkey (INT4), num=32) - -TABLE_SUBQUERY(3) as default.n - => Targets: default.n.n_nationkey (INT4) - => out schema: {(1) default.n.n_nationkey (INT4)} - => in schema: {(1) default.n.n_nationkey (INT4)} - PROJECTION(2) - => Targets: default.nation.n_nationkey (INT4) - => out schema: {(1) default.nation.n_nationkey (INT4)} - => in schema: {(1) default.nation.n_nationkey (INT4)} - SCAN(1) on default.nation - => target list: default.nation.n_nationkey (INT4) - => out schema: {(1) default.nation.n_nationkey (INT4)} - => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000003 [INTERMEDIATE] -======================================================= - -[Incoming] -[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.c.c_custkey (INT4), num=32) -[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.n.n_nationkey (INT4), num=32) - -[Outgoing] -[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.c.c_custkey (INT4), num=32) - -JOIN(9)(RIGHT_OUTER) - => Join Cond: default.n.n_nationkey (INT4) = default.c.c_custkey (INT4) - => target list: default.c.c_acctbal (FLOAT8), default.c.c_address (TEXT), default.c.c_comment (TEXT), default.c.c_custkey (INT4), default.c.c_mktsegment (TEXT), default.c.c_name (TEXT), default.c.c_nationkey (INT4), default.c.c_phone (TEXT), default.n.n_nationkey (INT4) - => out schema: {(9) default.c.c_acctbal (FLOAT8), default.c.c_address (TEXT), default.c.c_comment (TEXT), default.c.c_custkey (INT4), default.c.c_mktsegment (TEXT), default.c.c_name (TEXT), default.c.c_nationkey (INT4), default.c.c_phone (TEXT), default.n.n_nationkey (INT4)} - => in schema: {(9) default.c.c_acctbal (FLOAT8), default.c.c_address (TEXT), default.c.c_comment (TEXT), default.c.c_custkey (INT4), default.c.c_mktsegment (TEXT), default.c.c_name (TEXT), default.c.c_nationkey (INT4), default.c.c_phone (TEXT), default.n.n_nationkey (INT4)} - SCAN(13) on eb_0000000000000_0000_000002 - => out schema: {(1) default.n.n_nationkey (INT4)} - => in schema: {(1) default.n.n_nationkey (INT4)} - SCAN(12) on eb_0000000000000_0000_000001 - => out schema: {(8) default.c.c_acctbal (FLOAT8), default.c.c_address (TEXT), default.c.c_comment (TEXT), default.c.c_custkey (INT4), default.c.c_mktsegment (TEXT), default.c.c_name (TEXT), default.c.c_nationkey (INT4), default.c.c_phone (TEXT)} - => in schema: {(8) default.c.c_acctbal (FLOAT8), default.c.c_address (TEXT), default.c.c_comment (TEXT), default.c.c_custkey (INT4), default.c.c_mktsegment (TEXT), default.c.c_name (TEXT), default.c.c_nationkey (INT4), default.c.c_phone (TEXT)} - -======================================================= -Block Id: eb_0000000000000_0000_000004 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.r.r_regionkey (INT4), num=32) - -SCAN(5) on default.region as r - => target list: default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4) - => out schema: {(3) default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4)} - => in schema: {(3) default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4)} - ======================================================= Block Id: eb_0000000000000_0000_000005 [ROOT] ======================================================= -[Incoming] -[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.c.c_custkey (INT4), num=32) -[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.r.r_regionkey (INT4), num=32) +[Enforcers] + 0: type=Broadcast, tables=default.c + 1: type=Broadcast, tables=default.r JOIN(10)(INNER) => Join Cond: default.r.r_regionkey (INT4) = default.c.c_custkey (INT4) => target list: default.c.c_acctbal (FLOAT8), default.c.c_address (TEXT), default.c.c_comment (TEXT), default.c.c_custkey (INT4), default.c.c_mktsegment (TEXT), default.c.c_name (TEXT), default.c.c_nationkey (INT4), default.c.c_phone (TEXT), default.n.n_nationkey (INT4), default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4) => out schema: {(12) default.c.c_acctbal (FLOAT8), default.c.c_address (TEXT), default.c.c_comment (TEXT), default.c.c_custkey (INT4), default.c.c_mktsegment (TEXT), default.c.c_name (TEXT), default.c.c_nationkey (INT4), default.c.c_phone (TEXT), default.n.n_nationkey (INT4), default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4)} => in schema: {(12) default.c.c_acctbal (FLOAT8), default.c.c_address (TEXT), default.c.c_comment (TEXT), default.c.c_custkey (INT4), default.c.c_mktsegment (TEXT), default.c.c_name (TEXT), default.c.c_nationkey (INT4), default.c.c_phone (TEXT), default.n.n_nationkey (INT4), default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4)} - SCAN(15) on eb_0000000000000_0000_000004 + SCAN(5) on default.region as r + => target list: default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4) => out schema: {(3) default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4)} => in schema: {(3) default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4)} - SCAN(14) on eb_0000000000000_0000_000003 + JOIN(9)(RIGHT_OUTER) + => Join Cond: default.n.n_nationkey (INT4) = default.c.c_custkey (INT4) + => target list: default.c.c_acctbal (FLOAT8), default.c.c_address (TEXT), default.c.c_comment (TEXT), default.c.c_custkey (INT4), default.c.c_mktsegment (TEXT), default.c.c_name (TEXT), default.c.c_nationkey (INT4), default.c.c_phone (TEXT), default.n.n_nationkey (INT4) => out schema: {(9) default.c.c_acctbal (FLOAT8), default.c.c_address (TEXT), default.c.c_comment (TEXT), default.c.c_custkey (INT4), default.c.c_mktsegment (TEXT), default.c.c_name (TEXT), default.c.c_nationkey (INT4), default.c.c_phone (TEXT), default.n.n_nationkey (INT4)} => in schema: {(9) default.c.c_acctbal (FLOAT8), default.c.c_address (TEXT), default.c.c_comment (TEXT), default.c.c_custkey (INT4), default.c.c_mktsegment (TEXT), default.c.c_name (TEXT), default.c.c_nationkey (INT4), default.c.c_phone (TEXT), default.n.n_nationkey (INT4)} + TABLE_SUBQUERY(3) as default.n + => Targets: default.n.n_nationkey (INT4) + => out schema: {(1) default.n.n_nationkey (INT4)} + => in schema: {(1) default.n.n_nationkey (INT4)} + PROJECTION(2) + => Targets: default.nation.n_nationkey (INT4) + => out schema: {(1) default.nation.n_nationkey (INT4)} + => in schema: {(1) default.nation.n_nationkey (INT4)} + SCAN(1) on default.nation + => target list: default.nation.n_nationkey (INT4) + => out schema: {(1) default.nation.n_nationkey (INT4)} + => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + SCAN(0) on default.customer as c + => target list: default.c.c_acctbal (FLOAT8), default.c.c_address (TEXT), default.c.c_comment (TEXT), default.c.c_custkey (INT4), default.c.c_mktsegment (TEXT), default.c.c_name (TEXT), default.c.c_nationkey (INT4), default.c.c_phone (TEXT) + => out schema: {(8) default.c.c_acctbal (FLOAT8), default.c.c_address (TEXT), default.c.c_comment (TEXT), default.c.c_custkey (INT4), default.c.c_mktsegment (TEXT), default.c.c_name (TEXT), default.c.c_nationkey (INT4), default.c.c_phone (TEXT)} + => in schema: {(8) default.c.c_acctbal (FLOAT8), default.c.c_address (TEXT), default.c.c_comment (TEXT), default.c.c_custkey (INT4), default.c.c_mktsegment (TEXT), default.c.c_name (TEXT), default.c.c_nationkey (INT4), default.c.c_phone (TEXT)} ======================================================= Block Id: eb_0000000000000_0000_000006 [TERMINAL] diff --git a/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testJoinWithMultipleJoinTypes.Sort.plan b/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testJoinWithMultipleJoinTypes.Sort.plan index b2021fc76b..2c990f8ea3 100644 --- a/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testJoinWithMultipleJoinTypes.Sort.plan +++ b/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testJoinWithMultipleJoinTypes.Sort.plan @@ -33,107 +33,51 @@ Execution Block Graph (TERMINAL - eb_0000000000000_0000_000006) ------------------------------------------------------------------------------- |-eb_0000000000000_0000_000006 |-eb_0000000000000_0000_000005 - |-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 - |-eb_0000000000000_0000_000002 - |-eb_0000000000000_0000_000001 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000001 -2: eb_0000000000000_0000_000002 -3: eb_0000000000000_0000_000003 -4: eb_0000000000000_0000_000004 -5: eb_0000000000000_0000_000005 -6: eb_0000000000000_0000_000006 +1: eb_0000000000000_0000_000005 +2: eb_0000000000000_0000_000006 ------------------------------------------------------------------------------- -======================================================= -Block Id: eb_0000000000000_0000_000001 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.c.c_custkey (INT4), num=32) - -SCAN(0) on default.customer as c - => target list: default.c.c_acctbal (FLOAT8), default.c.c_address (TEXT), default.c.c_comment (TEXT), default.c.c_custkey (INT4), default.c.c_mktsegment (TEXT), default.c.c_name (TEXT), default.c.c_nationkey (INT4), default.c.c_phone (TEXT) - => out schema: {(8) default.c.c_acctbal (FLOAT8), default.c.c_address (TEXT), default.c.c_comment (TEXT), default.c.c_custkey (INT4), default.c.c_mktsegment (TEXT), default.c.c_name (TEXT), default.c.c_nationkey (INT4), default.c.c_phone (TEXT)} - => in schema: {(8) default.c.c_acctbal (FLOAT8), default.c.c_address (TEXT), default.c.c_comment (TEXT), default.c.c_custkey (INT4), default.c.c_mktsegment (TEXT), default.c.c_name (TEXT), default.c.c_nationkey (INT4), default.c.c_phone (TEXT)} - -======================================================= -Block Id: eb_0000000000000_0000_000002 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.n.n_nationkey (INT4), num=32) - -TABLE_SUBQUERY(3) as default.n - => Targets: default.n.n_nationkey (INT4) - => out schema: {(1) default.n.n_nationkey (INT4)} - => in schema: {(1) default.n.n_nationkey (INT4)} - PROJECTION(2) - => Targets: default.nation.n_nationkey (INT4) - => out schema: {(1) default.nation.n_nationkey (INT4)} - => in schema: {(1) default.nation.n_nationkey (INT4)} - SCAN(1) on default.nation - => target list: default.nation.n_nationkey (INT4) - => out schema: {(1) default.nation.n_nationkey (INT4)} - => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} - -======================================================= -Block Id: eb_0000000000000_0000_000003 [INTERMEDIATE] -======================================================= - -[Incoming] -[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.c.c_custkey (INT4), num=32) -[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.n.n_nationkey (INT4), num=32) - -[Outgoing] -[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.c.c_custkey (INT4), num=32) - -JOIN(9)(RIGHT_OUTER) - => Join Cond: default.n.n_nationkey (INT4) = default.c.c_custkey (INT4) - => target list: default.c.c_acctbal (FLOAT8), default.c.c_address (TEXT), default.c.c_comment (TEXT), default.c.c_custkey (INT4), default.c.c_mktsegment (TEXT), default.c.c_name (TEXT), default.c.c_nationkey (INT4), default.c.c_phone (TEXT), default.n.n_nationkey (INT4) - => out schema: {(9) default.c.c_acctbal (FLOAT8), default.c.c_address (TEXT), default.c.c_comment (TEXT), default.c.c_custkey (INT4), default.c.c_mktsegment (TEXT), default.c.c_name (TEXT), default.c.c_nationkey (INT4), default.c.c_phone (TEXT), default.n.n_nationkey (INT4)} - => in schema: {(9) default.c.c_acctbal (FLOAT8), default.c.c_address (TEXT), default.c.c_comment (TEXT), default.c.c_custkey (INT4), default.c.c_mktsegment (TEXT), default.c.c_name (TEXT), default.c.c_nationkey (INT4), default.c.c_phone (TEXT), default.n.n_nationkey (INT4)} - SCAN(13) on eb_0000000000000_0000_000002 - => out schema: {(1) default.n.n_nationkey (INT4)} - => in schema: {(1) default.n.n_nationkey (INT4)} - SCAN(12) on eb_0000000000000_0000_000001 - => out schema: {(8) default.c.c_acctbal (FLOAT8), default.c.c_address (TEXT), default.c.c_comment (TEXT), default.c.c_custkey (INT4), default.c.c_mktsegment (TEXT), default.c.c_name (TEXT), default.c.c_nationkey (INT4), default.c.c_phone (TEXT)} - => in schema: {(8) default.c.c_acctbal (FLOAT8), default.c.c_address (TEXT), default.c.c_comment (TEXT), default.c.c_custkey (INT4), default.c.c_mktsegment (TEXT), default.c.c_name (TEXT), default.c.c_nationkey (INT4), default.c.c_phone (TEXT)} - -======================================================= -Block Id: eb_0000000000000_0000_000004 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.r.r_regionkey (INT4), num=32) - -SCAN(5) on default.region as r - => target list: default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4) - => out schema: {(3) default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4)} - => in schema: {(3) default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4)} - ======================================================= Block Id: eb_0000000000000_0000_000005 [ROOT] ======================================================= -[Incoming] -[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.c.c_custkey (INT4), num=32) -[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.r.r_regionkey (INT4), num=32) +[Enforcers] + 0: type=Broadcast, tables=default.c + 1: type=Broadcast, tables=default.r JOIN(10)(INNER) => Join Cond: default.r.r_regionkey (INT4) = default.c.c_custkey (INT4) => target list: default.c.c_acctbal (FLOAT8), default.c.c_address (TEXT), default.c.c_comment (TEXT), default.c.c_custkey (INT4), default.c.c_mktsegment (TEXT), default.c.c_name (TEXT), default.c.c_nationkey (INT4), default.c.c_phone (TEXT), default.n.n_nationkey (INT4), default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4) => out schema: {(12) default.c.c_acctbal (FLOAT8), default.c.c_address (TEXT), default.c.c_comment (TEXT), default.c.c_custkey (INT4), default.c.c_mktsegment (TEXT), default.c.c_name (TEXT), default.c.c_nationkey (INT4), default.c.c_phone (TEXT), default.n.n_nationkey (INT4), default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4)} => in schema: {(12) default.c.c_acctbal (FLOAT8), default.c.c_address (TEXT), default.c.c_comment (TEXT), default.c.c_custkey (INT4), default.c.c_mktsegment (TEXT), default.c.c_name (TEXT), default.c.c_nationkey (INT4), default.c.c_phone (TEXT), default.n.n_nationkey (INT4), default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4)} - SCAN(15) on eb_0000000000000_0000_000004 + SCAN(5) on default.region as r + => target list: default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4) => out schema: {(3) default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4)} => in schema: {(3) default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4)} - SCAN(14) on eb_0000000000000_0000_000003 + JOIN(9)(RIGHT_OUTER) + => Join Cond: default.n.n_nationkey (INT4) = default.c.c_custkey (INT4) + => target list: default.c.c_acctbal (FLOAT8), default.c.c_address (TEXT), default.c.c_comment (TEXT), default.c.c_custkey (INT4), default.c.c_mktsegment (TEXT), default.c.c_name (TEXT), default.c.c_nationkey (INT4), default.c.c_phone (TEXT), default.n.n_nationkey (INT4) => out schema: {(9) default.c.c_acctbal (FLOAT8), default.c.c_address (TEXT), default.c.c_comment (TEXT), default.c.c_custkey (INT4), default.c.c_mktsegment (TEXT), default.c.c_name (TEXT), default.c.c_nationkey (INT4), default.c.c_phone (TEXT), default.n.n_nationkey (INT4)} => in schema: {(9) default.c.c_acctbal (FLOAT8), default.c.c_address (TEXT), default.c.c_comment (TEXT), default.c.c_custkey (INT4), default.c.c_mktsegment (TEXT), default.c.c_name (TEXT), default.c.c_nationkey (INT4), default.c.c_phone (TEXT), default.n.n_nationkey (INT4)} + TABLE_SUBQUERY(3) as default.n + => Targets: default.n.n_nationkey (INT4) + => out schema: {(1) default.n.n_nationkey (INT4)} + => in schema: {(1) default.n.n_nationkey (INT4)} + PROJECTION(2) + => Targets: default.nation.n_nationkey (INT4) + => out schema: {(1) default.nation.n_nationkey (INT4)} + => in schema: {(1) default.nation.n_nationkey (INT4)} + SCAN(1) on default.nation + => target list: default.nation.n_nationkey (INT4) + => out schema: {(1) default.nation.n_nationkey (INT4)} + => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + SCAN(0) on default.customer as c + => target list: default.c.c_acctbal (FLOAT8), default.c.c_address (TEXT), default.c.c_comment (TEXT), default.c.c_custkey (INT4), default.c.c_mktsegment (TEXT), default.c.c_name (TEXT), default.c.c_nationkey (INT4), default.c.c_phone (TEXT) + => out schema: {(8) default.c.c_acctbal (FLOAT8), default.c.c_address (TEXT), default.c.c_comment (TEXT), default.c.c_custkey (INT4), default.c.c_mktsegment (TEXT), default.c.c_name (TEXT), default.c.c_nationkey (INT4), default.c.c_phone (TEXT)} + => in schema: {(8) default.c.c_acctbal (FLOAT8), default.c.c_address (TEXT), default.c.c_comment (TEXT), default.c.c_custkey (INT4), default.c.c_mktsegment (TEXT), default.c.c_name (TEXT), default.c.c_nationkey (INT4), default.c.c_phone (TEXT)} ======================================================= Block Id: eb_0000000000000_0000_000006 [TERMINAL] diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testJoinFilterOfRowPreservedTable1.Hash.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testJoinFilterOfRowPreservedTable1.Hash.plan index 21e7d4f4ce..91199f65d4 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testJoinFilterOfRowPreservedTable1.Hash.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testJoinFilterOfRowPreservedTable1.Hash.plan @@ -41,7 +41,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.nation -SORT(8) +SORT(10) => Sort Keys: default.region.r_name (TEXT) (asc),default.nation.n_name (TEXT) (asc) JOIN(6)(LEFT_OUTER) => Join Cond: default.nation.n_regionkey (INT4) = default.region.r_regionkey (INT4) AND default.region.r_name (TEXT) IN (AMERICA, ASIA) @@ -69,7 +69,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.region.r_name (TEXT) (asc),default.nation.n_name (TEXT) (asc) - SCAN(9) on eb_0000000000000_0000_000003 + SCAN(11) on eb_0000000000000_0000_000003 => out schema: {(4) default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} => in schema: {(4) default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testJoinFilterOfRowPreservedTable1.Sort.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testJoinFilterOfRowPreservedTable1.Sort.plan index 21e7d4f4ce..91199f65d4 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testJoinFilterOfRowPreservedTable1.Sort.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testJoinFilterOfRowPreservedTable1.Sort.plan @@ -41,7 +41,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.nation -SORT(8) +SORT(10) => Sort Keys: default.region.r_name (TEXT) (asc),default.nation.n_name (TEXT) (asc) JOIN(6)(LEFT_OUTER) => Join Cond: default.nation.n_regionkey (INT4) = default.region.r_regionkey (INT4) AND default.region.r_name (TEXT) IN (AMERICA, ASIA) @@ -69,7 +69,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.region.r_name (TEXT) (asc),default.nation.n_name (TEXT) (asc) - SCAN(9) on eb_0000000000000_0000_000003 + SCAN(11) on eb_0000000000000_0000_000003 => out schema: {(4) default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} => in schema: {(4) default.nation.n_name (TEXT), default.nation.n_regionkey (INT4), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoin1.Hash.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoin1.Hash.plan index 539f3da890..ae407969c9 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoin1.Hash.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoin1.Hash.plan @@ -41,7 +41,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.orders -SORT(8) +SORT(10) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.orders.o_orderkey (INT4) (asc) JOIN(6)(LEFT_OUTER) => Join Cond: default.customer.c_custkey (INT4) = default.orders.o_orderkey (INT4) @@ -69,7 +69,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.orders.o_orderkey (INT4) (asc) - SCAN(9) on eb_0000000000000_0000_000003 + SCAN(11) on eb_0000000000000_0000_000003 => out schema: {(4) default.customer.c_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4), default.orders.o_orderstatus (TEXT)} => in schema: {(4) default.customer.c_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4), default.orders.o_orderstatus (TEXT)} diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoin1.Sort.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoin1.Sort.plan index 539f3da890..ae407969c9 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoin1.Sort.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoin1.Sort.plan @@ -41,7 +41,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.orders -SORT(8) +SORT(10) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.orders.o_orderkey (INT4) (asc) JOIN(6)(LEFT_OUTER) => Join Cond: default.customer.c_custkey (INT4) = default.orders.o_orderkey (INT4) @@ -69,7 +69,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.orders.o_orderkey (INT4) (asc) - SCAN(9) on eb_0000000000000_0000_000003 + SCAN(11) on eb_0000000000000_0000_000003 => out schema: {(4) default.customer.c_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4), default.orders.o_orderstatus (TEXT)} => in schema: {(4) default.customer.c_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4), default.orders.o_orderstatus (TEXT)} diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoin2.Hash.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoin2.Hash.plan index 3ebc3e4835..997d761e5f 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoin2.Hash.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoin2.Hash.plan @@ -34,19 +34,19 @@ JOIN(11)(LEFT_OUTER) explain ------------------------------- ------------------------------------------------------------------------------- -Execution Block Graph (TERMINAL - eb_0000000000000_0000_000004) +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000008) ------------------------------------------------------------------------------- -|-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 +|-eb_0000000000000_0000_000008 + |-eb_0000000000000_0000_000007 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 +1: eb_0000000000000_0000_000007 +2: eb_0000000000000_0000_000008 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [ROOT] +Block Id: eb_0000000000000_0000_000007 [ROOT] ======================================================= [Enforcers] @@ -87,5 +87,5 @@ JOIN(11)(LEFT_OUTER) => in schema: {(16) default.a.l_comment (TEXT), default.a.l_commitdate (TEXT), default.a.l_discount (FLOAT8), default.a.l_extendedprice (FLOAT8), default.a.l_linenumber (INT4), default.a.l_linestatus (TEXT), default.a.l_orderkey (INT4), default.a.l_partkey (INT4), default.a.l_quantity (FLOAT8), default.a.l_receiptdate (TEXT), default.a.l_returnflag (TEXT), default.a.l_shipdate (TEXT), default.a.l_shipinstruct (TEXT), default.a.l_shipmode (TEXT), default.a.l_suppkey (INT4), default.a.l_tax (FLOAT8)} ======================================================= -Block Id: eb_0000000000000_0000_000004 [TERMINAL] +Block Id: eb_0000000000000_0000_000008 [TERMINAL] ======================================================= diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoin2.Sort.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoin2.Sort.plan index 3ebc3e4835..997d761e5f 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoin2.Sort.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoin2.Sort.plan @@ -34,19 +34,19 @@ JOIN(11)(LEFT_OUTER) explain ------------------------------- ------------------------------------------------------------------------------- -Execution Block Graph (TERMINAL - eb_0000000000000_0000_000004) +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000008) ------------------------------------------------------------------------------- -|-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 +|-eb_0000000000000_0000_000008 + |-eb_0000000000000_0000_000007 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 +1: eb_0000000000000_0000_000007 +2: eb_0000000000000_0000_000008 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [ROOT] +Block Id: eb_0000000000000_0000_000007 [ROOT] ======================================================= [Enforcers] @@ -87,5 +87,5 @@ JOIN(11)(LEFT_OUTER) => in schema: {(16) default.a.l_comment (TEXT), default.a.l_commitdate (TEXT), default.a.l_discount (FLOAT8), default.a.l_extendedprice (FLOAT8), default.a.l_linenumber (INT4), default.a.l_linestatus (TEXT), default.a.l_orderkey (INT4), default.a.l_partkey (INT4), default.a.l_quantity (FLOAT8), default.a.l_receiptdate (TEXT), default.a.l_returnflag (TEXT), default.a.l_shipdate (TEXT), default.a.l_shipinstruct (TEXT), default.a.l_shipmode (TEXT), default.a.l_suppkey (INT4), default.a.l_tax (FLOAT8)} ======================================================= -Block Id: eb_0000000000000_0000_000004 [TERMINAL] +Block Id: eb_0000000000000_0000_000008 [TERMINAL] ======================================================= diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoin3.Hash.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoin3.Hash.plan index 1a8f3b5425..2418b98008 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoin3.Hash.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoin3.Hash.plan @@ -52,26 +52,26 @@ JOIN(17)(LEFT_OUTER) explain ------------------------------- ------------------------------------------------------------------------------- -Execution Block Graph (TERMINAL - eb_0000000000000_0000_000004) +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000012) ------------------------------------------------------------------------------- -|-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 +|-eb_0000000000000_0000_000012 + |-eb_0000000000000_0000_000011 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 +1: eb_0000000000000_0000_000011 +2: eb_0000000000000_0000_000012 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [ROOT] +Block Id: eb_0000000000000_0000_000011 [ROOT] ======================================================= [Enforcers] - 0: type=Broadcast, tables=default.b - 1: type=Broadcast, tables=default.c - 2: type=Broadcast, tables=default.d - 3: type=Broadcast, tables=default.e + 0: type=Broadcast, tables=default.e + 1: type=Broadcast, tables=default.b + 2: type=Broadcast, tables=default.c + 3: type=Broadcast, tables=default.d 4: type=Broadcast, tables=default.f JOIN(17)(LEFT_OUTER) @@ -125,5 +125,5 @@ JOIN(17)(LEFT_OUTER) => in schema: {(8) default.a.c_acctbal (FLOAT8), default.a.c_address (TEXT), default.a.c_comment (TEXT), default.a.c_custkey (INT4), default.a.c_mktsegment (TEXT), default.a.c_name (TEXT), default.a.c_nationkey (INT4), default.a.c_phone (TEXT)} ======================================================= -Block Id: eb_0000000000000_0000_000004 [TERMINAL] +Block Id: eb_0000000000000_0000_000012 [TERMINAL] ======================================================= diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoin3.Sort.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoin3.Sort.plan index 1a8f3b5425..2418b98008 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoin3.Sort.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoin3.Sort.plan @@ -52,26 +52,26 @@ JOIN(17)(LEFT_OUTER) explain ------------------------------- ------------------------------------------------------------------------------- -Execution Block Graph (TERMINAL - eb_0000000000000_0000_000004) +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000012) ------------------------------------------------------------------------------- -|-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 +|-eb_0000000000000_0000_000012 + |-eb_0000000000000_0000_000011 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 +1: eb_0000000000000_0000_000011 +2: eb_0000000000000_0000_000012 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [ROOT] +Block Id: eb_0000000000000_0000_000011 [ROOT] ======================================================= [Enforcers] - 0: type=Broadcast, tables=default.b - 1: type=Broadcast, tables=default.c - 2: type=Broadcast, tables=default.d - 3: type=Broadcast, tables=default.e + 0: type=Broadcast, tables=default.e + 1: type=Broadcast, tables=default.b + 2: type=Broadcast, tables=default.c + 3: type=Broadcast, tables=default.d 4: type=Broadcast, tables=default.f JOIN(17)(LEFT_OUTER) @@ -125,5 +125,5 @@ JOIN(17)(LEFT_OUTER) => in schema: {(8) default.a.c_acctbal (FLOAT8), default.a.c_address (TEXT), default.a.c_comment (TEXT), default.a.c_custkey (INT4), default.a.c_mktsegment (TEXT), default.a.c_name (TEXT), default.a.c_nationkey (INT4), default.a.c_phone (TEXT)} ======================================================= -Block Id: eb_0000000000000_0000_000004 [TERMINAL] +Block Id: eb_0000000000000_0000_000012 [TERMINAL] ======================================================= diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase1.1.Hash.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase1.1.Hash.plan index b6064cc57f..ea058ca738 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase1.1.Hash.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase1.1.Hash.plan @@ -25,19 +25,19 @@ JOIN(8)(LEFT_OUTER) explain ------------------------------- ------------------------------------------------------------------------------- -Execution Block Graph (TERMINAL - eb_0000000000000_0000_000004) +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000006) ------------------------------------------------------------------------------- -|-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 +|-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000005 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 +1: eb_0000000000000_0000_000005 +2: eb_0000000000000_0000_000006 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [ROOT] +Block Id: eb_0000000000000_0000_000005 [ROOT] ======================================================= [Enforcers] @@ -68,5 +68,5 @@ JOIN(8)(LEFT_OUTER) => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} ======================================================= -Block Id: eb_0000000000000_0000_000004 [TERMINAL] +Block Id: eb_0000000000000_0000_000006 [TERMINAL] ======================================================= diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase1.1.Sort.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase1.1.Sort.plan index b6064cc57f..ea058ca738 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase1.1.Sort.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase1.1.Sort.plan @@ -25,19 +25,19 @@ JOIN(8)(LEFT_OUTER) explain ------------------------------- ------------------------------------------------------------------------------- -Execution Block Graph (TERMINAL - eb_0000000000000_0000_000004) +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000006) ------------------------------------------------------------------------------- -|-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 +|-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000005 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 +1: eb_0000000000000_0000_000005 +2: eb_0000000000000_0000_000006 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [ROOT] +Block Id: eb_0000000000000_0000_000005 [ROOT] ======================================================= [Enforcers] @@ -68,5 +68,5 @@ JOIN(8)(LEFT_OUTER) => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} ======================================================= -Block Id: eb_0000000000000_0000_000004 [TERMINAL] +Block Id: eb_0000000000000_0000_000006 [TERMINAL] ======================================================= diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase2.1.Hash.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase2.1.Hash.plan index 0a22badee4..f5e05867a5 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase2.1.Hash.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase2.1.Hash.plan @@ -34,24 +34,24 @@ JOIN(11)(INNER) explain ------------------------------- ------------------------------------------------------------------------------- -Execution Block Graph (TERMINAL - eb_0000000000000_0000_000004) +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000008) ------------------------------------------------------------------------------- -|-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 +|-eb_0000000000000_0000_000008 + |-eb_0000000000000_0000_000007 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 +1: eb_0000000000000_0000_000007 +2: eb_0000000000000_0000_000008 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [ROOT] +Block Id: eb_0000000000000_0000_000007 [ROOT] ======================================================= [Enforcers] - 0: type=Broadcast, tables=default.t2 - 1: type=Broadcast, tables=default.t3 + 0: type=Broadcast, tables=default.t3 + 1: type=Broadcast, tables=default.t2 2: type=Broadcast, tables=default.t4 JOIN(11)(INNER) @@ -87,5 +87,5 @@ JOIN(11)(INNER) => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} ======================================================= -Block Id: eb_0000000000000_0000_000004 [TERMINAL] +Block Id: eb_0000000000000_0000_000008 [TERMINAL] ======================================================= diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase2.1.Sort.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase2.1.Sort.plan index 0a22badee4..f5e05867a5 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase2.1.Sort.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase2.1.Sort.plan @@ -34,24 +34,24 @@ JOIN(11)(INNER) explain ------------------------------- ------------------------------------------------------------------------------- -Execution Block Graph (TERMINAL - eb_0000000000000_0000_000004) +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000008) ------------------------------------------------------------------------------- -|-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 +|-eb_0000000000000_0000_000008 + |-eb_0000000000000_0000_000007 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 +1: eb_0000000000000_0000_000007 +2: eb_0000000000000_0000_000008 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [ROOT] +Block Id: eb_0000000000000_0000_000007 [ROOT] ======================================================= [Enforcers] - 0: type=Broadcast, tables=default.t2 - 1: type=Broadcast, tables=default.t3 + 0: type=Broadcast, tables=default.t3 + 1: type=Broadcast, tables=default.t2 2: type=Broadcast, tables=default.t4 JOIN(11)(INNER) @@ -87,5 +87,5 @@ JOIN(11)(INNER) => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} ======================================================= -Block Id: eb_0000000000000_0000_000004 [TERMINAL] +Block Id: eb_0000000000000_0000_000008 [TERMINAL] ======================================================= diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase2_1.1.Hash.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase2_1.1.Hash.plan index ab1b487fad..88fce6bb9b 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase2_1.1.Hash.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase2_1.1.Hash.plan @@ -37,25 +37,25 @@ JOIN(12)(LEFT_OUTER) explain ------------------------------- ------------------------------------------------------------------------------- -Execution Block Graph (TERMINAL - eb_0000000000000_0000_000004) +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000008) ------------------------------------------------------------------------------- -|-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 +|-eb_0000000000000_0000_000008 + |-eb_0000000000000_0000_000007 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 +1: eb_0000000000000_0000_000007 +2: eb_0000000000000_0000_000008 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [ROOT] +Block Id: eb_0000000000000_0000_000007 [ROOT] ======================================================= [Enforcers] - 0: type=Broadcast, tables=default.t1 - 1: type=Broadcast, tables=default.t4 - 2: type=Broadcast, tables=default.t3 + 0: type=Broadcast, tables=default.t4 + 1: type=Broadcast, tables=default.t3 + 2: type=Broadcast, tables=default.t1 3: type=Broadcast, tables=default.t2 JOIN(12)(LEFT_OUTER) @@ -94,5 +94,5 @@ JOIN(12)(LEFT_OUTER) => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} ======================================================= -Block Id: eb_0000000000000_0000_000004 [TERMINAL] +Block Id: eb_0000000000000_0000_000008 [TERMINAL] ======================================================= diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase2_1.1.Sort.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase2_1.1.Sort.plan index ab1b487fad..88fce6bb9b 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase2_1.1.Sort.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase2_1.1.Sort.plan @@ -37,25 +37,25 @@ JOIN(12)(LEFT_OUTER) explain ------------------------------- ------------------------------------------------------------------------------- -Execution Block Graph (TERMINAL - eb_0000000000000_0000_000004) +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000008) ------------------------------------------------------------------------------- -|-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 +|-eb_0000000000000_0000_000008 + |-eb_0000000000000_0000_000007 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 +1: eb_0000000000000_0000_000007 +2: eb_0000000000000_0000_000008 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [ROOT] +Block Id: eb_0000000000000_0000_000007 [ROOT] ======================================================= [Enforcers] - 0: type=Broadcast, tables=default.t1 - 1: type=Broadcast, tables=default.t4 - 2: type=Broadcast, tables=default.t3 + 0: type=Broadcast, tables=default.t4 + 1: type=Broadcast, tables=default.t3 + 2: type=Broadcast, tables=default.t1 3: type=Broadcast, tables=default.t2 JOIN(12)(LEFT_OUTER) @@ -94,5 +94,5 @@ JOIN(12)(LEFT_OUTER) => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} ======================================================= -Block Id: eb_0000000000000_0000_000004 [TERMINAL] +Block Id: eb_0000000000000_0000_000008 [TERMINAL] ======================================================= diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase3.1.Hash.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase3.1.Hash.plan index 50c41c9dcc..79ce689762 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase3.1.Hash.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase3.1.Hash.plan @@ -25,19 +25,19 @@ JOIN(8)(LEFT_OUTER) explain ------------------------------- ------------------------------------------------------------------------------- -Execution Block Graph (TERMINAL - eb_0000000000000_0000_000004) +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000006) ------------------------------------------------------------------------------- -|-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 +|-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000005 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 +1: eb_0000000000000_0000_000005 +2: eb_0000000000000_0000_000006 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [ROOT] +Block Id: eb_0000000000000_0000_000005 [ROOT] ======================================================= [Enforcers] @@ -68,5 +68,5 @@ JOIN(8)(LEFT_OUTER) => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} ======================================================= -Block Id: eb_0000000000000_0000_000004 [TERMINAL] +Block Id: eb_0000000000000_0000_000006 [TERMINAL] ======================================================= diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase3.1.Sort.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase3.1.Sort.plan index 50c41c9dcc..79ce689762 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase3.1.Sort.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase3.1.Sort.plan @@ -25,19 +25,19 @@ JOIN(8)(LEFT_OUTER) explain ------------------------------- ------------------------------------------------------------------------------- -Execution Block Graph (TERMINAL - eb_0000000000000_0000_000004) +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000006) ------------------------------------------------------------------------------- -|-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 +|-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000005 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 +1: eb_0000000000000_0000_000005 +2: eb_0000000000000_0000_000006 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [ROOT] +Block Id: eb_0000000000000_0000_000005 [ROOT] ======================================================= [Enforcers] @@ -68,5 +68,5 @@ JOIN(8)(LEFT_OUTER) => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} ======================================================= -Block Id: eb_0000000000000_0000_000004 [TERMINAL] +Block Id: eb_0000000000000_0000_000006 [TERMINAL] ======================================================= diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase4.1.Hash.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase4.1.Hash.plan index 1e2809e324..ad27e6f21f 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase4.1.Hash.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase4.1.Hash.plan @@ -26,19 +26,19 @@ JOIN(8)(LEFT_OUTER) explain ------------------------------- ------------------------------------------------------------------------------- -Execution Block Graph (TERMINAL - eb_0000000000000_0000_000004) +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000006) ------------------------------------------------------------------------------- -|-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 +|-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000005 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 +1: eb_0000000000000_0000_000005 +2: eb_0000000000000_0000_000006 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [ROOT] +Block Id: eb_0000000000000_0000_000005 [ROOT] ======================================================= [Enforcers] @@ -70,5 +70,5 @@ JOIN(8)(LEFT_OUTER) => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} ======================================================= -Block Id: eb_0000000000000_0000_000004 [TERMINAL] +Block Id: eb_0000000000000_0000_000006 [TERMINAL] ======================================================= diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase4.1.Sort.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase4.1.Sort.plan index 1e2809e324..ad27e6f21f 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase4.1.Sort.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase4.1.Sort.plan @@ -26,19 +26,19 @@ JOIN(8)(LEFT_OUTER) explain ------------------------------- ------------------------------------------------------------------------------- -Execution Block Graph (TERMINAL - eb_0000000000000_0000_000004) +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000006) ------------------------------------------------------------------------------- -|-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 +|-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000005 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 +1: eb_0000000000000_0000_000005 +2: eb_0000000000000_0000_000006 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [ROOT] +Block Id: eb_0000000000000_0000_000005 [ROOT] ======================================================= [Enforcers] @@ -70,5 +70,5 @@ JOIN(8)(LEFT_OUTER) => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} ======================================================= -Block Id: eb_0000000000000_0000_000004 [TERMINAL] +Block Id: eb_0000000000000_0000_000006 [TERMINAL] ======================================================= diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase5.1.Hash.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase5.1.Hash.plan index 88a955f1f1..88cf366e6c 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase5.1.Hash.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase5.1.Hash.plan @@ -26,19 +26,19 @@ JOIN(9)(LEFT_OUTER) explain ------------------------------- ------------------------------------------------------------------------------- -Execution Block Graph (TERMINAL - eb_0000000000000_0000_000004) +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000006) ------------------------------------------------------------------------------- -|-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 +|-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000005 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 +1: eb_0000000000000_0000_000005 +2: eb_0000000000000_0000_000006 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [ROOT] +Block Id: eb_0000000000000_0000_000005 [ROOT] ======================================================= [Enforcers] @@ -70,5 +70,5 @@ JOIN(9)(LEFT_OUTER) => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} ======================================================= -Block Id: eb_0000000000000_0000_000004 [TERMINAL] +Block Id: eb_0000000000000_0000_000006 [TERMINAL] ======================================================= diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase5.1.Sort.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase5.1.Sort.plan index 88a955f1f1..88cf366e6c 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase5.1.Sort.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase5.1.Sort.plan @@ -26,19 +26,19 @@ JOIN(9)(LEFT_OUTER) explain ------------------------------- ------------------------------------------------------------------------------- -Execution Block Graph (TERMINAL - eb_0000000000000_0000_000004) +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000006) ------------------------------------------------------------------------------- -|-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 +|-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000005 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 +1: eb_0000000000000_0000_000005 +2: eb_0000000000000_0000_000006 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [ROOT] +Block Id: eb_0000000000000_0000_000005 [ROOT] ======================================================= [Enforcers] @@ -70,5 +70,5 @@ JOIN(9)(LEFT_OUTER) => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} ======================================================= -Block Id: eb_0000000000000_0000_000004 [TERMINAL] +Block Id: eb_0000000000000_0000_000006 [TERMINAL] ======================================================= diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase6.1.Hash.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase6.1.Hash.plan index d9e214c12d..b23896b2d1 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase6.1.Hash.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase6.1.Hash.plan @@ -27,19 +27,19 @@ SELECTION(5) explain ------------------------------- ------------------------------------------------------------------------------- -Execution Block Graph (TERMINAL - eb_0000000000000_0000_000004) +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000006) ------------------------------------------------------------------------------- -|-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 +|-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000005 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 +1: eb_0000000000000_0000_000005 +2: eb_0000000000000_0000_000006 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [ROOT] +Block Id: eb_0000000000000_0000_000005 [ROOT] ======================================================= [Enforcers] @@ -72,5 +72,5 @@ SELECTION(5) => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} ======================================================= -Block Id: eb_0000000000000_0000_000004 [TERMINAL] +Block Id: eb_0000000000000_0000_000006 [TERMINAL] ======================================================= diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase6.1.Sort.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase6.1.Sort.plan index d9e214c12d..b23896b2d1 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase6.1.Sort.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase6.1.Sort.plan @@ -27,19 +27,19 @@ SELECTION(5) explain ------------------------------- ------------------------------------------------------------------------------- -Execution Block Graph (TERMINAL - eb_0000000000000_0000_000004) +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000006) ------------------------------------------------------------------------------- -|-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 +|-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000005 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 +1: eb_0000000000000_0000_000005 +2: eb_0000000000000_0000_000006 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [ROOT] +Block Id: eb_0000000000000_0000_000005 [ROOT] ======================================================= [Enforcers] @@ -72,5 +72,5 @@ SELECTION(5) => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} ======================================================= -Block Id: eb_0000000000000_0000_000004 [TERMINAL] +Block Id: eb_0000000000000_0000_000006 [TERMINAL] ======================================================= diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithConstantExpr1.Hash.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithConstantExpr1.Hash.plan index cc8a491694..9ce925e3c6 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithConstantExpr1.Hash.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithConstantExpr1.Hash.plan @@ -45,7 +45,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.orders -SORT(8) +SORT(10) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.orders.o_orderkey (INT4) (asc) JOIN(6)(LEFT_OUTER) => Join Cond: default.customer.c_custkey (INT4) = default.orders.o_orderkey (INT4) @@ -77,7 +77,7 @@ PROJECTION(4) => in schema: {(2) default.customer.c_custkey (INT4), default.orders.o_orderkey (INT4)} SORT(3) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.orders.o_orderkey (INT4) (asc) - SCAN(9) on eb_0000000000000_0000_000003 + SCAN(11) on eb_0000000000000_0000_000003 => out schema: {(2) default.customer.c_custkey (INT4), default.orders.o_orderkey (INT4)} => in schema: {(2) default.customer.c_custkey (INT4), default.orders.o_orderkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithConstantExpr1.Sort.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithConstantExpr1.Sort.plan index cc8a491694..9ce925e3c6 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithConstantExpr1.Sort.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithConstantExpr1.Sort.plan @@ -45,7 +45,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.orders -SORT(8) +SORT(10) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.orders.o_orderkey (INT4) (asc) JOIN(6)(LEFT_OUTER) => Join Cond: default.customer.c_custkey (INT4) = default.orders.o_orderkey (INT4) @@ -77,7 +77,7 @@ PROJECTION(4) => in schema: {(2) default.customer.c_custkey (INT4), default.orders.o_orderkey (INT4)} SORT(3) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.orders.o_orderkey (INT4) (asc) - SCAN(9) on eb_0000000000000_0000_000003 + SCAN(11) on eb_0000000000000_0000_000003 => out schema: {(2) default.customer.c_custkey (INT4), default.orders.o_orderkey (INT4)} => in schema: {(2) default.customer.c_custkey (INT4), default.orders.o_orderkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithEmptyTable1.Hash.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithEmptyTable1.Hash.plan index e35e6a547f..8861babb9d 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithEmptyTable1.Hash.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithEmptyTable1.Hash.plan @@ -41,7 +41,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.empty_orders -SORT(8) +SORT(10) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.empty_orders.o_orderkey (INT4) (asc) JOIN(6)(LEFT_OUTER) => Join Cond: default.customer.c_custkey (INT4) = default.empty_orders.o_orderkey (INT4) @@ -69,7 +69,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.empty_orders.o_orderkey (INT4) (asc) - SCAN(9) on eb_0000000000000_0000_000003 + SCAN(11) on eb_0000000000000_0000_000003 => out schema: {(4) default.customer.c_custkey (INT4), default.empty_orders.o_orderdate (TEXT), default.empty_orders.o_orderkey (INT4), default.empty_orders.o_orderstatus (TEXT)} => in schema: {(4) default.customer.c_custkey (INT4), default.empty_orders.o_orderdate (TEXT), default.empty_orders.o_orderkey (INT4), default.empty_orders.o_orderstatus (TEXT)} diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithEmptyTable1.Sort.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithEmptyTable1.Sort.plan index e35e6a547f..8861babb9d 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithEmptyTable1.Sort.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithEmptyTable1.Sort.plan @@ -41,7 +41,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.empty_orders -SORT(8) +SORT(10) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.empty_orders.o_orderkey (INT4) (asc) JOIN(6)(LEFT_OUTER) => Join Cond: default.customer.c_custkey (INT4) = default.empty_orders.o_orderkey (INT4) @@ -69,7 +69,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.empty_orders.o_orderkey (INT4) (asc) - SCAN(9) on eb_0000000000000_0000_000003 + SCAN(11) on eb_0000000000000_0000_000003 => out schema: {(4) default.customer.c_custkey (INT4), default.empty_orders.o_orderdate (TEXT), default.empty_orders.o_orderkey (INT4), default.empty_orders.o_orderstatus (TEXT)} => in schema: {(4) default.customer.c_custkey (INT4), default.empty_orders.o_orderdate (TEXT), default.empty_orders.o_orderkey (INT4), default.empty_orders.o_orderstatus (TEXT)} diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithEmptyTable2.Hash.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithEmptyTable2.Hash.plan index bca8356ab8..1fa968fd0c 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithEmptyTable2.Hash.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithEmptyTable2.Hash.plan @@ -48,7 +48,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.empty_orders -GROUP_BY(9)(c_custkey) +GROUP_BY(11)(c_custkey) => exprs: (sum(default.empty_orders.o_orderkey (INT4)),max(default.empty_orders.o_orderstatus (TEXT)),max(default.empty_orders.o_orderdate (TEXT))) => target list: default.customer.c_custkey (INT4), ?sum_5 (INT8), ?max_6 (TEXT), ?max_7 (TEXT) => out schema:{(4) default.customer.c_custkey (INT4), ?sum_5 (INT8), ?max_6 (TEXT), ?max_7 (TEXT)} @@ -77,14 +77,14 @@ Block Id: eb_0000000000000_0000_000004 [INTERMEDIATE] [Outgoing] [q_0000000000000_0000] 4 => 5 (type=RANGE_SHUFFLE, key=default.customer.c_custkey (INT4), num=32) -SORT(11) +SORT(13) => Sort Keys: default.customer.c_custkey (INT4) (asc) GROUP_BY(3)(c_custkey) => exprs: (sum(?sum_5 (INT8)),max(?max_6 (TEXT)),max(?max_7 (TEXT))) => target list: default.customer.c_custkey (INT4), ?sum (INT8), ?max_1 (TEXT), ?max_2 (TEXT) => out schema:{(4) ?max_1 (TEXT), ?max_2 (TEXT), ?sum (INT8), default.customer.c_custkey (INT4)} => in schema:{(4) default.customer.c_custkey (INT4), ?sum_5 (INT8), ?max_6 (TEXT), ?max_7 (TEXT)} - SCAN(10) on eb_0000000000000_0000_000003 + SCAN(12) on eb_0000000000000_0000_000003 => out schema: {(4) default.customer.c_custkey (INT4), ?sum_5 (INT8), ?max_6 (TEXT), ?max_7 (TEXT)} => in schema: {(4) default.customer.c_custkey (INT4), ?sum_5 (INT8), ?max_6 (TEXT), ?max_7 (TEXT)} @@ -100,7 +100,7 @@ Block Id: eb_0000000000000_0000_000005 [ROOT] SORT(4) => Sort Keys: default.customer.c_custkey (INT4) (asc) - SCAN(12) on eb_0000000000000_0000_000004 + SCAN(14) on eb_0000000000000_0000_000004 => out schema: {(4) ?max_1 (TEXT), ?max_2 (TEXT), ?sum (INT8), default.customer.c_custkey (INT4)} => in schema: {(4) ?max_1 (TEXT), ?max_2 (TEXT), ?sum (INT8), default.customer.c_custkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithEmptyTable2.Sort.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithEmptyTable2.Sort.plan index bca8356ab8..1fa968fd0c 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithEmptyTable2.Sort.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithEmptyTable2.Sort.plan @@ -48,7 +48,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.empty_orders -GROUP_BY(9)(c_custkey) +GROUP_BY(11)(c_custkey) => exprs: (sum(default.empty_orders.o_orderkey (INT4)),max(default.empty_orders.o_orderstatus (TEXT)),max(default.empty_orders.o_orderdate (TEXT))) => target list: default.customer.c_custkey (INT4), ?sum_5 (INT8), ?max_6 (TEXT), ?max_7 (TEXT) => out schema:{(4) default.customer.c_custkey (INT4), ?sum_5 (INT8), ?max_6 (TEXT), ?max_7 (TEXT)} @@ -77,14 +77,14 @@ Block Id: eb_0000000000000_0000_000004 [INTERMEDIATE] [Outgoing] [q_0000000000000_0000] 4 => 5 (type=RANGE_SHUFFLE, key=default.customer.c_custkey (INT4), num=32) -SORT(11) +SORT(13) => Sort Keys: default.customer.c_custkey (INT4) (asc) GROUP_BY(3)(c_custkey) => exprs: (sum(?sum_5 (INT8)),max(?max_6 (TEXT)),max(?max_7 (TEXT))) => target list: default.customer.c_custkey (INT4), ?sum (INT8), ?max_1 (TEXT), ?max_2 (TEXT) => out schema:{(4) ?max_1 (TEXT), ?max_2 (TEXT), ?sum (INT8), default.customer.c_custkey (INT4)} => in schema:{(4) default.customer.c_custkey (INT4), ?sum_5 (INT8), ?max_6 (TEXT), ?max_7 (TEXT)} - SCAN(10) on eb_0000000000000_0000_000003 + SCAN(12) on eb_0000000000000_0000_000003 => out schema: {(4) default.customer.c_custkey (INT4), ?sum_5 (INT8), ?max_6 (TEXT), ?max_7 (TEXT)} => in schema: {(4) default.customer.c_custkey (INT4), ?sum_5 (INT8), ?max_6 (TEXT), ?max_7 (TEXT)} @@ -100,7 +100,7 @@ Block Id: eb_0000000000000_0000_000005 [ROOT] SORT(4) => Sort Keys: default.customer.c_custkey (INT4) (asc) - SCAN(12) on eb_0000000000000_0000_000004 + SCAN(14) on eb_0000000000000_0000_000004 => out schema: {(4) ?max_1 (TEXT), ?max_2 (TEXT), ?sum (INT8), default.customer.c_custkey (INT4)} => in schema: {(4) ?max_1 (TEXT), ?max_2 (TEXT), ?sum (INT8), default.customer.c_custkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithEmptyTable3.Hash.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithEmptyTable3.Hash.plan index af210c2a7c..6640d698df 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithEmptyTable3.Hash.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithEmptyTable3.Hash.plan @@ -63,7 +63,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.empty_orders -GROUP_BY(12)(c_custkey) +GROUP_BY(14)(c_custkey) => exprs: (sum(default.empty_orders.o_orderkey (INT4)),max(default.empty_orders.o_orderstatus (TEXT)),max(default.empty_orders.o_orderdate (TEXT))) => target list: default.customer.c_custkey (INT4), ?sum_3 (INT8), ?max_4 (TEXT), ?max_5 (TEXT) => out schema:{(4) default.customer.c_custkey (INT4), ?sum_3 (INT8), ?max_4 (TEXT), ?max_5 (TEXT)} @@ -92,7 +92,7 @@ Block Id: eb_0000000000000_0000_000004 [INTERMEDIATE] [Outgoing] [q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.t1.c_custkey (INT4), num=32) -GROUP_BY(14)(c_custkey) +GROUP_BY(16)(c_custkey) => exprs: (count()) => target list: default.t1.c_custkey (INT4), ?count_6 (INT8) => out schema:{(2) default.t1.c_custkey (INT4), ?count_6 (INT8)} @@ -110,7 +110,7 @@ GROUP_BY(14)(c_custkey) => target list: default.customer.c_custkey (INT4), total1 (INT8), total2 (TEXT), total3 (TEXT) => out schema:{(4) default.customer.c_custkey (INT4), total1 (INT8), total2 (TEXT), total3 (TEXT)} => in schema:{(4) default.customer.c_custkey (INT4), ?sum_3 (INT8), ?max_4 (TEXT), ?max_5 (TEXT)} - SCAN(13) on eb_0000000000000_0000_000003 + SCAN(15) on eb_0000000000000_0000_000003 => out schema: {(4) default.customer.c_custkey (INT4), ?sum_3 (INT8), ?max_4 (TEXT), ?max_5 (TEXT)} => in schema: {(4) default.customer.c_custkey (INT4), ?sum_3 (INT8), ?max_4 (TEXT), ?max_5 (TEXT)} @@ -124,14 +124,14 @@ Block Id: eb_0000000000000_0000_000005 [INTERMEDIATE] [Outgoing] [q_0000000000000_0000] 5 => 6 (type=RANGE_SHUFFLE, key=default.t1.c_custkey (INT4), num=32) -SORT(16) +SORT(18) => Sort Keys: default.t1.c_custkey (INT4) (asc) GROUP_BY(6)(c_custkey) => exprs: (count(?count_6 (INT8))) => target list: default.t1.c_custkey (INT4), ?count (INT8) => out schema:{(2) ?count (INT8), default.t1.c_custkey (INT4)} => in schema:{(2) default.t1.c_custkey (INT4), ?count_6 (INT8)} - SCAN(15) on eb_0000000000000_0000_000004 + SCAN(17) on eb_0000000000000_0000_000004 => out schema: {(2) default.t1.c_custkey (INT4), ?count_6 (INT8)} => in schema: {(2) default.t1.c_custkey (INT4), ?count_6 (INT8)} @@ -151,7 +151,7 @@ PROJECTION(8) => in schema: {(2) ?count (INT8), default.t1.c_custkey (INT4)} SORT(7) => Sort Keys: default.t1.c_custkey (INT4) (asc) - SCAN(17) on eb_0000000000000_0000_000005 + SCAN(19) on eb_0000000000000_0000_000005 => out schema: {(2) ?count (INT8), default.t1.c_custkey (INT4)} => in schema: {(2) ?count (INT8), default.t1.c_custkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithEmptyTable3.Sort.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithEmptyTable3.Sort.plan index af210c2a7c..6640d698df 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithEmptyTable3.Sort.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithEmptyTable3.Sort.plan @@ -63,7 +63,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.empty_orders -GROUP_BY(12)(c_custkey) +GROUP_BY(14)(c_custkey) => exprs: (sum(default.empty_orders.o_orderkey (INT4)),max(default.empty_orders.o_orderstatus (TEXT)),max(default.empty_orders.o_orderdate (TEXT))) => target list: default.customer.c_custkey (INT4), ?sum_3 (INT8), ?max_4 (TEXT), ?max_5 (TEXT) => out schema:{(4) default.customer.c_custkey (INT4), ?sum_3 (INT8), ?max_4 (TEXT), ?max_5 (TEXT)} @@ -92,7 +92,7 @@ Block Id: eb_0000000000000_0000_000004 [INTERMEDIATE] [Outgoing] [q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.t1.c_custkey (INT4), num=32) -GROUP_BY(14)(c_custkey) +GROUP_BY(16)(c_custkey) => exprs: (count()) => target list: default.t1.c_custkey (INT4), ?count_6 (INT8) => out schema:{(2) default.t1.c_custkey (INT4), ?count_6 (INT8)} @@ -110,7 +110,7 @@ GROUP_BY(14)(c_custkey) => target list: default.customer.c_custkey (INT4), total1 (INT8), total2 (TEXT), total3 (TEXT) => out schema:{(4) default.customer.c_custkey (INT4), total1 (INT8), total2 (TEXT), total3 (TEXT)} => in schema:{(4) default.customer.c_custkey (INT4), ?sum_3 (INT8), ?max_4 (TEXT), ?max_5 (TEXT)} - SCAN(13) on eb_0000000000000_0000_000003 + SCAN(15) on eb_0000000000000_0000_000003 => out schema: {(4) default.customer.c_custkey (INT4), ?sum_3 (INT8), ?max_4 (TEXT), ?max_5 (TEXT)} => in schema: {(4) default.customer.c_custkey (INT4), ?sum_3 (INT8), ?max_4 (TEXT), ?max_5 (TEXT)} @@ -124,14 +124,14 @@ Block Id: eb_0000000000000_0000_000005 [INTERMEDIATE] [Outgoing] [q_0000000000000_0000] 5 => 6 (type=RANGE_SHUFFLE, key=default.t1.c_custkey (INT4), num=32) -SORT(16) +SORT(18) => Sort Keys: default.t1.c_custkey (INT4) (asc) GROUP_BY(6)(c_custkey) => exprs: (count(?count_6 (INT8))) => target list: default.t1.c_custkey (INT4), ?count (INT8) => out schema:{(2) ?count (INT8), default.t1.c_custkey (INT4)} => in schema:{(2) default.t1.c_custkey (INT4), ?count_6 (INT8)} - SCAN(15) on eb_0000000000000_0000_000004 + SCAN(17) on eb_0000000000000_0000_000004 => out schema: {(2) default.t1.c_custkey (INT4), ?count_6 (INT8)} => in schema: {(2) default.t1.c_custkey (INT4), ?count_6 (INT8)} @@ -151,7 +151,7 @@ PROJECTION(8) => in schema: {(2) ?count (INT8), default.t1.c_custkey (INT4)} SORT(7) => Sort Keys: default.t1.c_custkey (INT4) (asc) - SCAN(17) on eb_0000000000000_0000_000005 + SCAN(19) on eb_0000000000000_0000_000005 => out schema: {(2) ?count (INT8), default.t1.c_custkey (INT4)} => in schema: {(2) ?count (INT8), default.t1.c_custkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithEmptyTable4.Hash.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithEmptyTable4.Hash.plan index 3bb8c1d84b..5ea0fdd25e 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithEmptyTable4.Hash.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithEmptyTable4.Hash.plan @@ -75,7 +75,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.orders -GROUP_BY(15)() +GROUP_BY(17)() => exprs: (max(default.customer.c_custkey (INT4)),sum(default.orders.o_orderkey (INT4)),max(default.orders.o_orderstatus (TEXT)),max(default.orders.o_orderdate (TEXT))) => target list: ?max_12 (INT4), ?sum_13 (INT8), ?max_14 (TEXT), ?max_15 (TEXT) => out schema:{(4) ?max_12 (INT4), ?sum_13 (INT8), ?max_14 (TEXT), ?max_15 (TEXT)} @@ -106,7 +106,7 @@ GROUP_BY(9)() => target list: ?max (INT4), ?sum_1 (INT8), ?max_2 (TEXT), ?max_3 (TEXT) => out schema:{(4) ?max (INT4), ?max_2 (TEXT), ?max_3 (TEXT), ?sum_1 (INT8)} => in schema:{(4) ?max_12 (INT4), ?sum_13 (INT8), ?max_14 (TEXT), ?max_15 (TEXT)} - SCAN(16) on eb_0000000000000_0000_000003 + SCAN(18) on eb_0000000000000_0000_000003 => out schema: {(4) ?max_12 (INT4), ?sum_13 (INT8), ?max_14 (TEXT), ?max_15 (TEXT)} => in schema: {(4) ?max_12 (INT4), ?sum_13 (INT8), ?max_14 (TEXT), ?max_15 (TEXT)} @@ -120,7 +120,7 @@ Block Id: eb_0000000000000_0000_000007 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.empty_orders -GROUP_BY(17)() +GROUP_BY(21)() => exprs: (max(default.customer.c_custkey (INT4)),sum(default.empty_orders.o_orderkey (INT4)),max(default.empty_orders.o_orderstatus (TEXT)),max(default.empty_orders.o_orderdate (TEXT))) => target list: ?max_16 (INT4), ?sum_17 (INT8), ?max_18 (TEXT), ?max_19 (TEXT) => out schema:{(4) ?max_16 (INT4), ?sum_17 (INT8), ?max_18 (TEXT), ?max_19 (TEXT)} @@ -151,7 +151,7 @@ GROUP_BY(10)() => target list: ?max_5 (INT4), ?sum_6 (INT8), ?max_7 (TEXT), ?max_8 (TEXT) => out schema:{(4) ?max_5 (INT4), ?max_7 (TEXT), ?max_8 (TEXT), ?sum_6 (INT8)} => in schema:{(4) ?max_16 (INT4), ?sum_17 (INT8), ?max_18 (TEXT), ?max_19 (TEXT)} - SCAN(18) on eb_0000000000000_0000_000007 + SCAN(22) on eb_0000000000000_0000_000007 => out schema: {(4) ?max_16 (INT4), ?sum_17 (INT8), ?max_18 (TEXT), ?max_19 (TEXT)} => in schema: {(4) ?max_16 (INT4), ?sum_17 (INT8), ?max_18 (TEXT), ?max_19 (TEXT)} diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithEmptyTable4.Sort.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithEmptyTable4.Sort.plan index 3bb8c1d84b..5ea0fdd25e 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithEmptyTable4.Sort.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithEmptyTable4.Sort.plan @@ -75,7 +75,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.orders -GROUP_BY(15)() +GROUP_BY(17)() => exprs: (max(default.customer.c_custkey (INT4)),sum(default.orders.o_orderkey (INT4)),max(default.orders.o_orderstatus (TEXT)),max(default.orders.o_orderdate (TEXT))) => target list: ?max_12 (INT4), ?sum_13 (INT8), ?max_14 (TEXT), ?max_15 (TEXT) => out schema:{(4) ?max_12 (INT4), ?sum_13 (INT8), ?max_14 (TEXT), ?max_15 (TEXT)} @@ -106,7 +106,7 @@ GROUP_BY(9)() => target list: ?max (INT4), ?sum_1 (INT8), ?max_2 (TEXT), ?max_3 (TEXT) => out schema:{(4) ?max (INT4), ?max_2 (TEXT), ?max_3 (TEXT), ?sum_1 (INT8)} => in schema:{(4) ?max_12 (INT4), ?sum_13 (INT8), ?max_14 (TEXT), ?max_15 (TEXT)} - SCAN(16) on eb_0000000000000_0000_000003 + SCAN(18) on eb_0000000000000_0000_000003 => out schema: {(4) ?max_12 (INT4), ?sum_13 (INT8), ?max_14 (TEXT), ?max_15 (TEXT)} => in schema: {(4) ?max_12 (INT4), ?sum_13 (INT8), ?max_14 (TEXT), ?max_15 (TEXT)} @@ -120,7 +120,7 @@ Block Id: eb_0000000000000_0000_000007 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.empty_orders -GROUP_BY(17)() +GROUP_BY(21)() => exprs: (max(default.customer.c_custkey (INT4)),sum(default.empty_orders.o_orderkey (INT4)),max(default.empty_orders.o_orderstatus (TEXT)),max(default.empty_orders.o_orderdate (TEXT))) => target list: ?max_16 (INT4), ?sum_17 (INT8), ?max_18 (TEXT), ?max_19 (TEXT) => out schema:{(4) ?max_16 (INT4), ?sum_17 (INT8), ?max_18 (TEXT), ?max_19 (TEXT)} @@ -151,7 +151,7 @@ GROUP_BY(10)() => target list: ?max_5 (INT4), ?sum_6 (INT8), ?max_7 (TEXT), ?max_8 (TEXT) => out schema:{(4) ?max_5 (INT4), ?max_7 (TEXT), ?max_8 (TEXT), ?sum_6 (INT8)} => in schema:{(4) ?max_16 (INT4), ?sum_17 (INT8), ?max_18 (TEXT), ?max_19 (TEXT)} - SCAN(18) on eb_0000000000000_0000_000007 + SCAN(22) on eb_0000000000000_0000_000007 => out schema: {(4) ?max_16 (INT4), ?sum_17 (INT8), ?max_18 (TEXT), ?max_19 (TEXT)} => in schema: {(4) ?max_16 (INT4), ?sum_17 (INT8), ?max_18 (TEXT), ?max_19 (TEXT)} diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithEmptyTable5.Hash.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithEmptyTable5.Hash.plan index b7892e483a..a0476d1171 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithEmptyTable5.Hash.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithEmptyTable5.Hash.plan @@ -48,7 +48,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.empty_orders -GROUP_BY(9)(l_linenumber) +GROUP_BY(11)(l_linenumber) => exprs: (sum(default.empty_orders.o_orderkey (INT4)),max(default.empty_orders.o_orderstatus (TEXT)),max(default.empty_orders.o_orderdate (TEXT)),avg(default.lineitem.l_quantity (FLOAT8)),sum(default.lineitem.l_quantity (FLOAT8))) => target list: default.lineitem.l_linenumber (INT4), ?sum_7 (INT8), ?max_8 (TEXT), ?max_9 (TEXT), ?avg_10 (PROTOBUF), ?sum_11 (FLOAT8) => out schema:{(6) default.lineitem.l_linenumber (INT4), ?sum_7 (INT8), ?max_8 (TEXT), ?max_9 (TEXT), ?avg_10 (PROTOBUF), ?sum_11 (FLOAT8)} @@ -77,14 +77,14 @@ Block Id: eb_0000000000000_0000_000004 [INTERMEDIATE] [Outgoing] [q_0000000000000_0000] 4 => 5 (type=RANGE_SHUFFLE, key=default.lineitem.l_linenumber (INT4), num=32) -SORT(11) +SORT(13) => Sort Keys: default.lineitem.l_linenumber (INT4) (asc) GROUP_BY(3)(l_linenumber) => exprs: (sum(?sum_7 (INT8)),max(?max_8 (TEXT)),max(?max_9 (TEXT)),avg(?avg_10 (PROTOBUF)),sum(?sum_11 (FLOAT8))) => target list: default.lineitem.l_linenumber (INT4), ?sum (INT8), ?max_1 (TEXT), ?max_2 (TEXT), ?avg_3 (FLOAT8), ?sum_4 (FLOAT8) => out schema:{(6) ?avg_3 (FLOAT8), ?max_1 (TEXT), ?max_2 (TEXT), ?sum (INT8), ?sum_4 (FLOAT8), default.lineitem.l_linenumber (INT4)} => in schema:{(6) default.lineitem.l_linenumber (INT4), ?sum_7 (INT8), ?max_8 (TEXT), ?max_9 (TEXT), ?avg_10 (PROTOBUF), ?sum_11 (FLOAT8)} - SCAN(10) on eb_0000000000000_0000_000003 + SCAN(12) on eb_0000000000000_0000_000003 => out schema: {(6) default.lineitem.l_linenumber (INT4), ?sum_7 (INT8), ?max_8 (TEXT), ?max_9 (TEXT), ?avg_10 (PROTOBUF), ?sum_11 (FLOAT8)} => in schema: {(6) default.lineitem.l_linenumber (INT4), ?sum_7 (INT8), ?max_8 (TEXT), ?max_9 (TEXT), ?avg_10 (PROTOBUF), ?sum_11 (FLOAT8)} @@ -100,7 +100,7 @@ Block Id: eb_0000000000000_0000_000005 [ROOT] SORT(4) => Sort Keys: default.lineitem.l_linenumber (INT4) (asc) - SCAN(12) on eb_0000000000000_0000_000004 + SCAN(14) on eb_0000000000000_0000_000004 => out schema: {(6) ?avg_3 (FLOAT8), ?max_1 (TEXT), ?max_2 (TEXT), ?sum (INT8), ?sum_4 (FLOAT8), default.lineitem.l_linenumber (INT4)} => in schema: {(6) ?avg_3 (FLOAT8), ?max_1 (TEXT), ?max_2 (TEXT), ?sum (INT8), ?sum_4 (FLOAT8), default.lineitem.l_linenumber (INT4)} diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithEmptyTable5.Sort.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithEmptyTable5.Sort.plan index b7892e483a..a0476d1171 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithEmptyTable5.Sort.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithEmptyTable5.Sort.plan @@ -48,7 +48,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.empty_orders -GROUP_BY(9)(l_linenumber) +GROUP_BY(11)(l_linenumber) => exprs: (sum(default.empty_orders.o_orderkey (INT4)),max(default.empty_orders.o_orderstatus (TEXT)),max(default.empty_orders.o_orderdate (TEXT)),avg(default.lineitem.l_quantity (FLOAT8)),sum(default.lineitem.l_quantity (FLOAT8))) => target list: default.lineitem.l_linenumber (INT4), ?sum_7 (INT8), ?max_8 (TEXT), ?max_9 (TEXT), ?avg_10 (PROTOBUF), ?sum_11 (FLOAT8) => out schema:{(6) default.lineitem.l_linenumber (INT4), ?sum_7 (INT8), ?max_8 (TEXT), ?max_9 (TEXT), ?avg_10 (PROTOBUF), ?sum_11 (FLOAT8)} @@ -77,14 +77,14 @@ Block Id: eb_0000000000000_0000_000004 [INTERMEDIATE] [Outgoing] [q_0000000000000_0000] 4 => 5 (type=RANGE_SHUFFLE, key=default.lineitem.l_linenumber (INT4), num=32) -SORT(11) +SORT(13) => Sort Keys: default.lineitem.l_linenumber (INT4) (asc) GROUP_BY(3)(l_linenumber) => exprs: (sum(?sum_7 (INT8)),max(?max_8 (TEXT)),max(?max_9 (TEXT)),avg(?avg_10 (PROTOBUF)),sum(?sum_11 (FLOAT8))) => target list: default.lineitem.l_linenumber (INT4), ?sum (INT8), ?max_1 (TEXT), ?max_2 (TEXT), ?avg_3 (FLOAT8), ?sum_4 (FLOAT8) => out schema:{(6) ?avg_3 (FLOAT8), ?max_1 (TEXT), ?max_2 (TEXT), ?sum (INT8), ?sum_4 (FLOAT8), default.lineitem.l_linenumber (INT4)} => in schema:{(6) default.lineitem.l_linenumber (INT4), ?sum_7 (INT8), ?max_8 (TEXT), ?max_9 (TEXT), ?avg_10 (PROTOBUF), ?sum_11 (FLOAT8)} - SCAN(10) on eb_0000000000000_0000_000003 + SCAN(12) on eb_0000000000000_0000_000003 => out schema: {(6) default.lineitem.l_linenumber (INT4), ?sum_7 (INT8), ?max_8 (TEXT), ?max_9 (TEXT), ?avg_10 (PROTOBUF), ?sum_11 (FLOAT8)} => in schema: {(6) default.lineitem.l_linenumber (INT4), ?sum_7 (INT8), ?max_8 (TEXT), ?max_9 (TEXT), ?avg_10 (PROTOBUF), ?sum_11 (FLOAT8)} @@ -100,7 +100,7 @@ Block Id: eb_0000000000000_0000_000005 [ROOT] SORT(4) => Sort Keys: default.lineitem.l_linenumber (INT4) (asc) - SCAN(12) on eb_0000000000000_0000_000004 + SCAN(14) on eb_0000000000000_0000_000004 => out schema: {(6) ?avg_3 (FLOAT8), ?max_1 (TEXT), ?max_2 (TEXT), ?sum (INT8), ?sum_4 (FLOAT8), default.lineitem.l_linenumber (INT4)} => in schema: {(6) ?avg_3 (FLOAT8), ?max_1 (TEXT), ?max_2 (TEXT), ?sum (INT8), ?sum_4 (FLOAT8), default.lineitem.l_linenumber (INT4)} diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithNull1.Hash.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithNull1.Hash.plan index 61727204ec..4de481fa38 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithNull1.Hash.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithNull1.Hash.plan @@ -43,7 +43,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.orders -SORT(9) +SORT(11) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.orders.o_orderkey (INT4) (asc) SELECTION(3) => Search Cond: default.orders.o_orderkey (INT4) IS NULL @@ -73,7 +73,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(4) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.orders.o_orderkey (INT4) (asc) - SCAN(10) on eb_0000000000000_0000_000003 + SCAN(12) on eb_0000000000000_0000_000003 => out schema: {(4) ?coalesce (TEXT), default.customer.c_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4)} => in schema: {(4) ?coalesce (TEXT), default.customer.c_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithNull1.Sort.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithNull1.Sort.plan index 61727204ec..4de481fa38 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithNull1.Sort.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithNull1.Sort.plan @@ -43,7 +43,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.orders -SORT(9) +SORT(11) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.orders.o_orderkey (INT4) (asc) SELECTION(3) => Search Cond: default.orders.o_orderkey (INT4) IS NULL @@ -73,7 +73,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(4) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.orders.o_orderkey (INT4) (asc) - SCAN(10) on eb_0000000000000_0000_000003 + SCAN(12) on eb_0000000000000_0000_000003 => out schema: {(4) ?coalesce (TEXT), default.customer.c_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4)} => in schema: {(4) ?coalesce (TEXT), default.customer.c_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithNull2.Hash.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithNull2.Hash.plan index f92351ed29..f1e2800746 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithNull2.Hash.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithNull2.Hash.plan @@ -43,7 +43,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.orders -SORT(9) +SORT(11) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.orders.o_orderkey (INT4) (asc) SELECTION(3) => Search Cond: default.orders.o_orderdate (TEXT) IS NOT NULL AND default.orders.o_orderdate (TEXT)LIKE'1996%' @@ -73,7 +73,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(4) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.orders.o_orderkey (INT4) (asc) - SCAN(10) on eb_0000000000000_0000_000003 + SCAN(12) on eb_0000000000000_0000_000003 => out schema: {(4) ?coalesce (TEXT), default.customer.c_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4)} => in schema: {(4) ?coalesce (TEXT), default.customer.c_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithNull2.Sort.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithNull2.Sort.plan index f92351ed29..f1e2800746 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithNull2.Sort.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithNull2.Sort.plan @@ -43,7 +43,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.orders -SORT(9) +SORT(11) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.orders.o_orderkey (INT4) (asc) SELECTION(3) => Search Cond: default.orders.o_orderdate (TEXT) IS NOT NULL AND default.orders.o_orderdate (TEXT)LIKE'1996%' @@ -73,7 +73,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(4) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.orders.o_orderkey (INT4) (asc) - SCAN(10) on eb_0000000000000_0000_000003 + SCAN(12) on eb_0000000000000_0000_000003 => out schema: {(4) ?coalesce (TEXT), default.customer.c_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4)} => in schema: {(4) ?coalesce (TEXT), default.customer.c_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithNull3.Hash.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithNull3.Hash.plan index 23ef5fe2b9..5bace3685e 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithNull3.Hash.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithNull3.Hash.plan @@ -43,7 +43,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.orders -SORT(9) +SORT(11) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.orders.o_orderkey (INT4) (asc) SELECTION(3) => Search Cond: default.orders.o_orderkey (INT4) = 100 @@ -73,7 +73,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(4) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.orders.o_orderkey (INT4) (asc) - SCAN(10) on eb_0000000000000_0000_000003 + SCAN(12) on eb_0000000000000_0000_000003 => out schema: {(4) ?coalesce (TEXT), default.customer.c_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4)} => in schema: {(4) ?coalesce (TEXT), default.customer.c_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithNull3.Sort.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithNull3.Sort.plan index 23ef5fe2b9..5bace3685e 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithNull3.Sort.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinWithNull3.Sort.plan @@ -43,7 +43,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.orders -SORT(9) +SORT(11) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.orders.o_orderkey (INT4) (asc) SELECTION(3) => Search Cond: default.orders.o_orderkey (INT4) = 100 @@ -73,7 +73,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(4) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.orders.o_orderkey (INT4) (asc) - SCAN(10) on eb_0000000000000_0000_000003 + SCAN(12) on eb_0000000000000_0000_000003 => out schema: {(4) ?coalesce (TEXT), default.customer.c_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4)} => in schema: {(4) ?coalesce (TEXT), default.customer.c_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testMultipleBroadcastDataFileWithZeroLength2.Hash.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testMultipleBroadcastDataFileWithZeroLength2.Hash.plan index e9e2176d59..7b9c695261 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testMultipleBroadcastDataFileWithZeroLength2.Hash.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testMultipleBroadcastDataFileWithZeroLength2.Hash.plan @@ -27,19 +27,19 @@ SELECTION(5) explain ------------------------------- ------------------------------------------------------------------------------- -Execution Block Graph (TERMINAL - eb_0000000000000_0000_000004) +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000006) ------------------------------------------------------------------------------- -|-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 +|-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000005 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 +1: eb_0000000000000_0000_000005 +2: eb_0000000000000_0000_000006 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [ROOT] +Block Id: eb_0000000000000_0000_000005 [ROOT] ======================================================= [Enforcers] @@ -72,5 +72,5 @@ SELECTION(5) => in schema: {(8) default.a.c_acctbal (FLOAT8), default.a.c_address (TEXT), default.a.c_comment (TEXT), default.a.c_custkey (INT4), default.a.c_mktsegment (TEXT), default.a.c_name (TEXT), default.a.c_nationkey (INT4), default.a.c_phone (TEXT)} ======================================================= -Block Id: eb_0000000000000_0000_000004 [TERMINAL] +Block Id: eb_0000000000000_0000_000006 [TERMINAL] ======================================================= diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testMultipleBroadcastDataFileWithZeroLength2.Sort.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testMultipleBroadcastDataFileWithZeroLength2.Sort.plan index e9e2176d59..7b9c695261 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testMultipleBroadcastDataFileWithZeroLength2.Sort.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testMultipleBroadcastDataFileWithZeroLength2.Sort.plan @@ -27,19 +27,19 @@ SELECTION(5) explain ------------------------------- ------------------------------------------------------------------------------- -Execution Block Graph (TERMINAL - eb_0000000000000_0000_000004) +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000006) ------------------------------------------------------------------------------- -|-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 +|-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000005 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 +1: eb_0000000000000_0000_000005 +2: eb_0000000000000_0000_000006 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [ROOT] +Block Id: eb_0000000000000_0000_000005 [ROOT] ======================================================= [Enforcers] @@ -72,5 +72,5 @@ SELECTION(5) => in schema: {(8) default.a.c_acctbal (FLOAT8), default.a.c_address (TEXT), default.a.c_comment (TEXT), default.a.c_custkey (INT4), default.a.c_mktsegment (TEXT), default.a.c_name (TEXT), default.a.c_nationkey (INT4), default.a.c_phone (TEXT)} ======================================================= -Block Id: eb_0000000000000_0000_000004 [TERMINAL] +Block Id: eb_0000000000000_0000_000006 [TERMINAL] ======================================================= diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testOuterJoinAndCaseWhen1.Hash.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testOuterJoinAndCaseWhen1.Hash.plan index 58ccb3c763..08ded3e096 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testOuterJoinAndCaseWhen1.Hash.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testOuterJoinAndCaseWhen1.Hash.plan @@ -27,31 +27,31 @@ SORT(5) explain ------------------------------- ------------------------------------------------------------------------------- -Execution Block Graph (TERMINAL - eb_0000000000000_0000_000005) +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000007) ------------------------------------------------------------------------------- -|-eb_0000000000000_0000_000005 - |-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 +|-eb_0000000000000_0000_000007 + |-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000005 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 -3: eb_0000000000000_0000_000005 +1: eb_0000000000000_0000_000005 +2: eb_0000000000000_0000_000006 +3: eb_0000000000000_0000_000007 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [LEAF] +Block Id: eb_0000000000000_0000_000005 [LEAF] ======================================================= [Outgoing] -[q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.a.id (INT4), default.a.name (TEXT), num=32) +[q_0000000000000_0000] 5 => 6 (type=RANGE_SHUFFLE, key=default.a.id (INT4), default.a.name (TEXT), num=32) [Enforcers] 0: type=Broadcast, tables=default.b 1: type=Broadcast, tables=default.c -SORT(11) +SORT(15) => Sort Keys: default.a.id (INT4) (asc),default.a.name (TEXT) (asc) JOIN(9)(LEFT_OUTER) => Join Cond: default.b.id (INT4) = default.c.id (INT4) @@ -77,21 +77,21 @@ SORT(11) => in schema: {(4) default.a.id (INT4), default.a.name (TEXT), default.a.score (FLOAT4), default.a.type (TEXT)} ======================================================= -Block Id: eb_0000000000000_0000_000004 [ROOT] +Block Id: eb_0000000000000_0000_000006 [ROOT] ======================================================= [Incoming] -[q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.a.id (INT4), default.a.name (TEXT), num=32) +[q_0000000000000_0000] 5 => 6 (type=RANGE_SHUFFLE, key=default.a.id (INT4), default.a.name (TEXT), num=32) [Enforcers] - 0: sorted input=eb_0000000000000_0000_000003 + 0: sorted input=eb_0000000000000_0000_000005 SORT(5) => Sort Keys: default.a.id (INT4) (asc),default.a.name (TEXT) (asc) - SCAN(12) on eb_0000000000000_0000_000003 + SCAN(16) on eb_0000000000000_0000_000005 => out schema: {(6) c1 (TEXT), c2 (TEXT), default.a.id (INT4), default.a.name (TEXT), id2 (INT4), name2 (TEXT)} => in schema: {(6) c1 (TEXT), c2 (TEXT), default.a.id (INT4), default.a.name (TEXT), id2 (INT4), name2 (TEXT)} ======================================================= -Block Id: eb_0000000000000_0000_000005 [TERMINAL] +Block Id: eb_0000000000000_0000_000007 [TERMINAL] ======================================================= diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testOuterJoinAndCaseWhen1.Sort.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testOuterJoinAndCaseWhen1.Sort.plan index 58ccb3c763..08ded3e096 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testOuterJoinAndCaseWhen1.Sort.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testOuterJoinAndCaseWhen1.Sort.plan @@ -27,31 +27,31 @@ SORT(5) explain ------------------------------- ------------------------------------------------------------------------------- -Execution Block Graph (TERMINAL - eb_0000000000000_0000_000005) +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000007) ------------------------------------------------------------------------------- -|-eb_0000000000000_0000_000005 - |-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 +|-eb_0000000000000_0000_000007 + |-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000005 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 -3: eb_0000000000000_0000_000005 +1: eb_0000000000000_0000_000005 +2: eb_0000000000000_0000_000006 +3: eb_0000000000000_0000_000007 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [LEAF] +Block Id: eb_0000000000000_0000_000005 [LEAF] ======================================================= [Outgoing] -[q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.a.id (INT4), default.a.name (TEXT), num=32) +[q_0000000000000_0000] 5 => 6 (type=RANGE_SHUFFLE, key=default.a.id (INT4), default.a.name (TEXT), num=32) [Enforcers] 0: type=Broadcast, tables=default.b 1: type=Broadcast, tables=default.c -SORT(11) +SORT(15) => Sort Keys: default.a.id (INT4) (asc),default.a.name (TEXT) (asc) JOIN(9)(LEFT_OUTER) => Join Cond: default.b.id (INT4) = default.c.id (INT4) @@ -77,21 +77,21 @@ SORT(11) => in schema: {(4) default.a.id (INT4), default.a.name (TEXT), default.a.score (FLOAT4), default.a.type (TEXT)} ======================================================= -Block Id: eb_0000000000000_0000_000004 [ROOT] +Block Id: eb_0000000000000_0000_000006 [ROOT] ======================================================= [Incoming] -[q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.a.id (INT4), default.a.name (TEXT), num=32) +[q_0000000000000_0000] 5 => 6 (type=RANGE_SHUFFLE, key=default.a.id (INT4), default.a.name (TEXT), num=32) [Enforcers] - 0: sorted input=eb_0000000000000_0000_000003 + 0: sorted input=eb_0000000000000_0000_000005 SORT(5) => Sort Keys: default.a.id (INT4) (asc),default.a.name (TEXT) (asc) - SCAN(12) on eb_0000000000000_0000_000003 + SCAN(16) on eb_0000000000000_0000_000005 => out schema: {(6) c1 (TEXT), c2 (TEXT), default.a.id (INT4), default.a.name (TEXT), id2 (INT4), name2 (TEXT)} => in schema: {(6) c1 (TEXT), c2 (TEXT), default.a.id (INT4), default.a.name (TEXT), id2 (INT4), name2 (TEXT)} ======================================================= -Block Id: eb_0000000000000_0000_000005 [TERMINAL] +Block Id: eb_0000000000000_0000_000007 [TERMINAL] ======================================================= diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoin1.Hash.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoin1.Hash.plan index 98c044a38c..70507fec9f 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoin1.Hash.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoin1.Hash.plan @@ -41,7 +41,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.orders -SORT(8) +SORT(10) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.orders.o_orderkey (INT4) (asc) JOIN(6)(RIGHT_OUTER) => Join Cond: default.customer.c_custkey (INT4) = default.orders.o_orderkey (INT4) @@ -69,7 +69,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.orders.o_orderkey (INT4) (asc) - SCAN(9) on eb_0000000000000_0000_000003 + SCAN(11) on eb_0000000000000_0000_000003 => out schema: {(2) default.customer.c_custkey (INT4), default.orders.o_orderkey (INT4)} => in schema: {(2) default.customer.c_custkey (INT4), default.orders.o_orderkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoin1.Sort.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoin1.Sort.plan index 98c044a38c..70507fec9f 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoin1.Sort.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoin1.Sort.plan @@ -41,7 +41,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.orders -SORT(8) +SORT(10) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.orders.o_orderkey (INT4) (asc) JOIN(6)(RIGHT_OUTER) => Join Cond: default.customer.c_custkey (INT4) = default.orders.o_orderkey (INT4) @@ -69,7 +69,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.orders.o_orderkey (INT4) (asc) - SCAN(9) on eb_0000000000000_0000_000003 + SCAN(11) on eb_0000000000000_0000_000003 => out schema: {(2) default.customer.c_custkey (INT4), default.orders.o_orderkey (INT4)} => in schema: {(2) default.customer.c_custkey (INT4), default.orders.o_orderkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinWithEmptyTable1.Hash.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinWithEmptyTable1.Hash.plan index d33aa32fff..b271dccc53 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinWithEmptyTable1.Hash.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinWithEmptyTable1.Hash.plan @@ -41,7 +41,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.empty_orders -SORT(8) +SORT(10) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.empty_orders.o_orderkey (INT4) (asc) JOIN(6)(RIGHT_OUTER) => Join Cond: default.customer.c_custkey (INT4) = default.empty_orders.o_orderkey (INT4) @@ -69,7 +69,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.empty_orders.o_orderkey (INT4) (asc) - SCAN(9) on eb_0000000000000_0000_000003 + SCAN(11) on eb_0000000000000_0000_000003 => out schema: {(2) default.customer.c_custkey (INT4), default.empty_orders.o_orderkey (INT4)} => in schema: {(2) default.customer.c_custkey (INT4), default.empty_orders.o_orderkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinWithEmptyTable1.Sort.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinWithEmptyTable1.Sort.plan index d33aa32fff..b271dccc53 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinWithEmptyTable1.Sort.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinWithEmptyTable1.Sort.plan @@ -41,7 +41,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.empty_orders -SORT(8) +SORT(10) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.empty_orders.o_orderkey (INT4) (asc) JOIN(6)(RIGHT_OUTER) => Join Cond: default.customer.c_custkey (INT4) = default.empty_orders.o_orderkey (INT4) @@ -69,7 +69,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.empty_orders.o_orderkey (INT4) (asc) - SCAN(9) on eb_0000000000000_0000_000003 + SCAN(11) on eb_0000000000000_0000_000003 => out schema: {(2) default.customer.c_custkey (INT4), default.empty_orders.o_orderkey (INT4)} => in schema: {(2) default.customer.c_custkey (INT4), default.empty_orders.o_orderkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithConstantExpr2.Hash.plan b/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithConstantExpr2.Hash.plan index e4156aa9c8..4fb9de0ce5 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithConstantExpr2.Hash.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithConstantExpr2.Hash.plan @@ -31,61 +31,24 @@ Execution Block Graph (TERMINAL - eb_0000000000000_0000_000005) |-eb_0000000000000_0000_000005 |-eb_0000000000000_0000_000004 |-eb_0000000000000_0000_000003 - |-eb_0000000000000_0000_000002 - |-eb_0000000000000_0000_000001 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000001 -2: eb_0000000000000_0000_000002 -3: eb_0000000000000_0000_000003 -4: eb_0000000000000_0000_000004 -5: eb_0000000000000_0000_000005 +1: eb_0000000000000_0000_000003 +2: eb_0000000000000_0000_000004 +3: eb_0000000000000_0000_000005 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000001 [LEAF] +Block Id: eb_0000000000000_0000_000003 [LEAF] ======================================================= -[Outgoing] -[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.customer.c_custkey (INT4), num=32) - -SCAN(0) on default.customer - => target list: default.customer.c_custkey (INT4) - => out schema: {(1) default.customer.c_custkey (INT4)} - => in schema: {(8) default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT)} - -======================================================= -Block Id: eb_0000000000000_0000_000002 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.o.o_orderkey (INT4), num=32) - -TABLE_SUBQUERY(3) as default.o - => Targets: default.o.o_orderkey (INT4) - => out schema: {(1) default.o.o_orderkey (INT4)} - => in schema: {(9) default.o.o_clerk (TEXT), default.o.o_comment (TEXT), default.o.o_custkey (INT4), default.o.o_orderdate (TEXT), default.o.o_orderkey (INT4), default.o.o_orderpriority (TEXT), default.o.o_orderstatus (TEXT), default.o.o_shippriority (INT4), default.o.o_totalprice (FLOAT8)} - PROJECTION(2) - => Targets: default.orders.o_orderkey (INT4), default.orders.o_custkey (INT4), default.orders.o_orderstatus (TEXT), default.orders.o_totalprice (FLOAT8), default.orders.o_orderdate (TEXT), default.orders.o_orderpriority (TEXT), default.orders.o_clerk (TEXT), default.orders.o_shippriority (INT4), default.orders.o_comment (TEXT) - => out schema: {(9) default.orders.o_clerk (TEXT), default.orders.o_comment (TEXT), default.orders.o_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4), default.orders.o_orderpriority (TEXT), default.orders.o_orderstatus (TEXT), default.orders.o_shippriority (INT4), default.orders.o_totalprice (FLOAT8)} - => in schema: {(9) default.orders.o_clerk (TEXT), default.orders.o_comment (TEXT), default.orders.o_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4), default.orders.o_orderpriority (TEXT), default.orders.o_orderstatus (TEXT), default.orders.o_shippriority (INT4), default.orders.o_totalprice (FLOAT8)} - SCAN(1) on default.orders - => target list: default.orders.o_clerk (TEXT), default.orders.o_comment (TEXT), default.orders.o_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4), default.orders.o_orderpriority (TEXT), default.orders.o_orderstatus (TEXT), default.orders.o_shippriority (INT4), default.orders.o_totalprice (FLOAT8) - => out schema: {(9) default.orders.o_clerk (TEXT), default.orders.o_comment (TEXT), default.orders.o_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4), default.orders.o_orderpriority (TEXT), default.orders.o_orderstatus (TEXT), default.orders.o_shippriority (INT4), default.orders.o_totalprice (FLOAT8)} - => in schema: {(9) default.orders.o_clerk (TEXT), default.orders.o_comment (TEXT), default.orders.o_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4), default.orders.o_orderpriority (TEXT), default.orders.o_orderstatus (TEXT), default.orders.o_shippriority (INT4), default.orders.o_totalprice (FLOAT8)} - -======================================================= -Block Id: eb_0000000000000_0000_000003 [INTERMEDIATE] -======================================================= - -[Incoming] -[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.customer.c_custkey (INT4), num=32) -[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.o.o_orderkey (INT4), num=32) - [Outgoing] [q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.customer.c_custkey (INT4), default.o.o_orderkey (INT4), num=32) +[Enforcers] + 0: type=Broadcast, tables=default.orders + SORT(12) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.o.o_orderkey (INT4) (asc) JOIN(8)(LEFT_OUTER) @@ -93,12 +56,22 @@ SORT(12) => target list: default.customer.c_custkey (INT4), default.o.o_orderkey (INT4) => out schema: {(2) default.customer.c_custkey (INT4), default.o.o_orderkey (INT4)} => in schema: {(2) default.customer.c_custkey (INT4), default.o.o_orderkey (INT4)} - SCAN(11) on eb_0000000000000_0000_000002 + TABLE_SUBQUERY(3) as default.o + => Targets: default.o.o_orderkey (INT4) => out schema: {(1) default.o.o_orderkey (INT4)} - => in schema: {(1) default.o.o_orderkey (INT4)} - SCAN(10) on eb_0000000000000_0000_000001 + => in schema: {(9) default.o.o_clerk (TEXT), default.o.o_comment (TEXT), default.o.o_custkey (INT4), default.o.o_orderdate (TEXT), default.o.o_orderkey (INT4), default.o.o_orderpriority (TEXT), default.o.o_orderstatus (TEXT), default.o.o_shippriority (INT4), default.o.o_totalprice (FLOAT8)} + PROJECTION(2) + => Targets: default.orders.o_orderkey (INT4), default.orders.o_custkey (INT4), default.orders.o_orderstatus (TEXT), default.orders.o_totalprice (FLOAT8), default.orders.o_orderdate (TEXT), default.orders.o_orderpriority (TEXT), default.orders.o_clerk (TEXT), default.orders.o_shippriority (INT4), default.orders.o_comment (TEXT) + => out schema: {(9) default.orders.o_clerk (TEXT), default.orders.o_comment (TEXT), default.orders.o_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4), default.orders.o_orderpriority (TEXT), default.orders.o_orderstatus (TEXT), default.orders.o_shippriority (INT4), default.orders.o_totalprice (FLOAT8)} + => in schema: {(9) default.orders.o_clerk (TEXT), default.orders.o_comment (TEXT), default.orders.o_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4), default.orders.o_orderpriority (TEXT), default.orders.o_orderstatus (TEXT), default.orders.o_shippriority (INT4), default.orders.o_totalprice (FLOAT8)} + SCAN(1) on default.orders + => target list: default.orders.o_clerk (TEXT), default.orders.o_comment (TEXT), default.orders.o_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4), default.orders.o_orderpriority (TEXT), default.orders.o_orderstatus (TEXT), default.orders.o_shippriority (INT4), default.orders.o_totalprice (FLOAT8) + => out schema: {(9) default.orders.o_clerk (TEXT), default.orders.o_comment (TEXT), default.orders.o_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4), default.orders.o_orderpriority (TEXT), default.orders.o_orderstatus (TEXT), default.orders.o_shippriority (INT4), default.orders.o_totalprice (FLOAT8)} + => in schema: {(9) default.orders.o_clerk (TEXT), default.orders.o_comment (TEXT), default.orders.o_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4), default.orders.o_orderpriority (TEXT), default.orders.o_orderstatus (TEXT), default.orders.o_shippriority (INT4), default.orders.o_totalprice (FLOAT8)} + SCAN(0) on default.customer + => target list: default.customer.c_custkey (INT4) => out schema: {(1) default.customer.c_custkey (INT4)} - => in schema: {(1) default.customer.c_custkey (INT4)} + => in schema: {(8) default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT)} ======================================================= Block Id: eb_0000000000000_0000_000004 [ROOT] diff --git a/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithConstantExpr2.Sort.plan b/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithConstantExpr2.Sort.plan index e4156aa9c8..4fb9de0ce5 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithConstantExpr2.Sort.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithConstantExpr2.Sort.plan @@ -31,61 +31,24 @@ Execution Block Graph (TERMINAL - eb_0000000000000_0000_000005) |-eb_0000000000000_0000_000005 |-eb_0000000000000_0000_000004 |-eb_0000000000000_0000_000003 - |-eb_0000000000000_0000_000002 - |-eb_0000000000000_0000_000001 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000001 -2: eb_0000000000000_0000_000002 -3: eb_0000000000000_0000_000003 -4: eb_0000000000000_0000_000004 -5: eb_0000000000000_0000_000005 +1: eb_0000000000000_0000_000003 +2: eb_0000000000000_0000_000004 +3: eb_0000000000000_0000_000005 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000001 [LEAF] +Block Id: eb_0000000000000_0000_000003 [LEAF] ======================================================= -[Outgoing] -[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.customer.c_custkey (INT4), num=32) - -SCAN(0) on default.customer - => target list: default.customer.c_custkey (INT4) - => out schema: {(1) default.customer.c_custkey (INT4)} - => in schema: {(8) default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT)} - -======================================================= -Block Id: eb_0000000000000_0000_000002 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.o.o_orderkey (INT4), num=32) - -TABLE_SUBQUERY(3) as default.o - => Targets: default.o.o_orderkey (INT4) - => out schema: {(1) default.o.o_orderkey (INT4)} - => in schema: {(9) default.o.o_clerk (TEXT), default.o.o_comment (TEXT), default.o.o_custkey (INT4), default.o.o_orderdate (TEXT), default.o.o_orderkey (INT4), default.o.o_orderpriority (TEXT), default.o.o_orderstatus (TEXT), default.o.o_shippriority (INT4), default.o.o_totalprice (FLOAT8)} - PROJECTION(2) - => Targets: default.orders.o_orderkey (INT4), default.orders.o_custkey (INT4), default.orders.o_orderstatus (TEXT), default.orders.o_totalprice (FLOAT8), default.orders.o_orderdate (TEXT), default.orders.o_orderpriority (TEXT), default.orders.o_clerk (TEXT), default.orders.o_shippriority (INT4), default.orders.o_comment (TEXT) - => out schema: {(9) default.orders.o_clerk (TEXT), default.orders.o_comment (TEXT), default.orders.o_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4), default.orders.o_orderpriority (TEXT), default.orders.o_orderstatus (TEXT), default.orders.o_shippriority (INT4), default.orders.o_totalprice (FLOAT8)} - => in schema: {(9) default.orders.o_clerk (TEXT), default.orders.o_comment (TEXT), default.orders.o_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4), default.orders.o_orderpriority (TEXT), default.orders.o_orderstatus (TEXT), default.orders.o_shippriority (INT4), default.orders.o_totalprice (FLOAT8)} - SCAN(1) on default.orders - => target list: default.orders.o_clerk (TEXT), default.orders.o_comment (TEXT), default.orders.o_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4), default.orders.o_orderpriority (TEXT), default.orders.o_orderstatus (TEXT), default.orders.o_shippriority (INT4), default.orders.o_totalprice (FLOAT8) - => out schema: {(9) default.orders.o_clerk (TEXT), default.orders.o_comment (TEXT), default.orders.o_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4), default.orders.o_orderpriority (TEXT), default.orders.o_orderstatus (TEXT), default.orders.o_shippriority (INT4), default.orders.o_totalprice (FLOAT8)} - => in schema: {(9) default.orders.o_clerk (TEXT), default.orders.o_comment (TEXT), default.orders.o_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4), default.orders.o_orderpriority (TEXT), default.orders.o_orderstatus (TEXT), default.orders.o_shippriority (INT4), default.orders.o_totalprice (FLOAT8)} - -======================================================= -Block Id: eb_0000000000000_0000_000003 [INTERMEDIATE] -======================================================= - -[Incoming] -[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.customer.c_custkey (INT4), num=32) -[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.o.o_orderkey (INT4), num=32) - [Outgoing] [q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.customer.c_custkey (INT4), default.o.o_orderkey (INT4), num=32) +[Enforcers] + 0: type=Broadcast, tables=default.orders + SORT(12) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.o.o_orderkey (INT4) (asc) JOIN(8)(LEFT_OUTER) @@ -93,12 +56,22 @@ SORT(12) => target list: default.customer.c_custkey (INT4), default.o.o_orderkey (INT4) => out schema: {(2) default.customer.c_custkey (INT4), default.o.o_orderkey (INT4)} => in schema: {(2) default.customer.c_custkey (INT4), default.o.o_orderkey (INT4)} - SCAN(11) on eb_0000000000000_0000_000002 + TABLE_SUBQUERY(3) as default.o + => Targets: default.o.o_orderkey (INT4) => out schema: {(1) default.o.o_orderkey (INT4)} - => in schema: {(1) default.o.o_orderkey (INT4)} - SCAN(10) on eb_0000000000000_0000_000001 + => in schema: {(9) default.o.o_clerk (TEXT), default.o.o_comment (TEXT), default.o.o_custkey (INT4), default.o.o_orderdate (TEXT), default.o.o_orderkey (INT4), default.o.o_orderpriority (TEXT), default.o.o_orderstatus (TEXT), default.o.o_shippriority (INT4), default.o.o_totalprice (FLOAT8)} + PROJECTION(2) + => Targets: default.orders.o_orderkey (INT4), default.orders.o_custkey (INT4), default.orders.o_orderstatus (TEXT), default.orders.o_totalprice (FLOAT8), default.orders.o_orderdate (TEXT), default.orders.o_orderpriority (TEXT), default.orders.o_clerk (TEXT), default.orders.o_shippriority (INT4), default.orders.o_comment (TEXT) + => out schema: {(9) default.orders.o_clerk (TEXT), default.orders.o_comment (TEXT), default.orders.o_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4), default.orders.o_orderpriority (TEXT), default.orders.o_orderstatus (TEXT), default.orders.o_shippriority (INT4), default.orders.o_totalprice (FLOAT8)} + => in schema: {(9) default.orders.o_clerk (TEXT), default.orders.o_comment (TEXT), default.orders.o_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4), default.orders.o_orderpriority (TEXT), default.orders.o_orderstatus (TEXT), default.orders.o_shippriority (INT4), default.orders.o_totalprice (FLOAT8)} + SCAN(1) on default.orders + => target list: default.orders.o_clerk (TEXT), default.orders.o_comment (TEXT), default.orders.o_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4), default.orders.o_orderpriority (TEXT), default.orders.o_orderstatus (TEXT), default.orders.o_shippriority (INT4), default.orders.o_totalprice (FLOAT8) + => out schema: {(9) default.orders.o_clerk (TEXT), default.orders.o_comment (TEXT), default.orders.o_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4), default.orders.o_orderpriority (TEXT), default.orders.o_orderstatus (TEXT), default.orders.o_shippriority (INT4), default.orders.o_totalprice (FLOAT8)} + => in schema: {(9) default.orders.o_clerk (TEXT), default.orders.o_comment (TEXT), default.orders.o_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4), default.orders.o_orderpriority (TEXT), default.orders.o_orderstatus (TEXT), default.orders.o_shippriority (INT4), default.orders.o_totalprice (FLOAT8)} + SCAN(0) on default.customer + => target list: default.customer.c_custkey (INT4) => out schema: {(1) default.customer.c_custkey (INT4)} - => in schema: {(1) default.customer.c_custkey (INT4)} + => in schema: {(8) default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT)} ======================================================= Block Id: eb_0000000000000_0000_000004 [ROOT] diff --git a/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithConstantExpr3.Hash.plan b/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithConstantExpr3.Hash.plan index 0fe36c0009..6bde9f8d51 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithConstantExpr3.Hash.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithConstantExpr3.Hash.plan @@ -57,6 +57,9 @@ Block Id: eb_0000000000000_0000_000002 [LEAF] [Outgoing] [q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.customer.c_custkey (INT4), num=32) +[Enforcers] + 0: type=Broadcast, tables=default.customer + GROUP_BY(11)(c_custkey) => exprs: (min(default.customer.c_name (TEXT))) => target list: default.customer.c_custkey (INT4), ?min_2 (TEXT) diff --git a/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithConstantExpr3.Sort.plan b/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithConstantExpr3.Sort.plan index 0fe36c0009..6bde9f8d51 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithConstantExpr3.Sort.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithConstantExpr3.Sort.plan @@ -57,6 +57,9 @@ Block Id: eb_0000000000000_0000_000002 [LEAF] [Outgoing] [q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.customer.c_custkey (INT4), num=32) +[Enforcers] + 0: type=Broadcast, tables=default.customer + GROUP_BY(11)(c_custkey) => exprs: (min(default.customer.c_name (TEXT))) => target list: default.customer.c_custkey (INT4), ?min_2 (TEXT) diff --git a/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithEmptySubquery1.1.Hash.plan b/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithEmptySubquery1.1.Hash.plan index a978415bc7..2e0cc87499 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithEmptySubquery1.1.Hash.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithEmptySubquery1.1.Hash.plan @@ -37,88 +37,58 @@ Execution Block Graph (TERMINAL - eb_0000000000000_0000_000007) |-eb_0000000000000_0000_000007 |-eb_0000000000000_0000_000006 |-eb_0000000000000_0000_000005 - |-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000001 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000001 -2: eb_0000000000000_0000_000004 -3: eb_0000000000000_0000_000005 -4: eb_0000000000000_0000_000006 -5: eb_0000000000000_0000_000007 +1: eb_0000000000000_0000_000005 +2: eb_0000000000000_0000_000006 +3: eb_0000000000000_0000_000007 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000001 [LEAF] +Block Id: eb_0000000000000_0000_000005 [LEAF] ======================================================= [Outgoing] -[q_0000000000000_0000] 1 => 5 (type=HASH_SHUFFLE, key=default.a.id (INT4), num=32) - -SCAN(0) on default.jointable11 as a - => target list: default.a.id (INT4) - => out schema: {(1) default.a.id (INT4)} - => in schema: {(2) default.a.id (INT4), default.a.name (TEXT)} - -======================================================= -Block Id: eb_0000000000000_0000_000004 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.b.id (INT4), num=32) +[q_0000000000000_0000] 5 => 6 (type=RANGE_SHUFFLE, key=default.a.id (INT4), num=32) [Enforcers] - 0: type=Broadcast, tables=default.lineitem - 1: type=Broadcast, tables=default.jointable12 + 0: type=Broadcast, tables=default.jointable12 + 1: type=Broadcast, tables=default.lineitem -TABLE_SUBQUERY(5) as default.b - => Targets: default.b.id (INT4) - => out schema: {(1) default.b.id (INT4)} - => in schema: {(1) default.b.id (INT4)} - PROJECTION(4) - => Targets: default.jointable12.id (INT4) - => out schema: {(1) default.jointable12.id (INT4)} - => in schema: {(1) default.jointable12.id (INT4)} - JOIN(10)(INNER) - => Join Cond: default.jointable12.id (INT4) = default.lineitem.l_orderkey (INT4) - => target list: default.jointable12.id (INT4) - => out schema: {(1) default.jointable12.id (INT4)} - => in schema: {(2) default.jointable12.id (INT4), default.lineitem.l_orderkey (INT4)} - SCAN(1) on default.jointable12 - => filter: default.jointable12.id (INT4) > 10 - => target list: default.jointable12.id (INT4) - => out schema: {(1) default.jointable12.id (INT4)} - => in schema: {(2) default.jointable12.id (INT4), default.jointable12.name (TEXT)} - SCAN(2) on default.lineitem - => target list: default.lineitem.l_orderkey (INT4) - => out schema: {(1) default.lineitem.l_orderkey (INT4)} - => in schema: {(16) default.lineitem.l_comment (TEXT), default.lineitem.l_commitdate (TEXT), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_linenumber (INT4), default.lineitem.l_linestatus (TEXT), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.lineitem.l_quantity (FLOAT8), default.lineitem.l_receiptdate (TEXT), default.lineitem.l_returnflag (TEXT), default.lineitem.l_shipdate (TEXT), default.lineitem.l_shipinstruct (TEXT), default.lineitem.l_shipmode (TEXT), default.lineitem.l_suppkey (INT4), default.lineitem.l_tax (FLOAT8)} - -======================================================= -Block Id: eb_0000000000000_0000_000005 [INTERMEDIATE] -======================================================= - -[Incoming] -[q_0000000000000_0000] 1 => 5 (type=HASH_SHUFFLE, key=default.a.id (INT4), num=32) -[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.b.id (INT4), num=32) - -[Outgoing] -[q_0000000000000_0000] 5 => 6 (type=RANGE_SHUFFLE, key=default.a.id (INT4), num=32) - -SORT(15) +SORT(17) => Sort Keys: default.a.id (INT4) (asc) JOIN(11)(LEFT_OUTER) => Join Cond: default.a.id (INT4) = default.b.id (INT4) => target list: default.a.id (INT4), default.b.id (INT4) => out schema: {(2) default.a.id (INT4), default.b.id (INT4)} => in schema: {(2) default.a.id (INT4), default.b.id (INT4)} - SCAN(14) on eb_0000000000000_0000_000004 + TABLE_SUBQUERY(5) as default.b + => Targets: default.b.id (INT4) => out schema: {(1) default.b.id (INT4)} - => in schema: {(1) default.b.id (INT4)} - SCAN(13) on eb_0000000000000_0000_000001 + => in schema: {(1) default.b.id (INT4)} + PROJECTION(4) + => Targets: default.jointable12.id (INT4) + => out schema: {(1) default.jointable12.id (INT4)} + => in schema: {(1) default.jointable12.id (INT4)} + JOIN(10)(INNER) + => Join Cond: default.jointable12.id (INT4) = default.lineitem.l_orderkey (INT4) + => target list: default.jointable12.id (INT4) + => out schema: {(1) default.jointable12.id (INT4)} + => in schema: {(2) default.jointable12.id (INT4), default.lineitem.l_orderkey (INT4)} + SCAN(1) on default.jointable12 + => filter: default.jointable12.id (INT4) > 10 + => target list: default.jointable12.id (INT4) + => out schema: {(1) default.jointable12.id (INT4)} + => in schema: {(2) default.jointable12.id (INT4), default.jointable12.name (TEXT)} + SCAN(2) on default.lineitem + => target list: default.lineitem.l_orderkey (INT4) + => out schema: {(1) default.lineitem.l_orderkey (INT4)} + => in schema: {(16) default.lineitem.l_comment (TEXT), default.lineitem.l_commitdate (TEXT), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_linenumber (INT4), default.lineitem.l_linestatus (TEXT), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.lineitem.l_quantity (FLOAT8), default.lineitem.l_receiptdate (TEXT), default.lineitem.l_returnflag (TEXT), default.lineitem.l_shipdate (TEXT), default.lineitem.l_shipinstruct (TEXT), default.lineitem.l_shipmode (TEXT), default.lineitem.l_suppkey (INT4), default.lineitem.l_tax (FLOAT8)} + SCAN(0) on default.jointable11 as a + => target list: default.a.id (INT4) => out schema: {(1) default.a.id (INT4)} - => in schema: {(1) default.a.id (INT4)} + => in schema: {(2) default.a.id (INT4), default.a.name (TEXT)} ======================================================= Block Id: eb_0000000000000_0000_000006 [ROOT] @@ -132,7 +102,7 @@ Block Id: eb_0000000000000_0000_000006 [ROOT] SORT(7) => Sort Keys: default.a.id (INT4) (asc) - SCAN(16) on eb_0000000000000_0000_000005 + SCAN(18) on eb_0000000000000_0000_000005 => out schema: {(2) default.a.id (INT4), default.b.id (INT4)} => in schema: {(2) default.a.id (INT4), default.b.id (INT4)} diff --git a/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithEmptySubquery1.1.Sort.plan b/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithEmptySubquery1.1.Sort.plan index a978415bc7..2e0cc87499 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithEmptySubquery1.1.Sort.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithEmptySubquery1.1.Sort.plan @@ -37,88 +37,58 @@ Execution Block Graph (TERMINAL - eb_0000000000000_0000_000007) |-eb_0000000000000_0000_000007 |-eb_0000000000000_0000_000006 |-eb_0000000000000_0000_000005 - |-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000001 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000001 -2: eb_0000000000000_0000_000004 -3: eb_0000000000000_0000_000005 -4: eb_0000000000000_0000_000006 -5: eb_0000000000000_0000_000007 +1: eb_0000000000000_0000_000005 +2: eb_0000000000000_0000_000006 +3: eb_0000000000000_0000_000007 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000001 [LEAF] +Block Id: eb_0000000000000_0000_000005 [LEAF] ======================================================= [Outgoing] -[q_0000000000000_0000] 1 => 5 (type=HASH_SHUFFLE, key=default.a.id (INT4), num=32) - -SCAN(0) on default.jointable11 as a - => target list: default.a.id (INT4) - => out schema: {(1) default.a.id (INT4)} - => in schema: {(2) default.a.id (INT4), default.a.name (TEXT)} - -======================================================= -Block Id: eb_0000000000000_0000_000004 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.b.id (INT4), num=32) +[q_0000000000000_0000] 5 => 6 (type=RANGE_SHUFFLE, key=default.a.id (INT4), num=32) [Enforcers] - 0: type=Broadcast, tables=default.lineitem - 1: type=Broadcast, tables=default.jointable12 + 0: type=Broadcast, tables=default.jointable12 + 1: type=Broadcast, tables=default.lineitem -TABLE_SUBQUERY(5) as default.b - => Targets: default.b.id (INT4) - => out schema: {(1) default.b.id (INT4)} - => in schema: {(1) default.b.id (INT4)} - PROJECTION(4) - => Targets: default.jointable12.id (INT4) - => out schema: {(1) default.jointable12.id (INT4)} - => in schema: {(1) default.jointable12.id (INT4)} - JOIN(10)(INNER) - => Join Cond: default.jointable12.id (INT4) = default.lineitem.l_orderkey (INT4) - => target list: default.jointable12.id (INT4) - => out schema: {(1) default.jointable12.id (INT4)} - => in schema: {(2) default.jointable12.id (INT4), default.lineitem.l_orderkey (INT4)} - SCAN(1) on default.jointable12 - => filter: default.jointable12.id (INT4) > 10 - => target list: default.jointable12.id (INT4) - => out schema: {(1) default.jointable12.id (INT4)} - => in schema: {(2) default.jointable12.id (INT4), default.jointable12.name (TEXT)} - SCAN(2) on default.lineitem - => target list: default.lineitem.l_orderkey (INT4) - => out schema: {(1) default.lineitem.l_orderkey (INT4)} - => in schema: {(16) default.lineitem.l_comment (TEXT), default.lineitem.l_commitdate (TEXT), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_linenumber (INT4), default.lineitem.l_linestatus (TEXT), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.lineitem.l_quantity (FLOAT8), default.lineitem.l_receiptdate (TEXT), default.lineitem.l_returnflag (TEXT), default.lineitem.l_shipdate (TEXT), default.lineitem.l_shipinstruct (TEXT), default.lineitem.l_shipmode (TEXT), default.lineitem.l_suppkey (INT4), default.lineitem.l_tax (FLOAT8)} - -======================================================= -Block Id: eb_0000000000000_0000_000005 [INTERMEDIATE] -======================================================= - -[Incoming] -[q_0000000000000_0000] 1 => 5 (type=HASH_SHUFFLE, key=default.a.id (INT4), num=32) -[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.b.id (INT4), num=32) - -[Outgoing] -[q_0000000000000_0000] 5 => 6 (type=RANGE_SHUFFLE, key=default.a.id (INT4), num=32) - -SORT(15) +SORT(17) => Sort Keys: default.a.id (INT4) (asc) JOIN(11)(LEFT_OUTER) => Join Cond: default.a.id (INT4) = default.b.id (INT4) => target list: default.a.id (INT4), default.b.id (INT4) => out schema: {(2) default.a.id (INT4), default.b.id (INT4)} => in schema: {(2) default.a.id (INT4), default.b.id (INT4)} - SCAN(14) on eb_0000000000000_0000_000004 + TABLE_SUBQUERY(5) as default.b + => Targets: default.b.id (INT4) => out schema: {(1) default.b.id (INT4)} - => in schema: {(1) default.b.id (INT4)} - SCAN(13) on eb_0000000000000_0000_000001 + => in schema: {(1) default.b.id (INT4)} + PROJECTION(4) + => Targets: default.jointable12.id (INT4) + => out schema: {(1) default.jointable12.id (INT4)} + => in schema: {(1) default.jointable12.id (INT4)} + JOIN(10)(INNER) + => Join Cond: default.jointable12.id (INT4) = default.lineitem.l_orderkey (INT4) + => target list: default.jointable12.id (INT4) + => out schema: {(1) default.jointable12.id (INT4)} + => in schema: {(2) default.jointable12.id (INT4), default.lineitem.l_orderkey (INT4)} + SCAN(1) on default.jointable12 + => filter: default.jointable12.id (INT4) > 10 + => target list: default.jointable12.id (INT4) + => out schema: {(1) default.jointable12.id (INT4)} + => in schema: {(2) default.jointable12.id (INT4), default.jointable12.name (TEXT)} + SCAN(2) on default.lineitem + => target list: default.lineitem.l_orderkey (INT4) + => out schema: {(1) default.lineitem.l_orderkey (INT4)} + => in schema: {(16) default.lineitem.l_comment (TEXT), default.lineitem.l_commitdate (TEXT), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_linenumber (INT4), default.lineitem.l_linestatus (TEXT), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.lineitem.l_quantity (FLOAT8), default.lineitem.l_receiptdate (TEXT), default.lineitem.l_returnflag (TEXT), default.lineitem.l_shipdate (TEXT), default.lineitem.l_shipinstruct (TEXT), default.lineitem.l_shipmode (TEXT), default.lineitem.l_suppkey (INT4), default.lineitem.l_tax (FLOAT8)} + SCAN(0) on default.jointable11 as a + => target list: default.a.id (INT4) => out schema: {(1) default.a.id (INT4)} - => in schema: {(1) default.a.id (INT4)} + => in schema: {(2) default.a.id (INT4), default.a.name (TEXT)} ======================================================= Block Id: eb_0000000000000_0000_000006 [ROOT] @@ -132,7 +102,7 @@ Block Id: eb_0000000000000_0000_000006 [ROOT] SORT(7) => Sort Keys: default.a.id (INT4) (asc) - SCAN(16) on eb_0000000000000_0000_000005 + SCAN(18) on eb_0000000000000_0000_000005 => out schema: {(2) default.a.id (INT4), default.b.id (INT4)} => in schema: {(2) default.a.id (INT4), default.b.id (INT4)} diff --git a/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithEmptySubquery2.1.Hash.plan b/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithEmptySubquery2.1.Hash.plan index c4c25da907..63e5904cc3 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithEmptySubquery2.1.Hash.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithEmptySubquery2.1.Hash.plan @@ -34,82 +34,53 @@ Execution Block Graph (TERMINAL - eb_0000000000000_0000_000006) ------------------------------------------------------------------------------- |-eb_0000000000000_0000_000006 |-eb_0000000000000_0000_000005 - |-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 -3: eb_0000000000000_0000_000005 -4: eb_0000000000000_0000_000006 +1: eb_0000000000000_0000_000005 +2: eb_0000000000000_0000_000006 ------------------------------------------------------------------------------- -======================================================= -Block Id: eb_0000000000000_0000_000003 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.a.id (INT4), num=32) - -[Enforcers] - 0: type=Broadcast, tables=default.lineitem - 1: type=Broadcast, tables=default.jointable12 - -TABLE_SUBQUERY(4) as default.a - => Targets: default.a.id (INT4) - => out schema: {(1) default.a.id (INT4)} - => in schema: {(3) default.a.id (INT4), default.a.name (TEXT), default.a.l_shipdate (TEXT)} - PROJECTION(3) - => Targets: default.jointable12.id (INT4), default.jointable12.name (TEXT), default.lineitem.l_shipdate (TEXT) - => out schema: {(3) default.jointable12.id (INT4), default.jointable12.name (TEXT), default.lineitem.l_shipdate (TEXT)} - => in schema: {(3) default.jointable12.id (INT4), default.jointable12.name (TEXT), default.lineitem.l_shipdate (TEXT)} - JOIN(9)(INNER) - => Join Cond: default.jointable12.id (INT4) = default.lineitem.l_orderkey (INT4) - => target list: default.jointable12.id (INT4), default.jointable12.name (TEXT), default.lineitem.l_shipdate (TEXT) - => out schema: {(3) default.jointable12.id (INT4), default.jointable12.name (TEXT), default.lineitem.l_shipdate (TEXT)} - => in schema: {(4) default.jointable12.id (INT4), default.jointable12.name (TEXT), default.lineitem.l_orderkey (INT4), default.lineitem.l_shipdate (TEXT)} - SCAN(0) on default.jointable12 - => filter: default.jointable12.id (INT4) > 10 - => target list: default.jointable12.id (INT4), default.jointable12.name (TEXT) - => out schema: {(2) default.jointable12.id (INT4), default.jointable12.name (TEXT)} - => in schema: {(2) default.jointable12.id (INT4), default.jointable12.name (TEXT)} - SCAN(1) on default.lineitem - => target list: default.lineitem.l_orderkey (INT4), default.lineitem.l_shipdate (TEXT) - => out schema: {(2) default.lineitem.l_orderkey (INT4), default.lineitem.l_shipdate (TEXT)} - => in schema: {(16) default.lineitem.l_comment (TEXT), default.lineitem.l_commitdate (TEXT), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_linenumber (INT4), default.lineitem.l_linestatus (TEXT), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.lineitem.l_quantity (FLOAT8), default.lineitem.l_receiptdate (TEXT), default.lineitem.l_returnflag (TEXT), default.lineitem.l_shipdate (TEXT), default.lineitem.l_shipinstruct (TEXT), default.lineitem.l_shipmode (TEXT), default.lineitem.l_suppkey (INT4), default.lineitem.l_tax (FLOAT8)} - -======================================================= -Block Id: eb_0000000000000_0000_000004 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.b.id (INT4), num=32) - -SCAN(5) on default.jointable11 as b - => target list: default.b.id (INT4) - => out schema: {(1) default.b.id (INT4)} - => in schema: {(2) default.b.id (INT4), default.b.name (TEXT)} - ======================================================= Block Id: eb_0000000000000_0000_000005 [ROOT] ======================================================= -[Incoming] -[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.a.id (INT4), num=32) -[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.b.id (INT4), num=32) +[Enforcers] + 0: type=Broadcast, tables=default.jointable12 + 1: type=Broadcast, tables=default.lineitem + 2: type=Broadcast, tables=default.b JOIN(10)(LEFT_OUTER) => Join Cond: default.a.id (INT4) = default.b.id (INT4) => target list: default.a.id (INT4), default.b.id (INT4) => out schema: {(2) default.a.id (INT4), default.b.id (INT4)} => in schema: {(2) default.a.id (INT4), default.b.id (INT4)} - SCAN(13) on eb_0000000000000_0000_000004 + SCAN(5) on default.jointable11 as b + => target list: default.b.id (INT4) => out schema: {(1) default.b.id (INT4)} - => in schema: {(1) default.b.id (INT4)} - SCAN(12) on eb_0000000000000_0000_000003 + => in schema: {(2) default.b.id (INT4), default.b.name (TEXT)} + TABLE_SUBQUERY(4) as default.a + => Targets: default.a.id (INT4) => out schema: {(1) default.a.id (INT4)} - => in schema: {(1) default.a.id (INT4)} + => in schema: {(3) default.a.id (INT4), default.a.name (TEXT), default.a.l_shipdate (TEXT)} + PROJECTION(3) + => Targets: default.jointable12.id (INT4), default.jointable12.name (TEXT), default.lineitem.l_shipdate (TEXT) + => out schema: {(3) default.jointable12.id (INT4), default.jointable12.name (TEXT), default.lineitem.l_shipdate (TEXT)} + => in schema: {(3) default.jointable12.id (INT4), default.jointable12.name (TEXT), default.lineitem.l_shipdate (TEXT)} + JOIN(9)(INNER) + => Join Cond: default.jointable12.id (INT4) = default.lineitem.l_orderkey (INT4) + => target list: default.jointable12.id (INT4), default.jointable12.name (TEXT), default.lineitem.l_shipdate (TEXT) + => out schema: {(3) default.jointable12.id (INT4), default.jointable12.name (TEXT), default.lineitem.l_shipdate (TEXT)} + => in schema: {(4) default.jointable12.id (INT4), default.jointable12.name (TEXT), default.lineitem.l_orderkey (INT4), default.lineitem.l_shipdate (TEXT)} + SCAN(0) on default.jointable12 + => filter: default.jointable12.id (INT4) > 10 + => target list: default.jointable12.id (INT4), default.jointable12.name (TEXT) + => out schema: {(2) default.jointable12.id (INT4), default.jointable12.name (TEXT)} + => in schema: {(2) default.jointable12.id (INT4), default.jointable12.name (TEXT)} + SCAN(1) on default.lineitem + => target list: default.lineitem.l_orderkey (INT4), default.lineitem.l_shipdate (TEXT) + => out schema: {(2) default.lineitem.l_orderkey (INT4), default.lineitem.l_shipdate (TEXT)} + => in schema: {(16) default.lineitem.l_comment (TEXT), default.lineitem.l_commitdate (TEXT), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_linenumber (INT4), default.lineitem.l_linestatus (TEXT), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.lineitem.l_quantity (FLOAT8), default.lineitem.l_receiptdate (TEXT), default.lineitem.l_returnflag (TEXT), default.lineitem.l_shipdate (TEXT), default.lineitem.l_shipinstruct (TEXT), default.lineitem.l_shipmode (TEXT), default.lineitem.l_suppkey (INT4), default.lineitem.l_tax (FLOAT8)} ======================================================= Block Id: eb_0000000000000_0000_000006 [TERMINAL] diff --git a/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithEmptySubquery2.1.Sort.plan b/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithEmptySubquery2.1.Sort.plan index c4c25da907..63e5904cc3 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithEmptySubquery2.1.Sort.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithEmptySubquery2.1.Sort.plan @@ -34,82 +34,53 @@ Execution Block Graph (TERMINAL - eb_0000000000000_0000_000006) ------------------------------------------------------------------------------- |-eb_0000000000000_0000_000006 |-eb_0000000000000_0000_000005 - |-eb_0000000000000_0000_000004 - |-eb_0000000000000_0000_000003 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 -3: eb_0000000000000_0000_000005 -4: eb_0000000000000_0000_000006 +1: eb_0000000000000_0000_000005 +2: eb_0000000000000_0000_000006 ------------------------------------------------------------------------------- -======================================================= -Block Id: eb_0000000000000_0000_000003 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.a.id (INT4), num=32) - -[Enforcers] - 0: type=Broadcast, tables=default.lineitem - 1: type=Broadcast, tables=default.jointable12 - -TABLE_SUBQUERY(4) as default.a - => Targets: default.a.id (INT4) - => out schema: {(1) default.a.id (INT4)} - => in schema: {(3) default.a.id (INT4), default.a.name (TEXT), default.a.l_shipdate (TEXT)} - PROJECTION(3) - => Targets: default.jointable12.id (INT4), default.jointable12.name (TEXT), default.lineitem.l_shipdate (TEXT) - => out schema: {(3) default.jointable12.id (INT4), default.jointable12.name (TEXT), default.lineitem.l_shipdate (TEXT)} - => in schema: {(3) default.jointable12.id (INT4), default.jointable12.name (TEXT), default.lineitem.l_shipdate (TEXT)} - JOIN(9)(INNER) - => Join Cond: default.jointable12.id (INT4) = default.lineitem.l_orderkey (INT4) - => target list: default.jointable12.id (INT4), default.jointable12.name (TEXT), default.lineitem.l_shipdate (TEXT) - => out schema: {(3) default.jointable12.id (INT4), default.jointable12.name (TEXT), default.lineitem.l_shipdate (TEXT)} - => in schema: {(4) default.jointable12.id (INT4), default.jointable12.name (TEXT), default.lineitem.l_orderkey (INT4), default.lineitem.l_shipdate (TEXT)} - SCAN(0) on default.jointable12 - => filter: default.jointable12.id (INT4) > 10 - => target list: default.jointable12.id (INT4), default.jointable12.name (TEXT) - => out schema: {(2) default.jointable12.id (INT4), default.jointable12.name (TEXT)} - => in schema: {(2) default.jointable12.id (INT4), default.jointable12.name (TEXT)} - SCAN(1) on default.lineitem - => target list: default.lineitem.l_orderkey (INT4), default.lineitem.l_shipdate (TEXT) - => out schema: {(2) default.lineitem.l_orderkey (INT4), default.lineitem.l_shipdate (TEXT)} - => in schema: {(16) default.lineitem.l_comment (TEXT), default.lineitem.l_commitdate (TEXT), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_linenumber (INT4), default.lineitem.l_linestatus (TEXT), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.lineitem.l_quantity (FLOAT8), default.lineitem.l_receiptdate (TEXT), default.lineitem.l_returnflag (TEXT), default.lineitem.l_shipdate (TEXT), default.lineitem.l_shipinstruct (TEXT), default.lineitem.l_shipmode (TEXT), default.lineitem.l_suppkey (INT4), default.lineitem.l_tax (FLOAT8)} - -======================================================= -Block Id: eb_0000000000000_0000_000004 [LEAF] -======================================================= - -[Outgoing] -[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.b.id (INT4), num=32) - -SCAN(5) on default.jointable11 as b - => target list: default.b.id (INT4) - => out schema: {(1) default.b.id (INT4)} - => in schema: {(2) default.b.id (INT4), default.b.name (TEXT)} - ======================================================= Block Id: eb_0000000000000_0000_000005 [ROOT] ======================================================= -[Incoming] -[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.a.id (INT4), num=32) -[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.b.id (INT4), num=32) +[Enforcers] + 0: type=Broadcast, tables=default.jointable12 + 1: type=Broadcast, tables=default.lineitem + 2: type=Broadcast, tables=default.b JOIN(10)(LEFT_OUTER) => Join Cond: default.a.id (INT4) = default.b.id (INT4) => target list: default.a.id (INT4), default.b.id (INT4) => out schema: {(2) default.a.id (INT4), default.b.id (INT4)} => in schema: {(2) default.a.id (INT4), default.b.id (INT4)} - SCAN(13) on eb_0000000000000_0000_000004 + SCAN(5) on default.jointable11 as b + => target list: default.b.id (INT4) => out schema: {(1) default.b.id (INT4)} - => in schema: {(1) default.b.id (INT4)} - SCAN(12) on eb_0000000000000_0000_000003 + => in schema: {(2) default.b.id (INT4), default.b.name (TEXT)} + TABLE_SUBQUERY(4) as default.a + => Targets: default.a.id (INT4) => out schema: {(1) default.a.id (INT4)} - => in schema: {(1) default.a.id (INT4)} + => in schema: {(3) default.a.id (INT4), default.a.name (TEXT), default.a.l_shipdate (TEXT)} + PROJECTION(3) + => Targets: default.jointable12.id (INT4), default.jointable12.name (TEXT), default.lineitem.l_shipdate (TEXT) + => out schema: {(3) default.jointable12.id (INT4), default.jointable12.name (TEXT), default.lineitem.l_shipdate (TEXT)} + => in schema: {(3) default.jointable12.id (INT4), default.jointable12.name (TEXT), default.lineitem.l_shipdate (TEXT)} + JOIN(9)(INNER) + => Join Cond: default.jointable12.id (INT4) = default.lineitem.l_orderkey (INT4) + => target list: default.jointable12.id (INT4), default.jointable12.name (TEXT), default.lineitem.l_shipdate (TEXT) + => out schema: {(3) default.jointable12.id (INT4), default.jointable12.name (TEXT), default.lineitem.l_shipdate (TEXT)} + => in schema: {(4) default.jointable12.id (INT4), default.jointable12.name (TEXT), default.lineitem.l_orderkey (INT4), default.lineitem.l_shipdate (TEXT)} + SCAN(0) on default.jointable12 + => filter: default.jointable12.id (INT4) > 10 + => target list: default.jointable12.id (INT4), default.jointable12.name (TEXT) + => out schema: {(2) default.jointable12.id (INT4), default.jointable12.name (TEXT)} + => in schema: {(2) default.jointable12.id (INT4), default.jointable12.name (TEXT)} + SCAN(1) on default.lineitem + => target list: default.lineitem.l_orderkey (INT4), default.lineitem.l_shipdate (TEXT) + => out schema: {(2) default.lineitem.l_orderkey (INT4), default.lineitem.l_shipdate (TEXT)} + => in schema: {(16) default.lineitem.l_comment (TEXT), default.lineitem.l_commitdate (TEXT), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_linenumber (INT4), default.lineitem.l_linestatus (TEXT), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.lineitem.l_quantity (FLOAT8), default.lineitem.l_receiptdate (TEXT), default.lineitem.l_returnflag (TEXT), default.lineitem.l_shipdate (TEXT), default.lineitem.l_shipinstruct (TEXT), default.lineitem.l_shipmode (TEXT), default.lineitem.l_suppkey (INT4), default.lineitem.l_tax (FLOAT8)} ======================================================= Block Id: eb_0000000000000_0000_000006 [TERMINAL] From 2f6f90382e99cfc5c5f93d3511a80f828fc7326e Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 25 May 2015 18:50:32 +0900 Subject: [PATCH 31/37] All tests passed --- .../engine/planner/global/ExecutionBlock.java | 68 +-- .../engine/planner/global/MasterPlan.java | 5 + .../rewriter/rules/BroadcastJoinRule.java | 430 +++++++++--------- .../rewriter/rules/GlobalPlanRewriteUtil.java | 48 +- .../tajo/querymaster/Repartitioner.java | 65 +-- .../tajo/engine/query/TestOuterJoinQuery.java | 8 +- .../testBroadcastTwoPartJoin.Hash.plan | 3 +- .../testBroadcastTwoPartJoin.Sort.plan | 3 +- .../testComplexJoinCondition1.Hash.plan | 1 - .../testComplexJoinCondition1.Sort.plan | 1 - .../testComplexJoinCondition2.Hash.plan | 1 - .../testComplexJoinCondition2.Sort.plan | 1 - .../testComplexJoinCondition3.Hash.plan | 1 - .../testComplexJoinCondition3.Sort.plan | 1 - .../testComplexJoinCondition4.Hash.plan | 1 - .../testComplexJoinCondition4.Sort.plan | 1 - .../testCrossJoin.1.Hash.plan | 3 +- .../testCrossJoin.1.Sort.plan | 3 +- .../testCrossJoin.2.Hash.plan | 1 - .../testCrossJoin.2.Sort.plan | 1 - .../testCrossJoin.3.Hash.plan | 3 +- .../testCrossJoin.3.Sort.plan | 3 +- .../testCrossJoin.4.Hash.plan | 3 +- .../testCrossJoin.4.Sort.plan | 3 +- .../testCrossJoin.5.Hash.plan | 3 +- .../testCrossJoin.5.Sort.plan | 3 +- .../testCrossJoinAndCaseWhen.Hash.plan | 1 - .../testCrossJoinAndCaseWhen.Sort.plan | 1 - .../testCrossJoinWithAsterisk1.Hash.plan | 1 - .../testCrossJoinWithAsterisk1.Sort.plan | 1 - .../testCrossJoinWithAsterisk2.Hash.plan | 3 +- .../testCrossJoinWithAsterisk2.Sort.plan | 3 +- .../testCrossJoinWithAsterisk3.Hash.plan | 3 +- .../testCrossJoinWithAsterisk3.Sort.plan | 3 +- .../testCrossJoinWithAsterisk4.Hash.plan | 3 +- .../testCrossJoinWithAsterisk4.Sort.plan | 3 +- .../testCrossJoinWithEmptyTable1.Hash.plan | 1 - .../testCrossJoinWithEmptyTable1.Sort.plan | 1 - ...oinWithThetaJoinConditionInWhere.Hash.plan | 1 - ...oinWithThetaJoinConditionInWhere.Sort.plan | 1 - .../testDifferentTypesJoinCondition.Hash.plan | 1 - .../testDifferentTypesJoinCondition.Sort.plan | 1 - .../testInnerJoinAndCaseWhen.Hash.plan | 1 - .../testInnerJoinAndCaseWhen.Sort.plan | 1 - .../testInnerJoinWithEmptyTable.Hash.plan | 1 - .../testInnerJoinWithEmptyTable.Sort.plan | 1 - ...oinWithThetaJoinConditionInWhere.Hash.plan | 3 +- ...oinWithThetaJoinConditionInWhere.Sort.plan | 3 +- .../testJoinAsterisk.Hash.plan | 1 - .../testJoinAsterisk.Sort.plan | 1 - .../testJoinCoReferredEvals1.Hash.plan | 1 - .../testJoinCoReferredEvals1.Sort.plan | 1 - ...oinCoReferredEvalsWithSameExprs1.Hash.plan | 3 +- ...oinCoReferredEvalsWithSameExprs1.Sort.plan | 3 +- ...oinCoReferredEvalsWithSameExprs2.Hash.plan | 3 +- ...oinCoReferredEvalsWithSameExprs2.Sort.plan | 3 +- .../testJoinOnMultipleDatabases.Hash.plan | 9 +- .../testJoinOnMultipleDatabases.Sort.plan | 9 +- .../testJoinWithMultipleJoinQual1.Hash.plan | 3 +- .../testJoinWithMultipleJoinQual1.Sort.plan | 3 +- .../testJoinWithOrPredicates.Hash.plan | 3 +- .../testJoinWithOrPredicates.Sort.plan | 3 +- .../testNaturalJoin.Hash.plan | 3 +- .../testNaturalJoin.Sort.plan | 3 +- .../testTPCHQ2Join.Hash.plan | 3 +- .../testTPCHQ2Join.Sort.plan | 3 +- .../testWhereClauseJoin1.Hash.plan | 1 - .../testWhereClauseJoin1.Sort.plan | 1 - .../testWhereClauseJoin2.Hash.plan | 1 - .../testWhereClauseJoin2.Sort.plan | 1 - .../testWhereClauseJoin3.Hash.plan | 1 - .../testWhereClauseJoin3.Sort.plan | 1 - .../testWhereClauseJoin4.Hash.plan | 3 +- .../testWhereClauseJoin4.Sort.plan | 3 +- .../testWhereClauseJoin5.Hash.plan | 3 +- .../testWhereClauseJoin5.Sort.plan | 3 +- .../testWhereClauseJoin6.Hash.plan | 1 - .../testWhereClauseJoin6.Sort.plan | 1 - .../testBroadcastSubquery.Hash.plan | 3 +- .../testBroadcastSubquery.Sort.plan | 3 +- .../testBroadcastSubquery2.Hash.plan | 3 +- .../testBroadcastSubquery2.Sort.plan | 3 +- .../testComplexJoinCondition5.Hash.plan | 1 - .../testComplexJoinCondition5.Sort.plan | 1 - .../testJoinWithMultipleJoinQual2.Hash.plan | 5 +- .../testJoinWithMultipleJoinQual2.Sort.plan | 5 +- .../testJoinWithMultipleJoinQual3.Hash.plan | 7 +- .../testJoinWithMultipleJoinQual3.Sort.plan | 7 +- .../testJoinWithMultipleJoinQual4.Hash.plan | 7 +- .../testJoinWithMultipleJoinQual4.Sort.plan | 7 +- .../testComplexJoinsWithCaseWhen.Hash.plan | 5 +- .../testComplexJoinsWithCaseWhen.Sort.plan | 5 +- .../testInnerAndOuterWithEmpty.1.Hash.plan | 5 +- .../testInnerAndOuterWithEmpty.1.Sort.plan | 5 +- .../testFullOuterJoin1.Hash.plan | 58 ++- .../testFullOuterJoin1.Sort.plan | 58 ++- ...uterJoinPredicationCaseByCase1.1.Hash.plan | 153 +++++++ ...icationCaseByCase1.1.Hash_NoBroadcast.plan | 153 +++++++ ...uterJoinPredicationCaseByCase1.1.Sort.plan | 153 +++++++ ...icationCaseByCase1.1.Sort_NoBroadcast.plan | 153 +++++++ ...llOuterJoinPredicationCaseByCase1.1.result | 9 + ...testFullOuterJoinWithEmptyTable1.Hash.plan | 58 ++- ...testFullOuterJoinWithEmptyTable1.Sort.plan | 58 ++- ...erJoinPredicationCaseByCase2_1.1.Hash.plan | 3 +- ...erJoinPredicationCaseByCase2_1.1.Sort.plan | 3 +- ...uterJoinPredicationCaseByCase1.1.Hash.plan | 101 ++++ ...icationCaseByCase1.1.Hash_NoBroadcast.plan | 128 ++++++ ...uterJoinPredicationCaseByCase1.1.Sort.plan | 101 ++++ ...icationCaseByCase1.1.Sort_NoBroadcast.plan | 128 ++++++ ...htOuterJoinPredicationCaseByCase1.1.result | 4 + ...uterJoinPredicationCaseByCase2.1.Hash.plan | 76 ++++ ...icationCaseByCase2.1.Hash_NoBroadcast.plan | 132 ++++++ ...uterJoinPredicationCaseByCase2.1.Sort.plan | 76 ++++ ...icationCaseByCase2.1.Sort_NoBroadcast.plan | 132 ++++++ ...htOuterJoinPredicationCaseByCase2.1.result | 4 + ...uterJoinPredicationCaseByCase3.1.Hash.plan | 101 ++++ ...icationCaseByCase3.1.Hash_NoBroadcast.plan | 128 ++++++ ...uterJoinPredicationCaseByCase3.1.Sort.plan | 101 ++++ ...icationCaseByCase3.1.Sort_NoBroadcast.plan | 128 ++++++ ...htOuterJoinPredicationCaseByCase3.1.result | 4 + ...stLeftOuterJoinWithConstantExpr3.Hash.plan | 3 - ...stLeftOuterJoinWithConstantExpr3.Sort.plan | 3 - ...eftOuterJoinWithEmptySubquery2.1.Hash.plan | 3 +- ...eftOuterJoinWithEmptySubquery2.1.Sort.plan | 3 +- 124 files changed, 2502 insertions(+), 565 deletions(-) create mode 100644 tajo-core/src/test/resources/results/TestOuterJoinQuery/testFullOuterJoinPredicationCaseByCase1.1.Hash.plan create mode 100644 tajo-core/src/test/resources/results/TestOuterJoinQuery/testFullOuterJoinPredicationCaseByCase1.1.Hash_NoBroadcast.plan create mode 100644 tajo-core/src/test/resources/results/TestOuterJoinQuery/testFullOuterJoinPredicationCaseByCase1.1.Sort.plan create mode 100644 tajo-core/src/test/resources/results/TestOuterJoinQuery/testFullOuterJoinPredicationCaseByCase1.1.Sort_NoBroadcast.plan create mode 100644 tajo-core/src/test/resources/results/TestOuterJoinQuery/testFullOuterJoinPredicationCaseByCase1.1.result create mode 100644 tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase1.1.Hash.plan create mode 100644 tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase1.1.Hash_NoBroadcast.plan create mode 100644 tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase1.1.Sort.plan create mode 100644 tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase1.1.Sort_NoBroadcast.plan create mode 100644 tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase1.1.result create mode 100644 tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase2.1.Hash.plan create mode 100644 tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase2.1.Hash_NoBroadcast.plan create mode 100644 tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase2.1.Sort.plan create mode 100644 tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase2.1.Sort_NoBroadcast.plan create mode 100644 tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase2.1.result create mode 100644 tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase3.1.Hash.plan create mode 100644 tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase3.1.Hash_NoBroadcast.plan create mode 100644 tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase3.1.Sort.plan create mode 100644 tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase3.1.Sort_NoBroadcast.plan create mode 100644 tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase3.1.result diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java index a5f08517d4..2d02443d14 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java @@ -17,6 +17,7 @@ import org.apache.tajo.ExecutionBlockId; import org.apache.tajo.engine.planner.enforce.Enforcer; import org.apache.tajo.plan.logical.*; +import org.apache.tajo.util.TUtil; import java.util.*; @@ -41,7 +42,7 @@ public class ExecutionBlock { private boolean hasUnionPlan; private boolean isUnionOnly; - private Set broadcasted = new HashSet(); + private Map broadcastRelations = TUtil.newHashMap(); /* * The nullSupplying and preservedRow flags are used for finding which relations will be broadcasted. @@ -127,7 +128,7 @@ public StoreTableNode getStoreTableNode() { public int getNonBroadcastRelNum() { int nonBroadcastRelNum = 0; for (ScanNode scanNode : scanlist) { - if (!broadcasted.contains(scanNode.getCanonicalName())) { + if (!broadcastRelations.containsKey(scanNode.getCanonicalName())) { nonBroadcastRelNum++; } } @@ -150,67 +151,26 @@ public boolean isUnionOnly() { return isUnionOnly; } - public void addBroadcastRelation(String tableName) { - broadcasted.add(tableName); - enforcer.addBroadcast(tableName); + public void addBroadcastRelation(ScanNode relationNode) { + broadcastRelations.put(relationNode.getCanonicalName(), relationNode); + enforcer.addBroadcast(relationNode.getCanonicalName()); } - public void removeBroadcastRelation(String tableName) { - broadcasted.remove(tableName); - enforcer.removeBroadcast(tableName); + public void removeBroadcastRelation(ScanNode relationNode) { + broadcastRelations.remove(relationNode.getCanonicalName()); + enforcer.removeBroadcast(relationNode.getCanonicalName()); } - public boolean isBroadcastRelation(String tableName) { - return broadcasted.contains(tableName); + public boolean isBroadcastRelation(ScanNode relationNode) { + return broadcastRelations.containsKey(relationNode.getCanonicalName()); } public boolean hasBroadcastRelation() { - return broadcasted.size() > 0; + return broadcastRelations.size() > 0; } - /** - * Check this execution block is broadcastable. - * This execution block is broadcastable only when its every input is broadcastable. - * @return true if broadcastable - */ - public boolean isBroadcastable(final long broadcastThreshold) { - long totalTableVolume = 0; - for (ScanNode scanNode : scanlist) { - long volume = getTableVolume(scanNode); - if (volume == Long.MAX_VALUE) { - return false; - } - totalTableVolume += volume; - if (totalTableVolume > broadcastThreshold) { - return false; - } - } - return true; - } - - /** - * Get a volume of a table of a partitioned table - * @param scanNode ScanNode corresponding to a table - * @return table volume (bytes) - */ - private static long getTableVolume(ScanNode scanNode) { - if (scanNode.getTableDesc().hasStats()) { - long scanBytes = scanNode.getTableDesc().getStats().getNumBytes(); - if (scanNode.getType() == NodeType.PARTITIONS_SCAN) { - PartitionedTableScanNode pScanNode = (PartitionedTableScanNode) scanNode; - if (pScanNode.getInputPaths() == null || pScanNode.getInputPaths().length == 0) { - scanBytes = 0L; - } - } - - return scanBytes; - } else { - return Long.MAX_VALUE; - } - } - - public Collection getBroadcastTables() { - return broadcasted; + public Collection getBroadcastRelations() { + return broadcastRelations.values(); } public String toString() { diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/MasterPlan.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/MasterPlan.java index 97241955d5..6e9b74f7a0 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/MasterPlan.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/MasterPlan.java @@ -26,6 +26,7 @@ import org.apache.tajo.plan.LogicalPlan; import org.apache.tajo.plan.util.PlannerUtil; import org.apache.tajo.engine.planner.enforce.Enforcer; +import org.apache.tajo.util.graph.DirectedGraphVisitor; import org.apache.tajo.util.graph.SimpleDirectedGraph; import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.ipc.TajoWorkerProtocol; @@ -218,6 +219,10 @@ public ExecutionBlock getChild(ExecutionBlock executionBlock, int idx) { return getChild(executionBlock.getId(), idx); } + public void accept(ExecutionBlockId v, DirectedGraphVisitor visitor) { + execBlockGraph.accept(v, visitor); + } + @Override public String toString() { StringBuilder sb = new StringBuilder(); diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java index 8b5ba50441..3c835c4cf3 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java @@ -18,9 +18,12 @@ package org.apache.tajo.engine.planner.global.rewriter.rules; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import org.apache.tajo.ExecutionBlockId; import org.apache.tajo.OverridableConf; import org.apache.tajo.SessionVars; +import org.apache.tajo.algebra.JoinType; import org.apache.tajo.engine.planner.global.ExecutionBlock; import org.apache.tajo.engine.planner.global.GlobalPlanner; import org.apache.tajo.engine.planner.global.MasterPlan; @@ -29,14 +32,21 @@ import org.apache.tajo.plan.PlanningException; import org.apache.tajo.plan.logical.*; import org.apache.tajo.plan.util.PlannerUtil; +import org.apache.tajo.plan.visitor.BasicLogicalPlanVisitor; import org.apache.tajo.util.TUtil; +import org.apache.tajo.util.graph.DirectedGraphVisitor; import java.util.*; +/** + * Broadcast rules are as follows. + * A relation is broadcastable when its size is smaller than a given threshold. + */ public class BroadcastJoinRule implements GlobalPlanRewriteRule { - private long broadcastTableSizeThreshold; - private GlobalPlanRewriteUtil.ParentFinder parentFinder; - private RelationSizeComparator relSizeComparator; + private final static Log LOG = LogFactory.getLog(BroadcastJoinRule.class); + + private BroadcastJoinPlanBuilder planOptimizer; + private BroadcastJoinOptimizeFinalizer optimizeFinalizer; @Override public String getName() { @@ -48,12 +58,12 @@ public boolean isEligible(OverridableConf queryContext, MasterPlan plan) { if (queryContext.getBool(SessionVars.TEST_BROADCAST_JOIN_ENABLED)) { for (LogicalPlan.QueryBlock block : plan.getLogicalPlan().getQueryBlocks()) { if (block.hasNode(NodeType.JOIN)) { - broadcastTableSizeThreshold = queryContext.getLong(SessionVars.BROADCAST_TABLE_SIZE_LIMIT); - if (broadcastTableSizeThreshold > 0) { - if (parentFinder == null) { - parentFinder = new GlobalPlanRewriteUtil.ParentFinder(); - relSizeComparator = new RelationSizeComparator(); - } + long broadcastSizeThreshold = queryContext.getLong(SessionVars.BROADCAST_TABLE_SIZE_LIMIT); + if (broadcastSizeThreshold > 0) { + GlobalPlanRewriteUtil.ParentFinder parentFinder = new GlobalPlanRewriteUtil.ParentFinder(); + RelationSizeComparator relSizeComparator = new RelationSizeComparator(); + planOptimizer = new BroadcastJoinPlanBuilder(plan, relSizeComparator, parentFinder, broadcastSizeThreshold); + optimizeFinalizer = new BroadcastJoinOptimizeFinalizer(plan, relSizeComparator); return true; } } @@ -62,10 +72,10 @@ public boolean isEligible(OverridableConf queryContext, MasterPlan plan) { return false; } - // TODO: use MasterPlan.execBlockGraph.accept() @Override public MasterPlan rewrite(MasterPlan plan) throws PlanningException{ - rewrite(plan, plan.getTerminalBlock()); + plan.accept(plan.getRoot().getId(), planOptimizer); + plan.accept(plan.getRoot().getId(), optimizeFinalizer); return plan; } @@ -77,231 +87,231 @@ public int compare(ScanNode o1, ScanNode o2) { } } - private void rewrite(MasterPlan plan, ExecutionBlock current) throws PlanningException { - if (plan.isLeaf(current) && !current.isPreservedRow()) { - // in leaf execution blocks, find input tables which size is less than the predefined threshold. - for (ScanNode scanNode : current.getScanNodes()) { - if (GlobalPlanRewriteUtil.getTableVolume(scanNode) <= broadcastTableSizeThreshold) { - current.addBroadcastRelation(scanNode.getCanonicalName()); + /** + * + */ + private static class BroadcastJoinOptimizeFinalizer implements DirectedGraphVisitor { + private final MasterPlan plan; + private final RelationSizeComparator relSizeComparator; + + public BroadcastJoinOptimizeFinalizer(MasterPlan plan, RelationSizeComparator relationSizeComparator) { + this.plan = plan; + this.relSizeComparator = relationSizeComparator; + } + + @Override + public void visit(Stack stack, ExecutionBlockId currentId) { + ExecutionBlock current = plan.getExecBlock(currentId); + if (!plan.isTerminal(current)) { + // When every child is a broadcast candidate, enforce non-broadcast for the largest relation for the join to be + // computed at the node who stores such largest relation. + if (isFullyBroadcastable(current)) { + List broadcastCandidates = TUtil.newList(current.getBroadcastRelations()); + Collections.sort(broadcastCandidates, relSizeComparator); + + current.removeBroadcastRelation(broadcastCandidates.remove(broadcastCandidates.size()-1)); } } - } else { - // in intermediate execution blocks, merge broadcastable children's plan with the current plan. - for (ExecutionBlock child : plan.getChilds(current)) { - rewrite(plan, child); + } + } + + private static class BroadcastJoinPlanBuilder implements DirectedGraphVisitor { + private final MasterPlan plan; + private final RelationSizeComparator relSizeComparator; + private final long broadcastSizeThreshold; + private final GlobalPlanRewriteUtil.ParentFinder parentFinder; + + public BroadcastJoinPlanBuilder(MasterPlan plan, RelationSizeComparator relationSizeComparator, + GlobalPlanRewriteUtil.ParentFinder parentFinder, long broadcastSizeThreshold) { + this.plan = plan; + this.relSizeComparator = relationSizeComparator; + this.broadcastSizeThreshold = broadcastSizeThreshold; + this.parentFinder = parentFinder; + } + + @Override + public void visit(Stack stack, ExecutionBlockId executionBlockId) { + ExecutionBlock current = plan.getExecBlock(executionBlockId); + + if (plan.isLeaf(current)) { + visitLeafNode(current); + } else { + visitNonLeafNode(current); } - if (!plan.isTerminal(current) && current.hasJoin()) { - // TODO: handle unioned scans as a single scan - List broadcastCandidates = TUtil.newList(); - List childs = plan.getChilds(current); - Map relationBlockMap = TUtil.newHashMap(); - Map unionScanMap = current.getUnionScanMap(); - if (unionScanMap == null) { - unionScanMap = TUtil.newHashMap(); - } + } - // find all broadcast candidates from every child - for (ExecutionBlock child : childs) { - Map scanNodeMap = TUtil.newHashMap(); - for (ScanNode scanNode : child.getScanNodes()) { - scanNodeMap.put(scanNode.getCanonicalName(), scanNode); - } - for (String relName : child.getBroadcastTables()) { - broadcastCandidates.add(scanNodeMap.get(relName)); - relationBlockMap.put(scanNodeMap.get(relName), child); + private void visitLeafNode(ExecutionBlock current) { + // At leaf execution blocks, find input relations who's size is smaller than the predefined threshold. + if (!current.isPreservedRow()) { + // Preserved-row relations must not be broadcasted to avoid data duplication. + boolean fullyBroadcastable = true; + for (ScanNode scanNode : current.getScanNodes()) { + if (GlobalPlanRewriteUtil.getTableVolume(scanNode) <= broadcastSizeThreshold) { + current.addBroadcastRelation(scanNode); + } else { + fullyBroadcastable = false; } } - Collections.sort(broadcastCandidates, relSizeComparator); - - // Enforce broadcast for candidates in ascending order of relation size - long totalBroadcastVolume = 0; - int i, broadcastEnd = -1; - for (i = 0; i < broadcastCandidates.size(); i++) { - long volumeOfCandidate = GlobalPlanRewriteUtil.getTableVolume(broadcastCandidates.get(i)); - if (totalBroadcastVolume + volumeOfCandidate > broadcastTableSizeThreshold) { - broadcastEnd = i; - break; + if (fullyBroadcastable && current.getScanNodes().length == 1) { + try { + updateScanOfParentAsBroadcastable(plan, current); + } catch (PlanningException e) { +// throw new RuntimeException(e); + // This case is when the current has two or more inputs via union. + // It is able to be simply ignored. } - totalBroadcastVolume += volumeOfCandidate; - } - if (i == broadcastCandidates.size()) { - broadcastEnd = i-1; - } - - for (; i < broadcastCandidates.size(); i++) { - ScanNode nonBroadcast = broadcastCandidates.get(i); - ExecutionBlock enforceShuffleBlock = relationBlockMap.get(nonBroadcast); - enforceShuffleBlock.removeBroadcastRelation(nonBroadcast.getTableName()); } + } + } - // TODO: check all inputs are marked as broadcast - if (broadcastCandidates.size() > 0) { - for (ExecutionBlock child : childs) { -// if (child.hasBroadcastRelation()) { - List unionScans = TUtil.newList(); - ExecutionBlockId representativeId = null; - if (unionScanMap.containsKey(child.getId())) { - representativeId = unionScanMap.get(child.getId()); - } else if (unionScanMap.containsValue(child.getId())) { - representativeId = child.getId(); + private void visitNonLeafNode(ExecutionBlock current) { + // At non-leaf execution blocks, merge broadcastable children's plan with the current plan. + + if (!plan.isTerminal(current)) { + if (current.hasJoin()) { + List childs = plan.getChilds(current); + Map unionScanMap = current.getUnionScanMap(); + + // When the total size of broadcastable relations exceeds the threshold, enforce two-phase join for large ones + // in order to broadcast as many relations as possible. + List broadcastCandidates = TUtil.newList(); + for (ScanNode scanNode : current.getScanNodes()) { + long estimatedRelationSize = GlobalPlanRewriteUtil.getTableVolume(scanNode); + if (estimatedRelationSize > 0 && estimatedRelationSize <= broadcastSizeThreshold) { + broadcastCandidates.add(scanNode); + } + } + Collections.sort(broadcastCandidates, relSizeComparator); + + // Enforce broadcast for candidates in ascending order of relation size + long totalBroadcastVolume = 0; + int i; + for (i = 0; i < broadcastCandidates.size(); i++) { + long volumeOfCandidate = GlobalPlanRewriteUtil.getTableVolume(broadcastCandidates.get(i)); + if (totalBroadcastVolume + volumeOfCandidate > broadcastSizeThreshold) { + break; } + totalBroadcastVolume += volumeOfCandidate; + } - if (representativeId != null) { - for (Map.Entry entry : unionScanMap.entrySet()) { - if (entry.getValue().equals(representativeId)) { - unionScans.add(entry.getKey()); - } - } + for (; i < broadcastCandidates.size(); ) { + ScanNode nonBroadcast = broadcastCandidates.remove(i); + broadcastCandidates.remove(nonBroadcast); + } - // add unions - LogicalNode left, topUnion = null; - left = GlobalPlanner.buildInputExecutor(plan.getLogicalPlan(), plan.getChannel(unionScans.get(0), current.getId())); - for (i = 1; i < unionScans.size(); i++) { - // left must not be null - UnionNode unionNode = plan.getLogicalPlan().createNode(UnionNode.class); - unionNode.setLeftChild(left); - unionNode.setRightChild(GlobalPlanner.buildInputExecutor(plan.getLogicalPlan(), plan.getChannel(unionScans.get(i), current.getId()))); - unionNode.setInSchema(left.getOutSchema()); - unionNode.setOutSchema(left.getOutSchema()); - topUnion = unionNode; - left = unionNode; + if (current.hasBroadcastRelation()) { + // The current execution block and its every child are able to be merged. + for (ExecutionBlock child : childs) { + try { + addUnionNodeIfNecessary(unionScanMap, plan, child, current); + mergeTwoPhaseJoin(plan, child, current); + } catch (PlanningException e) { + throw new RuntimeException(e); } - - ScanNode scanForChild = GlobalPlanRewriteUtil.findScanForChildEb(plan.getExecBlock(representativeId), current); - PlannerUtil.replaceNode(plan.getLogicalPlan(), current.getPlan(), scanForChild, topUnion); - - current.getUnionScanMap().clear(); - current.setPlan(current.getPlan()); } - mergeTwoPhaseJoin(plan, child, current); + // We assume that if every input of an execution block is broadcastable, + // the output of the execution block is also broadcastable. + if (!current.isPreservedRow() && isFullyBroadcastable(current)) { + try { + updateScanOfParentAsBroadcastable(plan, current); + } catch (PlanningException e) { + throw new RuntimeException(e); + } + } + } + } else { + List relations = TUtil.newList(current.getBroadcastRelations()); + for (ScanNode eachRelation : relations) { + current.removeBroadcastRelation(eachRelation); } } - -// for (i = 0; i <= broadcastEnd; i++) { -// ExecutionBlock willBeMergedChild = relationBlockMap.get(broadcastCandidates.get(i)); -// mergeTwoPhaseJoin(plan, willBeMergedChild, current); -// } - - -// ExecutionBlock enforceNonBroadcast = null; -// ExecutionBlock broadcastCandidate = null; -// long smallestChildVolume = Long.MAX_VALUE; -// -// for (ExecutionBlock child : childs) { -// if (child.isBroadcastable(broadcastTableSizeThreshold)) { -// long inputVolume = GlobalPlanRewriteUtil.getInputVolume(child); -// if (smallestChildVolume > inputVolume) { -// smallestChildVolume = inputVolume; -// if (broadcastCandidate != null) { -// enforceNonBroadcast = broadcastCandidate; -// } -// broadcastCandidate = child; -// } -// } -// } -// if (broadcastCandidate != null) { -// if (enforceNonBroadcast != null) { -// List tables = TUtil.newList(enforceNonBroadcast.getBroadcastTables()); -// for (String broadcastTable : tables) { -//// enforceNonBroadcast.removeBroadcastRelation(broadcastTable); -// // TODO: remove the largest rel from broadcast when all inputs are broadcast -// } -// } -// -// } } } - } - private static boolean hasOuterJoin(ExecutionBlock block) { - LogicalNode found = PlannerUtil.findMostBottomNode(block.getPlan(), NodeType.JOIN); - if (found != null) { - JoinNode joinNode = (JoinNode) found; - return PlannerUtil.isOuterJoin(joinNode.getJoinType()); + private void updateScanOfParentAsBroadcastable(MasterPlan plan, ExecutionBlock current) throws PlanningException { + ExecutionBlock parent = plan.getParent(current); + if (parent != null && !plan.isTerminal(parent)) { + ScanNode scanForCurrent = GlobalPlanRewriteUtil.findScanForChildEb(current, parent); + parent.addBroadcastRelation(scanForCurrent); + } } - return false; - } - /** - * Merge child execution blocks. - * - * @param plan master plan - * @param child child block - * @param parent parent block who has join nodes - * @return - */ - private ExecutionBlock mergeTwoPhaseJoin(MasterPlan plan, ExecutionBlock child, ExecutionBlock parent) - throws PlanningException { - ScanNode scanForChild = GlobalPlanRewriteUtil.findScanForChildEb(child, parent); -// if (scanForChild == null) { -// if (parent.getUnionScanMap() != null && !parent.getUnionScanMap().isEmpty()) { -// ExecutionBlockId scanEbId = parent.getUnionScanMap().get(child.getId()); -// if (scanEbId != null) { -// ExecutionBlock scanEb = plan.getExecBlock(scanEbId); -// if (scanEb != null) { -// scanForChild = GlobalPlanRewriteUtil.findScanForChildEb(scanEb, parent); -// } -// } -// } -// } - if (scanForChild == null) { - throw new PlanningException("Cannot find any scan nodes for " + child.getId() + " in " + parent.getId()); - } + /** + * Merge child execution blocks. + * + * @param plan master plan + * @param child child block + * @param parent parent block who has join nodes + * @return + */ + private ExecutionBlock mergeTwoPhaseJoin(MasterPlan plan, ExecutionBlock child, ExecutionBlock parent) + throws PlanningException { + ScanNode scanForChild = GlobalPlanRewriteUtil.findScanForChildEb(child, parent); + + parentFinder.set(scanForChild); + parentFinder.find(parent.getPlan()); + LogicalNode parentOfScanForChild = parentFinder.getFound(); + + LogicalNode rootOfChild = child.getPlan(); + if (rootOfChild.getType() == NodeType.STORE) { + rootOfChild = ((StoreTableNode)rootOfChild).getChild(); + } - parentFinder.set(scanForChild); - parentFinder.find(parent.getPlan()); - LogicalNode parentOfScanForChild = parentFinder.getFound(); - if (parentOfScanForChild == null) { - throw new PlanningException("Cannot find the parent of " + scanForChild.getCanonicalName()); - } + GlobalPlanRewriteUtil.replaceChild(rootOfChild, scanForChild, parentOfScanForChild); - LogicalNode rootOfChild = child.getPlan(); - if (rootOfChild.getType() == NodeType.STORE) { - rootOfChild = ((StoreTableNode)rootOfChild).getChild(); + parent = GlobalPlanRewriteUtil.mergeExecutionBlocks(plan, child, parent); + parent.removeBroadcastRelation(scanForChild); + + parent.setPlan(parent.getPlan()); + + return parent; } -// if (parentOfScanForChild instanceof UnaryNode) { -// ((UnaryNode) parentOfScanForChild).setChild(rootOfChild); -// } else if (parentOfScanForChild instanceof BinaryNode) { -// BinaryNode binary = (BinaryNode) parentOfScanForChild; -// if (binary.getLeftChild().equals(scanForChild)) { -// binary.setLeftChild(rootOfChild); -// } else if (binary.getRightChild().equals(scanForChild)) { -// binary.setRightChild(rootOfChild); -// } else { -// throw new PlanningException(scanForChild.getPID() + " is not a child of " + parentOfScanForChild.getPID()); -// } -// } else { -// throw new PlanningException(parentOfScanForChild + " seems to not have any children"); -// } - GlobalPlanRewriteUtil.replaceChild(rootOfChild, scanForChild, parentOfScanForChild); - -// for (String broadcastable : child.getBroadcastTables()) { -// parent.addBroadcastRelation(broadcastable); -// } -// -// // connect parent and grand children -// List grandChilds = plan.getChilds(child); -// for (ExecutionBlock eachGrandChild : grandChilds) { -// plan.addConnect(eachGrandChild, parent, plan.getChannel(eachGrandChild, child).getShuffleType()); -// plan.disconnect(eachGrandChild, child); -// } -// -// plan.disconnect(child, parent); -// List channels = plan.getIncomingChannels(child.getId()); -// if (channels == null || channels.size() == 0) { -// channels = plan.getOutgoingChannels(child.getId()); -// if (channels == null || channels.size() == 0) { -// plan.removeExecBlock(child.getId()); -// } -// } - parent = GlobalPlanRewriteUtil.mergeExecutionBlocks(plan, child, parent); - - parent.setPlan(parent.getPlan()); - - return parent; - } + private void addUnionNodeIfNecessary(Map unionScanMap, MasterPlan plan, + ExecutionBlock child, ExecutionBlock current) + throws PlanningException { + if (unionScanMap != null) { + List unionScans = TUtil.newList(); + ExecutionBlockId representativeId = null; + if (unionScanMap.containsKey(child.getId())) { + representativeId = unionScanMap.get(child.getId()); + } else if (unionScanMap.containsValue(child.getId())) { + representativeId = child.getId(); + } + if (representativeId != null) { + for (Map.Entry entry : unionScanMap.entrySet()) { + if (entry.getValue().equals(representativeId)) { + unionScans.add(entry.getKey()); + } + } + + // add unions + LogicalNode left, topUnion = null; + left = GlobalPlanner.buildInputExecutor(plan.getLogicalPlan(), plan.getChannel(unionScans.get(0), current.getId())); + for (int i = 1; i < unionScans.size(); i++) { + // left must not be null + UnionNode unionNode = plan.getLogicalPlan().createNode(UnionNode.class); + unionNode.setLeftChild(left); + unionNode.setRightChild(GlobalPlanner.buildInputExecutor(plan.getLogicalPlan(), plan.getChannel(unionScans.get(i), current.getId()))); + unionNode.setInSchema(left.getOutSchema()); + unionNode.setOutSchema(left.getOutSchema()); + topUnion = unionNode; + left = unionNode; + } + ScanNode scanForChild = GlobalPlanRewriteUtil.findScanForChildEb(plan.getExecBlock(representativeId), current); + PlannerUtil.replaceNode(plan.getLogicalPlan(), current.getPlan(), scanForChild, topUnion); + + current.getUnionScanMap().clear(); + current.setPlan(current.getPlan()); + } + } + } + } + + private static boolean isFullyBroadcastable(ExecutionBlock block) { + return block.getBroadcastRelations().size() == block.getScanNodes().length; + } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanRewriteUtil.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanRewriteUtil.java index c6cb2a5ffc..58a74046a2 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanRewriteUtil.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanRewriteUtil.java @@ -24,12 +24,11 @@ import org.apache.tajo.plan.PlanningException; import org.apache.tajo.plan.logical.*; -import java.util.Collection; import java.util.List; public class GlobalPlanRewriteUtil { public static ExecutionBlock mergeExecutionBlocks(MasterPlan plan, ExecutionBlock child, ExecutionBlock parent) { - for (String broadcastable : child.getBroadcastTables()) { + for (ScanNode broadcastable : child.getBroadcastRelations()) { parent.addBroadcastRelation(broadcastable); } @@ -69,7 +68,7 @@ public static void replaceChild(LogicalNode newChild, ScanNode originalChild, Lo } } - public static ScanNode findScanForChildEb(ExecutionBlock child, ExecutionBlock parent) { + public static ScanNode findScanForChildEb(ExecutionBlock child, ExecutionBlock parent) throws PlanningException { ScanNode scanForChild = null; for (ScanNode scanNode : parent.getScanNodes()) { if (scanNode.getTableName().equals(child.getId().toString())) { @@ -77,6 +76,9 @@ public static ScanNode findScanForChildEb(ExecutionBlock child, ExecutionBlock p break; } } + if (scanForChild == null) { + throw new PlanningException("Cannot find any scan nodes for " + child.getId() + " in " + parent.getId()); + } return scanForChild; } @@ -86,34 +88,19 @@ public static ScanNode findScanForChildEb(ExecutionBlock child, ExecutionBlock p * @return table volume (bytes) */ public static long getTableVolume(ScanNode scanNode) { - long scanBytes = scanNode.getTableDesc().getStats().getNumBytes(); - if (scanNode.getType() == NodeType.PARTITIONS_SCAN) { - PartitionedTableScanNode pScanNode = (PartitionedTableScanNode)scanNode; - if (pScanNode.getInputPaths() == null || pScanNode.getInputPaths().length == 0) { - scanBytes = 0L; - } - } - - return scanBytes; - } - - public static long getBroadcastInputVolume(ExecutionBlock block) { - Collection broadcastRelations = block.getBroadcastTables(); - long volume = 0; - for (ScanNode scanNode : block.getScanNodes()) { - if (broadcastRelations.contains(scanNode.getCanonicalName())) { - volume += getTableVolume(scanNode); + if (scanNode.getTableDesc().hasStats()) { + long scanBytes = scanNode.getTableDesc().getStats().getNumBytes(); + if (scanNode.getType() == NodeType.PARTITIONS_SCAN) { + PartitionedTableScanNode pScanNode = (PartitionedTableScanNode) scanNode; + if (pScanNode.getInputPaths() == null || pScanNode.getInputPaths().length == 0) { + scanBytes = 0L; + } } - } - return volume; - } - public static long getInputVolume(ExecutionBlock block) { - long volume = 0; - for (ScanNode scanNode : block.getScanNodes()) { - volume += getTableVolume(scanNode); + return scanBytes; + } else { + return -1; } - return volume; } public static class ParentFinder implements LogicalNodeVisitor { @@ -129,7 +116,10 @@ public void find(LogicalNode root) { this.visit(root); } - public LogicalNode getFound() { + public LogicalNode getFound() throws PlanningException { + if (found == null) { + throw new PlanningException("Cannot find the parent of " + target.getPID()); + } return this.found; } diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java index d9baf3bcae..d554b448a4 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java @@ -188,37 +188,38 @@ public static void scheduleFragmentsForJoinQuery(TaskSchedulerContext schedulerC } // Assigning either fragments or fetch urls to query units - boolean isAllBroadcastTable = true; - for (int i = 0; i < scans.length; i++) { - if (!execBlock.isBroadcastRelation(scans[i].getCanonicalName())) { - isAllBroadcastTable = false; - break; - } - } - - - if (isAllBroadcastTable) { // if all relations of this EB are broadcasted - // set largest table to normal mode - long maxStats = Long.MIN_VALUE; - int maxStatsScanIdx = -1; - for (int i = 0; i < scans.length; i++) { - // finding largest table. - // If stats == 0, can't be base table. - if (stats[i] > 0 && stats[i] > maxStats) { - maxStats = stats[i]; - maxStatsScanIdx = i; - } - } - if (maxStatsScanIdx == -1) { - maxStatsScanIdx = 0; - } - int baseScanIdx = maxStatsScanIdx; - scans[baseScanIdx].setBroadcastTable(false); - execBlock.removeBroadcastRelation(scans[baseScanIdx].getCanonicalName()); - LOG.info(String.format("[Distributed Join Strategy] : Broadcast Join with all tables, base_table=%s, base_volume=%d", - scans[baseScanIdx].getCanonicalName(), stats[baseScanIdx])); - scheduleLeafTasksWithBroadcastTable(schedulerContext, stage, baseScanIdx, fragments); - } else if (!execBlock.getBroadcastTables().isEmpty()) { // If some relations of this EB are broadcasted +// boolean isAllBroadcastTable = true; +// for (int i = 0; i < scans.length; i++) { +// if (!execBlock.isBroadcastRelation(scans[i].getCanonicalName())) { +// isAllBroadcastTable = false; +// break; +// } +// } + + +// if (isAllBroadcastTable) { // if all relations of this EB are broadcasted +// // set largest table to normal mode +// long maxStats = Long.MIN_VALUE; +// int maxStatsScanIdx = -1; +// for (int i = 0; i < scans.length; i++) { +// // finding largest table. +// // If stats == 0, can't be base table. +// if (stats[i] > 0 && stats[i] > maxStats) { +// maxStats = stats[i]; +// maxStatsScanIdx = i; +// } +// } +// if (maxStatsScanIdx == -1) { +// maxStatsScanIdx = 0; +// } +// int baseScanIdx = maxStatsScanIdx; +// scans[baseScanIdx].setBroadcastTable(false); +// execBlock.removeBroadcastRelation(scans[baseScanIdx].getCanonicalName()); +// LOG.info(String.format("[Distributed Join Strategy] : Broadcast Join with all tables, base_table=%s, base_volume=%d", +// scans[baseScanIdx].getCanonicalName(), stats[baseScanIdx])); +// scheduleLeafTasksWithBroadcastTable(schedulerContext, stage, baseScanIdx, fragments); +// } else if (!execBlock.getBroadcastTables().isEmpty()) { // If some relations of this EB are broadcasted + if (execBlock.hasBroadcastRelation()) { // If some relations of this EB are broadcasted boolean hasNonLeafNode = false; List largeScanIndexList = new ArrayList(); List broadcastIndexList = new ArrayList(); @@ -235,7 +236,7 @@ public static void scheduleFragmentsForJoinQuery(TaskSchedulerContext schedulerC nonLeafScanNamesBuilder.append(namePrefix).append(scans[i].getCanonicalName()); namePrefix = ","; } - if (execBlock.isBroadcastRelation(scans[i].getCanonicalName())) { + if (execBlock.isBroadcastRelation(scans[i])) { broadcastIndexList.add(i); } else { // finding largest table. diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestOuterJoinQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestOuterJoinQuery.java index 077b7d2cb9..d646202d56 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestOuterJoinQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestOuterJoinQuery.java @@ -349,7 +349,7 @@ public final void testLeftOuterWithEmptyTable() throws Exception { } // TODO: this test is disabled due to a bug in broadcast join. It will be enabled after TAJO-1553 -// @Test + @Test @Option(withExplain = true, withExplainGlobal = true, parameterized = true) @SimpleTest(queries = { @QuerySpec("select t1.id, t1.name, t2.id, t3.id\n" + @@ -364,7 +364,7 @@ public final void testRightOuterJoinPredicationCaseByCase1() throws Exception { } // TODO: this test is disabled due to a bug in broadcast join. It will be enabled after TAJO-1553 -// @Test + @Test @Option(withExplain = true, withExplainGlobal = true, parameterized = true) @SimpleTest(queries = { @QuerySpec("select t1.id, t1.name, t3.id, t4.id\n" + @@ -383,7 +383,7 @@ public final void testRightOuterJoinPredicationCaseByCase2() throws Exception { } // TODO: this test is disabled due to a bug in broadcast join. It will be enabled after TAJO-1553 -// @Test + @Test @Option(withExplain = true, withExplainGlobal = true, parameterized = true) @SimpleTest(queries = { @QuerySpec("select t1.id, t1.name, t2.id, t3.id\n" + @@ -398,7 +398,7 @@ public final void testRightOuterJoinPredicationCaseByCase3() throws Exception { } // TODO: this test is disabled due to a bug in broadcast join. It will be enabled after TAJO-1553 -// @Test + @Test @Option(withExplain = true, withExplainGlobal = true, parameterized = true) @SimpleTest(queries = { @QuerySpec("select t1.id, t1.name, t3.id, t4.id\n" + diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testBroadcastTwoPartJoin.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testBroadcastTwoPartJoin.Hash.plan index ec92307e62..fa8c71e9a5 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testBroadcastTwoPartJoin.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testBroadcastTwoPartJoin.Hash.plan @@ -69,8 +69,7 @@ Block Id: eb_0000000000000_0000_000009 [LEAF] 0: type=Broadcast, tables=default.orders 1: type=Broadcast, tables=default.part 2: type=Broadcast, tables=default.lineitem - 3: type=Broadcast, tables=default.nation - 4: type=Broadcast, tables=default.customer + 3: type=Broadcast, tables=default.customer SORT(26) => Sort Keys: default.lineitem.l_orderkey (INT4) (asc),default.part.p_name (TEXT) (asc),default.nation.n_name (TEXT) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testBroadcastTwoPartJoin.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testBroadcastTwoPartJoin.Sort.plan index ec92307e62..fa8c71e9a5 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testBroadcastTwoPartJoin.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testBroadcastTwoPartJoin.Sort.plan @@ -69,8 +69,7 @@ Block Id: eb_0000000000000_0000_000009 [LEAF] 0: type=Broadcast, tables=default.orders 1: type=Broadcast, tables=default.part 2: type=Broadcast, tables=default.lineitem - 3: type=Broadcast, tables=default.nation - 4: type=Broadcast, tables=default.customer + 3: type=Broadcast, tables=default.customer SORT(26) => Sort Keys: default.lineitem.l_orderkey (INT4) (asc),default.part.p_name (TEXT) (asc),default.nation.n_name (TEXT) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition1.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition1.Hash.plan index a9c7fe95fd..e5b845568c 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition1.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition1.Hash.plan @@ -40,7 +40,6 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.n1 - 1: type=Broadcast, tables=default.n2 SORT(10) => Sort Keys: default.n1.n_nationkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition1.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition1.Sort.plan index a9c7fe95fd..e5b845568c 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition1.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition1.Sort.plan @@ -40,7 +40,6 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.n1 - 1: type=Broadcast, tables=default.n2 SORT(10) => Sort Keys: default.n1.n_nationkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition2.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition2.Hash.plan index 0e5f78a6bb..7c889332ab 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition2.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition2.Hash.plan @@ -40,7 +40,6 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.n1 - 1: type=Broadcast, tables=default.n2 SORT(10) => Sort Keys: default.n1.n_nationkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition2.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition2.Sort.plan index 0e5f78a6bb..7c889332ab 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition2.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition2.Sort.plan @@ -40,7 +40,6 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.n1 - 1: type=Broadcast, tables=default.n2 SORT(10) => Sort Keys: default.n1.n_nationkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition3.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition3.Hash.plan index e5ada3c991..f35d4862fb 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition3.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition3.Hash.plan @@ -40,7 +40,6 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.n1 - 1: type=Broadcast, tables=default.n2 SORT(10) => Sort Keys: default.n1.n_nationkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition3.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition3.Sort.plan index e5ada3c991..f35d4862fb 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition3.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition3.Sort.plan @@ -40,7 +40,6 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.n1 - 1: type=Broadcast, tables=default.n2 SORT(10) => Sort Keys: default.n1.n_nationkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition4.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition4.Hash.plan index aadfa008d5..fb73ee5b1b 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition4.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition4.Hash.plan @@ -40,7 +40,6 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.n1 - 1: type=Broadcast, tables=default.n2 SORT(10) => Sort Keys: default.n1.n_nationkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition4.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition4.Sort.plan index aadfa008d5..fb73ee5b1b 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition4.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testComplexJoinCondition4.Sort.plan @@ -40,7 +40,6 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.n1 - 1: type=Broadcast, tables=default.n2 SORT(10) => Sort Keys: default.n1.n_nationkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.1.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.1.Hash.plan index d76de99950..0fb22b3299 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.1.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.1.Hash.plan @@ -38,8 +38,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.nation.n_name (TEXT), default.region.r_name (TEXT), num=32) [Enforcers] - 0: type=Broadcast, tables=default.nation - 1: type=Broadcast, tables=default.region + 0: type=Broadcast, tables=default.region SORT(10) => Sort Keys: default.nation.n_name (TEXT) (asc),default.region.r_name (TEXT) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.1.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.1.Sort.plan index d76de99950..0fb22b3299 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.1.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.1.Sort.plan @@ -38,8 +38,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.nation.n_name (TEXT), default.region.r_name (TEXT), num=32) [Enforcers] - 0: type=Broadcast, tables=default.nation - 1: type=Broadcast, tables=default.region + 0: type=Broadcast, tables=default.region SORT(10) => Sort Keys: default.nation.n_name (TEXT) (asc),default.region.r_name (TEXT) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.2.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.2.Hash.plan index ed77747bd2..453e12feb4 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.2.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.2.Hash.plan @@ -32,7 +32,6 @@ Block Id: eb_0000000000000_0000_000003 [ROOT] [Enforcers] 0: type=Broadcast, tables=default.region - 1: type=Broadcast, tables=default.customer JOIN(5)(CROSS) => target list: default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT), default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.2.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.2.Sort.plan index ed77747bd2..453e12feb4 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.2.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.2.Sort.plan @@ -32,7 +32,6 @@ Block Id: eb_0000000000000_0000_000003 [ROOT] [Enforcers] 0: type=Broadcast, tables=default.region - 1: type=Broadcast, tables=default.customer JOIN(5)(CROSS) => target list: default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT), default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.3.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.3.Hash.plan index 1bf0332459..724e240a5c 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.3.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.3.Hash.plan @@ -31,8 +31,7 @@ Block Id: eb_0000000000000_0000_000003 [ROOT] ======================================================= [Enforcers] - 0: type=Broadcast, tables=default.customer - 1: type=Broadcast, tables=default.region + 0: type=Broadcast, tables=default.region JOIN(5)(CROSS) => target list: default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT), default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.3.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.3.Sort.plan index 1bf0332459..724e240a5c 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.3.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.3.Sort.plan @@ -31,8 +31,7 @@ Block Id: eb_0000000000000_0000_000003 [ROOT] ======================================================= [Enforcers] - 0: type=Broadcast, tables=default.customer - 1: type=Broadcast, tables=default.region + 0: type=Broadcast, tables=default.region JOIN(5)(CROSS) => target list: default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT), default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.4.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.4.Hash.plan index 1bf0332459..724e240a5c 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.4.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.4.Hash.plan @@ -31,8 +31,7 @@ Block Id: eb_0000000000000_0000_000003 [ROOT] ======================================================= [Enforcers] - 0: type=Broadcast, tables=default.customer - 1: type=Broadcast, tables=default.region + 0: type=Broadcast, tables=default.region JOIN(5)(CROSS) => target list: default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT), default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.4.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.4.Sort.plan index 1bf0332459..724e240a5c 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.4.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.4.Sort.plan @@ -31,8 +31,7 @@ Block Id: eb_0000000000000_0000_000003 [ROOT] ======================================================= [Enforcers] - 0: type=Broadcast, tables=default.customer - 1: type=Broadcast, tables=default.region + 0: type=Broadcast, tables=default.region JOIN(5)(CROSS) => target list: default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT), default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.5.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.5.Hash.plan index af24f2014a..35e397bd9b 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.5.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.5.Hash.plan @@ -38,8 +38,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.region.r_name (TEXT), default.region.r_regionkey (INT4), len (INT4), num=32) [Enforcers] - 0: type=Broadcast, tables=default.customer - 1: type=Broadcast, tables=default.region + 0: type=Broadcast, tables=default.region SORT(10) => Sort Keys: len (INT4) (asc),default.region.r_regionkey (INT4) (asc),default.region.r_name (TEXT) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.5.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.5.Sort.plan index af24f2014a..35e397bd9b 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.5.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoin.5.Sort.plan @@ -38,8 +38,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.region.r_name (TEXT), default.region.r_regionkey (INT4), len (INT4), num=32) [Enforcers] - 0: type=Broadcast, tables=default.customer - 1: type=Broadcast, tables=default.region + 0: type=Broadcast, tables=default.region SORT(10) => Sort Keys: len (INT4) (asc),default.region.r_regionkey (INT4) (asc),default.region.r_name (TEXT) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinAndCaseWhen.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinAndCaseWhen.Hash.plan index ebeae68c2a..af6c200226 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinAndCaseWhen.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinAndCaseWhen.Hash.plan @@ -40,7 +40,6 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.region - 1: type=Broadcast, tables=default.nation SORT(11) => Sort Keys: default.region.r_regionkey (INT4) (asc),default.nation.n_regionkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinAndCaseWhen.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinAndCaseWhen.Sort.plan index ebeae68c2a..af6c200226 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinAndCaseWhen.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinAndCaseWhen.Sort.plan @@ -40,7 +40,6 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.region - 1: type=Broadcast, tables=default.nation SORT(11) => Sort Keys: default.region.r_regionkey (INT4) (asc),default.nation.n_regionkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk1.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk1.Hash.plan index 0ff60b1604..b76c6e2544 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk1.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk1.Hash.plan @@ -39,7 +39,6 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.region - 1: type=Broadcast, tables=default.customer SORT(10) => Sort Keys: default.region.r_regionkey (INT4) (asc),default.region.r_name (TEXT) (asc),default.customer.c_custkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk1.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk1.Sort.plan index 0ff60b1604..b76c6e2544 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk1.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk1.Sort.plan @@ -39,7 +39,6 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.region - 1: type=Broadcast, tables=default.customer SORT(10) => Sort Keys: default.region.r_regionkey (INT4) (asc),default.region.r_name (TEXT) (asc),default.customer.c_custkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk2.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk2.Hash.plan index a197d6fbfe..b4d85658fb 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk2.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk2.Hash.plan @@ -38,8 +38,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.customer.c_custkey (INT4), default.region.r_name (TEXT), default.region.r_regionkey (INT4), num=32) [Enforcers] - 0: type=Broadcast, tables=default.customer - 1: type=Broadcast, tables=default.region + 0: type=Broadcast, tables=default.region SORT(10) => Sort Keys: default.region.r_regionkey (INT4) (asc),default.region.r_name (TEXT) (asc),default.customer.c_custkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk2.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk2.Sort.plan index a197d6fbfe..b4d85658fb 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk2.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk2.Sort.plan @@ -38,8 +38,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.customer.c_custkey (INT4), default.region.r_name (TEXT), default.region.r_regionkey (INT4), num=32) [Enforcers] - 0: type=Broadcast, tables=default.customer - 1: type=Broadcast, tables=default.region + 0: type=Broadcast, tables=default.region SORT(10) => Sort Keys: default.region.r_regionkey (INT4) (asc),default.region.r_name (TEXT) (asc),default.customer.c_custkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk3.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk3.Hash.plan index f285a1d2ac..d1c32a08b9 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk3.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk3.Hash.plan @@ -38,8 +38,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.customer.c_custkey (INT4), default.customer.c_name (TEXT), default.region.r_regionkey (INT4), num=32) [Enforcers] - 0: type=Broadcast, tables=default.customer - 1: type=Broadcast, tables=default.region + 0: type=Broadcast, tables=default.region SORT(10) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.customer.c_name (TEXT) (asc),default.region.r_regionkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk3.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk3.Sort.plan index f285a1d2ac..d1c32a08b9 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk3.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk3.Sort.plan @@ -38,8 +38,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.customer.c_custkey (INT4), default.customer.c_name (TEXT), default.region.r_regionkey (INT4), num=32) [Enforcers] - 0: type=Broadcast, tables=default.customer - 1: type=Broadcast, tables=default.region + 0: type=Broadcast, tables=default.region SORT(10) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.customer.c_name (TEXT) (asc),default.region.r_regionkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk4.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk4.Hash.plan index 370a12688f..4997c1d9e9 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk4.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk4.Hash.plan @@ -38,8 +38,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.customer.c_custkey (INT4), default.region.r_name (TEXT), default.region.r_regionkey (INT4), len (INT4), num=32) [Enforcers] - 0: type=Broadcast, tables=default.customer - 1: type=Broadcast, tables=default.region + 0: type=Broadcast, tables=default.region SORT(10) => Sort Keys: len (INT4) (asc),default.region.r_regionkey (INT4) (asc),default.region.r_name (TEXT) (asc),default.customer.c_custkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk4.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk4.Sort.plan index 370a12688f..4997c1d9e9 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk4.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithAsterisk4.Sort.plan @@ -38,8 +38,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.customer.c_custkey (INT4), default.region.r_name (TEXT), default.region.r_regionkey (INT4), len (INT4), num=32) [Enforcers] - 0: type=Broadcast, tables=default.customer - 1: type=Broadcast, tables=default.region + 0: type=Broadcast, tables=default.region SORT(10) => Sort Keys: len (INT4) (asc),default.region.r_regionkey (INT4) (asc),default.region.r_name (TEXT) (asc),default.customer.c_custkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithEmptyTable1.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithEmptyTable1.Hash.plan index 8906ddfe4b..a684a0ca66 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithEmptyTable1.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithEmptyTable1.Hash.plan @@ -40,7 +40,6 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.empty_orders - 1: type=Broadcast, tables=default.customer SORT(11) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.empty_orders.o_orderkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithEmptyTable1.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithEmptyTable1.Sort.plan index 8906ddfe4b..a684a0ca66 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithEmptyTable1.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithEmptyTable1.Sort.plan @@ -40,7 +40,6 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.empty_orders - 1: type=Broadcast, tables=default.customer SORT(11) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.empty_orders.o_orderkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithThetaJoinConditionInWhere.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithThetaJoinConditionInWhere.Hash.plan index f39025cd6a..e3cbb61f7c 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithThetaJoinConditionInWhere.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithThetaJoinConditionInWhere.Hash.plan @@ -38,7 +38,6 @@ Block Id: eb_0000000000000_0000_000003 [ROOT] [Enforcers] 0: type=Broadcast, tables=default.a - 1: type=Broadcast, tables=default.b PROJECTION(3) => Targets: a_name (TEXT), b_name (TEXT) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithThetaJoinConditionInWhere.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithThetaJoinConditionInWhere.Sort.plan index f39025cd6a..e3cbb61f7c 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithThetaJoinConditionInWhere.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testCrossJoinWithThetaJoinConditionInWhere.Sort.plan @@ -38,7 +38,6 @@ Block Id: eb_0000000000000_0000_000003 [ROOT] [Enforcers] 0: type=Broadcast, tables=default.a - 1: type=Broadcast, tables=default.b PROJECTION(3) => Targets: a_name (TEXT), b_name (TEXT) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testDifferentTypesJoinCondition.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testDifferentTypesJoinCondition.Hash.plan index 2478ddaeb3..69ede305f2 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testDifferentTypesJoinCondition.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testDifferentTypesJoinCondition.Hash.plan @@ -33,7 +33,6 @@ Block Id: eb_0000000000000_0000_000003 [ROOT] [Enforcers] 0: type=Broadcast, tables=default.t4 - 1: type=Broadcast, tables=default.t3 JOIN(5)(INNER) => Join Cond: default.t3.id (INT8) = ?int8_1 (INT8) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testDifferentTypesJoinCondition.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testDifferentTypesJoinCondition.Sort.plan index 2478ddaeb3..69ede305f2 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testDifferentTypesJoinCondition.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testDifferentTypesJoinCondition.Sort.plan @@ -33,7 +33,6 @@ Block Id: eb_0000000000000_0000_000003 [ROOT] [Enforcers] 0: type=Broadcast, tables=default.t4 - 1: type=Broadcast, tables=default.t3 JOIN(5)(INNER) => Join Cond: default.t3.id (INT8) = ?int8_1 (INT8) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testInnerJoinAndCaseWhen.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testInnerJoinAndCaseWhen.Hash.plan index ebeae68c2a..af6c200226 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testInnerJoinAndCaseWhen.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testInnerJoinAndCaseWhen.Hash.plan @@ -40,7 +40,6 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.region - 1: type=Broadcast, tables=default.nation SORT(11) => Sort Keys: default.region.r_regionkey (INT4) (asc),default.nation.n_regionkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testInnerJoinAndCaseWhen.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testInnerJoinAndCaseWhen.Sort.plan index ebeae68c2a..af6c200226 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testInnerJoinAndCaseWhen.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testInnerJoinAndCaseWhen.Sort.plan @@ -40,7 +40,6 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.region - 1: type=Broadcast, tables=default.nation SORT(11) => Sort Keys: default.region.r_regionkey (INT4) (asc),default.nation.n_regionkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testInnerJoinWithEmptyTable.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testInnerJoinWithEmptyTable.Hash.plan index 8906ddfe4b..a684a0ca66 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testInnerJoinWithEmptyTable.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testInnerJoinWithEmptyTable.Hash.plan @@ -40,7 +40,6 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.empty_orders - 1: type=Broadcast, tables=default.customer SORT(11) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.empty_orders.o_orderkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testInnerJoinWithEmptyTable.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testInnerJoinWithEmptyTable.Sort.plan index 8906ddfe4b..a684a0ca66 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testInnerJoinWithEmptyTable.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testInnerJoinWithEmptyTable.Sort.plan @@ -40,7 +40,6 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.empty_orders - 1: type=Broadcast, tables=default.customer SORT(11) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.empty_orders.o_orderkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testInnerJoinWithThetaJoinConditionInWhere.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testInnerJoinWithThetaJoinConditionInWhere.Hash.plan index accf06eb80..469668c18e 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testInnerJoinWithThetaJoinConditionInWhere.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testInnerJoinWithThetaJoinConditionInWhere.Hash.plan @@ -34,8 +34,7 @@ Block Id: eb_0000000000000_0000_000003 [ROOT] ======================================================= [Enforcers] - 0: type=Broadcast, tables=default.b - 1: type=Broadcast, tables=default.a + 0: type=Broadcast, tables=default.a SELECTION(3) => Search Cond: default.a.r_name (TEXT) <= default.b.r_name (TEXT) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testInnerJoinWithThetaJoinConditionInWhere.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testInnerJoinWithThetaJoinConditionInWhere.Sort.plan index accf06eb80..469668c18e 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testInnerJoinWithThetaJoinConditionInWhere.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testInnerJoinWithThetaJoinConditionInWhere.Sort.plan @@ -34,8 +34,7 @@ Block Id: eb_0000000000000_0000_000003 [ROOT] ======================================================= [Enforcers] - 0: type=Broadcast, tables=default.b - 1: type=Broadcast, tables=default.a + 0: type=Broadcast, tables=default.a SELECTION(3) => Search Cond: default.a.r_name (TEXT) <= default.b.r_name (TEXT) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinAsterisk.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinAsterisk.Hash.plan index 0ce7b529b6..244bfa9bc7 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinAsterisk.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinAsterisk.Hash.plan @@ -33,7 +33,6 @@ Block Id: eb_0000000000000_0000_000003 [ROOT] [Enforcers] 0: type=Broadcast, tables=default.a - 1: type=Broadcast, tables=default.b JOIN(5)(INNER) => Join Cond: default.b.n_nationkey (INT4) = default.a.c_nationkey (INT4) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinAsterisk.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinAsterisk.Sort.plan index 0ce7b529b6..244bfa9bc7 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinAsterisk.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinAsterisk.Sort.plan @@ -33,7 +33,6 @@ Block Id: eb_0000000000000_0000_000003 [ROOT] [Enforcers] 0: type=Broadcast, tables=default.a - 1: type=Broadcast, tables=default.b JOIN(5)(INNER) => Join Cond: default.b.n_nationkey (INT4) = default.a.c_nationkey (INT4) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvals1.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvals1.Hash.plan index 6da9a5d982..d5e2b9b15a 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvals1.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvals1.Hash.plan @@ -40,7 +40,6 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.region - 1: type=Broadcast, tables=default.nation SORT(11) => Sort Keys: default.region.r_regionkey (INT4) (asc),default.nation.n_regionkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvals1.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvals1.Sort.plan index 6da9a5d982..d5e2b9b15a 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvals1.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvals1.Sort.plan @@ -40,7 +40,6 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.region - 1: type=Broadcast, tables=default.nation SORT(11) => Sort Keys: default.region.r_regionkey (INT4) (asc),default.nation.n_regionkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvalsWithSameExprs1.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvalsWithSameExprs1.Hash.plan index 15e5651a63..a8f9f8bbfa 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvalsWithSameExprs1.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvalsWithSameExprs1.Hash.plan @@ -44,8 +44,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.nation.n_regionkey (INT4), v1 (INT4), num=32) [Enforcers] - 0: type=Broadcast, tables=default.nation - 1: type=Broadcast, tables=default.region + 0: type=Broadcast, tables=default.region SORT(11) => Sort Keys: v1 (INT4) (asc),default.nation.n_regionkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvalsWithSameExprs1.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvalsWithSameExprs1.Sort.plan index 15e5651a63..a8f9f8bbfa 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvalsWithSameExprs1.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvalsWithSameExprs1.Sort.plan @@ -44,8 +44,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.nation.n_regionkey (INT4), v1 (INT4), num=32) [Enforcers] - 0: type=Broadcast, tables=default.nation - 1: type=Broadcast, tables=default.region + 0: type=Broadcast, tables=default.region SORT(11) => Sort Keys: v1 (INT4) (asc),default.nation.n_regionkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvalsWithSameExprs2.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvalsWithSameExprs2.Hash.plan index bf84ad173d..d6252ea807 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvalsWithSameExprs2.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvalsWithSameExprs2.Hash.plan @@ -51,8 +51,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [q_0000000000000_0000] 3 => 4 (type=HASH_SHUFFLE, key=?plus (INT4), default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4), result (INT4), v1 (INT4), num=32) [Enforcers] - 0: type=Broadcast, tables=default.nation - 1: type=Broadcast, tables=default.region + 0: type=Broadcast, tables=default.region GROUP_BY(12)(?plus,n_regionkey,r_regionkey,result,v1) => exprs: (sum(?plus (INT4))) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvalsWithSameExprs2.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvalsWithSameExprs2.Sort.plan index bf84ad173d..d6252ea807 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvalsWithSameExprs2.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinCoReferredEvalsWithSameExprs2.Sort.plan @@ -51,8 +51,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [q_0000000000000_0000] 3 => 4 (type=HASH_SHUFFLE, key=?plus (INT4), default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4), result (INT4), v1 (INT4), num=32) [Enforcers] - 0: type=Broadcast, tables=default.nation - 1: type=Broadcast, tables=default.region + 0: type=Broadcast, tables=default.region GROUP_BY(12)(?plus,n_regionkey,r_regionkey,result,v1) => exprs: (sum(?plus (INT4))) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinOnMultipleDatabases.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinOnMultipleDatabases.Hash.plan index bc8deb0cbc..f3b0e9dcdc 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinOnMultipleDatabases.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinOnMultipleDatabases.Hash.plan @@ -66,11 +66,10 @@ Block Id: eb_0000000000000_0000_000009 [LEAF] [q_0000000000000_0000] 9 => 10 (type=RANGE_SHUFFLE, key=default.nation.n_name (TEXT), joins.part_.p_partkey (INT4), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_name (TEXT), num=32) [Enforcers] - 0: type=Broadcast, tables=default.nation - 1: type=Broadcast, tables=joins.supplier_ - 2: type=Broadcast, tables=default.region - 3: type=Broadcast, tables=joins.part_ - 4: type=Broadcast, tables=default.partsupp + 0: type=Broadcast, tables=joins.supplier_ + 1: type=Broadcast, tables=default.region + 2: type=Broadcast, tables=joins.part_ + 3: type=Broadcast, tables=default.partsupp SORT(26) => Sort Keys: joins.supplier_.s_acctbal (FLOAT8) (asc),joins.supplier_.s_name (TEXT) (asc),default.nation.n_name (TEXT) (asc),joins.part_.p_partkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinOnMultipleDatabases.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinOnMultipleDatabases.Sort.plan index bc8deb0cbc..f3b0e9dcdc 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinOnMultipleDatabases.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinOnMultipleDatabases.Sort.plan @@ -66,11 +66,10 @@ Block Id: eb_0000000000000_0000_000009 [LEAF] [q_0000000000000_0000] 9 => 10 (type=RANGE_SHUFFLE, key=default.nation.n_name (TEXT), joins.part_.p_partkey (INT4), joins.supplier_.s_acctbal (FLOAT8), joins.supplier_.s_name (TEXT), num=32) [Enforcers] - 0: type=Broadcast, tables=default.nation - 1: type=Broadcast, tables=joins.supplier_ - 2: type=Broadcast, tables=default.region - 3: type=Broadcast, tables=joins.part_ - 4: type=Broadcast, tables=default.partsupp + 0: type=Broadcast, tables=joins.supplier_ + 1: type=Broadcast, tables=default.region + 2: type=Broadcast, tables=joins.part_ + 3: type=Broadcast, tables=default.partsupp SORT(26) => Sort Keys: joins.supplier_.s_acctbal (FLOAT8) (asc),joins.supplier_.s_name (TEXT) (asc),default.nation.n_name (TEXT) (asc),joins.part_.p_partkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinWithMultipleJoinQual1.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinWithMultipleJoinQual1.Hash.plan index d9e413389e..ead39f7d13 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinWithMultipleJoinQual1.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinWithMultipleJoinQual1.Hash.plan @@ -69,8 +69,7 @@ Block Id: eb_0000000000000_0000_000009 [LEAF] 0: type=Broadcast, tables=default.ps 1: type=Broadcast, tables=default.p 2: type=Broadcast, tables=default.s - 3: type=Broadcast, tables=default.n - 4: type=Broadcast, tables=default.r + 3: type=Broadcast, tables=default.r SORT(26) => Sort Keys: default.s.s_acctbal (FLOAT8) (asc),default.s.s_name (TEXT) (asc),default.n.n_name (TEXT) (asc),default.p.p_partkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinWithMultipleJoinQual1.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinWithMultipleJoinQual1.Sort.plan index d9e413389e..ead39f7d13 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinWithMultipleJoinQual1.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinWithMultipleJoinQual1.Sort.plan @@ -69,8 +69,7 @@ Block Id: eb_0000000000000_0000_000009 [LEAF] 0: type=Broadcast, tables=default.ps 1: type=Broadcast, tables=default.p 2: type=Broadcast, tables=default.s - 3: type=Broadcast, tables=default.n - 4: type=Broadcast, tables=default.r + 3: type=Broadcast, tables=default.r SORT(26) => Sort Keys: default.s.s_acctbal (FLOAT8) (asc),default.s.s_name (TEXT) (asc),default.n.n_name (TEXT) (asc),default.p.p_partkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinWithOrPredicates.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinWithOrPredicates.Hash.plan index de323b192e..c86757cbfc 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinWithOrPredicates.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinWithOrPredicates.Hash.plan @@ -45,8 +45,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.n1.n_nationkey (INT4), num=32) [Enforcers] - 0: type=Broadcast, tables=default.n2 - 1: type=Broadcast, tables=default.n1 + 0: type=Broadcast, tables=default.n1 SORT(11) => Sort Keys: default.n1.n_nationkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinWithOrPredicates.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinWithOrPredicates.Sort.plan index de323b192e..c86757cbfc 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinWithOrPredicates.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testJoinWithOrPredicates.Sort.plan @@ -45,8 +45,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.n1.n_nationkey (INT4), num=32) [Enforcers] - 0: type=Broadcast, tables=default.n2 - 1: type=Broadcast, tables=default.n1 + 0: type=Broadcast, tables=default.n1 SORT(11) => Sort Keys: default.n1.n_nationkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testNaturalJoin.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testNaturalJoin.Hash.plan index e459241fbb..1ef6092da5 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testNaturalJoin.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testNaturalJoin.Hash.plan @@ -39,8 +39,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.n2.n_name (TEXT), num=32) [Enforcers] - 0: type=Broadcast, tables=default.n2 - 1: type=Broadcast, tables=default.n1 + 0: type=Broadcast, tables=default.n1 SORT(10) => Sort Keys: default.n2.n_name (TEXT) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testNaturalJoin.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testNaturalJoin.Sort.plan index e459241fbb..1ef6092da5 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testNaturalJoin.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testNaturalJoin.Sort.plan @@ -39,8 +39,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.n2.n_name (TEXT), num=32) [Enforcers] - 0: type=Broadcast, tables=default.n2 - 1: type=Broadcast, tables=default.n1 + 0: type=Broadcast, tables=default.n1 SORT(10) => Sort Keys: default.n2.n_name (TEXT) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testTPCHQ2Join.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testTPCHQ2Join.Hash.plan index d8f2c7b253..b19c33fd68 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testTPCHQ2Join.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testTPCHQ2Join.Hash.plan @@ -69,8 +69,7 @@ Block Id: eb_0000000000000_0000_000009 [LEAF] 0: type=Broadcast, tables=default.part 1: type=Broadcast, tables=default.supplier 2: type=Broadcast, tables=default.partsupp - 3: type=Broadcast, tables=default.nation - 4: type=Broadcast, tables=default.region + 3: type=Broadcast, tables=default.region SORT(26) => Sort Keys: default.supplier.s_acctbal (FLOAT8) (asc),default.supplier.s_name (TEXT) (asc),default.nation.n_name (TEXT) (asc),default.part.p_partkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testTPCHQ2Join.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testTPCHQ2Join.Sort.plan index d8f2c7b253..b19c33fd68 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testTPCHQ2Join.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testTPCHQ2Join.Sort.plan @@ -69,8 +69,7 @@ Block Id: eb_0000000000000_0000_000009 [LEAF] 0: type=Broadcast, tables=default.part 1: type=Broadcast, tables=default.supplier 2: type=Broadcast, tables=default.partsupp - 3: type=Broadcast, tables=default.nation - 4: type=Broadcast, tables=default.region + 3: type=Broadcast, tables=default.region SORT(26) => Sort Keys: default.supplier.s_acctbal (FLOAT8) (asc),default.supplier.s_name (TEXT) (asc),default.nation.n_name (TEXT) (asc),default.part.p_partkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin1.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin1.Hash.plan index 6d9a0190f8..6da36f78ce 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin1.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin1.Hash.plan @@ -40,7 +40,6 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.region - 1: type=Broadcast, tables=default.nation SORT(11) => Sort Keys: default.nation.n_name (TEXT) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin1.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin1.Sort.plan index 6d9a0190f8..6da36f78ce 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin1.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin1.Sort.plan @@ -40,7 +40,6 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.region - 1: type=Broadcast, tables=default.nation SORT(11) => Sort Keys: default.nation.n_name (TEXT) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin2.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin2.Hash.plan index cd036afc70..549fe31562 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin2.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin2.Hash.plan @@ -40,7 +40,6 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.region - 1: type=Broadcast, tables=default.nation SORT(11) => Sort Keys: default.nation.n_name (TEXT) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin2.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin2.Sort.plan index cd036afc70..549fe31562 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin2.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin2.Sort.plan @@ -40,7 +40,6 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.region - 1: type=Broadcast, tables=default.nation SORT(11) => Sort Keys: default.nation.n_name (TEXT) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin3.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin3.Hash.plan index bebae840b6..458abea3f8 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin3.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin3.Hash.plan @@ -40,7 +40,6 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.region - 1: type=Broadcast, tables=default.nation SORT(11) => Sort Keys: default.nation.n_name (TEXT) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin3.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin3.Sort.plan index bebae840b6..458abea3f8 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin3.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin3.Sort.plan @@ -40,7 +40,6 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.region - 1: type=Broadcast, tables=default.nation SORT(11) => Sort Keys: default.nation.n_name (TEXT) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin4.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin4.Hash.plan index 0978d54994..9ca4d412a2 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin4.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin4.Hash.plan @@ -39,8 +39,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.nation.n_name (TEXT), num=32) [Enforcers] - 0: type=Broadcast, tables=default.nation - 1: type=Broadcast, tables=default.region + 0: type=Broadcast, tables=default.region SORT(11) => Sort Keys: default.nation.n_name (TEXT) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin4.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin4.Sort.plan index 0978d54994..9ca4d412a2 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin4.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin4.Sort.plan @@ -39,8 +39,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.nation.n_name (TEXT), num=32) [Enforcers] - 0: type=Broadcast, tables=default.nation - 1: type=Broadcast, tables=default.region + 0: type=Broadcast, tables=default.region SORT(11) => Sort Keys: default.nation.n_name (TEXT) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin5.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin5.Hash.plan index c948e64476..93829fc319 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin5.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin5.Hash.plan @@ -49,8 +49,7 @@ Block Id: eb_0000000000000_0000_000005 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.supplier - 1: type=Broadcast, tables=default.partsupp - 2: type=Broadcast, tables=default.part + 1: type=Broadcast, tables=default.part SORT(16) => Sort Keys: default.supplier.s_acctbal (FLOAT8) (asc),default.supplier.s_name (TEXT) (asc),default.part.p_partkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin5.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin5.Sort.plan index c948e64476..93829fc319 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin5.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin5.Sort.plan @@ -49,8 +49,7 @@ Block Id: eb_0000000000000_0000_000005 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.supplier - 1: type=Broadcast, tables=default.partsupp - 2: type=Broadcast, tables=default.part + 1: type=Broadcast, tables=default.part SORT(16) => Sort Keys: default.supplier.s_acctbal (FLOAT8) (asc),default.supplier.s_name (TEXT) (asc),default.part.p_partkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin6.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin6.Hash.plan index 08bd65573e..73d8693ecb 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin6.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin6.Hash.plan @@ -60,7 +60,6 @@ Block Id: eb_0000000000000_0000_000007 [LEAF] 0: type=Broadcast, tables=default.part 1: type=Broadcast, tables=default.supplier 2: type=Broadcast, tables=default.partsupp - 3: type=Broadcast, tables=default.nation SORT(21) => Sort Keys: default.supplier.s_acctbal (FLOAT8) (asc),default.supplier.s_name (TEXT) (asc),default.part.p_partkey (INT4) (asc),default.nation.n_name (TEXT) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin6.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin6.Sort.plan index 08bd65573e..73d8693ecb 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin6.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinQuery/testWhereClauseJoin6.Sort.plan @@ -60,7 +60,6 @@ Block Id: eb_0000000000000_0000_000007 [LEAF] 0: type=Broadcast, tables=default.part 1: type=Broadcast, tables=default.supplier 2: type=Broadcast, tables=default.partsupp - 3: type=Broadcast, tables=default.nation SORT(21) => Sort Keys: default.supplier.s_acctbal (FLOAT8) (asc),default.supplier.s_name (TEXT) (asc),default.part.p_partkey (INT4) (asc),default.nation.n_name (TEXT) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery.Hash.plan index ab65357321..2b7db47b3d 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery.Hash.plan @@ -53,8 +53,7 @@ Block Id: eb_0000000000000_0000_000005 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.orders - 1: type=Broadcast, tables=default.lineitem - 2: type=Broadcast, tables=default.part + 1: type=Broadcast, tables=default.part SORT(18) => Sort Keys: default.lineitem.l_orderkey (INT4) (asc),default.a.o_custkey (INT4) (asc),default.part.p_name (TEXT) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery.Sort.plan index ab65357321..2b7db47b3d 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery.Sort.plan @@ -53,8 +53,7 @@ Block Id: eb_0000000000000_0000_000005 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.orders - 1: type=Broadcast, tables=default.lineitem - 2: type=Broadcast, tables=default.part + 1: type=Broadcast, tables=default.part SORT(18) => Sort Keys: default.lineitem.l_orderkey (INT4) (asc),default.a.o_custkey (INT4) (asc),default.part.p_name (TEXT) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery2.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery2.Hash.plan index dd370aa2be..56c6ac137a 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery2.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery2.Hash.plan @@ -91,8 +91,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [q_0000000000000_0000] 3 => 10 (type=HASH_SHUFFLE) [Enforcers] - 0: type=Broadcast, tables=default.a - 1: type=Broadcast, tables=default.part + 0: type=Broadcast, tables=default.part GROUP_BY(27)(l_orderkey) => exprs: (avg(default.e.l_quantity (FLOAT8))) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery2.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery2.Sort.plan index dd370aa2be..56c6ac137a 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery2.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery2.Sort.plan @@ -91,8 +91,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [q_0000000000000_0000] 3 => 10 (type=HASH_SHUFFLE) [Enforcers] - 0: type=Broadcast, tables=default.a - 1: type=Broadcast, tables=default.part + 0: type=Broadcast, tables=default.part GROUP_BY(27)(l_orderkey) => exprs: (avg(default.e.l_quantity (FLOAT8))) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition5.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition5.Hash.plan index 406e704cae..4e72e0877d 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition5.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition5.Hash.plan @@ -44,7 +44,6 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.nation - 1: type=Broadcast, tables=default.n1 SORT(12) => Sort Keys: default.n1.n_nationkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition5.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition5.Sort.plan index 406e704cae..4e72e0877d 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition5.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition5.Sort.plan @@ -44,7 +44,6 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.nation - 1: type=Broadcast, tables=default.n1 SORT(12) => Sort Keys: default.n1.n_nationkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual2.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual2.Hash.plan index b647b53fc0..f19b5cda83 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual2.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual2.Hash.plan @@ -46,9 +46,8 @@ Block Id: eb_0000000000000_0000_000005 [ROOT] ======================================================= [Enforcers] - 0: type=Broadcast, tables=default.n - 1: type=Broadcast, tables=default.r - 2: type=Broadcast, tables=default.s + 0: type=Broadcast, tables=default.r + 1: type=Broadcast, tables=default.s JOIN(11)(INNER) => Join Cond: default.s.s_nationkey (INT4) = default.t.n_nationkey (INT4) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual2.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual2.Sort.plan index b647b53fc0..f19b5cda83 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual2.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual2.Sort.plan @@ -46,9 +46,8 @@ Block Id: eb_0000000000000_0000_000005 [ROOT] ======================================================= [Enforcers] - 0: type=Broadcast, tables=default.n - 1: type=Broadcast, tables=default.r - 2: type=Broadcast, tables=default.s + 0: type=Broadcast, tables=default.r + 1: type=Broadcast, tables=default.s JOIN(11)(INNER) => Join Cond: default.s.s_nationkey (INT4) = default.t.n_nationkey (INT4) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual3.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual3.Hash.plan index 4356e9c135..cfbb8497e1 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual3.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual3.Hash.plan @@ -55,10 +55,9 @@ Block Id: eb_0000000000000_0000_000007 [ROOT] ======================================================= [Enforcers] - 0: type=Broadcast, tables=default.n - 1: type=Broadcast, tables=default.r - 2: type=Broadcast, tables=default.ps - 3: type=Broadcast, tables=default.s + 0: type=Broadcast, tables=default.r + 1: type=Broadcast, tables=default.ps + 2: type=Broadcast, tables=default.s JOIN(14)(INNER) => Join Cond: default.s.s_nationkey (INT4) = default.t.n_nationkey (INT4) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual3.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual3.Sort.plan index 4356e9c135..cfbb8497e1 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual3.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual3.Sort.plan @@ -55,10 +55,9 @@ Block Id: eb_0000000000000_0000_000007 [ROOT] ======================================================= [Enforcers] - 0: type=Broadcast, tables=default.n - 1: type=Broadcast, tables=default.r - 2: type=Broadcast, tables=default.ps - 3: type=Broadcast, tables=default.s + 0: type=Broadcast, tables=default.r + 1: type=Broadcast, tables=default.ps + 2: type=Broadcast, tables=default.s JOIN(14)(INNER) => Join Cond: default.s.s_nationkey (INT4) = default.t.n_nationkey (INT4) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual4.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual4.Hash.plan index a44067d673..a9b08762af 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual4.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual4.Hash.plan @@ -57,10 +57,9 @@ Block Id: eb_0000000000000_0000_000007 [ROOT] ======================================================= [Enforcers] - 0: type=Broadcast, tables=default.n - 1: type=Broadcast, tables=default.r - 2: type=Broadcast, tables=default.ps - 3: type=Broadcast, tables=default.s + 0: type=Broadcast, tables=default.r + 1: type=Broadcast, tables=default.ps + 2: type=Broadcast, tables=default.s SELECTION(9) => Search Cond: default.t.n_nationkey (INT4) > default.s.s_suppkey (INT4) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual4.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual4.Sort.plan index a44067d673..a9b08762af 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual4.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testJoinWithMultipleJoinQual4.Sort.plan @@ -57,10 +57,9 @@ Block Id: eb_0000000000000_0000_000007 [ROOT] ======================================================= [Enforcers] - 0: type=Broadcast, tables=default.n - 1: type=Broadcast, tables=default.r - 2: type=Broadcast, tables=default.ps - 3: type=Broadcast, tables=default.s + 0: type=Broadcast, tables=default.r + 1: type=Broadcast, tables=default.ps + 2: type=Broadcast, tables=default.s SELECTION(9) => Search Cond: default.t.n_nationkey (INT4) > default.s.s_suppkey (INT4) diff --git a/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testComplexJoinsWithCaseWhen.Hash.plan b/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testComplexJoinsWithCaseWhen.Hash.plan index a56bba9e7a..4ded90a1df 100644 --- a/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testComplexJoinsWithCaseWhen.Hash.plan +++ b/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testComplexJoinsWithCaseWhen.Hash.plan @@ -48,9 +48,8 @@ Block Id: eb_0000000000000_0000_000005 [LEAF] [q_0000000000000_0000] 5 => 6 (type=RANGE_SHUFFLE, key=default.region.r_name (TEXT), s1 (TEXT), num=32) [Enforcers] - 0: type=Broadcast, tables=default.nation - 1: type=Broadcast, tables=default.region - 2: type=Broadcast, tables=default.supplier + 0: type=Broadcast, tables=default.region + 1: type=Broadcast, tables=default.supplier SORT(15) => Sort Keys: default.region.r_name (TEXT) (asc),s1 (TEXT) (asc) diff --git a/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testComplexJoinsWithCaseWhen.Sort.plan b/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testComplexJoinsWithCaseWhen.Sort.plan index a56bba9e7a..4ded90a1df 100644 --- a/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testComplexJoinsWithCaseWhen.Sort.plan +++ b/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testComplexJoinsWithCaseWhen.Sort.plan @@ -48,9 +48,8 @@ Block Id: eb_0000000000000_0000_000005 [LEAF] [q_0000000000000_0000] 5 => 6 (type=RANGE_SHUFFLE, key=default.region.r_name (TEXT), s1 (TEXT), num=32) [Enforcers] - 0: type=Broadcast, tables=default.nation - 1: type=Broadcast, tables=default.region - 2: type=Broadcast, tables=default.supplier + 0: type=Broadcast, tables=default.region + 1: type=Broadcast, tables=default.supplier SORT(15) => Sort Keys: default.region.r_name (TEXT) (asc),s1 (TEXT) (asc) diff --git a/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testInnerAndOuterWithEmpty.1.Hash.plan b/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testInnerAndOuterWithEmpty.1.Hash.plan index e6ee3dbf61..1d5762f6e3 100644 --- a/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testInnerAndOuterWithEmpty.1.Hash.plan +++ b/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testInnerAndOuterWithEmpty.1.Hash.plan @@ -42,9 +42,8 @@ Block Id: eb_0000000000000_0000_000005 [ROOT] ======================================================= [Enforcers] - 0: type=Broadcast, tables=default.a - 1: type=Broadcast, tables=default.b - 2: type=Broadcast, tables=default.c + 0: type=Broadcast, tables=default.b + 1: type=Broadcast, tables=default.c JOIN(8)(LEFT_OUTER) => Join Cond: default.a.l_orderkey (INT4) = default.c.c_custkey (INT4) diff --git a/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testInnerAndOuterWithEmpty.1.Sort.plan b/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testInnerAndOuterWithEmpty.1.Sort.plan index e6ee3dbf61..1d5762f6e3 100644 --- a/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testInnerAndOuterWithEmpty.1.Sort.plan +++ b/tajo-core/src/test/resources/results/TestMultipleJoinTypes/testInnerAndOuterWithEmpty.1.Sort.plan @@ -42,9 +42,8 @@ Block Id: eb_0000000000000_0000_000005 [ROOT] ======================================================= [Enforcers] - 0: type=Broadcast, tables=default.a - 1: type=Broadcast, tables=default.b - 2: type=Broadcast, tables=default.c + 0: type=Broadcast, tables=default.b + 1: type=Broadcast, tables=default.c JOIN(8)(LEFT_OUTER) => Join Cond: default.a.l_orderkey (INT4) = default.c.c_custkey (INT4) diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testFullOuterJoin1.Hash.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testFullOuterJoin1.Hash.plan index 9b709e7e28..9fbfec508a 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testFullOuterJoin1.Hash.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testFullOuterJoin1.Hash.plan @@ -23,40 +23,66 @@ Execution Block Graph (TERMINAL - eb_0000000000000_0000_000005) |-eb_0000000000000_0000_000005 |-eb_0000000000000_0000_000004 |-eb_0000000000000_0000_000003 + |-eb_0000000000000_0000_000002 + |-eb_0000000000000_0000_000001 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 -3: eb_0000000000000_0000_000005 +1: eb_0000000000000_0000_000001 +2: eb_0000000000000_0000_000002 +3: eb_0000000000000_0000_000003 +4: eb_0000000000000_0000_000004 +5: eb_0000000000000_0000_000005 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [LEAF] +Block Id: eb_0000000000000_0000_000001 [LEAF] ======================================================= [Outgoing] -[q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.customer.c_custkey (INT4), default.orders.o_orderkey (INT4), num=32) +[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.orders.o_orderkey (INT4), num=32) -[Enforcers] - 0: type=Broadcast, tables=default.orders - 1: type=Broadcast, tables=default.customer +SCAN(0) on default.orders + => target list: default.orders.o_orderkey (INT4) + => out schema: {(1) default.orders.o_orderkey (INT4)} + => in schema: {(9) default.orders.o_clerk (TEXT), default.orders.o_comment (TEXT), default.orders.o_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4), default.orders.o_orderpriority (TEXT), default.orders.o_orderstatus (TEXT), default.orders.o_shippriority (INT4), default.orders.o_totalprice (FLOAT8)} + +======================================================= +Block Id: eb_0000000000000_0000_000002 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.customer.c_custkey (INT4), num=32) + +SCAN(1) on default.customer + => target list: default.customer.c_custkey (INT4) + => out schema: {(1) default.customer.c_custkey (INT4)} + => in schema: {(8) default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000003 [INTERMEDIATE] +======================================================= -SORT(8) +[Incoming] +[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.orders.o_orderkey (INT4), num=32) +[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.customer.c_custkey (INT4), num=32) + +[Outgoing] +[q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.customer.c_custkey (INT4), default.orders.o_orderkey (INT4), num=32) + +SORT(10) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.orders.o_orderkey (INT4) (asc) JOIN(6)(FULL_OUTER) => Join Cond: default.customer.c_custkey (INT4) = default.orders.o_orderkey (INT4) => target list: default.customer.c_custkey (INT4), default.orders.o_orderkey (INT4) => out schema: {(2) default.customer.c_custkey (INT4), default.orders.o_orderkey (INT4)} => in schema: {(2) default.customer.c_custkey (INT4), default.orders.o_orderkey (INT4)} - SCAN(1) on default.customer - => target list: default.customer.c_custkey (INT4) + SCAN(9) on eb_0000000000000_0000_000002 => out schema: {(1) default.customer.c_custkey (INT4)} - => in schema: {(8) default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT)} - SCAN(0) on default.orders - => target list: default.orders.o_orderkey (INT4) + => in schema: {(1) default.customer.c_custkey (INT4)} + SCAN(8) on eb_0000000000000_0000_000001 => out schema: {(1) default.orders.o_orderkey (INT4)} - => in schema: {(9) default.orders.o_clerk (TEXT), default.orders.o_comment (TEXT), default.orders.o_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4), default.orders.o_orderpriority (TEXT), default.orders.o_orderstatus (TEXT), default.orders.o_shippriority (INT4), default.orders.o_totalprice (FLOAT8)} + => in schema: {(1) default.orders.o_orderkey (INT4)} ======================================================= Block Id: eb_0000000000000_0000_000004 [ROOT] @@ -70,7 +96,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.orders.o_orderkey (INT4) (asc) - SCAN(9) on eb_0000000000000_0000_000003 + SCAN(11) on eb_0000000000000_0000_000003 => out schema: {(2) default.customer.c_custkey (INT4), default.orders.o_orderkey (INT4)} => in schema: {(2) default.customer.c_custkey (INT4), default.orders.o_orderkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testFullOuterJoin1.Sort.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testFullOuterJoin1.Sort.plan index 9b709e7e28..9fbfec508a 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testFullOuterJoin1.Sort.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testFullOuterJoin1.Sort.plan @@ -23,40 +23,66 @@ Execution Block Graph (TERMINAL - eb_0000000000000_0000_000005) |-eb_0000000000000_0000_000005 |-eb_0000000000000_0000_000004 |-eb_0000000000000_0000_000003 + |-eb_0000000000000_0000_000002 + |-eb_0000000000000_0000_000001 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 -3: eb_0000000000000_0000_000005 +1: eb_0000000000000_0000_000001 +2: eb_0000000000000_0000_000002 +3: eb_0000000000000_0000_000003 +4: eb_0000000000000_0000_000004 +5: eb_0000000000000_0000_000005 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [LEAF] +Block Id: eb_0000000000000_0000_000001 [LEAF] ======================================================= [Outgoing] -[q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.customer.c_custkey (INT4), default.orders.o_orderkey (INT4), num=32) +[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.orders.o_orderkey (INT4), num=32) -[Enforcers] - 0: type=Broadcast, tables=default.orders - 1: type=Broadcast, tables=default.customer +SCAN(0) on default.orders + => target list: default.orders.o_orderkey (INT4) + => out schema: {(1) default.orders.o_orderkey (INT4)} + => in schema: {(9) default.orders.o_clerk (TEXT), default.orders.o_comment (TEXT), default.orders.o_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4), default.orders.o_orderpriority (TEXT), default.orders.o_orderstatus (TEXT), default.orders.o_shippriority (INT4), default.orders.o_totalprice (FLOAT8)} + +======================================================= +Block Id: eb_0000000000000_0000_000002 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.customer.c_custkey (INT4), num=32) + +SCAN(1) on default.customer + => target list: default.customer.c_custkey (INT4) + => out schema: {(1) default.customer.c_custkey (INT4)} + => in schema: {(8) default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000003 [INTERMEDIATE] +======================================================= -SORT(8) +[Incoming] +[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.orders.o_orderkey (INT4), num=32) +[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.customer.c_custkey (INT4), num=32) + +[Outgoing] +[q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.customer.c_custkey (INT4), default.orders.o_orderkey (INT4), num=32) + +SORT(10) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.orders.o_orderkey (INT4) (asc) JOIN(6)(FULL_OUTER) => Join Cond: default.customer.c_custkey (INT4) = default.orders.o_orderkey (INT4) => target list: default.customer.c_custkey (INT4), default.orders.o_orderkey (INT4) => out schema: {(2) default.customer.c_custkey (INT4), default.orders.o_orderkey (INT4)} => in schema: {(2) default.customer.c_custkey (INT4), default.orders.o_orderkey (INT4)} - SCAN(1) on default.customer - => target list: default.customer.c_custkey (INT4) + SCAN(9) on eb_0000000000000_0000_000002 => out schema: {(1) default.customer.c_custkey (INT4)} - => in schema: {(8) default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT)} - SCAN(0) on default.orders - => target list: default.orders.o_orderkey (INT4) + => in schema: {(1) default.customer.c_custkey (INT4)} + SCAN(8) on eb_0000000000000_0000_000001 => out schema: {(1) default.orders.o_orderkey (INT4)} - => in schema: {(9) default.orders.o_clerk (TEXT), default.orders.o_comment (TEXT), default.orders.o_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4), default.orders.o_orderpriority (TEXT), default.orders.o_orderstatus (TEXT), default.orders.o_shippriority (INT4), default.orders.o_totalprice (FLOAT8)} + => in schema: {(1) default.orders.o_orderkey (INT4)} ======================================================= Block Id: eb_0000000000000_0000_000004 [ROOT] @@ -70,7 +96,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.orders.o_orderkey (INT4) (asc) - SCAN(9) on eb_0000000000000_0000_000003 + SCAN(11) on eb_0000000000000_0000_000003 => out schema: {(2) default.customer.c_custkey (INT4), default.orders.o_orderkey (INT4)} => in schema: {(2) default.customer.c_custkey (INT4), default.orders.o_orderkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testFullOuterJoinPredicationCaseByCase1.1.Hash.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testFullOuterJoinPredicationCaseByCase1.1.Hash.plan new file mode 100644 index 0000000000..fa727c0009 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testFullOuterJoinPredicationCaseByCase1.1.Hash.plan @@ -0,0 +1,153 @@ +explain +------------------------------- +SORT(5) + => Sort Keys: default.t4.id (INT4) (asc) + JOIN(9)(FULL_OUTER) + => Join Cond: default.t3.id (INT4) = default.t4.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4) + => out schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4)} + => in schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4)} + SCAN(3) on default.jointable14 as t4 + => target list: default.t4.id (INT4) + => out schema: {(1) default.t4.id (INT4)} + => in schema: {(2) default.t4.id (INT4), default.t4.name (TEXT)} + JOIN(8)(FULL_OUTER) + => Join Cond: default.t1.id (INT4) = default.t3.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4) + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4)} + SCAN(1) on default.jointable13 as t3 + => target list: default.t3.id (INT4) + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(2) default.t3.id (INT4), default.t3.name (TEXT)} + SCAN(0) on default.jointable11 as t1 + => target list: default.t1.id (INT4), default.t1.name (TEXT) + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} +explain +------------------------------- +------------------------------------------------------------------------------- +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000007) +------------------------------------------------------------------------------- +|-eb_0000000000000_0000_000007 + |-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000005 + |-eb_0000000000000_0000_000004 + |-eb_0000000000000_0000_000003 + |-eb_0000000000000_0000_000002 + |-eb_0000000000000_0000_000001 +------------------------------------------------------------------------------- +Order of Execution +------------------------------------------------------------------------------- +1: eb_0000000000000_0000_000001 +2: eb_0000000000000_0000_000002 +3: eb_0000000000000_0000_000003 +4: eb_0000000000000_0000_000004 +5: eb_0000000000000_0000_000005 +6: eb_0000000000000_0000_000006 +7: eb_0000000000000_0000_000007 +------------------------------------------------------------------------------- + +======================================================= +Block Id: eb_0000000000000_0000_000001 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.t1.id (INT4), num=32) + +SCAN(0) on default.jointable11 as t1 + => target list: default.t1.id (INT4), default.t1.name (TEXT) + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000002 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.t3.id (INT4), num=32) + +SCAN(1) on default.jointable13 as t3 + => target list: default.t3.id (INT4) + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(2) default.t3.id (INT4), default.t3.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000003 [INTERMEDIATE] +======================================================= + +[Incoming] +[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.t1.id (INT4), num=32) +[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.t3.id (INT4), num=32) + +[Outgoing] +[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.t3.id (INT4), num=32) + +JOIN(8)(FULL_OUTER) + => Join Cond: default.t1.id (INT4) = default.t3.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4) + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4)} + SCAN(12) on eb_0000000000000_0000_000002 + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(1) default.t3.id (INT4)} + SCAN(11) on eb_0000000000000_0000_000001 + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000004 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.t4.id (INT4), num=32) + +SCAN(3) on default.jointable14 as t4 + => target list: default.t4.id (INT4) + => out schema: {(1) default.t4.id (INT4)} + => in schema: {(2) default.t4.id (INT4), default.t4.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000005 [INTERMEDIATE] +======================================================= + +[Incoming] +[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.t3.id (INT4), num=32) +[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.t4.id (INT4), num=32) + +[Outgoing] +[q_0000000000000_0000] 5 => 6 (type=RANGE_SHUFFLE, key=default.t4.id (INT4), num=32) + +SORT(15) + => Sort Keys: default.t4.id (INT4) (asc) + JOIN(9)(FULL_OUTER) + => Join Cond: default.t3.id (INT4) = default.t4.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4) + => out schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4)} + => in schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4)} + SCAN(14) on eb_0000000000000_0000_000004 + => out schema: {(1) default.t4.id (INT4)} + => in schema: {(1) default.t4.id (INT4)} + SCAN(13) on eb_0000000000000_0000_000003 + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000006 [ROOT] +======================================================= + +[Incoming] +[q_0000000000000_0000] 5 => 6 (type=RANGE_SHUFFLE, key=default.t4.id (INT4), num=32) + +[Enforcers] + 0: sorted input=eb_0000000000000_0000_000005 + +SORT(5) + => Sort Keys: default.t4.id (INT4) (asc) + SCAN(16) on eb_0000000000000_0000_000005 + => out schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4)} + => in schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000007 [TERMINAL] +======================================================= diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testFullOuterJoinPredicationCaseByCase1.1.Hash_NoBroadcast.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testFullOuterJoinPredicationCaseByCase1.1.Hash_NoBroadcast.plan new file mode 100644 index 0000000000..fa727c0009 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testFullOuterJoinPredicationCaseByCase1.1.Hash_NoBroadcast.plan @@ -0,0 +1,153 @@ +explain +------------------------------- +SORT(5) + => Sort Keys: default.t4.id (INT4) (asc) + JOIN(9)(FULL_OUTER) + => Join Cond: default.t3.id (INT4) = default.t4.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4) + => out schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4)} + => in schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4)} + SCAN(3) on default.jointable14 as t4 + => target list: default.t4.id (INT4) + => out schema: {(1) default.t4.id (INT4)} + => in schema: {(2) default.t4.id (INT4), default.t4.name (TEXT)} + JOIN(8)(FULL_OUTER) + => Join Cond: default.t1.id (INT4) = default.t3.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4) + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4)} + SCAN(1) on default.jointable13 as t3 + => target list: default.t3.id (INT4) + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(2) default.t3.id (INT4), default.t3.name (TEXT)} + SCAN(0) on default.jointable11 as t1 + => target list: default.t1.id (INT4), default.t1.name (TEXT) + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} +explain +------------------------------- +------------------------------------------------------------------------------- +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000007) +------------------------------------------------------------------------------- +|-eb_0000000000000_0000_000007 + |-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000005 + |-eb_0000000000000_0000_000004 + |-eb_0000000000000_0000_000003 + |-eb_0000000000000_0000_000002 + |-eb_0000000000000_0000_000001 +------------------------------------------------------------------------------- +Order of Execution +------------------------------------------------------------------------------- +1: eb_0000000000000_0000_000001 +2: eb_0000000000000_0000_000002 +3: eb_0000000000000_0000_000003 +4: eb_0000000000000_0000_000004 +5: eb_0000000000000_0000_000005 +6: eb_0000000000000_0000_000006 +7: eb_0000000000000_0000_000007 +------------------------------------------------------------------------------- + +======================================================= +Block Id: eb_0000000000000_0000_000001 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.t1.id (INT4), num=32) + +SCAN(0) on default.jointable11 as t1 + => target list: default.t1.id (INT4), default.t1.name (TEXT) + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000002 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.t3.id (INT4), num=32) + +SCAN(1) on default.jointable13 as t3 + => target list: default.t3.id (INT4) + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(2) default.t3.id (INT4), default.t3.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000003 [INTERMEDIATE] +======================================================= + +[Incoming] +[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.t1.id (INT4), num=32) +[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.t3.id (INT4), num=32) + +[Outgoing] +[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.t3.id (INT4), num=32) + +JOIN(8)(FULL_OUTER) + => Join Cond: default.t1.id (INT4) = default.t3.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4) + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4)} + SCAN(12) on eb_0000000000000_0000_000002 + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(1) default.t3.id (INT4)} + SCAN(11) on eb_0000000000000_0000_000001 + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000004 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.t4.id (INT4), num=32) + +SCAN(3) on default.jointable14 as t4 + => target list: default.t4.id (INT4) + => out schema: {(1) default.t4.id (INT4)} + => in schema: {(2) default.t4.id (INT4), default.t4.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000005 [INTERMEDIATE] +======================================================= + +[Incoming] +[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.t3.id (INT4), num=32) +[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.t4.id (INT4), num=32) + +[Outgoing] +[q_0000000000000_0000] 5 => 6 (type=RANGE_SHUFFLE, key=default.t4.id (INT4), num=32) + +SORT(15) + => Sort Keys: default.t4.id (INT4) (asc) + JOIN(9)(FULL_OUTER) + => Join Cond: default.t3.id (INT4) = default.t4.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4) + => out schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4)} + => in schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4)} + SCAN(14) on eb_0000000000000_0000_000004 + => out schema: {(1) default.t4.id (INT4)} + => in schema: {(1) default.t4.id (INT4)} + SCAN(13) on eb_0000000000000_0000_000003 + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000006 [ROOT] +======================================================= + +[Incoming] +[q_0000000000000_0000] 5 => 6 (type=RANGE_SHUFFLE, key=default.t4.id (INT4), num=32) + +[Enforcers] + 0: sorted input=eb_0000000000000_0000_000005 + +SORT(5) + => Sort Keys: default.t4.id (INT4) (asc) + SCAN(16) on eb_0000000000000_0000_000005 + => out schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4)} + => in schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000007 [TERMINAL] +======================================================= diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testFullOuterJoinPredicationCaseByCase1.1.Sort.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testFullOuterJoinPredicationCaseByCase1.1.Sort.plan new file mode 100644 index 0000000000..fa727c0009 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testFullOuterJoinPredicationCaseByCase1.1.Sort.plan @@ -0,0 +1,153 @@ +explain +------------------------------- +SORT(5) + => Sort Keys: default.t4.id (INT4) (asc) + JOIN(9)(FULL_OUTER) + => Join Cond: default.t3.id (INT4) = default.t4.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4) + => out schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4)} + => in schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4)} + SCAN(3) on default.jointable14 as t4 + => target list: default.t4.id (INT4) + => out schema: {(1) default.t4.id (INT4)} + => in schema: {(2) default.t4.id (INT4), default.t4.name (TEXT)} + JOIN(8)(FULL_OUTER) + => Join Cond: default.t1.id (INT4) = default.t3.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4) + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4)} + SCAN(1) on default.jointable13 as t3 + => target list: default.t3.id (INT4) + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(2) default.t3.id (INT4), default.t3.name (TEXT)} + SCAN(0) on default.jointable11 as t1 + => target list: default.t1.id (INT4), default.t1.name (TEXT) + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} +explain +------------------------------- +------------------------------------------------------------------------------- +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000007) +------------------------------------------------------------------------------- +|-eb_0000000000000_0000_000007 + |-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000005 + |-eb_0000000000000_0000_000004 + |-eb_0000000000000_0000_000003 + |-eb_0000000000000_0000_000002 + |-eb_0000000000000_0000_000001 +------------------------------------------------------------------------------- +Order of Execution +------------------------------------------------------------------------------- +1: eb_0000000000000_0000_000001 +2: eb_0000000000000_0000_000002 +3: eb_0000000000000_0000_000003 +4: eb_0000000000000_0000_000004 +5: eb_0000000000000_0000_000005 +6: eb_0000000000000_0000_000006 +7: eb_0000000000000_0000_000007 +------------------------------------------------------------------------------- + +======================================================= +Block Id: eb_0000000000000_0000_000001 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.t1.id (INT4), num=32) + +SCAN(0) on default.jointable11 as t1 + => target list: default.t1.id (INT4), default.t1.name (TEXT) + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000002 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.t3.id (INT4), num=32) + +SCAN(1) on default.jointable13 as t3 + => target list: default.t3.id (INT4) + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(2) default.t3.id (INT4), default.t3.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000003 [INTERMEDIATE] +======================================================= + +[Incoming] +[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.t1.id (INT4), num=32) +[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.t3.id (INT4), num=32) + +[Outgoing] +[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.t3.id (INT4), num=32) + +JOIN(8)(FULL_OUTER) + => Join Cond: default.t1.id (INT4) = default.t3.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4) + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4)} + SCAN(12) on eb_0000000000000_0000_000002 + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(1) default.t3.id (INT4)} + SCAN(11) on eb_0000000000000_0000_000001 + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000004 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.t4.id (INT4), num=32) + +SCAN(3) on default.jointable14 as t4 + => target list: default.t4.id (INT4) + => out schema: {(1) default.t4.id (INT4)} + => in schema: {(2) default.t4.id (INT4), default.t4.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000005 [INTERMEDIATE] +======================================================= + +[Incoming] +[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.t3.id (INT4), num=32) +[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.t4.id (INT4), num=32) + +[Outgoing] +[q_0000000000000_0000] 5 => 6 (type=RANGE_SHUFFLE, key=default.t4.id (INT4), num=32) + +SORT(15) + => Sort Keys: default.t4.id (INT4) (asc) + JOIN(9)(FULL_OUTER) + => Join Cond: default.t3.id (INT4) = default.t4.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4) + => out schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4)} + => in schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4)} + SCAN(14) on eb_0000000000000_0000_000004 + => out schema: {(1) default.t4.id (INT4)} + => in schema: {(1) default.t4.id (INT4)} + SCAN(13) on eb_0000000000000_0000_000003 + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000006 [ROOT] +======================================================= + +[Incoming] +[q_0000000000000_0000] 5 => 6 (type=RANGE_SHUFFLE, key=default.t4.id (INT4), num=32) + +[Enforcers] + 0: sorted input=eb_0000000000000_0000_000005 + +SORT(5) + => Sort Keys: default.t4.id (INT4) (asc) + SCAN(16) on eb_0000000000000_0000_000005 + => out schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4)} + => in schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000007 [TERMINAL] +======================================================= diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testFullOuterJoinPredicationCaseByCase1.1.Sort_NoBroadcast.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testFullOuterJoinPredicationCaseByCase1.1.Sort_NoBroadcast.plan new file mode 100644 index 0000000000..fa727c0009 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testFullOuterJoinPredicationCaseByCase1.1.Sort_NoBroadcast.plan @@ -0,0 +1,153 @@ +explain +------------------------------- +SORT(5) + => Sort Keys: default.t4.id (INT4) (asc) + JOIN(9)(FULL_OUTER) + => Join Cond: default.t3.id (INT4) = default.t4.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4) + => out schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4)} + => in schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4)} + SCAN(3) on default.jointable14 as t4 + => target list: default.t4.id (INT4) + => out schema: {(1) default.t4.id (INT4)} + => in schema: {(2) default.t4.id (INT4), default.t4.name (TEXT)} + JOIN(8)(FULL_OUTER) + => Join Cond: default.t1.id (INT4) = default.t3.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4) + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4)} + SCAN(1) on default.jointable13 as t3 + => target list: default.t3.id (INT4) + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(2) default.t3.id (INT4), default.t3.name (TEXT)} + SCAN(0) on default.jointable11 as t1 + => target list: default.t1.id (INT4), default.t1.name (TEXT) + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} +explain +------------------------------- +------------------------------------------------------------------------------- +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000007) +------------------------------------------------------------------------------- +|-eb_0000000000000_0000_000007 + |-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000005 + |-eb_0000000000000_0000_000004 + |-eb_0000000000000_0000_000003 + |-eb_0000000000000_0000_000002 + |-eb_0000000000000_0000_000001 +------------------------------------------------------------------------------- +Order of Execution +------------------------------------------------------------------------------- +1: eb_0000000000000_0000_000001 +2: eb_0000000000000_0000_000002 +3: eb_0000000000000_0000_000003 +4: eb_0000000000000_0000_000004 +5: eb_0000000000000_0000_000005 +6: eb_0000000000000_0000_000006 +7: eb_0000000000000_0000_000007 +------------------------------------------------------------------------------- + +======================================================= +Block Id: eb_0000000000000_0000_000001 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.t1.id (INT4), num=32) + +SCAN(0) on default.jointable11 as t1 + => target list: default.t1.id (INT4), default.t1.name (TEXT) + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000002 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.t3.id (INT4), num=32) + +SCAN(1) on default.jointable13 as t3 + => target list: default.t3.id (INT4) + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(2) default.t3.id (INT4), default.t3.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000003 [INTERMEDIATE] +======================================================= + +[Incoming] +[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.t1.id (INT4), num=32) +[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.t3.id (INT4), num=32) + +[Outgoing] +[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.t3.id (INT4), num=32) + +JOIN(8)(FULL_OUTER) + => Join Cond: default.t1.id (INT4) = default.t3.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4) + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4)} + SCAN(12) on eb_0000000000000_0000_000002 + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(1) default.t3.id (INT4)} + SCAN(11) on eb_0000000000000_0000_000001 + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000004 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.t4.id (INT4), num=32) + +SCAN(3) on default.jointable14 as t4 + => target list: default.t4.id (INT4) + => out schema: {(1) default.t4.id (INT4)} + => in schema: {(2) default.t4.id (INT4), default.t4.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000005 [INTERMEDIATE] +======================================================= + +[Incoming] +[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.t3.id (INT4), num=32) +[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.t4.id (INT4), num=32) + +[Outgoing] +[q_0000000000000_0000] 5 => 6 (type=RANGE_SHUFFLE, key=default.t4.id (INT4), num=32) + +SORT(15) + => Sort Keys: default.t4.id (INT4) (asc) + JOIN(9)(FULL_OUTER) + => Join Cond: default.t3.id (INT4) = default.t4.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4) + => out schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4)} + => in schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4)} + SCAN(14) on eb_0000000000000_0000_000004 + => out schema: {(1) default.t4.id (INT4)} + => in schema: {(1) default.t4.id (INT4)} + SCAN(13) on eb_0000000000000_0000_000003 + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000006 [ROOT] +======================================================= + +[Incoming] +[q_0000000000000_0000] 5 => 6 (type=RANGE_SHUFFLE, key=default.t4.id (INT4), num=32) + +[Enforcers] + 0: sorted input=eb_0000000000000_0000_000005 + +SORT(5) + => Sort Keys: default.t4.id (INT4) (asc) + SCAN(16) on eb_0000000000000_0000_000005 + => out schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4)} + => in schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000007 [TERMINAL] +======================================================= diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testFullOuterJoinPredicationCaseByCase1.1.result b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testFullOuterJoinPredicationCaseByCase1.1.result new file mode 100644 index 0000000000..8d9d2942a7 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testFullOuterJoinPredicationCaseByCase1.1.result @@ -0,0 +1,9 @@ +id,name,id,id +------------------------------- +null,null,null,1 +2,table11-2,2,2 +3,table11-3,3,3 +null,null,null,4 +1,table11-1,null,null +4,table11-4,null,null +5,table11-5,null,null diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testFullOuterJoinWithEmptyTable1.Hash.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testFullOuterJoinWithEmptyTable1.Hash.plan index 38ffa150d7..580a217f55 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testFullOuterJoinWithEmptyTable1.Hash.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testFullOuterJoinWithEmptyTable1.Hash.plan @@ -23,40 +23,66 @@ Execution Block Graph (TERMINAL - eb_0000000000000_0000_000005) |-eb_0000000000000_0000_000005 |-eb_0000000000000_0000_000004 |-eb_0000000000000_0000_000003 + |-eb_0000000000000_0000_000002 + |-eb_0000000000000_0000_000001 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 -3: eb_0000000000000_0000_000005 +1: eb_0000000000000_0000_000001 +2: eb_0000000000000_0000_000002 +3: eb_0000000000000_0000_000003 +4: eb_0000000000000_0000_000004 +5: eb_0000000000000_0000_000005 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [LEAF] +Block Id: eb_0000000000000_0000_000001 [LEAF] ======================================================= [Outgoing] -[q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.customer.c_custkey (INT4), default.empty_orders.o_orderkey (INT4), num=32) +[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.empty_orders.o_orderkey (INT4), num=32) -[Enforcers] - 0: type=Broadcast, tables=default.empty_orders - 1: type=Broadcast, tables=default.customer +SCAN(0) on default.empty_orders + => target list: default.empty_orders.o_orderkey (INT4) + => out schema: {(1) default.empty_orders.o_orderkey (INT4)} + => in schema: {(9) default.empty_orders.o_clerk (TEXT), default.empty_orders.o_comment (TEXT), default.empty_orders.o_custkey (INT4), default.empty_orders.o_orderdate (TEXT), default.empty_orders.o_orderkey (INT4), default.empty_orders.o_orderpriority (TEXT), default.empty_orders.o_orderstatus (TEXT), default.empty_orders.o_shippriority (INT4), default.empty_orders.o_totalprice (FLOAT8)} + +======================================================= +Block Id: eb_0000000000000_0000_000002 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.customer.c_custkey (INT4), num=32) + +SCAN(1) on default.customer + => target list: default.customer.c_custkey (INT4) + => out schema: {(1) default.customer.c_custkey (INT4)} + => in schema: {(8) default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000003 [INTERMEDIATE] +======================================================= -SORT(8) +[Incoming] +[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.empty_orders.o_orderkey (INT4), num=32) +[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.customer.c_custkey (INT4), num=32) + +[Outgoing] +[q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.customer.c_custkey (INT4), default.empty_orders.o_orderkey (INT4), num=32) + +SORT(10) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.empty_orders.o_orderkey (INT4) (asc) JOIN(6)(FULL_OUTER) => Join Cond: default.customer.c_custkey (INT4) = default.empty_orders.o_orderkey (INT4) => target list: default.customer.c_custkey (INT4), default.empty_orders.o_orderkey (INT4) => out schema: {(2) default.customer.c_custkey (INT4), default.empty_orders.o_orderkey (INT4)} => in schema: {(2) default.customer.c_custkey (INT4), default.empty_orders.o_orderkey (INT4)} - SCAN(1) on default.customer - => target list: default.customer.c_custkey (INT4) + SCAN(9) on eb_0000000000000_0000_000002 => out schema: {(1) default.customer.c_custkey (INT4)} - => in schema: {(8) default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT)} - SCAN(0) on default.empty_orders - => target list: default.empty_orders.o_orderkey (INT4) + => in schema: {(1) default.customer.c_custkey (INT4)} + SCAN(8) on eb_0000000000000_0000_000001 => out schema: {(1) default.empty_orders.o_orderkey (INT4)} - => in schema: {(9) default.empty_orders.o_clerk (TEXT), default.empty_orders.o_comment (TEXT), default.empty_orders.o_custkey (INT4), default.empty_orders.o_orderdate (TEXT), default.empty_orders.o_orderkey (INT4), default.empty_orders.o_orderpriority (TEXT), default.empty_orders.o_orderstatus (TEXT), default.empty_orders.o_shippriority (INT4), default.empty_orders.o_totalprice (FLOAT8)} + => in schema: {(1) default.empty_orders.o_orderkey (INT4)} ======================================================= Block Id: eb_0000000000000_0000_000004 [ROOT] @@ -70,7 +96,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.empty_orders.o_orderkey (INT4) (asc) - SCAN(9) on eb_0000000000000_0000_000003 + SCAN(11) on eb_0000000000000_0000_000003 => out schema: {(2) default.customer.c_custkey (INT4), default.empty_orders.o_orderkey (INT4)} => in schema: {(2) default.customer.c_custkey (INT4), default.empty_orders.o_orderkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testFullOuterJoinWithEmptyTable1.Sort.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testFullOuterJoinWithEmptyTable1.Sort.plan index 38ffa150d7..580a217f55 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testFullOuterJoinWithEmptyTable1.Sort.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testFullOuterJoinWithEmptyTable1.Sort.plan @@ -23,40 +23,66 @@ Execution Block Graph (TERMINAL - eb_0000000000000_0000_000005) |-eb_0000000000000_0000_000005 |-eb_0000000000000_0000_000004 |-eb_0000000000000_0000_000003 + |-eb_0000000000000_0000_000002 + |-eb_0000000000000_0000_000001 ------------------------------------------------------------------------------- Order of Execution ------------------------------------------------------------------------------- -1: eb_0000000000000_0000_000003 -2: eb_0000000000000_0000_000004 -3: eb_0000000000000_0000_000005 +1: eb_0000000000000_0000_000001 +2: eb_0000000000000_0000_000002 +3: eb_0000000000000_0000_000003 +4: eb_0000000000000_0000_000004 +5: eb_0000000000000_0000_000005 ------------------------------------------------------------------------------- ======================================================= -Block Id: eb_0000000000000_0000_000003 [LEAF] +Block Id: eb_0000000000000_0000_000001 [LEAF] ======================================================= [Outgoing] -[q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.customer.c_custkey (INT4), default.empty_orders.o_orderkey (INT4), num=32) +[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.empty_orders.o_orderkey (INT4), num=32) -[Enforcers] - 0: type=Broadcast, tables=default.empty_orders - 1: type=Broadcast, tables=default.customer +SCAN(0) on default.empty_orders + => target list: default.empty_orders.o_orderkey (INT4) + => out schema: {(1) default.empty_orders.o_orderkey (INT4)} + => in schema: {(9) default.empty_orders.o_clerk (TEXT), default.empty_orders.o_comment (TEXT), default.empty_orders.o_custkey (INT4), default.empty_orders.o_orderdate (TEXT), default.empty_orders.o_orderkey (INT4), default.empty_orders.o_orderpriority (TEXT), default.empty_orders.o_orderstatus (TEXT), default.empty_orders.o_shippriority (INT4), default.empty_orders.o_totalprice (FLOAT8)} + +======================================================= +Block Id: eb_0000000000000_0000_000002 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.customer.c_custkey (INT4), num=32) + +SCAN(1) on default.customer + => target list: default.customer.c_custkey (INT4) + => out schema: {(1) default.customer.c_custkey (INT4)} + => in schema: {(8) default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000003 [INTERMEDIATE] +======================================================= -SORT(8) +[Incoming] +[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.empty_orders.o_orderkey (INT4), num=32) +[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.customer.c_custkey (INT4), num=32) + +[Outgoing] +[q_0000000000000_0000] 3 => 4 (type=RANGE_SHUFFLE, key=default.customer.c_custkey (INT4), default.empty_orders.o_orderkey (INT4), num=32) + +SORT(10) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.empty_orders.o_orderkey (INT4) (asc) JOIN(6)(FULL_OUTER) => Join Cond: default.customer.c_custkey (INT4) = default.empty_orders.o_orderkey (INT4) => target list: default.customer.c_custkey (INT4), default.empty_orders.o_orderkey (INT4) => out schema: {(2) default.customer.c_custkey (INT4), default.empty_orders.o_orderkey (INT4)} => in schema: {(2) default.customer.c_custkey (INT4), default.empty_orders.o_orderkey (INT4)} - SCAN(1) on default.customer - => target list: default.customer.c_custkey (INT4) + SCAN(9) on eb_0000000000000_0000_000002 => out schema: {(1) default.customer.c_custkey (INT4)} - => in schema: {(8) default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT)} - SCAN(0) on default.empty_orders - => target list: default.empty_orders.o_orderkey (INT4) + => in schema: {(1) default.customer.c_custkey (INT4)} + SCAN(8) on eb_0000000000000_0000_000001 => out schema: {(1) default.empty_orders.o_orderkey (INT4)} - => in schema: {(9) default.empty_orders.o_clerk (TEXT), default.empty_orders.o_comment (TEXT), default.empty_orders.o_custkey (INT4), default.empty_orders.o_orderdate (TEXT), default.empty_orders.o_orderkey (INT4), default.empty_orders.o_orderpriority (TEXT), default.empty_orders.o_orderstatus (TEXT), default.empty_orders.o_shippriority (INT4), default.empty_orders.o_totalprice (FLOAT8)} + => in schema: {(1) default.empty_orders.o_orderkey (INT4)} ======================================================= Block Id: eb_0000000000000_0000_000004 [ROOT] @@ -70,7 +96,7 @@ Block Id: eb_0000000000000_0000_000004 [ROOT] SORT(3) => Sort Keys: default.customer.c_custkey (INT4) (asc),default.empty_orders.o_orderkey (INT4) (asc) - SCAN(9) on eb_0000000000000_0000_000003 + SCAN(11) on eb_0000000000000_0000_000003 => out schema: {(2) default.customer.c_custkey (INT4), default.empty_orders.o_orderkey (INT4)} => in schema: {(2) default.customer.c_custkey (INT4), default.empty_orders.o_orderkey (INT4)} diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase2_1.1.Hash.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase2_1.1.Hash.plan index 88fce6bb9b..a35e200dcd 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase2_1.1.Hash.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase2_1.1.Hash.plan @@ -55,8 +55,7 @@ Block Id: eb_0000000000000_0000_000007 [ROOT] [Enforcers] 0: type=Broadcast, tables=default.t4 1: type=Broadcast, tables=default.t3 - 2: type=Broadcast, tables=default.t1 - 3: type=Broadcast, tables=default.t2 + 2: type=Broadcast, tables=default.t2 JOIN(12)(LEFT_OUTER) => Join Cond: default.t1.id (INT4) = default.t2.id (INT4) diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase2_1.1.Sort.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase2_1.1.Sort.plan index 88fce6bb9b..a35e200dcd 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase2_1.1.Sort.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testLeftOuterJoinPredicationCaseByCase2_1.1.Sort.plan @@ -55,8 +55,7 @@ Block Id: eb_0000000000000_0000_000007 [ROOT] [Enforcers] 0: type=Broadcast, tables=default.t4 1: type=Broadcast, tables=default.t3 - 2: type=Broadcast, tables=default.t1 - 3: type=Broadcast, tables=default.t2 + 2: type=Broadcast, tables=default.t2 JOIN(12)(LEFT_OUTER) => Join Cond: default.t1.id (INT4) = default.t2.id (INT4) diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase1.1.Hash.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase1.1.Hash.plan new file mode 100644 index 0000000000..e0bb3a08b6 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase1.1.Hash.plan @@ -0,0 +1,101 @@ +explain +------------------------------- +JOIN(8)(RIGHT_OUTER) + => Join Cond: default.t1.id (INT4) = default.t3.id (INT4) AND default.t2.id (INT4) = default.t3.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4) + => out schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4)} + => in schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4)} + SCAN(3) on default.jointable13 as t3 + => target list: default.t3.id (INT4) + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(2) default.t3.id (INT4), default.t3.name (TEXT)} + JOIN(7)(RIGHT_OUTER) + => Join Cond: default.t1.id (INT4) = default.t2.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4) + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + SCAN(1) on default.jointable12 as t2 + => target list: default.t2.id (INT4) + => out schema: {(1) default.t2.id (INT4)} + => in schema: {(2) default.t2.id (INT4), default.t2.name (TEXT)} + SCAN(0) on default.jointable11 as t1 + => target list: default.t1.id (INT4), default.t1.name (TEXT) + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} +explain +------------------------------- +------------------------------------------------------------------------------- +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000006) +------------------------------------------------------------------------------- +|-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000005 + |-eb_0000000000000_0000_000004 + |-eb_0000000000000_0000_000003 +------------------------------------------------------------------------------- +Order of Execution +------------------------------------------------------------------------------- +1: eb_0000000000000_0000_000003 +2: eb_0000000000000_0000_000004 +3: eb_0000000000000_0000_000005 +4: eb_0000000000000_0000_000006 +------------------------------------------------------------------------------- + +======================================================= +Block Id: eb_0000000000000_0000_000003 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.t1.id (INT4), default.t2.id (INT4), num=32) + +[Enforcers] + 0: type=Broadcast, tables=default.t1 + +JOIN(7)(RIGHT_OUTER) + => Join Cond: default.t1.id (INT4) = default.t2.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4) + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + SCAN(1) on default.jointable12 as t2 + => target list: default.t2.id (INT4) + => out schema: {(1) default.t2.id (INT4)} + => in schema: {(2) default.t2.id (INT4), default.t2.name (TEXT)} + SCAN(0) on default.jointable11 as t1 + => target list: default.t1.id (INT4), default.t1.name (TEXT) + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000004 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.t3.id (INT4), default.t3.id (INT4), num=32) + +SCAN(3) on default.jointable13 as t3 + => target list: default.t3.id (INT4) + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(2) default.t3.id (INT4), default.t3.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000005 [ROOT] +======================================================= + +[Incoming] +[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.t1.id (INT4), default.t2.id (INT4), num=32) +[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.t3.id (INT4), default.t3.id (INT4), num=32) + +JOIN(8)(RIGHT_OUTER) + => Join Cond: default.t1.id (INT4) = default.t3.id (INT4) AND default.t2.id (INT4) = default.t3.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4) + => out schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4)} + => in schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4)} + SCAN(13) on eb_0000000000000_0000_000004 + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(1) default.t3.id (INT4)} + SCAN(12) on eb_0000000000000_0000_000003 + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000006 [TERMINAL] +======================================================= diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase1.1.Hash_NoBroadcast.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase1.1.Hash_NoBroadcast.plan new file mode 100644 index 0000000000..735f3059e1 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase1.1.Hash_NoBroadcast.plan @@ -0,0 +1,128 @@ +explain +------------------------------- +JOIN(8)(RIGHT_OUTER) + => Join Cond: default.t1.id (INT4) = default.t3.id (INT4) AND default.t2.id (INT4) = default.t3.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4) + => out schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4)} + => in schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4)} + SCAN(3) on default.jointable13 as t3 + => target list: default.t3.id (INT4) + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(2) default.t3.id (INT4), default.t3.name (TEXT)} + JOIN(7)(RIGHT_OUTER) + => Join Cond: default.t1.id (INT4) = default.t2.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4) + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + SCAN(1) on default.jointable12 as t2 + => target list: default.t2.id (INT4) + => out schema: {(1) default.t2.id (INT4)} + => in schema: {(2) default.t2.id (INT4), default.t2.name (TEXT)} + SCAN(0) on default.jointable11 as t1 + => target list: default.t1.id (INT4), default.t1.name (TEXT) + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} +explain +------------------------------- +------------------------------------------------------------------------------- +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000006) +------------------------------------------------------------------------------- +|-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000005 + |-eb_0000000000000_0000_000004 + |-eb_0000000000000_0000_000003 + |-eb_0000000000000_0000_000002 + |-eb_0000000000000_0000_000001 +------------------------------------------------------------------------------- +Order of Execution +------------------------------------------------------------------------------- +1: eb_0000000000000_0000_000001 +2: eb_0000000000000_0000_000002 +3: eb_0000000000000_0000_000003 +4: eb_0000000000000_0000_000004 +5: eb_0000000000000_0000_000005 +6: eb_0000000000000_0000_000006 +------------------------------------------------------------------------------- + +======================================================= +Block Id: eb_0000000000000_0000_000001 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.t1.id (INT4), num=32) + +SCAN(0) on default.jointable11 as t1 + => target list: default.t1.id (INT4), default.t1.name (TEXT) + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000002 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.t2.id (INT4), num=32) + +SCAN(1) on default.jointable12 as t2 + => target list: default.t2.id (INT4) + => out schema: {(1) default.t2.id (INT4)} + => in schema: {(2) default.t2.id (INT4), default.t2.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000003 [INTERMEDIATE] +======================================================= + +[Incoming] +[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.t1.id (INT4), num=32) +[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.t2.id (INT4), num=32) + +[Outgoing] +[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.t1.id (INT4), default.t2.id (INT4), num=32) + +JOIN(7)(RIGHT_OUTER) + => Join Cond: default.t1.id (INT4) = default.t2.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4) + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + SCAN(11) on eb_0000000000000_0000_000002 + => out schema: {(1) default.t2.id (INT4)} + => in schema: {(1) default.t2.id (INT4)} + SCAN(10) on eb_0000000000000_0000_000001 + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000004 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.t3.id (INT4), default.t3.id (INT4), num=32) + +SCAN(3) on default.jointable13 as t3 + => target list: default.t3.id (INT4) + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(2) default.t3.id (INT4), default.t3.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000005 [ROOT] +======================================================= + +[Incoming] +[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.t1.id (INT4), default.t2.id (INT4), num=32) +[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.t3.id (INT4), default.t3.id (INT4), num=32) + +JOIN(8)(RIGHT_OUTER) + => Join Cond: default.t1.id (INT4) = default.t3.id (INT4) AND default.t2.id (INT4) = default.t3.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4) + => out schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4)} + => in schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4)} + SCAN(13) on eb_0000000000000_0000_000004 + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(1) default.t3.id (INT4)} + SCAN(12) on eb_0000000000000_0000_000003 + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000006 [TERMINAL] +======================================================= diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase1.1.Sort.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase1.1.Sort.plan new file mode 100644 index 0000000000..e0bb3a08b6 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase1.1.Sort.plan @@ -0,0 +1,101 @@ +explain +------------------------------- +JOIN(8)(RIGHT_OUTER) + => Join Cond: default.t1.id (INT4) = default.t3.id (INT4) AND default.t2.id (INT4) = default.t3.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4) + => out schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4)} + => in schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4)} + SCAN(3) on default.jointable13 as t3 + => target list: default.t3.id (INT4) + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(2) default.t3.id (INT4), default.t3.name (TEXT)} + JOIN(7)(RIGHT_OUTER) + => Join Cond: default.t1.id (INT4) = default.t2.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4) + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + SCAN(1) on default.jointable12 as t2 + => target list: default.t2.id (INT4) + => out schema: {(1) default.t2.id (INT4)} + => in schema: {(2) default.t2.id (INT4), default.t2.name (TEXT)} + SCAN(0) on default.jointable11 as t1 + => target list: default.t1.id (INT4), default.t1.name (TEXT) + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} +explain +------------------------------- +------------------------------------------------------------------------------- +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000006) +------------------------------------------------------------------------------- +|-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000005 + |-eb_0000000000000_0000_000004 + |-eb_0000000000000_0000_000003 +------------------------------------------------------------------------------- +Order of Execution +------------------------------------------------------------------------------- +1: eb_0000000000000_0000_000003 +2: eb_0000000000000_0000_000004 +3: eb_0000000000000_0000_000005 +4: eb_0000000000000_0000_000006 +------------------------------------------------------------------------------- + +======================================================= +Block Id: eb_0000000000000_0000_000003 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.t1.id (INT4), default.t2.id (INT4), num=32) + +[Enforcers] + 0: type=Broadcast, tables=default.t1 + +JOIN(7)(RIGHT_OUTER) + => Join Cond: default.t1.id (INT4) = default.t2.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4) + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + SCAN(1) on default.jointable12 as t2 + => target list: default.t2.id (INT4) + => out schema: {(1) default.t2.id (INT4)} + => in schema: {(2) default.t2.id (INT4), default.t2.name (TEXT)} + SCAN(0) on default.jointable11 as t1 + => target list: default.t1.id (INT4), default.t1.name (TEXT) + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000004 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.t3.id (INT4), default.t3.id (INT4), num=32) + +SCAN(3) on default.jointable13 as t3 + => target list: default.t3.id (INT4) + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(2) default.t3.id (INT4), default.t3.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000005 [ROOT] +======================================================= + +[Incoming] +[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.t1.id (INT4), default.t2.id (INT4), num=32) +[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.t3.id (INT4), default.t3.id (INT4), num=32) + +JOIN(8)(RIGHT_OUTER) + => Join Cond: default.t1.id (INT4) = default.t3.id (INT4) AND default.t2.id (INT4) = default.t3.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4) + => out schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4)} + => in schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4)} + SCAN(13) on eb_0000000000000_0000_000004 + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(1) default.t3.id (INT4)} + SCAN(12) on eb_0000000000000_0000_000003 + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000006 [TERMINAL] +======================================================= diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase1.1.Sort_NoBroadcast.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase1.1.Sort_NoBroadcast.plan new file mode 100644 index 0000000000..735f3059e1 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase1.1.Sort_NoBroadcast.plan @@ -0,0 +1,128 @@ +explain +------------------------------- +JOIN(8)(RIGHT_OUTER) + => Join Cond: default.t1.id (INT4) = default.t3.id (INT4) AND default.t2.id (INT4) = default.t3.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4) + => out schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4)} + => in schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4)} + SCAN(3) on default.jointable13 as t3 + => target list: default.t3.id (INT4) + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(2) default.t3.id (INT4), default.t3.name (TEXT)} + JOIN(7)(RIGHT_OUTER) + => Join Cond: default.t1.id (INT4) = default.t2.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4) + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + SCAN(1) on default.jointable12 as t2 + => target list: default.t2.id (INT4) + => out schema: {(1) default.t2.id (INT4)} + => in schema: {(2) default.t2.id (INT4), default.t2.name (TEXT)} + SCAN(0) on default.jointable11 as t1 + => target list: default.t1.id (INT4), default.t1.name (TEXT) + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} +explain +------------------------------- +------------------------------------------------------------------------------- +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000006) +------------------------------------------------------------------------------- +|-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000005 + |-eb_0000000000000_0000_000004 + |-eb_0000000000000_0000_000003 + |-eb_0000000000000_0000_000002 + |-eb_0000000000000_0000_000001 +------------------------------------------------------------------------------- +Order of Execution +------------------------------------------------------------------------------- +1: eb_0000000000000_0000_000001 +2: eb_0000000000000_0000_000002 +3: eb_0000000000000_0000_000003 +4: eb_0000000000000_0000_000004 +5: eb_0000000000000_0000_000005 +6: eb_0000000000000_0000_000006 +------------------------------------------------------------------------------- + +======================================================= +Block Id: eb_0000000000000_0000_000001 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.t1.id (INT4), num=32) + +SCAN(0) on default.jointable11 as t1 + => target list: default.t1.id (INT4), default.t1.name (TEXT) + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000002 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.t2.id (INT4), num=32) + +SCAN(1) on default.jointable12 as t2 + => target list: default.t2.id (INT4) + => out schema: {(1) default.t2.id (INT4)} + => in schema: {(2) default.t2.id (INT4), default.t2.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000003 [INTERMEDIATE] +======================================================= + +[Incoming] +[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.t1.id (INT4), num=32) +[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.t2.id (INT4), num=32) + +[Outgoing] +[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.t1.id (INT4), default.t2.id (INT4), num=32) + +JOIN(7)(RIGHT_OUTER) + => Join Cond: default.t1.id (INT4) = default.t2.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4) + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + SCAN(11) on eb_0000000000000_0000_000002 + => out schema: {(1) default.t2.id (INT4)} + => in schema: {(1) default.t2.id (INT4)} + SCAN(10) on eb_0000000000000_0000_000001 + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000004 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.t3.id (INT4), default.t3.id (INT4), num=32) + +SCAN(3) on default.jointable13 as t3 + => target list: default.t3.id (INT4) + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(2) default.t3.id (INT4), default.t3.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000005 [ROOT] +======================================================= + +[Incoming] +[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.t1.id (INT4), default.t2.id (INT4), num=32) +[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.t3.id (INT4), default.t3.id (INT4), num=32) + +JOIN(8)(RIGHT_OUTER) + => Join Cond: default.t1.id (INT4) = default.t3.id (INT4) AND default.t2.id (INT4) = default.t3.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4) + => out schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4)} + => in schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4)} + SCAN(13) on eb_0000000000000_0000_000004 + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(1) default.t3.id (INT4)} + SCAN(12) on eb_0000000000000_0000_000003 + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000006 [TERMINAL] +======================================================= diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase1.1.result b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase1.1.result new file mode 100644 index 0000000000..c5da2c4db6 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase1.1.result @@ -0,0 +1,4 @@ +id,name,id,id +------------------------------- +2,table11-2,2,2 +null,null,null,3 diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase2.1.Hash.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase2.1.Hash.plan new file mode 100644 index 0000000000..22fc8cec55 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase2.1.Hash.plan @@ -0,0 +1,76 @@ +explain +------------------------------- +JOIN(9)(RIGHT_OUTER) + => Join Cond: default.t3.id (INT4) = 2 AND default.t4.id (INT4) = default.t3.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4) + => out schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4)} + => in schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4)} + SCAN(3) on default.jointable13 as t3 + => filter: default.t3.id (INT4) > 1 + => target list: default.t3.id (INT4) + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(2) default.t3.id (INT4), default.t3.name (TEXT)} + JOIN(8)(INNER) + => Join Cond: default.t1.id (INT4) = default.t4.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t4.id (INT4) + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t4.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t4.id (INT4)} + SCAN(1) on default.jointable14 as t4 + => filter: default.t4.id (INT4) > 1 + => target list: default.t4.id (INT4) + => out schema: {(1) default.t4.id (INT4)} + => in schema: {(2) default.t4.id (INT4), default.t4.name (TEXT)} + SCAN(0) on default.jointable11 as t1 + => target list: default.t1.id (INT4), default.t1.name (TEXT) + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} +explain +------------------------------- +------------------------------------------------------------------------------- +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000006) +------------------------------------------------------------------------------- +|-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000005 +------------------------------------------------------------------------------- +Order of Execution +------------------------------------------------------------------------------- +1: eb_0000000000000_0000_000005 +2: eb_0000000000000_0000_000006 +------------------------------------------------------------------------------- + +======================================================= +Block Id: eb_0000000000000_0000_000005 [ROOT] +======================================================= + +[Enforcers] + 0: type=Broadcast, tables=default.t4 + 1: type=Broadcast, tables=default.t1 + +JOIN(9)(RIGHT_OUTER) + => Join Cond: default.t3.id (INT4) = 2 AND default.t4.id (INT4) = default.t3.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4) + => out schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4)} + => in schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4)} + SCAN(3) on default.jointable13 as t3 + => filter: default.t3.id (INT4) > 1 + => target list: default.t3.id (INT4) + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(2) default.t3.id (INT4), default.t3.name (TEXT)} + JOIN(8)(INNER) + => Join Cond: default.t1.id (INT4) = default.t4.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t4.id (INT4) + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t4.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t4.id (INT4)} + SCAN(1) on default.jointable14 as t4 + => filter: default.t4.id (INT4) > 1 + => target list: default.t4.id (INT4) + => out schema: {(1) default.t4.id (INT4)} + => in schema: {(2) default.t4.id (INT4), default.t4.name (TEXT)} + SCAN(0) on default.jointable11 as t1 + => target list: default.t1.id (INT4), default.t1.name (TEXT) + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000006 [TERMINAL] +======================================================= diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase2.1.Hash_NoBroadcast.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase2.1.Hash_NoBroadcast.plan new file mode 100644 index 0000000000..f5486f993f --- /dev/null +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase2.1.Hash_NoBroadcast.plan @@ -0,0 +1,132 @@ +explain +------------------------------- +JOIN(9)(RIGHT_OUTER) + => Join Cond: default.t3.id (INT4) = 2 AND default.t4.id (INT4) = default.t3.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4) + => out schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4)} + => in schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4)} + SCAN(3) on default.jointable13 as t3 + => filter: default.t3.id (INT4) > 1 + => target list: default.t3.id (INT4) + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(2) default.t3.id (INT4), default.t3.name (TEXT)} + JOIN(8)(INNER) + => Join Cond: default.t1.id (INT4) = default.t4.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t4.id (INT4) + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t4.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t4.id (INT4)} + SCAN(1) on default.jointable14 as t4 + => filter: default.t4.id (INT4) > 1 + => target list: default.t4.id (INT4) + => out schema: {(1) default.t4.id (INT4)} + => in schema: {(2) default.t4.id (INT4), default.t4.name (TEXT)} + SCAN(0) on default.jointable11 as t1 + => target list: default.t1.id (INT4), default.t1.name (TEXT) + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} +explain +------------------------------- +------------------------------------------------------------------------------- +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000006) +------------------------------------------------------------------------------- +|-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000005 + |-eb_0000000000000_0000_000004 + |-eb_0000000000000_0000_000003 + |-eb_0000000000000_0000_000002 + |-eb_0000000000000_0000_000001 +------------------------------------------------------------------------------- +Order of Execution +------------------------------------------------------------------------------- +1: eb_0000000000000_0000_000001 +2: eb_0000000000000_0000_000002 +3: eb_0000000000000_0000_000003 +4: eb_0000000000000_0000_000004 +5: eb_0000000000000_0000_000005 +6: eb_0000000000000_0000_000006 +------------------------------------------------------------------------------- + +======================================================= +Block Id: eb_0000000000000_0000_000001 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.t1.id (INT4), num=32) + +SCAN(0) on default.jointable11 as t1 + => target list: default.t1.id (INT4), default.t1.name (TEXT) + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000002 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.t4.id (INT4), num=32) + +SCAN(1) on default.jointable14 as t4 + => filter: default.t4.id (INT4) > 1 + => target list: default.t4.id (INT4) + => out schema: {(1) default.t4.id (INT4)} + => in schema: {(2) default.t4.id (INT4), default.t4.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000003 [INTERMEDIATE] +======================================================= + +[Incoming] +[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.t1.id (INT4), num=32) +[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.t4.id (INT4), num=32) + +[Outgoing] +[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.t4.id (INT4), num=32) + +JOIN(8)(INNER) + => Join Cond: default.t1.id (INT4) = default.t4.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t4.id (INT4) + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t4.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t4.id (INT4)} + SCAN(12) on eb_0000000000000_0000_000002 + => out schema: {(1) default.t4.id (INT4)} + => in schema: {(1) default.t4.id (INT4)} + SCAN(11) on eb_0000000000000_0000_000001 + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000004 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.t3.id (INT4), num=32) + +SCAN(3) on default.jointable13 as t3 + => filter: default.t3.id (INT4) > 1 + => target list: default.t3.id (INT4) + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(2) default.t3.id (INT4), default.t3.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000005 [ROOT] +======================================================= + +[Incoming] +[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.t4.id (INT4), num=32) +[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.t3.id (INT4), num=32) + +JOIN(9)(RIGHT_OUTER) + => Join Cond: default.t3.id (INT4) = 2 AND default.t4.id (INT4) = default.t3.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4) + => out schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4)} + => in schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4)} + SCAN(14) on eb_0000000000000_0000_000004 + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(1) default.t3.id (INT4)} + SCAN(13) on eb_0000000000000_0000_000003 + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t4.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t4.id (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000006 [TERMINAL] +======================================================= diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase2.1.Sort.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase2.1.Sort.plan new file mode 100644 index 0000000000..22fc8cec55 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase2.1.Sort.plan @@ -0,0 +1,76 @@ +explain +------------------------------- +JOIN(9)(RIGHT_OUTER) + => Join Cond: default.t3.id (INT4) = 2 AND default.t4.id (INT4) = default.t3.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4) + => out schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4)} + => in schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4)} + SCAN(3) on default.jointable13 as t3 + => filter: default.t3.id (INT4) > 1 + => target list: default.t3.id (INT4) + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(2) default.t3.id (INT4), default.t3.name (TEXT)} + JOIN(8)(INNER) + => Join Cond: default.t1.id (INT4) = default.t4.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t4.id (INT4) + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t4.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t4.id (INT4)} + SCAN(1) on default.jointable14 as t4 + => filter: default.t4.id (INT4) > 1 + => target list: default.t4.id (INT4) + => out schema: {(1) default.t4.id (INT4)} + => in schema: {(2) default.t4.id (INT4), default.t4.name (TEXT)} + SCAN(0) on default.jointable11 as t1 + => target list: default.t1.id (INT4), default.t1.name (TEXT) + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} +explain +------------------------------- +------------------------------------------------------------------------------- +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000006) +------------------------------------------------------------------------------- +|-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000005 +------------------------------------------------------------------------------- +Order of Execution +------------------------------------------------------------------------------- +1: eb_0000000000000_0000_000005 +2: eb_0000000000000_0000_000006 +------------------------------------------------------------------------------- + +======================================================= +Block Id: eb_0000000000000_0000_000005 [ROOT] +======================================================= + +[Enforcers] + 0: type=Broadcast, tables=default.t4 + 1: type=Broadcast, tables=default.t1 + +JOIN(9)(RIGHT_OUTER) + => Join Cond: default.t3.id (INT4) = 2 AND default.t4.id (INT4) = default.t3.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4) + => out schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4)} + => in schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4)} + SCAN(3) on default.jointable13 as t3 + => filter: default.t3.id (INT4) > 1 + => target list: default.t3.id (INT4) + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(2) default.t3.id (INT4), default.t3.name (TEXT)} + JOIN(8)(INNER) + => Join Cond: default.t1.id (INT4) = default.t4.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t4.id (INT4) + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t4.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t4.id (INT4)} + SCAN(1) on default.jointable14 as t4 + => filter: default.t4.id (INT4) > 1 + => target list: default.t4.id (INT4) + => out schema: {(1) default.t4.id (INT4)} + => in schema: {(2) default.t4.id (INT4), default.t4.name (TEXT)} + SCAN(0) on default.jointable11 as t1 + => target list: default.t1.id (INT4), default.t1.name (TEXT) + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000006 [TERMINAL] +======================================================= diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase2.1.Sort_NoBroadcast.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase2.1.Sort_NoBroadcast.plan new file mode 100644 index 0000000000..f5486f993f --- /dev/null +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase2.1.Sort_NoBroadcast.plan @@ -0,0 +1,132 @@ +explain +------------------------------- +JOIN(9)(RIGHT_OUTER) + => Join Cond: default.t3.id (INT4) = 2 AND default.t4.id (INT4) = default.t3.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4) + => out schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4)} + => in schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4)} + SCAN(3) on default.jointable13 as t3 + => filter: default.t3.id (INT4) > 1 + => target list: default.t3.id (INT4) + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(2) default.t3.id (INT4), default.t3.name (TEXT)} + JOIN(8)(INNER) + => Join Cond: default.t1.id (INT4) = default.t4.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t4.id (INT4) + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t4.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t4.id (INT4)} + SCAN(1) on default.jointable14 as t4 + => filter: default.t4.id (INT4) > 1 + => target list: default.t4.id (INT4) + => out schema: {(1) default.t4.id (INT4)} + => in schema: {(2) default.t4.id (INT4), default.t4.name (TEXT)} + SCAN(0) on default.jointable11 as t1 + => target list: default.t1.id (INT4), default.t1.name (TEXT) + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} +explain +------------------------------- +------------------------------------------------------------------------------- +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000006) +------------------------------------------------------------------------------- +|-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000005 + |-eb_0000000000000_0000_000004 + |-eb_0000000000000_0000_000003 + |-eb_0000000000000_0000_000002 + |-eb_0000000000000_0000_000001 +------------------------------------------------------------------------------- +Order of Execution +------------------------------------------------------------------------------- +1: eb_0000000000000_0000_000001 +2: eb_0000000000000_0000_000002 +3: eb_0000000000000_0000_000003 +4: eb_0000000000000_0000_000004 +5: eb_0000000000000_0000_000005 +6: eb_0000000000000_0000_000006 +------------------------------------------------------------------------------- + +======================================================= +Block Id: eb_0000000000000_0000_000001 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.t1.id (INT4), num=32) + +SCAN(0) on default.jointable11 as t1 + => target list: default.t1.id (INT4), default.t1.name (TEXT) + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000002 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.t4.id (INT4), num=32) + +SCAN(1) on default.jointable14 as t4 + => filter: default.t4.id (INT4) > 1 + => target list: default.t4.id (INT4) + => out schema: {(1) default.t4.id (INT4)} + => in schema: {(2) default.t4.id (INT4), default.t4.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000003 [INTERMEDIATE] +======================================================= + +[Incoming] +[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.t1.id (INT4), num=32) +[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.t4.id (INT4), num=32) + +[Outgoing] +[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.t4.id (INT4), num=32) + +JOIN(8)(INNER) + => Join Cond: default.t1.id (INT4) = default.t4.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t4.id (INT4) + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t4.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t4.id (INT4)} + SCAN(12) on eb_0000000000000_0000_000002 + => out schema: {(1) default.t4.id (INT4)} + => in schema: {(1) default.t4.id (INT4)} + SCAN(11) on eb_0000000000000_0000_000001 + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000004 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.t3.id (INT4), num=32) + +SCAN(3) on default.jointable13 as t3 + => filter: default.t3.id (INT4) > 1 + => target list: default.t3.id (INT4) + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(2) default.t3.id (INT4), default.t3.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000005 [ROOT] +======================================================= + +[Incoming] +[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.t4.id (INT4), num=32) +[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.t3.id (INT4), num=32) + +JOIN(9)(RIGHT_OUTER) + => Join Cond: default.t3.id (INT4) = 2 AND default.t4.id (INT4) = default.t3.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4) + => out schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4)} + => in schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t3.id (INT4), default.t4.id (INT4)} + SCAN(14) on eb_0000000000000_0000_000004 + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(1) default.t3.id (INT4)} + SCAN(13) on eb_0000000000000_0000_000003 + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t4.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t4.id (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000006 [TERMINAL] +======================================================= diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase2.1.result b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase2.1.result new file mode 100644 index 0000000000..4fc079a9f5 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase2.1.result @@ -0,0 +1,4 @@ +id,name,id,id +------------------------------- +2,table11-2,2,2 +null,null,3,null diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase3.1.Hash.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase3.1.Hash.plan new file mode 100644 index 0000000000..1c2fd7a370 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase3.1.Hash.plan @@ -0,0 +1,101 @@ +explain +------------------------------- +JOIN(8)(RIGHT_OUTER) + => Join Cond: default.t1.id (INT4) = default.t3.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4) + => out schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4)} + => in schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4)} + SCAN(3) on default.jointable13 as t3 + => target list: default.t3.id (INT4) + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(2) default.t3.id (INT4), default.t3.name (TEXT)} + JOIN(7)(RIGHT_OUTER) + => Join Cond: default.t1.id (INT4) = default.t2.id (INT4) AND concat(default.t1.name (TEXT),CAST (default.t2.id (INT4) AS TEXT)) = table11-11 OR concat(default.t1.name (TEXT),CAST (default.t2.id (INT4) AS TEXT)) = table11-33 + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4) + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + SCAN(1) on default.jointable12 as t2 + => target list: default.t2.id (INT4) + => out schema: {(1) default.t2.id (INT4)} + => in schema: {(2) default.t2.id (INT4), default.t2.name (TEXT)} + SCAN(0) on default.jointable11 as t1 + => target list: default.t1.id (INT4), default.t1.name (TEXT) + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} +explain +------------------------------- +------------------------------------------------------------------------------- +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000006) +------------------------------------------------------------------------------- +|-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000005 + |-eb_0000000000000_0000_000004 + |-eb_0000000000000_0000_000003 +------------------------------------------------------------------------------- +Order of Execution +------------------------------------------------------------------------------- +1: eb_0000000000000_0000_000003 +2: eb_0000000000000_0000_000004 +3: eb_0000000000000_0000_000005 +4: eb_0000000000000_0000_000006 +------------------------------------------------------------------------------- + +======================================================= +Block Id: eb_0000000000000_0000_000003 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.t1.id (INT4), num=32) + +[Enforcers] + 0: type=Broadcast, tables=default.t1 + +JOIN(7)(RIGHT_OUTER) + => Join Cond: default.t1.id (INT4) = default.t2.id (INT4) AND concat(default.t1.name (TEXT),CAST (default.t2.id (INT4) AS TEXT)) = table11-11 OR concat(default.t1.name (TEXT),CAST (default.t2.id (INT4) AS TEXT)) = table11-33 + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4) + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + SCAN(1) on default.jointable12 as t2 + => target list: default.t2.id (INT4) + => out schema: {(1) default.t2.id (INT4)} + => in schema: {(2) default.t2.id (INT4), default.t2.name (TEXT)} + SCAN(0) on default.jointable11 as t1 + => target list: default.t1.id (INT4), default.t1.name (TEXT) + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000004 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.t3.id (INT4), num=32) + +SCAN(3) on default.jointable13 as t3 + => target list: default.t3.id (INT4) + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(2) default.t3.id (INT4), default.t3.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000005 [ROOT] +======================================================= + +[Incoming] +[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.t1.id (INT4), num=32) +[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.t3.id (INT4), num=32) + +JOIN(8)(RIGHT_OUTER) + => Join Cond: default.t1.id (INT4) = default.t3.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4) + => out schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4)} + => in schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4)} + SCAN(13) on eb_0000000000000_0000_000004 + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(1) default.t3.id (INT4)} + SCAN(12) on eb_0000000000000_0000_000003 + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000006 [TERMINAL] +======================================================= diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase3.1.Hash_NoBroadcast.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase3.1.Hash_NoBroadcast.plan new file mode 100644 index 0000000000..5a589ff07d --- /dev/null +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase3.1.Hash_NoBroadcast.plan @@ -0,0 +1,128 @@ +explain +------------------------------- +JOIN(8)(RIGHT_OUTER) + => Join Cond: default.t1.id (INT4) = default.t3.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4) + => out schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4)} + => in schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4)} + SCAN(3) on default.jointable13 as t3 + => target list: default.t3.id (INT4) + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(2) default.t3.id (INT4), default.t3.name (TEXT)} + JOIN(7)(RIGHT_OUTER) + => Join Cond: default.t1.id (INT4) = default.t2.id (INT4) AND concat(default.t1.name (TEXT),CAST (default.t2.id (INT4) AS TEXT)) = table11-11 OR concat(default.t1.name (TEXT),CAST (default.t2.id (INT4) AS TEXT)) = table11-33 + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4) + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + SCAN(1) on default.jointable12 as t2 + => target list: default.t2.id (INT4) + => out schema: {(1) default.t2.id (INT4)} + => in schema: {(2) default.t2.id (INT4), default.t2.name (TEXT)} + SCAN(0) on default.jointable11 as t1 + => target list: default.t1.id (INT4), default.t1.name (TEXT) + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} +explain +------------------------------- +------------------------------------------------------------------------------- +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000006) +------------------------------------------------------------------------------- +|-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000005 + |-eb_0000000000000_0000_000004 + |-eb_0000000000000_0000_000003 + |-eb_0000000000000_0000_000002 + |-eb_0000000000000_0000_000001 +------------------------------------------------------------------------------- +Order of Execution +------------------------------------------------------------------------------- +1: eb_0000000000000_0000_000001 +2: eb_0000000000000_0000_000002 +3: eb_0000000000000_0000_000003 +4: eb_0000000000000_0000_000004 +5: eb_0000000000000_0000_000005 +6: eb_0000000000000_0000_000006 +------------------------------------------------------------------------------- + +======================================================= +Block Id: eb_0000000000000_0000_000001 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.t1.id (INT4), num=32) + +SCAN(0) on default.jointable11 as t1 + => target list: default.t1.id (INT4), default.t1.name (TEXT) + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000002 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.t2.id (INT4), num=32) + +SCAN(1) on default.jointable12 as t2 + => target list: default.t2.id (INT4) + => out schema: {(1) default.t2.id (INT4)} + => in schema: {(2) default.t2.id (INT4), default.t2.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000003 [INTERMEDIATE] +======================================================= + +[Incoming] +[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.t1.id (INT4), num=32) +[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.t2.id (INT4), num=32) + +[Outgoing] +[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.t1.id (INT4), num=32) + +JOIN(7)(RIGHT_OUTER) + => Join Cond: default.t1.id (INT4) = default.t2.id (INT4) AND concat(default.t1.name (TEXT),CAST (default.t2.id (INT4) AS TEXT)) = table11-11 OR concat(default.t1.name (TEXT),CAST (default.t2.id (INT4) AS TEXT)) = table11-33 + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4) + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + SCAN(11) on eb_0000000000000_0000_000002 + => out schema: {(1) default.t2.id (INT4)} + => in schema: {(1) default.t2.id (INT4)} + SCAN(10) on eb_0000000000000_0000_000001 + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000004 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.t3.id (INT4), num=32) + +SCAN(3) on default.jointable13 as t3 + => target list: default.t3.id (INT4) + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(2) default.t3.id (INT4), default.t3.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000005 [ROOT] +======================================================= + +[Incoming] +[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.t1.id (INT4), num=32) +[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.t3.id (INT4), num=32) + +JOIN(8)(RIGHT_OUTER) + => Join Cond: default.t1.id (INT4) = default.t3.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4) + => out schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4)} + => in schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4)} + SCAN(13) on eb_0000000000000_0000_000004 + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(1) default.t3.id (INT4)} + SCAN(12) on eb_0000000000000_0000_000003 + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000006 [TERMINAL] +======================================================= diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase3.1.Sort.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase3.1.Sort.plan new file mode 100644 index 0000000000..1c2fd7a370 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase3.1.Sort.plan @@ -0,0 +1,101 @@ +explain +------------------------------- +JOIN(8)(RIGHT_OUTER) + => Join Cond: default.t1.id (INT4) = default.t3.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4) + => out schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4)} + => in schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4)} + SCAN(3) on default.jointable13 as t3 + => target list: default.t3.id (INT4) + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(2) default.t3.id (INT4), default.t3.name (TEXT)} + JOIN(7)(RIGHT_OUTER) + => Join Cond: default.t1.id (INT4) = default.t2.id (INT4) AND concat(default.t1.name (TEXT),CAST (default.t2.id (INT4) AS TEXT)) = table11-11 OR concat(default.t1.name (TEXT),CAST (default.t2.id (INT4) AS TEXT)) = table11-33 + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4) + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + SCAN(1) on default.jointable12 as t2 + => target list: default.t2.id (INT4) + => out schema: {(1) default.t2.id (INT4)} + => in schema: {(2) default.t2.id (INT4), default.t2.name (TEXT)} + SCAN(0) on default.jointable11 as t1 + => target list: default.t1.id (INT4), default.t1.name (TEXT) + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} +explain +------------------------------- +------------------------------------------------------------------------------- +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000006) +------------------------------------------------------------------------------- +|-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000005 + |-eb_0000000000000_0000_000004 + |-eb_0000000000000_0000_000003 +------------------------------------------------------------------------------- +Order of Execution +------------------------------------------------------------------------------- +1: eb_0000000000000_0000_000003 +2: eb_0000000000000_0000_000004 +3: eb_0000000000000_0000_000005 +4: eb_0000000000000_0000_000006 +------------------------------------------------------------------------------- + +======================================================= +Block Id: eb_0000000000000_0000_000003 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.t1.id (INT4), num=32) + +[Enforcers] + 0: type=Broadcast, tables=default.t1 + +JOIN(7)(RIGHT_OUTER) + => Join Cond: default.t1.id (INT4) = default.t2.id (INT4) AND concat(default.t1.name (TEXT),CAST (default.t2.id (INT4) AS TEXT)) = table11-11 OR concat(default.t1.name (TEXT),CAST (default.t2.id (INT4) AS TEXT)) = table11-33 + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4) + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + SCAN(1) on default.jointable12 as t2 + => target list: default.t2.id (INT4) + => out schema: {(1) default.t2.id (INT4)} + => in schema: {(2) default.t2.id (INT4), default.t2.name (TEXT)} + SCAN(0) on default.jointable11 as t1 + => target list: default.t1.id (INT4), default.t1.name (TEXT) + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000004 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.t3.id (INT4), num=32) + +SCAN(3) on default.jointable13 as t3 + => target list: default.t3.id (INT4) + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(2) default.t3.id (INT4), default.t3.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000005 [ROOT] +======================================================= + +[Incoming] +[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.t1.id (INT4), num=32) +[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.t3.id (INT4), num=32) + +JOIN(8)(RIGHT_OUTER) + => Join Cond: default.t1.id (INT4) = default.t3.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4) + => out schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4)} + => in schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4)} + SCAN(13) on eb_0000000000000_0000_000004 + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(1) default.t3.id (INT4)} + SCAN(12) on eb_0000000000000_0000_000003 + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000006 [TERMINAL] +======================================================= diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase3.1.Sort_NoBroadcast.plan b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase3.1.Sort_NoBroadcast.plan new file mode 100644 index 0000000000..5a589ff07d --- /dev/null +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase3.1.Sort_NoBroadcast.plan @@ -0,0 +1,128 @@ +explain +------------------------------- +JOIN(8)(RIGHT_OUTER) + => Join Cond: default.t1.id (INT4) = default.t3.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4) + => out schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4)} + => in schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4)} + SCAN(3) on default.jointable13 as t3 + => target list: default.t3.id (INT4) + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(2) default.t3.id (INT4), default.t3.name (TEXT)} + JOIN(7)(RIGHT_OUTER) + => Join Cond: default.t1.id (INT4) = default.t2.id (INT4) AND concat(default.t1.name (TEXT),CAST (default.t2.id (INT4) AS TEXT)) = table11-11 OR concat(default.t1.name (TEXT),CAST (default.t2.id (INT4) AS TEXT)) = table11-33 + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4) + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + SCAN(1) on default.jointable12 as t2 + => target list: default.t2.id (INT4) + => out schema: {(1) default.t2.id (INT4)} + => in schema: {(2) default.t2.id (INT4), default.t2.name (TEXT)} + SCAN(0) on default.jointable11 as t1 + => target list: default.t1.id (INT4), default.t1.name (TEXT) + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} +explain +------------------------------- +------------------------------------------------------------------------------- +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000006) +------------------------------------------------------------------------------- +|-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000005 + |-eb_0000000000000_0000_000004 + |-eb_0000000000000_0000_000003 + |-eb_0000000000000_0000_000002 + |-eb_0000000000000_0000_000001 +------------------------------------------------------------------------------- +Order of Execution +------------------------------------------------------------------------------- +1: eb_0000000000000_0000_000001 +2: eb_0000000000000_0000_000002 +3: eb_0000000000000_0000_000003 +4: eb_0000000000000_0000_000004 +5: eb_0000000000000_0000_000005 +6: eb_0000000000000_0000_000006 +------------------------------------------------------------------------------- + +======================================================= +Block Id: eb_0000000000000_0000_000001 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.t1.id (INT4), num=32) + +SCAN(0) on default.jointable11 as t1 + => target list: default.t1.id (INT4), default.t1.name (TEXT) + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000002 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.t2.id (INT4), num=32) + +SCAN(1) on default.jointable12 as t2 + => target list: default.t2.id (INT4) + => out schema: {(1) default.t2.id (INT4)} + => in schema: {(2) default.t2.id (INT4), default.t2.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000003 [INTERMEDIATE] +======================================================= + +[Incoming] +[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.t1.id (INT4), num=32) +[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.t2.id (INT4), num=32) + +[Outgoing] +[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.t1.id (INT4), num=32) + +JOIN(7)(RIGHT_OUTER) + => Join Cond: default.t1.id (INT4) = default.t2.id (INT4) AND concat(default.t1.name (TEXT),CAST (default.t2.id (INT4) AS TEXT)) = table11-11 OR concat(default.t1.name (TEXT),CAST (default.t2.id (INT4) AS TEXT)) = table11-33 + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4) + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + SCAN(11) on eb_0000000000000_0000_000002 + => out schema: {(1) default.t2.id (INT4)} + => in schema: {(1) default.t2.id (INT4)} + SCAN(10) on eb_0000000000000_0000_000001 + => out schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + => in schema: {(2) default.t1.id (INT4), default.t1.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000004 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.t3.id (INT4), num=32) + +SCAN(3) on default.jointable13 as t3 + => target list: default.t3.id (INT4) + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(2) default.t3.id (INT4), default.t3.name (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000005 [ROOT] +======================================================= + +[Incoming] +[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.t1.id (INT4), num=32) +[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.t3.id (INT4), num=32) + +JOIN(8)(RIGHT_OUTER) + => Join Cond: default.t1.id (INT4) = default.t3.id (INT4) + => target list: default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4) + => out schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4)} + => in schema: {(4) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4), default.t3.id (INT4)} + SCAN(13) on eb_0000000000000_0000_000004 + => out schema: {(1) default.t3.id (INT4)} + => in schema: {(1) default.t3.id (INT4)} + SCAN(12) on eb_0000000000000_0000_000003 + => out schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + => in schema: {(3) default.t1.id (INT4), default.t1.name (TEXT), default.t2.id (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000006 [TERMINAL] +======================================================= diff --git a/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase3.1.result b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase3.1.result new file mode 100644 index 0000000000..c5da2c4db6 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestOuterJoinQuery/testRightOuterJoinPredicationCaseByCase3.1.result @@ -0,0 +1,4 @@ +id,name,id,id +------------------------------- +2,table11-2,2,2 +null,null,null,3 diff --git a/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithConstantExpr3.Hash.plan b/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithConstantExpr3.Hash.plan index 6bde9f8d51..0fe36c0009 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithConstantExpr3.Hash.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithConstantExpr3.Hash.plan @@ -57,9 +57,6 @@ Block Id: eb_0000000000000_0000_000002 [LEAF] [Outgoing] [q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.customer.c_custkey (INT4), num=32) -[Enforcers] - 0: type=Broadcast, tables=default.customer - GROUP_BY(11)(c_custkey) => exprs: (min(default.customer.c_name (TEXT))) => target list: default.customer.c_custkey (INT4), ?min_2 (TEXT) diff --git a/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithConstantExpr3.Sort.plan b/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithConstantExpr3.Sort.plan index 6bde9f8d51..0fe36c0009 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithConstantExpr3.Sort.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithConstantExpr3.Sort.plan @@ -57,9 +57,6 @@ Block Id: eb_0000000000000_0000_000002 [LEAF] [Outgoing] [q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.customer.c_custkey (INT4), num=32) -[Enforcers] - 0: type=Broadcast, tables=default.customer - GROUP_BY(11)(c_custkey) => exprs: (min(default.customer.c_name (TEXT))) => target list: default.customer.c_custkey (INT4), ?min_2 (TEXT) diff --git a/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithEmptySubquery2.1.Hash.plan b/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithEmptySubquery2.1.Hash.plan index 63e5904cc3..45b95c2191 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithEmptySubquery2.1.Hash.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithEmptySubquery2.1.Hash.plan @@ -47,8 +47,7 @@ Block Id: eb_0000000000000_0000_000005 [ROOT] [Enforcers] 0: type=Broadcast, tables=default.jointable12 - 1: type=Broadcast, tables=default.lineitem - 2: type=Broadcast, tables=default.b + 1: type=Broadcast, tables=default.b JOIN(10)(LEFT_OUTER) => Join Cond: default.a.id (INT4) = default.b.id (INT4) diff --git a/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithEmptySubquery2.1.Sort.plan b/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithEmptySubquery2.1.Sort.plan index 63e5904cc3..45b95c2191 100644 --- a/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithEmptySubquery2.1.Sort.plan +++ b/tajo-core/src/test/resources/results/TestOuterJoinWithSubQuery/testLeftOuterJoinWithEmptySubquery2.1.Sort.plan @@ -47,8 +47,7 @@ Block Id: eb_0000000000000_0000_000005 [ROOT] [Enforcers] 0: type=Broadcast, tables=default.jointable12 - 1: type=Broadcast, tables=default.lineitem - 2: type=Broadcast, tables=default.b + 1: type=Broadcast, tables=default.b JOIN(10)(LEFT_OUTER) => Join Cond: default.a.id (INT4) = default.b.id (INT4) From 260ae7f3717d1b5e75e633f95d14296b7ae4ff9c Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 25 May 2015 19:37:35 +0900 Subject: [PATCH 32/37] Remove commented out codes --- .../BroadcastJoinMarkCandidateVisitor.java | 72 ---- .../planner/BroadcastJoinPlanVisitor.java | 60 --- .../engine/planner/global/GlobalPlanner.java | 241 +----------- .../BaseGlobalPlanRewriteRuleProvider.java | 2 - .../rewriter/rules/BroadcastJoinRule.java | 6 +- .../global/rewriter/rules/EbMergeRule.java | 110 ------ .../rewriter/rules/GlobalPlanRewriteUtil.java | 45 +++ .../rewriter/rules/UnionReduceRule.java | 105 ------ .../physical/HashFullOuterJoinExec.java | 24 -- .../engine/planner/physical/HashJoinExec.java | 21 -- .../physical/HashLeftOuterJoinExec.java | 41 --- .../tajo/querymaster/Repartitioner.java | 35 +- .../org/apache/tajo/querymaster/Stage.java | 7 +- .../org/apache/tajo/benchmark/TestTPCH.java | 25 +- .../query/TestInnerJoinWithSubQuery.java | 7 + .../tajo/engine/query/TestSortQuery.java | 7 + .../tajo/engine/query/TestUnionQuery.java | 13 + .../apache/tajo/master/TestGlobalPlanner.java | 347 ------------------ .../testThetaJoinKeyPairs.sql | 20 + .../testSubQuerySortAfterGroupMultiBlocks.sql | 5 + .../resources/queries/TestTPCH/testTPCHQ5.sql | 24 ++ .../testComplexUnion1.sql} | 2 - .../testComplexUnion2.sql} | 0 .../testThetaJoinKeyPairs.Hash.plan | 142 +++++++ ...estThetaJoinKeyPairs.Hash_NoBroadcast.plan | 196 ++++++++++ .../testThetaJoinKeyPairs.Sort.plan | 142 +++++++ ...estThetaJoinKeyPairs.Sort_NoBroadcast.plan | 196 ++++++++++ .../testThetaJoinKeyPairs.result | 27 ++ ...testSubQuerySortAfterGroupMultiBlocks.plan | 112 ++++++ ...stSubQuerySortAfterGroupMultiBlocks.result | 5 + .../results/TestTPCH/testQ1OrderBy.plan | 88 +++++ .../results/TestTPCH/testQ1OrderBy.result | 2 +- .../results/TestTPCH/testQ2FourJoins.plan | 228 ++++++++++++ .../results/TestTPCH/testQ2FourJoins.result | 2 +- .../results/TestTPCH/testTPCH14Expr.plan | 119 ++++++ .../results/TestTPCH/testTPCH14Expr.result | 2 +- .../results/TestTPCH/testTPCHQ5.plan | 332 +++++++++++++++++ .../results/TestTPCH/testTPCHQ5.result | 2 + .../TestUnionQuery/testComplexUnion1.plan | 80 ++++ .../TestUnionQuery/testComplexUnion1.result | 6 + .../TestUnionQuery/testComplexUnion2.plan | 116 ++++++ .../TestUnionQuery/testComplexUnion2.result | 7 + .../expr/AggregationFunctionCallEval.java | 3 - .../apache/tajo/plan/logical/JoinNode.java | 22 +- .../plan/serder/LogicalNodeSerializer.java | 8 - 45 files changed, 1944 insertions(+), 1112 deletions(-) delete mode 100644 tajo-core/src/main/java/org/apache/tajo/engine/planner/BroadcastJoinMarkCandidateVisitor.java delete mode 100644 tajo-core/src/main/java/org/apache/tajo/engine/planner/BroadcastJoinPlanVisitor.java delete mode 100644 tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/EbMergeRule.java delete mode 100644 tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/UnionReduceRule.java delete mode 100644 tajo-core/src/test/java/org/apache/tajo/master/TestGlobalPlanner.java create mode 100644 tajo-core/src/test/resources/queries/TestInnerJoinWithSubQuery/testThetaJoinKeyPairs.sql create mode 100644 tajo-core/src/test/resources/queries/TestSortQuery/testSubQuerySortAfterGroupMultiBlocks.sql create mode 100644 tajo-core/src/test/resources/queries/TestTPCH/testTPCHQ5.sql rename tajo-core/src/test/resources/queries/{default/complex_union_1.sql => TestUnionQuery/testComplexUnion1.sql} (91%) rename tajo-core/src/test/resources/queries/{default/complex_union_2.sql => TestUnionQuery/testComplexUnion2.sql} (100%) create mode 100644 tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testThetaJoinKeyPairs.Hash.plan create mode 100644 tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testThetaJoinKeyPairs.Hash_NoBroadcast.plan create mode 100644 tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testThetaJoinKeyPairs.Sort.plan create mode 100644 tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testThetaJoinKeyPairs.Sort_NoBroadcast.plan create mode 100644 tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testThetaJoinKeyPairs.result create mode 100644 tajo-core/src/test/resources/results/TestSortQuery/testSubQuerySortAfterGroupMultiBlocks.plan create mode 100644 tajo-core/src/test/resources/results/TestSortQuery/testSubQuerySortAfterGroupMultiBlocks.result create mode 100644 tajo-core/src/test/resources/results/TestTPCH/testQ1OrderBy.plan create mode 100644 tajo-core/src/test/resources/results/TestTPCH/testQ2FourJoins.plan create mode 100644 tajo-core/src/test/resources/results/TestTPCH/testTPCH14Expr.plan create mode 100644 tajo-core/src/test/resources/results/TestTPCH/testTPCHQ5.plan create mode 100644 tajo-core/src/test/resources/results/TestTPCH/testTPCHQ5.result create mode 100644 tajo-core/src/test/resources/results/TestUnionQuery/testComplexUnion1.plan create mode 100644 tajo-core/src/test/resources/results/TestUnionQuery/testComplexUnion1.result create mode 100644 tajo-core/src/test/resources/results/TestUnionQuery/testComplexUnion2.plan create mode 100644 tajo-core/src/test/resources/results/TestUnionQuery/testComplexUnion2.result diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/BroadcastJoinMarkCandidateVisitor.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/BroadcastJoinMarkCandidateVisitor.java deleted file mode 100644 index bdc6bae3d7..0000000000 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/BroadcastJoinMarkCandidateVisitor.java +++ /dev/null @@ -1,72 +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; -// -//import org.apache.tajo.engine.planner.global.GlobalPlanner; -//import org.apache.tajo.plan.visitor.BasicLogicalPlanVisitor; -//import org.apache.tajo.plan.LogicalPlan; -//import org.apache.tajo.plan.PlanningException; -//import org.apache.tajo.plan.logical.JoinNode; -//import org.apache.tajo.plan.logical.LogicalNode; -//import org.apache.tajo.plan.logical.NodeType; -//import org.apache.tajo.plan.logical.ScanNode; -// -//import java.util.Stack; -// -//public class BroadcastJoinMarkCandidateVisitor extends BasicLogicalPlanVisitor { -// @Override -// public LogicalNode visitJoin(GlobalPlanner.GlobalPlanContext context, LogicalPlan plan, LogicalPlan.QueryBlock block, -// JoinNode node, Stack stack) throws PlanningException { -// LogicalNode leftChild = node.getLeftChild(); -// LogicalNode rightChild = node.getRightChild(); -// -// if (ScanNode.isScanNode(leftChild) && ScanNode.isScanNode(rightChild)) { -// node.setCandidateBroadcast(true); -// return node; -// } -// -// if(!ScanNode.isScanNode(leftChild)) { -// visit(context, plan, block, leftChild, stack); -// } -// -// if(!ScanNode.isScanNode(rightChild)) { -// visit(context, plan, block, rightChild, stack); -// } -// -// if(isBroadcastCandidateNode(leftChild) && -// isBroadcastCandidateNode(rightChild)) { -// node.setCandidateBroadcast(true); -// } -// -// return node; -// } -// -// public static boolean isBroadcastCandidateNode(LogicalNode node) { -// if(node.getType() == NodeType.SCAN || -// node.getType() == NodeType.PARTITIONS_SCAN) { -// return true; -// } -// -// if(node.getType() == NodeType.JOIN && ((JoinNode)node).isCandidateBroadcast()) { -// return true; -// } -// -// return false; -// } -//} diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/BroadcastJoinPlanVisitor.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/BroadcastJoinPlanVisitor.java deleted file mode 100644 index 6d7ed62ab6..0000000000 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/BroadcastJoinPlanVisitor.java +++ /dev/null @@ -1,60 +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; -// -//import org.apache.tajo.engine.planner.global.GlobalPlanner; -//import org.apache.tajo.plan.visitor.BasicLogicalPlanVisitor; -//import org.apache.tajo.plan.LogicalPlan; -//import org.apache.tajo.plan.PlanningException; -//import org.apache.tajo.plan.logical.JoinNode; -//import org.apache.tajo.plan.logical.LogicalNode; -//import org.apache.tajo.plan.logical.NodeType; -//import org.apache.tajo.plan.logical.ScanNode; -// -//import java.util.Stack; -// -//public class BroadcastJoinPlanVisitor extends BasicLogicalPlanVisitor { -// @Override -// public LogicalNode visitJoin(GlobalPlanner.GlobalPlanContext context, LogicalPlan plan, LogicalPlan.QueryBlock block, -// JoinNode node, Stack stack) throws PlanningException { -// LogicalNode leftChild = node.getLeftChild(); -// LogicalNode rightChild = node.getRightChild(); -// -// if (leftChild.getType() == NodeType.JOIN && ScanNode.isScanNode(rightChild)) { -// node.getBroadcastCandidateTargets().add(node); -// } -// LogicalNode parentNode = stack.peek(); -// if (parentNode != null && parentNode.getType() == NodeType.JOIN) { -// node.getBroadcastCandidateTargets().addAll(((JoinNode)parentNode).getBroadcastCandidateTargets()); -// } -// -// Stack currentStack = new Stack(); -// currentStack.push(node); -// if(!ScanNode.isScanNode(leftChild)) { -// visit(context, plan, block, leftChild, currentStack); -// } -// -// if(!ScanNode.isScanNode(rightChild)) { -// visit(context, plan, block, rightChild, currentStack); -// } -// currentStack.pop(); -// -// return node; -// } -//} diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java index 257edf40fd..89e887a710 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java @@ -130,19 +130,6 @@ public void build(QueryContext queryContext, MasterPlan masterPlan) throws IOExc LogicalNode inputPlan = PlannerUtil.clone(masterPlan.getLogicalPlan(), masterPlan.getLogicalPlan().getRootBlock().getRoot()); -// boolean broadcastEnabled = masterPlan.getContext().getBool(SessionVars.TEST_BROADCAST_JOIN_ENABLED); -// if (broadcastEnabled) { -// // pre-visit the master plan in order to find tables to be broadcasted -// // this visiting does not make any execution block and change plan. -// BroadcastJoinMarkCandidateVisitor markCandidateVisitor = new BroadcastJoinMarkCandidateVisitor(); -// markCandidateVisitor.visit(globalPlanContext, -// masterPlan.getLogicalPlan(), masterPlan.getLogicalPlan().getRootBlock(), inputPlan, new Stack()); -// -// BroadcastJoinPlanVisitor broadcastJoinPlanVisitor = new BroadcastJoinPlanVisitor(); -// broadcastJoinPlanVisitor.visit(globalPlanContext, -// masterPlan.getLogicalPlan(), masterPlan.getLogicalPlan().getRootBlock(), inputPlan, new Stack()); -// } - // create a distributed execution plan by visiting each logical node. // Its output is a graph, where each vertex is an execution block, and each edge is a data channel. // MasterPlan contains them. @@ -208,187 +195,12 @@ private DataChannel createDataChannelFromJoin(ExecutionBlock leftBlock, Executio return channel; } - /** - * It calculates the total volume of all descendent relation nodes. - */ - public static long computeDescendentVolume(LogicalNode node) throws PlanningException { - - if (node instanceof RelationNode) { - switch (node.getType()) { - case SCAN: - ScanNode scanNode = (ScanNode) node; - if (scanNode.getTableDesc().getStats() == null) { - // TODO - this case means that data is not located in HDFS. So, we need additional - // broadcast method. - return Long.MAX_VALUE; - } else { - return scanNode.getTableDesc().getStats().getNumBytes(); - } - case PARTITIONS_SCAN: - PartitionedTableScanNode pScanNode = (PartitionedTableScanNode) node; - if (pScanNode.getTableDesc().getStats() == null) { - // TODO - this case means that data is not located in HDFS. So, we need additional - // broadcast method. - return Long.MAX_VALUE; - } else { - // if there is no selected partition - if (pScanNode.getInputPaths() == null || pScanNode.getInputPaths().length == 0) { - return 0; - } else { - return pScanNode.getTableDesc().getStats().getNumBytes(); - } - } - case TABLE_SUBQUERY: - return computeDescendentVolume(((TableSubQueryNode) node).getSubQuery()); - default: - throw new IllegalArgumentException("Not RelationNode"); - } - } else if (node instanceof UnaryNode) { - return computeDescendentVolume(((UnaryNode) node).getChild()); - } else if (node instanceof BinaryNode) { - BinaryNode binaryNode = (BinaryNode) node; - return computeDescendentVolume(binaryNode.getLeftChild()) + computeDescendentVolume(binaryNode.getRightChild()); - } - - throw new PlanningException("Invalid State"); - } - - private static boolean checkIfCanBeOneOfBroadcastJoin(LogicalNode node) { - return node.getType() == NodeType.SCAN || node.getType() == NodeType.PARTITIONS_SCAN; - } - - /** - * Get a volume of a table of a partitioned table - * @param scanNode ScanNode corresponding to a table - * @return table volume (bytes) - */ - private static long getTableVolume(ScanNode scanNode) { - long scanBytes = scanNode.getTableDesc().getStats().getNumBytes(); - if (scanNode.getType() == NodeType.PARTITIONS_SCAN) { - PartitionedTableScanNode pScanNode = (PartitionedTableScanNode)scanNode; - if (pScanNode.getInputPaths() == null || pScanNode.getInputPaths().length == 0) { - scanBytes = 0L; - } - } - - return scanBytes; - } - private ExecutionBlock buildJoinPlan(GlobalPlanContext context, JoinNode joinNode, ExecutionBlock leftBlock, ExecutionBlock rightBlock) throws PlanningException { MasterPlan masterPlan = context.plan; ExecutionBlock currentBlock; -// boolean broadcastEnabled = context.getPlan().getContext().getBool(SessionVars.TEST_BROADCAST_JOIN_ENABLED); -// long broadcastTableSizeLimit = context.getPlan().getContext().getLong(SessionVars.BROADCAST_TABLE_SIZE_LIMIT); - - // to check when the tajo.dist-query.join.broadcast.auto property is true -// if (broadcastEnabled && joinNode.isCandidateBroadcast()) { -// LogicalNode leftNode = joinNode.getLeftChild(); -// LogicalNode rightNode = joinNode.getRightChild(); -// -// List broadcastTargetScanNodes = new ArrayList(); -// int numLargeTables = 0; -// boolean leftBroadcast = false; -// boolean rightBroadcast = false; -// -// // TODO - in the the current implementation, a broadcast join on a bush join tree is not supported yet. -// // -// // Join -// // / \ -// // Join Join -// // / \ / \ -// // Scan Scan Scan Scan -// -// -// // Checking Left Side of Join -// if (ScanNode.isScanNode(leftNode)) { -// ScanNode scanNode = (ScanNode)leftNode; -// if (joinNode.getJoinType() == JoinType.LEFT_OUTER || getTableVolume(scanNode) >= broadcastTableSizeLimit) { -// numLargeTables++; -// } else { -// leftBroadcast = true; -// broadcastTargetScanNodes.add(scanNode); -// LOG.info("JoinNode's left table " + scanNode.getCanonicalName() + " (" -// + getTableVolume(scanNode) + ") is marked a broadcasted table"); -// } -// } -// -// // Checking Right Side OF Join -// if (ScanNode.isScanNode(rightNode)) { -// ScanNode scanNode = (ScanNode)rightNode; -// if (joinNode.getJoinType() == JoinType.RIGHT_OUTER || getTableVolume(scanNode) >= broadcastTableSizeLimit) { -// numLargeTables++; -// } else { -// rightBroadcast = true; -// broadcastTargetScanNodes.add(scanNode); -// LOG.info("JoinNode's right table " + scanNode.getCanonicalName() + " (" -// + getTableVolume(scanNode) + ") is marked a broadcasted table"); -// } -// } -// -// JoinNode blockJoinNode = null; -// if (!leftBroadcast && !rightBroadcast) { -// // In the case of large, large, small, small -// // all small tables broadcast to right large table -// numLargeTables = 1; -// } -// for(LogicalNode eachNode: joinNode.getBroadcastCandidateTargets()) { -// if (eachNode.getPID() == joinNode.getPID()) { -// continue; -// } -// if (numLargeTables >= 2) { -// break; -// } -// JoinNode broadcastJoinNode = (JoinNode)eachNode; -// ScanNode scanNode = broadcastJoinNode.getRightChild(); -// if (getTableVolume(scanNode) < broadcastTableSizeLimit) { -// broadcastTargetScanNodes.add(scanNode); -// blockJoinNode = broadcastJoinNode; -// LOG.info("The table " + scanNode.getCanonicalName() + " (" -// + getTableVolume(scanNode) + ") is marked a broadcasted table"); -// } else { -// numLargeTables++; -// if (numLargeTables < 2) { -// blockJoinNode = broadcastJoinNode; -// } -// } -// } -// -// if (!broadcastTargetScanNodes.isEmpty()) { -// // make new execution block -// currentBlock = masterPlan.newExecutionBlock(); -// -// if (!leftBroadcast && !rightBroadcast) { -// DataChannel leftChannel = createDataChannelFromJoin(leftBlock, rightBlock, currentBlock, joinNode, true); -// ScanNode leftScan = buildInputExecutor(masterPlan.getLogicalPlan(), leftChannel); -// joinNode.setLeftChild(leftScan); -// masterPlan.addConnect(leftChannel); -// -// DataChannel rightChannel = createDataChannelFromJoin(leftBlock, rightBlock, currentBlock, joinNode, false); -// ScanNode rightScan = buildInputExecutor(masterPlan.getLogicalPlan(), rightChannel); -// joinNode.setRightChild(rightScan); -// masterPlan.addConnect(rightChannel); -// } -// -// if (blockJoinNode != null) { -// LOG.info("Set execution's plan with join " + blockJoinNode + " for broadcast join"); -// // set current execution block's plan with last broadcast join node -// currentBlock.setPlan(blockJoinNode); -// } else { -// currentBlock.setPlan(joinNode); -// } -// -// for (ScanNode eachBroadcastTarget: broadcastTargetScanNodes) { -// currentBlock.addBroadcastRelation(eachBroadcastTarget.getCanonicalName()); -// context.execBlockMap.remove(eachBroadcastTarget.getPID()); -// } -// -// return currentBlock; -// } -// } - LogicalNode leftNode = joinNode.getLeftChild(); LogicalNode rightNode = joinNode.getRightChild(); @@ -815,7 +627,7 @@ private ExecutionBlock buildGroupBy(GlobalPlanContext context, ExecutionBlock la if (hasUnionChild(firstPhaseGroupby)) { currentBlock = buildGroupbyAndUnionPlan(masterPlan, lastBlock, firstPhaseGroupby, groupbyNode); } else { -// general hash-shuffled aggregation + // general hash-shuffled aggregation currentBlock = buildTwoPhaseGroupby(masterPlan, lastBlock, firstPhaseGroupby, groupbyNode); } } @@ -1354,35 +1166,9 @@ public LogicalNode visitJoin(GlobalPlanContext context, LogicalPlan plan, Logica LogicalNode leftChild = visit(context, plan, block, node.getLeftChild(), stack); ExecutionBlock leftChildBlock = context.execBlockMap.get(leftChild.getPID()); -// if (leftChild.getType() == NodeType.JOIN && checkIfCanBeOneOfBroadcastJoin(node.getRightChild())) { -// ScanNode scanNode = node.getRightChild(); -// if (leftChildBlock.isBroadcastRelation(scanNode.getCanonicalName())) { -// context.execBlockMap.put(node.getPID(), leftChildBlock); -// return node; -// } -// -// // if left execution block's plan is replaced with parent node(join node) -// if (leftChildBlock.getPlan().getPID() == node.getPID()) { -// context.execBlockMap.put(node.getPID(), leftChildBlock); -// return node; -// } -// } - LogicalNode rightChild = visit(context, plan, block, node.getRightChild(), stack); ExecutionBlock rightChildBlock = context.execBlockMap.get(rightChild.getPID()); -// // In the case of broadcast join leftChildBlock can be replaced with upper join node. -// // So if the current join node is a child node of leftChildBlock's plan(join node) -// // the current join node already participates in broadcast join. -// LogicalNode leftChildBlockNode = leftChildBlock.getPlan(); -// // If child block is union, child block has not plan -// if (leftChildBlockNode != null && leftChildBlockNode.getType() == NodeType.JOIN) { -// if (leftChildBlockNode.getPID() > node.getPID()) { -// context.execBlockMap.put(node.getPID(), leftChildBlock); -// return node; -// } -// } - if (node.getJoinType() == JoinType.LEFT_OUTER) { leftChildBlock.setPreservedRow(); rightChildBlock.setNullSuppllying(); @@ -1623,29 +1409,4 @@ public LogicalNode visitInsert(GlobalPlanContext context, LogicalPlan plan, Logi return node; } } - -// @SuppressWarnings("unused") -// private static class ConsecutiveUnionFinder extends BasicLogicalPlanVisitor, LogicalNode> { -// @Override -// public LogicalNode visitUnion(List unionNodeList, LogicalPlan plan, LogicalPlan.QueryBlock queryBlock, -// UnionNode node, Stack stack) -// throws PlanningException { -// if (node.getType() == NodeType.UNION) { -// unionNodeList.add(node); -// } -// -// stack.push(node); -// TableSubQueryNode leftSubQuery = node.getLeftChild(); -// TableSubQueryNode rightSubQuery = node.getRightChild(); -// if (leftSubQuery.getSubQuery().getType() == NodeType.UNION) { -// visit(unionNodeList, plan, queryBlock, leftSubQuery, stack); -// } -// if (rightSubQuery.getSubQuery().getType() == NodeType.UNION) { -// visit(unionNodeList, plan, queryBlock, rightSubQuery, stack); -// } -// stack.pop(); -// -// return node; -// } -// } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/BaseGlobalPlanRewriteRuleProvider.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/BaseGlobalPlanRewriteRuleProvider.java index 6d922b1ed0..1ae405632d 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/BaseGlobalPlanRewriteRuleProvider.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/BaseGlobalPlanRewriteRuleProvider.java @@ -21,7 +21,6 @@ import com.google.common.collect.Lists; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.engine.planner.global.rewriter.rules.BroadcastJoinRule; -//import org.apache.tajo.engine.planner.global.rewriter.rules.UnionReduceRule; import org.apache.tajo.util.TUtil; import java.util.Collection; @@ -38,7 +37,6 @@ public BaseGlobalPlanRewriteRuleProvider(TajoConf conf) { @Override public Collection> getRules() { List> rules = Lists.newArrayList(); -// rules.add(UnionReduceRule.class); rules.add(BroadcastJoinRule.class); return rules; } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java index 3c835c4cf3..576f2d60d9 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java @@ -23,7 +23,6 @@ import org.apache.tajo.ExecutionBlockId; import org.apache.tajo.OverridableConf; import org.apache.tajo.SessionVars; -import org.apache.tajo.algebra.JoinType; import org.apache.tajo.engine.planner.global.ExecutionBlock; import org.apache.tajo.engine.planner.global.GlobalPlanner; import org.apache.tajo.engine.planner.global.MasterPlan; @@ -32,7 +31,6 @@ import org.apache.tajo.plan.PlanningException; import org.apache.tajo.plan.logical.*; import org.apache.tajo.plan.util.PlannerUtil; -import org.apache.tajo.plan.visitor.BasicLogicalPlanVisitor; import org.apache.tajo.util.TUtil; import org.apache.tajo.util.graph.DirectedGraphVisitor; @@ -156,9 +154,7 @@ private void visitLeafNode(ExecutionBlock current) { try { updateScanOfParentAsBroadcastable(plan, current); } catch (PlanningException e) { -// throw new RuntimeException(e); - // This case is when the current has two or more inputs via union. - // It is able to be simply ignored. + // This case is when the current has two or more inputs via union, and simply ignored. } } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/EbMergeRule.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/EbMergeRule.java deleted file mode 100644 index 82bcd79afb..0000000000 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/EbMergeRule.java +++ /dev/null @@ -1,110 +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.global.rewriter.rules; - -import org.apache.tajo.OverridableConf; -import org.apache.tajo.engine.planner.global.ExecutionBlock; -import org.apache.tajo.engine.planner.global.MasterPlan; -import org.apache.tajo.engine.planner.global.rewriter.GlobalPlanRewriteRule; -import org.apache.tajo.ipc.TajoWorkerProtocol; -import org.apache.tajo.plan.PlanningException; -import org.apache.tajo.plan.expr.AggregationFunctionCallEval; -import org.apache.tajo.plan.logical.*; - -public class EbMergeRule implements GlobalPlanRewriteRule { - private GlobalPlanRewriteUtil.ParentFinder parentFinder = new GlobalPlanRewriteUtil.ParentFinder(); - - @Override - public String getName() { - return "EbMergeRule"; - } - - @Override - public boolean isEligible(OverridableConf queryContext, MasterPlan plan) { - return true; - } - - @Override - public MasterPlan rewrite(MasterPlan plan) throws PlanningException { - return null; - } - - private ExecutionBlock mergeTwoPhaseNonJoin(MasterPlan plan, ExecutionBlock child, ExecutionBlock parent) - throws PlanningException { - - ScanNode scanForChild = GlobalPlanRewriteUtil.findScanForChildEb(child, parent); - if (scanForChild == null) { - throw new PlanningException("Cannot find any scan nodes for " + child.getId() + " in " + parent.getId()); - } - - parentFinder.set(scanForChild); - parentFinder.find(parent.getPlan()); - LogicalNode parentOfScanForChild = parentFinder.getFound(); - if (parentOfScanForChild == null) { - throw new PlanningException("Cannot find the parent of " + scanForChild.getCanonicalName()); - } - - LogicalNode rootOfChild = child.getPlan(); - if (rootOfChild.getType() == NodeType.STORE) { - rootOfChild = ((StoreTableNode)rootOfChild).getChild(); - } - LogicalNode mergedPlan; - if (rootOfChild.getType() == parentOfScanForChild.getType()) { - // merge two-phase plan into one-phase plan. - // remove the second-phase plan. - LogicalNode firstPhaseNode = rootOfChild; - LogicalNode secondPhaseNode = parentOfScanForChild; - - parentFinder.set(parentOfScanForChild); - parentFinder.find(parent.getPlan()); - parentOfScanForChild = parentFinder.getFound(); - - if (parentOfScanForChild == null) { - // assume that the node which will be merged is the root node of the plan of the parent eb. - mergedPlan = firstPhaseNode; - } else { - GlobalPlanRewriteUtil.replaceChild(firstPhaseNode, scanForChild, parentOfScanForChild); - mergedPlan = parent.getPlan(); - } - - if (firstPhaseNode.getType() == NodeType.GROUP_BY) { - GroupbyNode firstPhaseGroupby = (GroupbyNode) firstPhaseNode; - GroupbyNode secondPhaseGroupby = (GroupbyNode) secondPhaseNode; - for (AggregationFunctionCallEval aggFunc : firstPhaseGroupby.getAggFunctions()) { - aggFunc.setFirstAndLastPhase(); - } - firstPhaseGroupby.setTargets(secondPhaseGroupby.getTargets()); - firstPhaseGroupby.setOutSchema(secondPhaseGroupby.getOutSchema()); - } - } else { - mergedPlan = parent.getPlan(); - } - - parent = GlobalPlanRewriteUtil.mergeExecutionBlocks(plan, child, parent); - - if (parent.getEnforcer().hasEnforceProperty(TajoWorkerProtocol.EnforceProperty.EnforceType.SORTED_INPUT)) { - parent.getEnforcer().removeSortedInput(scanForChild.getTableName()); - } - - parent.setPlan(mergedPlan); - - return parent; - } - -} diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanRewriteUtil.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanRewriteUtil.java index 58a74046a2..4b22c12b23 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanRewriteUtil.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanRewriteUtil.java @@ -103,6 +103,51 @@ public static long getTableVolume(ScanNode scanNode) { } } + /** + * It calculates the total volume of all descendent relation nodes. + */ + public static long computeDescendentVolume(LogicalNode node) throws PlanningException { + + if (node instanceof RelationNode) { + switch (node.getType()) { + case SCAN: + ScanNode scanNode = (ScanNode) node; + if (scanNode.getTableDesc().getStats() == null) { + // TODO - this case means that data is not located in HDFS. So, we need additional + // broadcast method. + return Long.MAX_VALUE; + } else { + return scanNode.getTableDesc().getStats().getNumBytes(); + } + case PARTITIONS_SCAN: + PartitionedTableScanNode pScanNode = (PartitionedTableScanNode) node; + if (pScanNode.getTableDesc().getStats() == null) { + // TODO - this case means that data is not located in HDFS. So, we need additional + // broadcast method. + return Long.MAX_VALUE; + } else { + // if there is no selected partition + if (pScanNode.getInputPaths() == null || pScanNode.getInputPaths().length == 0) { + return 0; + } else { + return pScanNode.getTableDesc().getStats().getNumBytes(); + } + } + case TABLE_SUBQUERY: + return computeDescendentVolume(((TableSubQueryNode) node).getSubQuery()); + default: + throw new IllegalArgumentException("Not RelationNode"); + } + } else if (node instanceof UnaryNode) { + return computeDescendentVolume(((UnaryNode) node).getChild()); + } else if (node instanceof BinaryNode) { + BinaryNode binaryNode = (BinaryNode) node; + return computeDescendentVolume(binaryNode.getLeftChild()) + computeDescendentVolume(binaryNode.getRightChild()); + } + + throw new PlanningException("Invalid State"); + } + public static class ParentFinder implements LogicalNodeVisitor { private LogicalNode target; private LogicalNode found; diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/UnionReduceRule.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/UnionReduceRule.java deleted file mode 100644 index 45e9ff07c0..0000000000 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/UnionReduceRule.java +++ /dev/null @@ -1,105 +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.global.rewriter.rules; -// -//import org.apache.tajo.ExecutionBlockId; -//import org.apache.tajo.OverridableConf; -//import org.apache.tajo.engine.planner.global.ExecutionBlock; -//import org.apache.tajo.engine.planner.global.MasterPlan; -//import org.apache.tajo.engine.planner.global.rewriter.GlobalPlanRewriteRule; -//import org.apache.tajo.plan.LogicalPlan; -//import org.apache.tajo.plan.PlanningException; -//import org.apache.tajo.plan.logical.LogicalNode; -//import org.apache.tajo.plan.logical.NodeType; -//import org.apache.tajo.plan.logical.UnionNode; -//import org.apache.tajo.plan.util.PlannerUtil; -//import org.apache.tajo.util.TUtil; -//import org.apache.tajo.util.graph.DirectedGraphVisitor; -// -//import java.util.List; -//import java.util.Map; -//import java.util.Stack; -// -//public class UnionReduceRule implements GlobalPlanRewriteRule { -// -// @Override -// public String getName() { -// return "UnionReduceRule"; -// } -// -// @Override -// public boolean isEligible(OverridableConf queryContext, MasterPlan plan) { -// for (LogicalPlan.QueryBlock block : plan.getLogicalPlan().getQueryBlocks()) { -// if (block.hasNode(NodeType.UNION)) { -// return true; -// } -// } -// return false; -// } -// -// @Override -// public MasterPlan rewrite(MasterPlan plan) throws PlanningException { -// Rewriter.rewrite(plan); -// return plan; -// } -// -// static class Rewriter implements DirectedGraphVisitor { -// private static Rewriter instance; -// private static MasterPlan plan; -// -// private Rewriter() {} -// -// public static void rewrite(MasterPlan plan) { -// if (instance == null) { -// instance = new Rewriter(); -// } -// instance.plan = plan; -// instance.visit(new Stack(), plan.getTerminalBlock().getId()); -// } -// -// @Override -// public void visit(Stack stack, ExecutionBlockId executionBlockId) { -// // must have the form of -// /* -// parent_op -// | -// union -// / \ -// child_op child_op -// */ -// ExecutionBlock current = instance.plan.getExecBlock(executionBlockId); -// if (current.hasUnion()) { -// Map newBlocks = TUtil.newHashMap(); -// List childBlocks = instance.plan.getChilds(current); -// // create new execution blocks for each child_op -// // TODO: consider union sequence -// UnionNode unionNode = PlannerUtil.findTopNode(current.getPlan(), NodeType.UNION); -// LogicalNode parentOfUnion = PlannerUtil.findTopParentNode(current.getPlan(), NodeType.UNION); -// newBlocks.put(unionNode.getLeftChild().getPID(), instance.plan.newExecutionBlock()); -// newBlocks.put(unionNode.getRightChild().getPID(), instance.plan.newExecutionBlock()); -// -// // push parent_op as the parent of each child_op and remove union -// -// -// // connect new execution blocks and parents of the current block -// -// } -// } -// } -//} 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 ee3a04f12f..1645263fbd 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 @@ -103,29 +103,6 @@ public Tuple next() throws IOException { iterator = nullIterator(rightNumCols); continue; } -//<<<<<<< HEAD -// -// /* -// * TODO -// * Currently, some physical executors can return new instances of tuple, but others not. -// * This sometimes causes wrong results due to the singleton Tuple instance. -// * The below line is a temporal solution to fix this problem. -// * This will be improved at https://issues.apache.org/jira/browse/TAJO-1343. -// */ -// try { -// tuple = tuple.clone(); -// } catch (CloneNotSupportedException e) { -// throw new IOException(e); -// } -// 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) { @@ -136,7 +113,6 @@ public Tuple next() throws IOException { if (!rightTuples.hasNext()) { iterator = nullIterator(rightNumCols); continue; -//>>>>>>> 5491f0e7507c7efa1b2306d4c1f1d25240e482a9 } iterator = rightTuples; hashed.setFirst(true); // match found 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 c56034aea3..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 @@ -58,32 +58,11 @@ public Tuple next() throws IOException { frameTuple.setLeft(leftTuple); -//<<<<<<< HEAD -// /* -// * TODO -// * Currently, some physical executors can return new instances of tuple, but others not. -// * This sometimes causes wrong results due to the singleton Tuple instance. -// * The below line is a temporal solution to fix this problem. -// * This will be improved at https://issues.apache.org/jira/browse/TAJO-1343. -// */ -// try { -// tuple = tuple.clone(); -// } catch (CloneNotSupportedException e) { -// throw new IOException(e); -// } -// 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; -//>>>>>>> 5491f0e7507c7efa1b2306d4c1f1d25240e482a9 } } 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 516fab406e..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 @@ -55,46 +55,6 @@ public Tuple next() throws IOException { } frameTuple.setLeft(leftTuple); -//<<<<<<< HEAD -// /** -// * Build a hash table for right input relation. -// * If the right child is not scan exec, -// * -// * @return -// * @throws IOException -// */ -// 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])); -// } -// -// List newValue = map.get(keyTuple); -// -// /* -// * TODO -// * Currently, some physical executors can return new instances of tuple, but others not. -// * This sometimes causes wrong results due to the singleton Tuple instance. -// * The below line is a temporal solution to fix this problem. -// * This will be improved at https://issues.apache.org/jira/browse/TAJO-1343. -// */ -// try { -// tuple = tuple.clone(); -// } catch (CloneNotSupportedException e) { -// throw new IOException(e); -// } -// if (newValue != null) { -// newValue.add(tuple); -// } else { -// newValue = new ArrayList(); -// newValue.add(tuple); -// map.put(keyTuple, newValue); -//======= if (leftFiltered(leftTuple)) { iterator = nullIterator(rightNumCols); continue; @@ -108,7 +68,6 @@ public Tuple next() throws IOException { //output a tuple with the nulls padded rightTuple iterator = nullIterator(rightNumCols); continue; -//>>>>>>> 5491f0e7507c7efa1b2306d4c1f1d25240e482a9 } iterator = rightTuples; } diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java index d554b448a4..4fe150bd39 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java @@ -38,6 +38,7 @@ import org.apache.tajo.engine.planner.global.ExecutionBlock; import org.apache.tajo.engine.planner.global.GlobalPlanner; import org.apache.tajo.engine.planner.global.MasterPlan; +import org.apache.tajo.engine.planner.global.rewriter.rules.GlobalPlanRewriteUtil; import org.apache.tajo.engine.utils.TupleUtil; import org.apache.tajo.exception.InternalException; import org.apache.tajo.ipc.TajoWorkerProtocol.DistinctGroupbyEnforcer.MultipleAggregationStage; @@ -107,7 +108,7 @@ public static void scheduleFragmentsForJoinQuery(TaskSchedulerContext schedulerC fragments[i] = new FileFragment(scans[i].getCanonicalName(), tablePath, 0, 0, new String[]{UNKNOWN_HOST}); } else { try { - stats[i] = GlobalPlanner.computeDescendentVolume(scans[i]); + stats[i] = GlobalPlanRewriteUtil.computeDescendentVolume(scans[i]); } catch (PlanningException e) { throw new IOException(e); } @@ -188,37 +189,6 @@ public static void scheduleFragmentsForJoinQuery(TaskSchedulerContext schedulerC } // Assigning either fragments or fetch urls to query units -// boolean isAllBroadcastTable = true; -// for (int i = 0; i < scans.length; i++) { -// if (!execBlock.isBroadcastRelation(scans[i].getCanonicalName())) { -// isAllBroadcastTable = false; -// break; -// } -// } - - -// if (isAllBroadcastTable) { // if all relations of this EB are broadcasted -// // set largest table to normal mode -// long maxStats = Long.MIN_VALUE; -// int maxStatsScanIdx = -1; -// for (int i = 0; i < scans.length; i++) { -// // finding largest table. -// // If stats == 0, can't be base table. -// if (stats[i] > 0 && stats[i] > maxStats) { -// maxStats = stats[i]; -// maxStatsScanIdx = i; -// } -// } -// if (maxStatsScanIdx == -1) { -// maxStatsScanIdx = 0; -// } -// int baseScanIdx = maxStatsScanIdx; -// scans[baseScanIdx].setBroadcastTable(false); -// execBlock.removeBroadcastRelation(scans[baseScanIdx].getCanonicalName()); -// LOG.info(String.format("[Distributed Join Strategy] : Broadcast Join with all tables, base_table=%s, base_volume=%d", -// scans[baseScanIdx].getCanonicalName(), stats[baseScanIdx])); -// scheduleLeafTasksWithBroadcastTable(schedulerContext, stage, baseScanIdx, fragments); -// } else if (!execBlock.getBroadcastTables().isEmpty()) { // If some relations of this EB are broadcasted if (execBlock.hasBroadcastRelation()) { // If some relations of this EB are broadcasted boolean hasNonLeafNode = false; List largeScanIndexList = new ArrayList(); @@ -384,7 +354,6 @@ private static void scheduleSymmetricRepartitionJoin(QueryMasterTask.QueryMaster // Getting the desire number of join tasks according to the volumn // of a larger table -// int largerIdx = leftStats >= rightStats ? 0 : 1; long largerStat = leftStats >= rightStats ? leftStats : rightStats; int desireJoinTaskVolumn = stage.getMasterPlan().getContext().getInt(SessionVars.JOIN_TASK_INPUT_SIZE); diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java index 5d42e0f491..5a0fc38de3 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java @@ -884,7 +884,6 @@ public static int calculateShuffleOutputNum(Stage stage, DataChannel channel) { // We assume this execution block the first stage of join if two or more tables are included in this block, if (parent != null && (parent.getNonBroadcastRelNum()) >= 2) { -// if (parent != null && parent.getScanNodes().length >= 2) { List childs = masterPlan.getChilds(parent); // for outer @@ -991,11 +990,9 @@ private static void schedule(Stage stage) throws IOException { MasterPlan masterPlan = stage.getMasterPlan(); ExecutionBlock execBlock = stage.getBlock(); if (stage.getMasterPlan().isLeaf(execBlock.getId()) && execBlock.getScanNodes().length == 1) { // Case 1: Just Scan - // Some execution blocks can have broadcast table even though they don't have any join nodes -// if (stage.getMasterPlan().isLeaf(execBlock.getId()) && execBlock.getNonBroadcastRelNum() <= 1) { // Case 1: Just Scan + // Some execution blocks can have broadcast table even though they don't have any join nodes scheduleFragmentsForLeafQuery(stage); } else if (execBlock.getScanNodes().length > 1) { // Case 2: Join -// } else if (execBlock.getNonBroadcastRelNum() > 1) { // Case 2: Join Repartitioner.scheduleFragmentsForJoinQuery(stage.schedulerContext, stage); } else { // Case 3: Others (Sort or Aggregation) int numTasks = getNonLeafTaskNum(stage); @@ -1081,8 +1078,6 @@ private static void scheduleFragmentsForLeafQuery(Stage stage) throws IOExceptio ExecutionBlock execBlock = stage.getBlock(); ScanNode[] scans = execBlock.getScanNodes(); Preconditions.checkArgument(scans.length == 1, "Must be Scan Query"); -// Preconditions.checkArgument(execBlock.getNonBroadcastRelNum() <= 1, -// "A leaf stage should not scan two or more large fragments"); ScanNode scan = scans[0]; TableDesc table = stage.context.getTableDescMap().get(scan.getCanonicalName()); diff --git a/tajo-core/src/test/java/org/apache/tajo/benchmark/TestTPCH.java b/tajo-core/src/test/java/org/apache/tajo/benchmark/TestTPCH.java index 4ca9b3e7bd..9b7db2de0b 100644 --- a/tajo-core/src/test/java/org/apache/tajo/benchmark/TestTPCH.java +++ b/tajo-core/src/test/java/org/apache/tajo/benchmark/TestTPCH.java @@ -34,23 +34,30 @@ public TestTPCH() { } @Test + @Option(withExplain = true, withExplainGlobal = true) + @SimpleTest public void testQ1OrderBy() throws Exception { - ResultSet res = executeQuery(); - assertResultSet(res); - cleanupQuery(res); + runSimpleTests(); } @Test + @Option(withExplain = true, withExplainGlobal = true) + @SimpleTest public void testQ2FourJoins() throws Exception { - ResultSet res = executeQuery(); - assertResultSet(res); - cleanupQuery(res); + runSimpleTests(); } @Test + @Option(withExplain = true, withExplainGlobal = true) + @SimpleTest public void testTPCH14Expr() throws Exception { - ResultSet res = executeQuery(); - assertResultSet(res); - cleanupQuery(res); + runSimpleTests(); + } + + @Test + @Option(withExplain = true, withExplainGlobal = true) + @SimpleTest + public void testTPCHQ5() throws Exception { + runSimpleTests(); } } \ No newline at end of file diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInnerJoinWithSubQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInnerJoinWithSubQuery.java index a8e2a3b979..2bcb5d9262 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInnerJoinWithSubQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInnerJoinWithSubQuery.java @@ -122,4 +122,11 @@ public final void testBroadcastSubquery() throws Exception { public final void testBroadcastSubquery2() throws Exception { runSimpleTests(); } + + @Test + @Option(withExplain = true, withExplainGlobal = true, parameterized = true) + @SimpleTest() + public final void testThetaJoinKeyPairs() throws Exception { + runSimpleTests(); + } } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSortQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSortQuery.java index fa77fda1ea..24b525d8fb 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSortQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSortQuery.java @@ -370,4 +370,11 @@ public final void testSortWithConstKeys() throws Exception { assertResultSet(res); cleanupQuery(res); } + + @Test + @Option(withExplain = true, withExplainGlobal = true) + @SimpleTest() + public final void testSubQuerySortAfterGroupMultiBlocks() throws Exception { + runSimpleTests(); + } } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestUnionQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestUnionQuery.java index 03a80d1938..03c9c24b01 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestUnionQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestUnionQuery.java @@ -512,4 +512,17 @@ public void testTajo1368Case2() throws Exception { res.close(); } + @Test + @Option(withExplain = true, withExplainGlobal = true) + @SimpleTest + public void testComplexUnion1() throws Exception { + runSimpleTests(); + } + + @Test + @Option(withExplain = true, withExplainGlobal = true) + @SimpleTest + public void testComplexUnion2() throws Exception { + runSimpleTests(); + } } diff --git a/tajo-core/src/test/java/org/apache/tajo/master/TestGlobalPlanner.java b/tajo-core/src/test/java/org/apache/tajo/master/TestGlobalPlanner.java deleted file mode 100644 index 25ef9bf0ae..0000000000 --- a/tajo-core/src/test/java/org/apache/tajo/master/TestGlobalPlanner.java +++ /dev/null @@ -1,347 +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.master; -// -//import org.apache.commons.logging.Log; -//import org.apache.commons.logging.LogFactory; -//import org.apache.tajo.LocalTajoTestingUtility; -//import org.apache.tajo.TajoTestingCluster; -//import org.apache.tajo.algebra.Expr; -//import org.apache.tajo.benchmark.TPCH; -//import org.apache.tajo.catalog.*; -//import org.apache.tajo.catalog.partition.PartitionMethodDesc; -//import org.apache.tajo.catalog.proto.CatalogProtos; -//import org.apache.tajo.catalog.statistics.TableStats; -//import org.apache.tajo.common.TajoDataTypes; -//import org.apache.tajo.engine.function.FunctionLoader; -//import org.apache.tajo.engine.parser.SQLAnalyzer; -//import org.apache.tajo.engine.planner.global.DataChannel; -//import org.apache.tajo.engine.planner.global.ExecutionBlock; -//import org.apache.tajo.engine.planner.global.GlobalPlanner; -//import org.apache.tajo.engine.planner.global.MasterPlan; -//import org.apache.tajo.engine.query.QueryContext; -//import org.apache.tajo.plan.*; -//import org.apache.tajo.plan.expr.BinaryEval; -//import org.apache.tajo.plan.expr.EvalType; -//import org.apache.tajo.plan.expr.FieldEval; -//import org.apache.tajo.plan.util.PlannerUtil; -//import org.apache.tajo.util.CommonTestingUtil; -//import org.apache.tajo.util.FileUtil; -//import org.apache.tajo.util.TUtil; -//import org.junit.AfterClass; -//import org.junit.BeforeClass; -//import org.junit.Test; -// -//import java.io.File; -//import java.io.IOException; -//import java.util.Map; -// -//import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME; -//import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME; -//import static org.junit.Assert.assertFalse; -//import static org.junit.Assert.assertTrue; -// -//public class TestGlobalPlanner { -// private static Log LOG = LogFactory.getLog(TestGlobalPlanner.class); -// -// private static TajoTestingCluster util; -// private static CatalogService catalog; -// private static SQLAnalyzer sqlAnalyzer; -// private static LogicalPlanner planner; -// private static LogicalOptimizer optimizer; -// private static TPCH tpch; -// private static GlobalPlanner globalPlanner; -// -// @BeforeClass -// public static void setUp() throws Exception { -// util = new TajoTestingCluster(); -// util.startCatalogCluster(); -// catalog = util.getMiniCatalogCluster().getCatalog(); -// for (FunctionDesc funcDesc : FunctionLoader.findLegacyFunctions()) { -// catalog.createFunction(funcDesc); -// } -// catalog.createTablespace(DEFAULT_TABLESPACE_NAME, "hdfs://localhost:1234/warehouse"); -// catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME); -// -// // TPC-H Schema for Complex Queries -// String [] tables = { -// "part", "supplier", "partsupp", "nation", "region", "lineitem", "orders", "customer", "customer_parts" -// }; -// int [] volumes = { -// 100, 200, 50, 5, 5, 800, 300, 100, 707 -// }; -// tpch = new TPCH(); -// tpch.loadSchemas(); -// tpch.loadOutSchema(); -// for (int i = 0; i < tables.length; i++) { -// TableMeta m = CatalogUtil.newTableMeta("CSV"); -// TableStats stats = new TableStats(); -// stats.setNumBytes(volumes[i]); -// TableDesc d = CatalogUtil.newTableDesc( -// CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, tables[i]), tpch.getSchema(tables[i]), m, -// CommonTestingUtil.getTestDir()); -// d.setStats(stats); -// -// if (tables[i].equals(TPCH.CUSTOMER_PARTS)) { -// Schema expressionSchema = new Schema(); -// expressionSchema.addColumn("c_nationkey", TajoDataTypes.Type.INT4); -// PartitionMethodDesc partitionMethodDesc = new PartitionMethodDesc( -// DEFAULT_DATABASE_NAME, -// tables[i], -// CatalogProtos.PartitionType.COLUMN, -// "c_nationkey", -// expressionSchema); -// -// d.setPartitionMethod(partitionMethodDesc); -// } -// catalog.createTable(d); -// } -// -// sqlAnalyzer = new SQLAnalyzer(); -// planner = new LogicalPlanner(catalog); -// optimizer = new LogicalOptimizer(util.getConfiguration()); -// globalPlanner = new GlobalPlanner(util.getConfiguration(), catalog); -// } -// -// @AfterClass -// public static void tearDown() { -// util.shutdownCatalogCluster(); -// } -// -// private MasterPlan buildPlan(String sql) throws PlanningException, IOException { -// Expr expr = sqlAnalyzer.parse(sql); -// QueryContext context = LocalTajoTestingUtility.createDummyContext(util.getConfiguration()); -// LogicalPlan plan = planner.createPlan(context, expr); -// optimizer.optimize(context, plan); -// MasterPlan masterPlan = new MasterPlan(LocalTajoTestingUtility.newQueryId(), context, plan); -// globalPlanner.build(context, masterPlan); -// return masterPlan; -// } -// -// @Test -// public void testSelectDistinct() throws Exception { -// buildPlan("select distinct l_orderkey from lineitem"); -// } -// -// @Test -// public void testSortAfterGroupBy() throws Exception { -// buildPlan("select max(l_quantity) as max_quantity, l_orderkey from lineitem group by l_orderkey order by max_quantity"); -// } -// -// @Test -// public void testSortLimit() throws Exception { -// buildPlan("select max(l_quantity) as max_quantity, l_orderkey from lineitem group by l_orderkey order by max_quantity limit 3"); -// } -// -// @Test -// public void testJoin() throws Exception { -// buildPlan("select n_name, r_name, n_regionkey, r_regionkey from nation, region"); -// } -// -// @Test -// public void testThetaJoinKeyPairs() throws Exception { -// StringBuilder sb = new StringBuilder(); -// sb.append("select n_nationkey, n_name, n_regionkey, t.cnt"); -// sb.append(" from nation n"); -// sb.append(" join"); -// sb.append(" ("); -// sb.append(" select r_regionkey, count(*) as cnt"); -// sb.append(" from nation n"); -// sb.append(" join region r on (n.n_regionkey = r.r_regionkey)"); -// sb.append(" group by r_regionkey"); -// sb.append(" ) t on (n.n_regionkey = t.r_regionkey)"); -// sb.append(" and n.n_nationkey > t.cnt "); -// sb.append(" order by n_nationkey"); -// -// MasterPlan plan = buildPlan(sb.toString()); -// ExecutionBlock root = plan.getRoot(); -// -// Map evalMap = TUtil.newHashMap(); -// BinaryEval eval1 = new BinaryEval(EvalType.EQUAL -// , new FieldEval(new Column("default.n.n_regionkey", TajoDataTypes.Type.INT4)) -// , new FieldEval(new Column("default.t.r_regionkey", TajoDataTypes.Type.INT4)) -// ); -// evalMap.put(eval1, Boolean.FALSE); -// -// BinaryEval eval2 = new BinaryEval(EvalType.EQUAL -// , new FieldEval(new Column("default.n.n_nationkey", TajoDataTypes.Type.INT4)) -// , new FieldEval(new Column("default.t.cnt", TajoDataTypes.Type.INT4)) -// ); -// evalMap.put(eval2, Boolean.FALSE); -// -// visitChildExecutionBLock(plan, root, evalMap); -// -// // Find required shuffleKey. -// assertTrue(evalMap.get(eval1).booleanValue()); -// -// // Find that ShuffleKeys only includes equi-join conditions -// assertFalse(evalMap.get(eval2).booleanValue()); -// } -// -// private void visitChildExecutionBLock(MasterPlan plan, ExecutionBlock parentBlock, -// Map qualMap) throws Exception { -// boolean isExistLeftField, isExistRightField; -// -// for (Map.Entry entry : qualMap.entrySet()) { -// FieldEval leftField = (FieldEval)entry.getKey().getLeftExpr(); -// FieldEval rightField = (FieldEval)entry.getKey().getRightExpr(); -// -// for (ExecutionBlock block : plan.getChilds(parentBlock)) { -// isExistLeftField = false; -// isExistRightField = false; -// -// if (plan.getIncomingChannels(block.getId()) != null) { -// for (DataChannel channel :plan.getIncomingChannels(block.getId())) { -// if (channel.getShuffleKeys() != null) { -// for (Column column : channel.getShuffleKeys()) { -// if (column.getQualifiedName().equals(leftField.getColumnRef().getQualifiedName())) { -// isExistLeftField = true; -// } else if (column.getQualifiedName(). -// equals(rightField.getColumnRef().getQualifiedName())) { -// isExistRightField = true; -// } -// } -// } -// } -// -// if(isExistLeftField && isExistRightField) { -// qualMap.put(entry.getKey(), Boolean.TRUE); -// } -// } -// -// visitChildExecutionBLock(plan, block, qualMap); -// } -// } -// } -// -// @Test -// public void testUnion() throws IOException, PlanningException { -// buildPlan("select o_custkey as num from orders union select c_custkey as num from customer union select p_partkey as num from part"); -// } -// -// @Test -// public void testSubQuery() throws IOException, PlanningException { -// buildPlan("select l.l_orderkey from (select * from lineitem) l"); -// } -// -// @Test -// public void testSubQueryJoin() throws IOException, PlanningException { -// buildPlan("select l.l_orderkey from (select * from lineitem) l join (select * from orders) o on l.l_orderkey = o.o_orderkey"); -// } -// -// @Test -// public void testSubQueryGroupBy() throws IOException, PlanningException { -// buildPlan("select sum(l_extendedprice*l_discount) as revenue from (select * from lineitem) as l"); -// } -// -// @Test -// public void testSubQueryGroupBy2() throws IOException, PlanningException { -// buildPlan("select l_orderkey, sum(l_extendedprice*l_discount) as revenue from (select * from lineitem) as l group by l_orderkey"); -// } -// -// @Test -// public void testSubQuerySortAfterGroup() throws IOException, PlanningException { -// buildPlan("select l_orderkey, sum(l_extendedprice*l_discount) as revenue from (select * from lineitem) as l group by l_orderkey order by l_orderkey"); -// } -// -// @Test -// public void testSubQuerySortAfterGroupMultiBlocks() throws IOException, PlanningException { -// buildPlan( -// "select l_orderkey, revenue from (" + -// "select l_orderkey, sum(l_extendedprice*l_discount) as revenue from lineitem group by l_orderkey" -// +") l1" -// -// ); -// } -// -// @Test -// public void testSubQuerySortAfterGroupMultiBlocks2() throws IOException, PlanningException { -// buildPlan( -// "select l_orderkey, revenue from (" + -// "select l_orderkey, revenue from (" + -// "select l_orderkey, sum(l_extendedprice*l_discount) as revenue from lineitem group by l_orderkey" -// +") l1" + -// ") l2 order by l_orderkey" -// -// ); -// } -// -// @Test -// public void testComplexUnion1() throws Exception { -// buildPlan(FileUtil.readTextFile(new File("src/test/resources/queries/default/complex_union_1.sql"))); -// } -// -// @Test -// public void testComplexUnion2() throws Exception { -// buildPlan(FileUtil.readTextFile(new File("src/test/resources/queries/default/complex_union_2.sql"))); -// } -// -// @Test -// public void testUnionGroupBy1() throws Exception { -// buildPlan("select l_orderkey, sum(l_extendedprice*l_discount) as revenue from (" + -// "select * from lineitem " + -// "union " + -// "select * from lineitem ) l group by l_orderkey"); -// } -// -// @Test -// public void testTPCH_Q5() throws Exception { -// buildPlan(FileUtil.readTextFile(new File("benchmark/tpch/q5.sql"))); -// } -// -// @Test -// public void testCheckIfSimpleQuery() throws Exception { -// MasterPlan plan = buildPlan("select * from customer"); -// assertTrue(PlannerUtil.checkIfSimpleQuery(plan.getLogicalPlan())); -// -// //partition table -// plan = buildPlan("select * from customer_parts"); -// assertTrue(PlannerUtil.checkIfSimpleQuery(plan.getLogicalPlan())); -// -// plan = buildPlan("select * from customer where c_nationkey = 1"); -// assertFalse(PlannerUtil.checkIfSimpleQuery(plan.getLogicalPlan())); -// -// // c_nationkey is partition column -// plan = buildPlan("select * from customer_parts where c_nationkey = 1"); -// assertTrue(PlannerUtil.checkIfSimpleQuery(plan.getLogicalPlan())); -// -// // same column order -// plan = buildPlan("select c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment" + -// " from customer"); -// assertTrue(PlannerUtil.checkIfSimpleQuery(plan.getLogicalPlan())); -// -// plan = buildPlan("select c_custkey, c_name, c_address, c_phone, c_acctbal, c_mktsegment, c_comment, c_nationkey " + -// " from customer_parts"); -// assertTrue(PlannerUtil.checkIfSimpleQuery(plan.getLogicalPlan())); -// -// // different column order -// plan = buildPlan("select c_name, c_custkey, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment" + -// " from customer"); -// assertFalse(PlannerUtil.checkIfSimpleQuery(plan.getLogicalPlan())); -// -// plan = buildPlan("select c_name, c_custkey, c_address, c_phone, c_acctbal, c_mktsegment, c_comment, c_nationkey " + -// " from customer_parts"); -// assertFalse(PlannerUtil.checkIfSimpleQuery(plan.getLogicalPlan())); -// -// plan = buildPlan("insert into customer_parts " + -// " select c_name, c_custkey, c_address, c_phone, c_acctbal, c_mktsegment, c_comment, c_nationkey " + -// " from customer"); -// assertFalse(PlannerUtil.checkIfSimpleQuery(plan.getLogicalPlan())); -// } -//} diff --git a/tajo-core/src/test/resources/queries/TestInnerJoinWithSubQuery/testThetaJoinKeyPairs.sql b/tajo-core/src/test/resources/queries/TestInnerJoinWithSubQuery/testThetaJoinKeyPairs.sql new file mode 100644 index 0000000000..802e2b0c90 --- /dev/null +++ b/tajo-core/src/test/resources/queries/TestInnerJoinWithSubQuery/testThetaJoinKeyPairs.sql @@ -0,0 +1,20 @@ +select + n_nationkey, n_name, n_regionkey, t.cnt +from + nation n + join + ( + select + r_regionkey, count(*) as cnt + from + nation n + join + region r + on (n.n_regionkey = r.r_regionkey) + group by + r_regionkey + ) t +on + (n.n_regionkey = t.r_regionkey) and n.n_nationkey > t.cnt +order by + n_nationkey \ No newline at end of file diff --git a/tajo-core/src/test/resources/queries/TestSortQuery/testSubQuerySortAfterGroupMultiBlocks.sql b/tajo-core/src/test/resources/queries/TestSortQuery/testSubQuerySortAfterGroupMultiBlocks.sql new file mode 100644 index 0000000000..3e7eaab303 --- /dev/null +++ b/tajo-core/src/test/resources/queries/TestSortQuery/testSubQuerySortAfterGroupMultiBlocks.sql @@ -0,0 +1,5 @@ +select l_orderkey, revenue from ( + select l_orderkey, revenue from ( + select l_orderkey, sum(l_extendedprice*l_discount) as revenue from lineitem group by l_orderkey + ) l1 +) l2 order by l_orderkey \ No newline at end of file diff --git a/tajo-core/src/test/resources/queries/TestTPCH/testTPCHQ5.sql b/tajo-core/src/test/resources/queries/TestTPCH/testTPCHQ5.sql new file mode 100644 index 0000000000..e7d3d56888 --- /dev/null +++ b/tajo-core/src/test/resources/queries/TestTPCH/testTPCHQ5.sql @@ -0,0 +1,24 @@ +select + n_name, + sum(l_extendedprice * (1 - l_discount)) as revenue +from + customer, + orders, + lineitem, + supplier, + nation, + region +where + c_custkey = o_custkey and + l_orderkey = o_orderkey and + l_suppkey = s_suppkey and + c_nationkey = s_nationkey and + s_nationkey = n_nationkey and + n_regionkey = r_regionkey and + r_name = 'ASIA' and + o_orderdate >= '1994-01-01' and + o_orderdate < '1995-01-01' +group by + n_name +order by + revenue desc \ No newline at end of file diff --git a/tajo-core/src/test/resources/queries/default/complex_union_1.sql b/tajo-core/src/test/resources/queries/TestUnionQuery/testComplexUnion1.sql similarity index 91% rename from tajo-core/src/test/resources/queries/default/complex_union_1.sql rename to tajo-core/src/test/resources/queries/TestUnionQuery/testComplexUnion1.sql index 9020ad7964..0ba2909eb0 100644 --- a/tajo-core/src/test/resources/queries/default/complex_union_1.sql +++ b/tajo-core/src/test/resources/queries/TestUnionQuery/testComplexUnion1.sql @@ -1,5 +1,3 @@ -create table xdr_url as - SELECT l_orderkey, l_partkey, diff --git a/tajo-core/src/test/resources/queries/default/complex_union_2.sql b/tajo-core/src/test/resources/queries/TestUnionQuery/testComplexUnion2.sql similarity index 100% rename from tajo-core/src/test/resources/queries/default/complex_union_2.sql rename to tajo-core/src/test/resources/queries/TestUnionQuery/testComplexUnion2.sql diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testThetaJoinKeyPairs.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testThetaJoinKeyPairs.Hash.plan new file mode 100644 index 0000000000..8ca7e29eb2 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testThetaJoinKeyPairs.Hash.plan @@ -0,0 +1,142 @@ +explain +------------------------------- +SORT(8) + => Sort Keys: default.n.n_nationkey (INT4) (asc) + JOIN(12)(INNER) + => Join Cond: default.n.n_regionkey (INT4) = default.t.r_regionkey (INT4) + => target list: default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.t.cnt (INT8) + => out schema: {(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.t.cnt (INT8)} + => in schema: {(5) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.t.cnt (INT8), default.t.r_regionkey (INT4)} + SCAN(0) on default.nation as n + => target list: default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) + => out schema: {(3) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + => in schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + TABLE_SUBQUERY(6) as default.t + => Targets: default.t.cnt (INT8), default.t.r_regionkey (INT4) + => out schema: {(2) default.t.cnt (INT8), default.t.r_regionkey (INT4)} + => in schema: {(2) default.t.cnt (INT8), default.t.r_regionkey (INT4)} + GROUP_BY(4)(r_regionkey) + => exprs: (count()) + => target list: default.r.r_regionkey (INT4), cnt (INT8) + => out schema:{(2) cnt (INT8), default.r.r_regionkey (INT4)} + => in schema:{(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4)} + JOIN(11)(INNER) + => Join Cond: default.n.n_regionkey (INT4) = default.r.r_regionkey (INT4) + => target list: default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4) + => out schema: {(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4)} + => in schema: {(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4)} + SCAN(2) on default.region as r + => target list: default.r.r_regionkey (INT4) + => out schema: {(1) default.r.r_regionkey (INT4)} + => in schema: {(3) default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4)} + SCAN(1) on default.nation as n + => target list: default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) + => out schema: {(3) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + => in schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} +explain +------------------------------- +------------------------------------------------------------------------------- +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000008) +------------------------------------------------------------------------------- +|-eb_0000000000000_0000_000008 + |-eb_0000000000000_0000_000007 + |-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000003 +------------------------------------------------------------------------------- +Order of Execution +------------------------------------------------------------------------------- +1: eb_0000000000000_0000_000003 +2: eb_0000000000000_0000_000006 +3: eb_0000000000000_0000_000007 +4: eb_0000000000000_0000_000008 +------------------------------------------------------------------------------- + +======================================================= +Block Id: eb_0000000000000_0000_000003 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 3 => 6 (type=HASH_SHUFFLE) + +[Enforcers] + 0: type=Broadcast, tables=default.r + +GROUP_BY(16)(r_regionkey) + => exprs: (count()) + => target list: default.r.r_regionkey (INT4), ?count_4 (INT8) + => out schema:{(2) default.r.r_regionkey (INT4), ?count_4 (INT8)} + => in schema:{(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4)} + JOIN(11)(INNER) + => Join Cond: default.n.n_regionkey (INT4) = default.r.r_regionkey (INT4) + => target list: default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4) + => out schema: {(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4)} + => in schema: {(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4)} + SCAN(2) on default.region as r + => target list: default.r.r_regionkey (INT4) + => out schema: {(1) default.r.r_regionkey (INT4)} + => in schema: {(3) default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4)} + SCAN(1) on default.nation as n + => target list: default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) + => out schema: {(3) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + => in schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000006 [INTERMEDIATE] +======================================================= + +[Incoming] +[q_0000000000000_0000] 3 => 6 (type=HASH_SHUFFLE) + +[Outgoing] +[q_0000000000000_0000] 6 => 7 (type=RANGE_SHUFFLE, key=default.n.n_nationkey (INT4), num=32) + +[Enforcers] + 0: type=Broadcast, tables=default.n + +SORT(20) + => Sort Keys: default.n.n_nationkey (INT4) (asc) + JOIN(12)(INNER) + => Join Cond: default.n.n_regionkey (INT4) = default.t.r_regionkey (INT4) + => target list: default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.t.cnt (INT8) + => out schema: {(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.t.cnt (INT8)} + => in schema: {(5) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.t.cnt (INT8), default.t.r_regionkey (INT4)} + SCAN(0) on default.nation as n + => target list: default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) + => out schema: {(3) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + => in schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + TABLE_SUBQUERY(6) as default.t + => Targets: default.t.cnt (INT8), default.t.r_regionkey (INT4) + => out schema: {(2) default.t.cnt (INT8), default.t.r_regionkey (INT4)} + => in schema: {(2) default.t.cnt (INT8), default.t.r_regionkey (INT4)} + PROJECTION(5) + => Targets: default.r.r_regionkey (INT4), cnt (INT8) + => out schema: {(2) cnt (INT8), default.r.r_regionkey (INT4)} + => in schema: {(2) cnt (INT8), default.r.r_regionkey (INT4)} + GROUP_BY(4)(r_regionkey) + => exprs: (count(?count_4 (INT8))) + => target list: default.r.r_regionkey (INT4), cnt (INT8) + => out schema:{(2) cnt (INT8), default.r.r_regionkey (INT4)} + => in schema:{(2) default.r.r_regionkey (INT4), ?count_4 (INT8)} + SCAN(17) on eb_0000000000000_0000_000003 + => out schema: {(2) default.r.r_regionkey (INT4), ?count_4 (INT8)} + => in schema: {(2) default.r.r_regionkey (INT4), ?count_4 (INT8)} + +======================================================= +Block Id: eb_0000000000000_0000_000007 [ROOT] +======================================================= + +[Incoming] +[q_0000000000000_0000] 6 => 7 (type=RANGE_SHUFFLE, key=default.n.n_nationkey (INT4), num=32) + +[Enforcers] + 0: sorted input=eb_0000000000000_0000_000006 + +SORT(8) + => Sort Keys: default.n.n_nationkey (INT4) (asc) + SCAN(21) on eb_0000000000000_0000_000006 + => out schema: {(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.t.cnt (INT8)} + => in schema: {(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.t.cnt (INT8)} + +======================================================= +Block Id: eb_0000000000000_0000_000008 [TERMINAL] +======================================================= diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testThetaJoinKeyPairs.Hash_NoBroadcast.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testThetaJoinKeyPairs.Hash_NoBroadcast.plan new file mode 100644 index 0000000000..6509afc064 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testThetaJoinKeyPairs.Hash_NoBroadcast.plan @@ -0,0 +1,196 @@ +explain +------------------------------- +SORT(8) + => Sort Keys: default.n.n_nationkey (INT4) (asc) + JOIN(12)(INNER) + => Join Cond: default.n.n_regionkey (INT4) = default.t.r_regionkey (INT4) + => target list: default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.t.cnt (INT8) + => out schema: {(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.t.cnt (INT8)} + => in schema: {(5) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.t.cnt (INT8), default.t.r_regionkey (INT4)} + SCAN(0) on default.nation as n + => target list: default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) + => out schema: {(3) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + => in schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + TABLE_SUBQUERY(6) as default.t + => Targets: default.t.cnt (INT8), default.t.r_regionkey (INT4) + => out schema: {(2) default.t.cnt (INT8), default.t.r_regionkey (INT4)} + => in schema: {(2) default.t.cnt (INT8), default.t.r_regionkey (INT4)} + GROUP_BY(4)(r_regionkey) + => exprs: (count()) + => target list: default.r.r_regionkey (INT4), cnt (INT8) + => out schema:{(2) cnt (INT8), default.r.r_regionkey (INT4)} + => in schema:{(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4)} + JOIN(11)(INNER) + => Join Cond: default.n.n_regionkey (INT4) = default.r.r_regionkey (INT4) + => target list: default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4) + => out schema: {(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4)} + => in schema: {(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4)} + SCAN(2) on default.region as r + => target list: default.r.r_regionkey (INT4) + => out schema: {(1) default.r.r_regionkey (INT4)} + => in schema: {(3) default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4)} + SCAN(1) on default.nation as n + => target list: default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) + => out schema: {(3) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + => in schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} +explain +------------------------------- +------------------------------------------------------------------------------- +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000008) +------------------------------------------------------------------------------- +|-eb_0000000000000_0000_000008 + |-eb_0000000000000_0000_000007 + |-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000005 + |-eb_0000000000000_0000_000004 + |-eb_0000000000000_0000_000003 + |-eb_0000000000000_0000_000002 + |-eb_0000000000000_0000_000001 +------------------------------------------------------------------------------- +Order of Execution +------------------------------------------------------------------------------- +1: eb_0000000000000_0000_000001 +2: eb_0000000000000_0000_000002 +3: eb_0000000000000_0000_000003 +4: eb_0000000000000_0000_000004 +5: eb_0000000000000_0000_000005 +6: eb_0000000000000_0000_000006 +7: eb_0000000000000_0000_000007 +8: eb_0000000000000_0000_000008 +------------------------------------------------------------------------------- + +======================================================= +Block Id: eb_0000000000000_0000_000001 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.n.n_regionkey (INT4), num=32) + +SCAN(1) on default.nation as n + => target list: default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) + => out schema: {(3) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + => in schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000002 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.r.r_regionkey (INT4), num=32) + +SCAN(2) on default.region as r + => target list: default.r.r_regionkey (INT4) + => out schema: {(1) default.r.r_regionkey (INT4)} + => in schema: {(3) default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000003 [INTERMEDIATE] +======================================================= + +[Incoming] +[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.n.n_regionkey (INT4), num=32) +[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.r.r_regionkey (INT4), num=32) + +[Outgoing] +[q_0000000000000_0000] 3 => 4 (type=HASH_SHUFFLE, key=default.r.r_regionkey (INT4), num=32) + +GROUP_BY(16)(r_regionkey) + => exprs: (count()) + => target list: default.r.r_regionkey (INT4), ?count_4 (INT8) + => out schema:{(2) default.r.r_regionkey (INT4), ?count_4 (INT8)} + => in schema:{(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4)} + JOIN(11)(INNER) + => Join Cond: default.n.n_regionkey (INT4) = default.r.r_regionkey (INT4) + => target list: default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4) + => out schema: {(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4)} + => in schema: {(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4)} + SCAN(15) on eb_0000000000000_0000_000002 + => out schema: {(1) default.r.r_regionkey (INT4)} + => in schema: {(1) default.r.r_regionkey (INT4)} + SCAN(14) on eb_0000000000000_0000_000001 + => out schema: {(3) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + => in schema: {(3) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000004 [INTERMEDIATE] +======================================================= + +[Incoming] +[q_0000000000000_0000] 3 => 4 (type=HASH_SHUFFLE, key=default.r.r_regionkey (INT4), num=32) + +[Outgoing] +[q_0000000000000_0000] 4 => 6 (type=HASH_SHUFFLE, key=default.t.r_regionkey (INT4), num=32) + +TABLE_SUBQUERY(6) as default.t + => Targets: default.t.cnt (INT8), default.t.r_regionkey (INT4) + => out schema: {(2) default.t.cnt (INT8), default.t.r_regionkey (INT4)} + => in schema: {(2) default.t.cnt (INT8), default.t.r_regionkey (INT4)} + PROJECTION(5) + => Targets: default.r.r_regionkey (INT4), cnt (INT8) + => out schema: {(2) cnt (INT8), default.r.r_regionkey (INT4)} + => in schema: {(2) cnt (INT8), default.r.r_regionkey (INT4)} + GROUP_BY(4)(r_regionkey) + => exprs: (count(?count_4 (INT8))) + => target list: default.r.r_regionkey (INT4), cnt (INT8) + => out schema:{(2) cnt (INT8), default.r.r_regionkey (INT4)} + => in schema:{(2) default.r.r_regionkey (INT4), ?count_4 (INT8)} + SCAN(17) on eb_0000000000000_0000_000003 + => out schema: {(2) default.r.r_regionkey (INT4), ?count_4 (INT8)} + => in schema: {(2) default.r.r_regionkey (INT4), ?count_4 (INT8)} + +======================================================= +Block Id: eb_0000000000000_0000_000005 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 5 => 6 (type=HASH_SHUFFLE, key=default.n.n_regionkey (INT4), num=32) + +SCAN(0) on default.nation as n + => target list: default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) + => out schema: {(3) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + => in schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000006 [INTERMEDIATE] +======================================================= + +[Incoming] +[q_0000000000000_0000] 4 => 6 (type=HASH_SHUFFLE, key=default.t.r_regionkey (INT4), num=32) +[q_0000000000000_0000] 5 => 6 (type=HASH_SHUFFLE, key=default.n.n_regionkey (INT4), num=32) + +[Outgoing] +[q_0000000000000_0000] 6 => 7 (type=RANGE_SHUFFLE, key=default.n.n_nationkey (INT4), num=32) + +SORT(20) + => Sort Keys: default.n.n_nationkey (INT4) (asc) + JOIN(12)(INNER) + => Join Cond: default.n.n_regionkey (INT4) = default.t.r_regionkey (INT4) + => target list: default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.t.cnt (INT8) + => out schema: {(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.t.cnt (INT8)} + => in schema: {(5) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.t.cnt (INT8), default.t.r_regionkey (INT4)} + SCAN(19) on eb_0000000000000_0000_000005 + => out schema: {(3) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + => in schema: {(3) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + SCAN(18) on eb_0000000000000_0000_000004 + => out schema: {(2) default.t.cnt (INT8), default.t.r_regionkey (INT4)} + => in schema: {(2) default.t.cnt (INT8), default.t.r_regionkey (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000007 [ROOT] +======================================================= + +[Incoming] +[q_0000000000000_0000] 6 => 7 (type=RANGE_SHUFFLE, key=default.n.n_nationkey (INT4), num=32) + +[Enforcers] + 0: sorted input=eb_0000000000000_0000_000006 + +SORT(8) + => Sort Keys: default.n.n_nationkey (INT4) (asc) + SCAN(21) on eb_0000000000000_0000_000006 + => out schema: {(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.t.cnt (INT8)} + => in schema: {(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.t.cnt (INT8)} + +======================================================= +Block Id: eb_0000000000000_0000_000008 [TERMINAL] +======================================================= diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testThetaJoinKeyPairs.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testThetaJoinKeyPairs.Sort.plan new file mode 100644 index 0000000000..8ca7e29eb2 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testThetaJoinKeyPairs.Sort.plan @@ -0,0 +1,142 @@ +explain +------------------------------- +SORT(8) + => Sort Keys: default.n.n_nationkey (INT4) (asc) + JOIN(12)(INNER) + => Join Cond: default.n.n_regionkey (INT4) = default.t.r_regionkey (INT4) + => target list: default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.t.cnt (INT8) + => out schema: {(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.t.cnt (INT8)} + => in schema: {(5) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.t.cnt (INT8), default.t.r_regionkey (INT4)} + SCAN(0) on default.nation as n + => target list: default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) + => out schema: {(3) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + => in schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + TABLE_SUBQUERY(6) as default.t + => Targets: default.t.cnt (INT8), default.t.r_regionkey (INT4) + => out schema: {(2) default.t.cnt (INT8), default.t.r_regionkey (INT4)} + => in schema: {(2) default.t.cnt (INT8), default.t.r_regionkey (INT4)} + GROUP_BY(4)(r_regionkey) + => exprs: (count()) + => target list: default.r.r_regionkey (INT4), cnt (INT8) + => out schema:{(2) cnt (INT8), default.r.r_regionkey (INT4)} + => in schema:{(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4)} + JOIN(11)(INNER) + => Join Cond: default.n.n_regionkey (INT4) = default.r.r_regionkey (INT4) + => target list: default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4) + => out schema: {(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4)} + => in schema: {(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4)} + SCAN(2) on default.region as r + => target list: default.r.r_regionkey (INT4) + => out schema: {(1) default.r.r_regionkey (INT4)} + => in schema: {(3) default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4)} + SCAN(1) on default.nation as n + => target list: default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) + => out schema: {(3) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + => in schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} +explain +------------------------------- +------------------------------------------------------------------------------- +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000008) +------------------------------------------------------------------------------- +|-eb_0000000000000_0000_000008 + |-eb_0000000000000_0000_000007 + |-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000003 +------------------------------------------------------------------------------- +Order of Execution +------------------------------------------------------------------------------- +1: eb_0000000000000_0000_000003 +2: eb_0000000000000_0000_000006 +3: eb_0000000000000_0000_000007 +4: eb_0000000000000_0000_000008 +------------------------------------------------------------------------------- + +======================================================= +Block Id: eb_0000000000000_0000_000003 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 3 => 6 (type=HASH_SHUFFLE) + +[Enforcers] + 0: type=Broadcast, tables=default.r + +GROUP_BY(16)(r_regionkey) + => exprs: (count()) + => target list: default.r.r_regionkey (INT4), ?count_4 (INT8) + => out schema:{(2) default.r.r_regionkey (INT4), ?count_4 (INT8)} + => in schema:{(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4)} + JOIN(11)(INNER) + => Join Cond: default.n.n_regionkey (INT4) = default.r.r_regionkey (INT4) + => target list: default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4) + => out schema: {(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4)} + => in schema: {(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4)} + SCAN(2) on default.region as r + => target list: default.r.r_regionkey (INT4) + => out schema: {(1) default.r.r_regionkey (INT4)} + => in schema: {(3) default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4)} + SCAN(1) on default.nation as n + => target list: default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) + => out schema: {(3) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + => in schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000006 [INTERMEDIATE] +======================================================= + +[Incoming] +[q_0000000000000_0000] 3 => 6 (type=HASH_SHUFFLE) + +[Outgoing] +[q_0000000000000_0000] 6 => 7 (type=RANGE_SHUFFLE, key=default.n.n_nationkey (INT4), num=32) + +[Enforcers] + 0: type=Broadcast, tables=default.n + +SORT(20) + => Sort Keys: default.n.n_nationkey (INT4) (asc) + JOIN(12)(INNER) + => Join Cond: default.n.n_regionkey (INT4) = default.t.r_regionkey (INT4) + => target list: default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.t.cnt (INT8) + => out schema: {(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.t.cnt (INT8)} + => in schema: {(5) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.t.cnt (INT8), default.t.r_regionkey (INT4)} + SCAN(0) on default.nation as n + => target list: default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) + => out schema: {(3) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + => in schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + TABLE_SUBQUERY(6) as default.t + => Targets: default.t.cnt (INT8), default.t.r_regionkey (INT4) + => out schema: {(2) default.t.cnt (INT8), default.t.r_regionkey (INT4)} + => in schema: {(2) default.t.cnt (INT8), default.t.r_regionkey (INT4)} + PROJECTION(5) + => Targets: default.r.r_regionkey (INT4), cnt (INT8) + => out schema: {(2) cnt (INT8), default.r.r_regionkey (INT4)} + => in schema: {(2) cnt (INT8), default.r.r_regionkey (INT4)} + GROUP_BY(4)(r_regionkey) + => exprs: (count(?count_4 (INT8))) + => target list: default.r.r_regionkey (INT4), cnt (INT8) + => out schema:{(2) cnt (INT8), default.r.r_regionkey (INT4)} + => in schema:{(2) default.r.r_regionkey (INT4), ?count_4 (INT8)} + SCAN(17) on eb_0000000000000_0000_000003 + => out schema: {(2) default.r.r_regionkey (INT4), ?count_4 (INT8)} + => in schema: {(2) default.r.r_regionkey (INT4), ?count_4 (INT8)} + +======================================================= +Block Id: eb_0000000000000_0000_000007 [ROOT] +======================================================= + +[Incoming] +[q_0000000000000_0000] 6 => 7 (type=RANGE_SHUFFLE, key=default.n.n_nationkey (INT4), num=32) + +[Enforcers] + 0: sorted input=eb_0000000000000_0000_000006 + +SORT(8) + => Sort Keys: default.n.n_nationkey (INT4) (asc) + SCAN(21) on eb_0000000000000_0000_000006 + => out schema: {(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.t.cnt (INT8)} + => in schema: {(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.t.cnt (INT8)} + +======================================================= +Block Id: eb_0000000000000_0000_000008 [TERMINAL] +======================================================= diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testThetaJoinKeyPairs.Sort_NoBroadcast.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testThetaJoinKeyPairs.Sort_NoBroadcast.plan new file mode 100644 index 0000000000..6509afc064 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testThetaJoinKeyPairs.Sort_NoBroadcast.plan @@ -0,0 +1,196 @@ +explain +------------------------------- +SORT(8) + => Sort Keys: default.n.n_nationkey (INT4) (asc) + JOIN(12)(INNER) + => Join Cond: default.n.n_regionkey (INT4) = default.t.r_regionkey (INT4) + => target list: default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.t.cnt (INT8) + => out schema: {(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.t.cnt (INT8)} + => in schema: {(5) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.t.cnt (INT8), default.t.r_regionkey (INT4)} + SCAN(0) on default.nation as n + => target list: default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) + => out schema: {(3) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + => in schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + TABLE_SUBQUERY(6) as default.t + => Targets: default.t.cnt (INT8), default.t.r_regionkey (INT4) + => out schema: {(2) default.t.cnt (INT8), default.t.r_regionkey (INT4)} + => in schema: {(2) default.t.cnt (INT8), default.t.r_regionkey (INT4)} + GROUP_BY(4)(r_regionkey) + => exprs: (count()) + => target list: default.r.r_regionkey (INT4), cnt (INT8) + => out schema:{(2) cnt (INT8), default.r.r_regionkey (INT4)} + => in schema:{(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4)} + JOIN(11)(INNER) + => Join Cond: default.n.n_regionkey (INT4) = default.r.r_regionkey (INT4) + => target list: default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4) + => out schema: {(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4)} + => in schema: {(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4)} + SCAN(2) on default.region as r + => target list: default.r.r_regionkey (INT4) + => out schema: {(1) default.r.r_regionkey (INT4)} + => in schema: {(3) default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4)} + SCAN(1) on default.nation as n + => target list: default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) + => out schema: {(3) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + => in schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} +explain +------------------------------- +------------------------------------------------------------------------------- +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000008) +------------------------------------------------------------------------------- +|-eb_0000000000000_0000_000008 + |-eb_0000000000000_0000_000007 + |-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000005 + |-eb_0000000000000_0000_000004 + |-eb_0000000000000_0000_000003 + |-eb_0000000000000_0000_000002 + |-eb_0000000000000_0000_000001 +------------------------------------------------------------------------------- +Order of Execution +------------------------------------------------------------------------------- +1: eb_0000000000000_0000_000001 +2: eb_0000000000000_0000_000002 +3: eb_0000000000000_0000_000003 +4: eb_0000000000000_0000_000004 +5: eb_0000000000000_0000_000005 +6: eb_0000000000000_0000_000006 +7: eb_0000000000000_0000_000007 +8: eb_0000000000000_0000_000008 +------------------------------------------------------------------------------- + +======================================================= +Block Id: eb_0000000000000_0000_000001 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.n.n_regionkey (INT4), num=32) + +SCAN(1) on default.nation as n + => target list: default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) + => out schema: {(3) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + => in schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000002 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.r.r_regionkey (INT4), num=32) + +SCAN(2) on default.region as r + => target list: default.r.r_regionkey (INT4) + => out schema: {(1) default.r.r_regionkey (INT4)} + => in schema: {(3) default.r.r_comment (TEXT), default.r.r_name (TEXT), default.r.r_regionkey (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000003 [INTERMEDIATE] +======================================================= + +[Incoming] +[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.n.n_regionkey (INT4), num=32) +[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.r.r_regionkey (INT4), num=32) + +[Outgoing] +[q_0000000000000_0000] 3 => 4 (type=HASH_SHUFFLE, key=default.r.r_regionkey (INT4), num=32) + +GROUP_BY(16)(r_regionkey) + => exprs: (count()) + => target list: default.r.r_regionkey (INT4), ?count_4 (INT8) + => out schema:{(2) default.r.r_regionkey (INT4), ?count_4 (INT8)} + => in schema:{(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4)} + JOIN(11)(INNER) + => Join Cond: default.n.n_regionkey (INT4) = default.r.r_regionkey (INT4) + => target list: default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4) + => out schema: {(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4)} + => in schema: {(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.r.r_regionkey (INT4)} + SCAN(15) on eb_0000000000000_0000_000002 + => out schema: {(1) default.r.r_regionkey (INT4)} + => in schema: {(1) default.r.r_regionkey (INT4)} + SCAN(14) on eb_0000000000000_0000_000001 + => out schema: {(3) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + => in schema: {(3) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000004 [INTERMEDIATE] +======================================================= + +[Incoming] +[q_0000000000000_0000] 3 => 4 (type=HASH_SHUFFLE, key=default.r.r_regionkey (INT4), num=32) + +[Outgoing] +[q_0000000000000_0000] 4 => 6 (type=HASH_SHUFFLE, key=default.t.r_regionkey (INT4), num=32) + +TABLE_SUBQUERY(6) as default.t + => Targets: default.t.cnt (INT8), default.t.r_regionkey (INT4) + => out schema: {(2) default.t.cnt (INT8), default.t.r_regionkey (INT4)} + => in schema: {(2) default.t.cnt (INT8), default.t.r_regionkey (INT4)} + PROJECTION(5) + => Targets: default.r.r_regionkey (INT4), cnt (INT8) + => out schema: {(2) cnt (INT8), default.r.r_regionkey (INT4)} + => in schema: {(2) cnt (INT8), default.r.r_regionkey (INT4)} + GROUP_BY(4)(r_regionkey) + => exprs: (count(?count_4 (INT8))) + => target list: default.r.r_regionkey (INT4), cnt (INT8) + => out schema:{(2) cnt (INT8), default.r.r_regionkey (INT4)} + => in schema:{(2) default.r.r_regionkey (INT4), ?count_4 (INT8)} + SCAN(17) on eb_0000000000000_0000_000003 + => out schema: {(2) default.r.r_regionkey (INT4), ?count_4 (INT8)} + => in schema: {(2) default.r.r_regionkey (INT4), ?count_4 (INT8)} + +======================================================= +Block Id: eb_0000000000000_0000_000005 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 5 => 6 (type=HASH_SHUFFLE, key=default.n.n_regionkey (INT4), num=32) + +SCAN(0) on default.nation as n + => target list: default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4) + => out schema: {(3) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + => in schema: {(4) default.n.n_comment (TEXT), default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000006 [INTERMEDIATE] +======================================================= + +[Incoming] +[q_0000000000000_0000] 4 => 6 (type=HASH_SHUFFLE, key=default.t.r_regionkey (INT4), num=32) +[q_0000000000000_0000] 5 => 6 (type=HASH_SHUFFLE, key=default.n.n_regionkey (INT4), num=32) + +[Outgoing] +[q_0000000000000_0000] 6 => 7 (type=RANGE_SHUFFLE, key=default.n.n_nationkey (INT4), num=32) + +SORT(20) + => Sort Keys: default.n.n_nationkey (INT4) (asc) + JOIN(12)(INNER) + => Join Cond: default.n.n_regionkey (INT4) = default.t.r_regionkey (INT4) + => target list: default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.t.cnt (INT8) + => out schema: {(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.t.cnt (INT8)} + => in schema: {(5) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.t.cnt (INT8), default.t.r_regionkey (INT4)} + SCAN(19) on eb_0000000000000_0000_000005 + => out schema: {(3) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + => in schema: {(3) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4)} + SCAN(18) on eb_0000000000000_0000_000004 + => out schema: {(2) default.t.cnt (INT8), default.t.r_regionkey (INT4)} + => in schema: {(2) default.t.cnt (INT8), default.t.r_regionkey (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000007 [ROOT] +======================================================= + +[Incoming] +[q_0000000000000_0000] 6 => 7 (type=RANGE_SHUFFLE, key=default.n.n_nationkey (INT4), num=32) + +[Enforcers] + 0: sorted input=eb_0000000000000_0000_000006 + +SORT(8) + => Sort Keys: default.n.n_nationkey (INT4) (asc) + SCAN(21) on eb_0000000000000_0000_000006 + => out schema: {(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.t.cnt (INT8)} + => in schema: {(4) default.n.n_name (TEXT), default.n.n_nationkey (INT4), default.n.n_regionkey (INT4), default.t.cnt (INT8)} + +======================================================= +Block Id: eb_0000000000000_0000_000008 [TERMINAL] +======================================================= diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testThetaJoinKeyPairs.result b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testThetaJoinKeyPairs.result new file mode 100644 index 0000000000..f3a26c839d --- /dev/null +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testThetaJoinKeyPairs.result @@ -0,0 +1,27 @@ +n_nationkey,n_name,n_regionkey,cnt +------------------------------- +0,ALGERIA,0,5 +1,ARGENTINA,1,5 +2,BRAZIL,1,5 +3,CANADA,1,5 +4,EGYPT,4,5 +5,ETHIOPIA,0,5 +6,FRANCE,3,5 +7,GERMANY,3,5 +8,INDIA,2,5 +9,INDONESIA,2,5 +10,IRAN,4,5 +11,IRAQ,4,5 +12,JAPAN,2,5 +13,JORDAN,4,5 +14,KENYA,0,5 +15,MOROCCO,0,5 +16,MOZAMBIQUE,0,5 +17,PERU,1,5 +18,CHINA,2,5 +19,ROMANIA,3,5 +20,SAUDI ARABIA,4,5 +21,VIETNAM,2,5 +22,RUSSIA,3,5 +23,UNITED KINGDOM,3,5 +24,UNITED STATES,1,5 diff --git a/tajo-core/src/test/resources/results/TestSortQuery/testSubQuerySortAfterGroupMultiBlocks.plan b/tajo-core/src/test/resources/results/TestSortQuery/testSubQuerySortAfterGroupMultiBlocks.plan new file mode 100644 index 0000000000..17a76585da --- /dev/null +++ b/tajo-core/src/test/resources/results/TestSortQuery/testSubQuerySortAfterGroupMultiBlocks.plan @@ -0,0 +1,112 @@ +explain +------------------------------- +SORT(6) + => Sort Keys: default.l2.l_orderkey (INT4) (asc) + TABLE_SUBQUERY(5) as default.l2 + => Targets: default.l2.l_orderkey (INT4), default.l2.revenue (FLOAT8) + => out schema: {(2) default.l2.l_orderkey (INT4), default.l2.revenue (FLOAT8)} + => in schema: {(2) default.l2.l_orderkey (INT4), default.l2.revenue (FLOAT8)} + TABLE_SUBQUERY(3) as default.l1 + => Targets: default.l1.l_orderkey (INT4), default.l1.revenue (FLOAT8) + => out schema: {(2) default.l1.l_orderkey (INT4), default.l1.revenue (FLOAT8)} + => in schema: {(2) default.l1.l_orderkey (INT4), default.l1.revenue (FLOAT8)} + GROUP_BY(1)(l_orderkey) + => exprs: (sum(?multiply (FLOAT8))) + => target list: default.lineitem.l_orderkey (INT4), revenue (FLOAT8) + => out schema:{(2) default.lineitem.l_orderkey (INT4), revenue (FLOAT8)} + => in schema:{(4) ?multiply (FLOAT8), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_orderkey (INT4)} + SCAN(0) on default.lineitem + => target list: default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_orderkey (INT4), default.lineitem.l_extendedprice (FLOAT8) * default.lineitem.l_discount (FLOAT8) as ?multiply + => out schema: {(4) ?multiply (FLOAT8), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_orderkey (INT4)} + => in schema: {(16) default.lineitem.l_comment (TEXT), default.lineitem.l_commitdate (TEXT), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_linenumber (INT4), default.lineitem.l_linestatus (TEXT), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.lineitem.l_quantity (FLOAT8), default.lineitem.l_receiptdate (TEXT), default.lineitem.l_returnflag (TEXT), default.lineitem.l_shipdate (TEXT), default.lineitem.l_shipinstruct (TEXT), default.lineitem.l_shipmode (TEXT), default.lineitem.l_suppkey (INT4), default.lineitem.l_tax (FLOAT8)} +explain +------------------------------- +------------------------------------------------------------------------------- +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000004) +------------------------------------------------------------------------------- +|-eb_0000000000000_0000_000004 + |-eb_0000000000000_0000_000003 + |-eb_0000000000000_0000_000002 + |-eb_0000000000000_0000_000001 +------------------------------------------------------------------------------- +Order of Execution +------------------------------------------------------------------------------- +1: eb_0000000000000_0000_000001 +2: eb_0000000000000_0000_000002 +3: eb_0000000000000_0000_000003 +4: eb_0000000000000_0000_000004 +------------------------------------------------------------------------------- + +======================================================= +Block Id: eb_0000000000000_0000_000001 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 1 => 2 (type=HASH_SHUFFLE, key=default.lineitem.l_orderkey (INT4), num=32) + +GROUP_BY(10)(l_orderkey) + => exprs: (sum(?multiply (FLOAT8))) + => target list: default.lineitem.l_orderkey (INT4), ?sum_1 (FLOAT8) + => out schema:{(2) default.lineitem.l_orderkey (INT4), ?sum_1 (FLOAT8)} + => in schema:{(4) ?multiply (FLOAT8), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_orderkey (INT4)} + SCAN(0) on default.lineitem + => target list: default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_orderkey (INT4), default.lineitem.l_extendedprice (FLOAT8) * default.lineitem.l_discount (FLOAT8) as ?multiply + => out schema: {(4) ?multiply (FLOAT8), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_orderkey (INT4)} + => in schema: {(16) default.lineitem.l_comment (TEXT), default.lineitem.l_commitdate (TEXT), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_linenumber (INT4), default.lineitem.l_linestatus (TEXT), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.lineitem.l_quantity (FLOAT8), default.lineitem.l_receiptdate (TEXT), default.lineitem.l_returnflag (TEXT), default.lineitem.l_shipdate (TEXT), default.lineitem.l_shipinstruct (TEXT), default.lineitem.l_shipmode (TEXT), default.lineitem.l_suppkey (INT4), default.lineitem.l_tax (FLOAT8)} + +======================================================= +Block Id: eb_0000000000000_0000_000002 [INTERMEDIATE] +======================================================= + +[Incoming] +[q_0000000000000_0000] 1 => 2 (type=HASH_SHUFFLE, key=default.lineitem.l_orderkey (INT4), num=32) + +[Outgoing] +[q_0000000000000_0000] 2 => 3 (type=RANGE_SHUFFLE, key=default.l2.l_orderkey (INT4), num=32) + +SORT(12) + => Sort Keys: default.l2.l_orderkey (INT4) (asc) + TABLE_SUBQUERY(5) as default.l2 + => Targets: default.l2.l_orderkey (INT4), default.l2.revenue (FLOAT8) + => out schema: {(2) default.l2.l_orderkey (INT4), default.l2.revenue (FLOAT8)} + => in schema: {(2) default.l2.l_orderkey (INT4), default.l2.revenue (FLOAT8)} + PROJECTION(4) + => Targets: default.l1.l_orderkey (INT4), default.l1.revenue (FLOAT8) + => out schema: {(2) default.l1.l_orderkey (INT4), default.l1.revenue (FLOAT8)} + => in schema: {(2) default.l1.l_orderkey (INT4), default.l1.revenue (FLOAT8)} + TABLE_SUBQUERY(3) as default.l1 + => Targets: default.l1.l_orderkey (INT4), default.l1.revenue (FLOAT8) + => out schema: {(2) default.l1.l_orderkey (INT4), default.l1.revenue (FLOAT8)} + => in schema: {(2) default.l1.l_orderkey (INT4), default.l1.revenue (FLOAT8)} + PROJECTION(2) + => Targets: default.lineitem.l_orderkey (INT4), revenue (FLOAT8) + => out schema: {(2) default.lineitem.l_orderkey (INT4), revenue (FLOAT8)} + => in schema: {(2) default.lineitem.l_orderkey (INT4), revenue (FLOAT8)} + GROUP_BY(1)(l_orderkey) + => exprs: (sum(?sum_1 (FLOAT8))) + => target list: default.lineitem.l_orderkey (INT4), revenue (FLOAT8) + => out schema:{(2) default.lineitem.l_orderkey (INT4), revenue (FLOAT8)} + => in schema:{(2) default.lineitem.l_orderkey (INT4), ?sum_1 (FLOAT8)} + SCAN(11) on eb_0000000000000_0000_000001 + => out schema: {(2) default.lineitem.l_orderkey (INT4), ?sum_1 (FLOAT8)} + => in schema: {(2) default.lineitem.l_orderkey (INT4), ?sum_1 (FLOAT8)} + +======================================================= +Block Id: eb_0000000000000_0000_000003 [ROOT] +======================================================= + +[Incoming] +[q_0000000000000_0000] 2 => 3 (type=RANGE_SHUFFLE, key=default.l2.l_orderkey (INT4), num=32) + +[Enforcers] + 0: sorted input=eb_0000000000000_0000_000002 + +SORT(6) + => Sort Keys: default.l2.l_orderkey (INT4) (asc) + SCAN(13) on eb_0000000000000_0000_000002 + => out schema: {(2) default.l2.l_orderkey (INT4), default.l2.revenue (FLOAT8)} + => in schema: {(2) default.l2.l_orderkey (INT4), default.l2.revenue (FLOAT8)} + +======================================================= +Block Id: eb_0000000000000_0000_000004 [TERMINAL] +======================================================= diff --git a/tajo-core/src/test/resources/results/TestSortQuery/testSubQuerySortAfterGroupMultiBlocks.result b/tajo-core/src/test/resources/results/TestSortQuery/testSubQuerySortAfterGroupMultiBlocks.result new file mode 100644 index 0000000000..2b635266dd --- /dev/null +++ b/tajo-core/src/test/resources/results/TestSortQuery/testSubQuerySortAfterGroupMultiBlocks.result @@ -0,0 +1,5 @@ +l_orderkey,revenue +------------------------------- +1,4985.2136 +2,0.0 +3,7923.13 diff --git a/tajo-core/src/test/resources/results/TestTPCH/testQ1OrderBy.plan b/tajo-core/src/test/resources/results/TestTPCH/testQ1OrderBy.plan new file mode 100644 index 0000000000..4cf1bacfaf --- /dev/null +++ b/tajo-core/src/test/resources/results/TestTPCH/testQ1OrderBy.plan @@ -0,0 +1,88 @@ +explain +------------------------------- +SORT(2) + => Sort Keys: default.lineitem.l_returnflag (TEXT) (asc),default.lineitem.l_linestatus (TEXT) (asc) + GROUP_BY(1)(l_returnflag,l_linestatus) + => exprs: (count()) + => target list: default.lineitem.l_returnflag (TEXT), default.lineitem.l_linestatus (TEXT), count_order (INT8) + => out schema:{(3) count_order (INT8), default.lineitem.l_linestatus (TEXT), default.lineitem.l_returnflag (TEXT)} + => in schema:{(2) default.lineitem.l_linestatus (TEXT), default.lineitem.l_returnflag (TEXT)} + SCAN(0) on default.lineitem + => target list: default.lineitem.l_linestatus (TEXT), default.lineitem.l_returnflag (TEXT) + => out schema: {(2) default.lineitem.l_linestatus (TEXT), default.lineitem.l_returnflag (TEXT)} + => in schema: {(16) default.lineitem.l_comment (TEXT), default.lineitem.l_commitdate (TEXT), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_linenumber (INT4), default.lineitem.l_linestatus (TEXT), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.lineitem.l_quantity (FLOAT8), default.lineitem.l_receiptdate (TEXT), default.lineitem.l_returnflag (TEXT), default.lineitem.l_shipdate (TEXT), default.lineitem.l_shipinstruct (TEXT), default.lineitem.l_shipmode (TEXT), default.lineitem.l_suppkey (INT4), default.lineitem.l_tax (FLOAT8)} +explain +------------------------------- +------------------------------------------------------------------------------- +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000004) +------------------------------------------------------------------------------- +|-eb_0000000000000_0000_000004 + |-eb_0000000000000_0000_000003 + |-eb_0000000000000_0000_000002 + |-eb_0000000000000_0000_000001 +------------------------------------------------------------------------------- +Order of Execution +------------------------------------------------------------------------------- +1: eb_0000000000000_0000_000001 +2: eb_0000000000000_0000_000002 +3: eb_0000000000000_0000_000003 +4: eb_0000000000000_0000_000004 +------------------------------------------------------------------------------- + +======================================================= +Block Id: eb_0000000000000_0000_000001 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 1 => 2 (type=HASH_SHUFFLE, key=default.lineitem.l_linestatus (TEXT), default.lineitem.l_returnflag (TEXT), num=32) + +GROUP_BY(6)(l_linestatus,l_returnflag) + => exprs: (count()) + => target list: default.lineitem.l_returnflag (TEXT), default.lineitem.l_linestatus (TEXT), ?count (INT8) + => out schema:{(3) default.lineitem.l_returnflag (TEXT), default.lineitem.l_linestatus (TEXT), ?count (INT8)} + => in schema:{(2) default.lineitem.l_linestatus (TEXT), default.lineitem.l_returnflag (TEXT)} + SCAN(0) on default.lineitem + => target list: default.lineitem.l_linestatus (TEXT), default.lineitem.l_returnflag (TEXT) + => out schema: {(2) default.lineitem.l_linestatus (TEXT), default.lineitem.l_returnflag (TEXT)} + => in schema: {(16) default.lineitem.l_comment (TEXT), default.lineitem.l_commitdate (TEXT), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_linenumber (INT4), default.lineitem.l_linestatus (TEXT), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.lineitem.l_quantity (FLOAT8), default.lineitem.l_receiptdate (TEXT), default.lineitem.l_returnflag (TEXT), default.lineitem.l_shipdate (TEXT), default.lineitem.l_shipinstruct (TEXT), default.lineitem.l_shipmode (TEXT), default.lineitem.l_suppkey (INT4), default.lineitem.l_tax (FLOAT8)} + +======================================================= +Block Id: eb_0000000000000_0000_000002 [INTERMEDIATE] +======================================================= + +[Incoming] +[q_0000000000000_0000] 1 => 2 (type=HASH_SHUFFLE, key=default.lineitem.l_linestatus (TEXT), default.lineitem.l_returnflag (TEXT), num=32) + +[Outgoing] +[q_0000000000000_0000] 2 => 3 (type=RANGE_SHUFFLE, key=default.lineitem.l_linestatus (TEXT), default.lineitem.l_returnflag (TEXT), num=32) + +SORT(8) + => Sort Keys: default.lineitem.l_returnflag (TEXT) (asc),default.lineitem.l_linestatus (TEXT) (asc) + GROUP_BY(1)(l_returnflag,l_linestatus) + => exprs: (count(?count (INT8))) + => target list: default.lineitem.l_returnflag (TEXT), default.lineitem.l_linestatus (TEXT), count_order (INT8) + => out schema:{(3) count_order (INT8), default.lineitem.l_linestatus (TEXT), default.lineitem.l_returnflag (TEXT)} + => in schema:{(3) default.lineitem.l_returnflag (TEXT), default.lineitem.l_linestatus (TEXT), ?count (INT8)} + SCAN(7) on eb_0000000000000_0000_000001 + => out schema: {(3) default.lineitem.l_returnflag (TEXT), default.lineitem.l_linestatus (TEXT), ?count (INT8)} + => in schema: {(3) default.lineitem.l_returnflag (TEXT), default.lineitem.l_linestatus (TEXT), ?count (INT8)} + +======================================================= +Block Id: eb_0000000000000_0000_000003 [ROOT] +======================================================= + +[Incoming] +[q_0000000000000_0000] 2 => 3 (type=RANGE_SHUFFLE, key=default.lineitem.l_linestatus (TEXT), default.lineitem.l_returnflag (TEXT), num=32) + +[Enforcers] + 0: sorted input=eb_0000000000000_0000_000002 + +SORT(2) + => Sort Keys: default.lineitem.l_returnflag (TEXT) (asc),default.lineitem.l_linestatus (TEXT) (asc) + SCAN(9) on eb_0000000000000_0000_000002 + => out schema: {(3) count_order (INT8), default.lineitem.l_linestatus (TEXT), default.lineitem.l_returnflag (TEXT)} + => in schema: {(3) count_order (INT8), default.lineitem.l_linestatus (TEXT), default.lineitem.l_returnflag (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000004 [TERMINAL] +======================================================= diff --git a/tajo-core/src/test/resources/results/TestTPCH/testQ1OrderBy.result b/tajo-core/src/test/resources/results/TestTPCH/testQ1OrderBy.result index 94b5e84ea7..a34b5144b8 100644 --- a/tajo-core/src/test/resources/results/TestTPCH/testQ1OrderBy.result +++ b/tajo-core/src/test/resources/results/TestTPCH/testQ1OrderBy.result @@ -1,4 +1,4 @@ l_returnflag,l_linestatus,count_order ------------------------------- N,O,3 -R,F,2 \ No newline at end of file +R,F,2 diff --git a/tajo-core/src/test/resources/results/TestTPCH/testQ2FourJoins.plan b/tajo-core/src/test/resources/results/TestTPCH/testQ2FourJoins.plan new file mode 100644 index 0000000000..a2beae2ce2 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestTPCH/testQ2FourJoins.plan @@ -0,0 +1,228 @@ +explain +------------------------------- +JOIN(14)(INNER) + => Join Cond: default.supplier.s_nationkey (INT4) = default.nation.n_nationkey (INT4) + => target list: default.nation.n_name (TEXT), default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.part.p_size (INT4), default.part.p_type (TEXT), default.partsupp.ps_supplycost (FLOAT8), default.region.r_name (TEXT), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_phone (TEXT) + => out schema: {(12) default.nation.n_name (TEXT), default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.part.p_size (INT4), default.part.p_type (TEXT), default.partsupp.ps_supplycost (FLOAT8), default.region.r_name (TEXT), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_phone (TEXT)} + => in schema: {(14) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.part.p_size (INT4), default.part.p_type (TEXT), default.partsupp.ps_supplycost (FLOAT8), default.region.r_name (TEXT), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT)} + JOIN(13)(INNER) + => Join Cond: default.nation.n_regionkey (INT4) = default.region.r_regionkey (INT4) + => target list: default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.region.r_name (TEXT) + => out schema: {(3) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.region.r_name (TEXT)} + => in schema: {(5) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} + SCAN(0) on default.region + => filter: default.region.r_name (TEXT) = AMERICA + => target list: default.region.r_name (TEXT), default.region.r_regionkey (INT4) + => out schema: {(2) default.region.r_name (TEXT), default.region.r_regionkey (INT4)} + => in schema: {(3) default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} + SCAN(1) on default.nation + => target list: default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4) + => out schema: {(3) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + JOIN(12)(INNER) + => Join Cond: default.part.p_partkey (INT4) = default.partsupp.ps_partkey (INT4) + => target list: default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.part.p_size (INT4), default.part.p_type (TEXT), default.partsupp.ps_supplycost (FLOAT8), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT) + => out schema: {(11) default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.part.p_size (INT4), default.part.p_type (TEXT), default.partsupp.ps_supplycost (FLOAT8), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT)} + => in schema: {(12) default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.part.p_size (INT4), default.part.p_type (TEXT), default.partsupp.ps_partkey (INT4), default.partsupp.ps_supplycost (FLOAT8), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT)} + SCAN(7) on default.part + => filter: default.part.p_size (INT4) = 15 AND default.part.p_type (TEXT)LIKE'%BRASS' + => target list: default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.part.p_size (INT4), default.part.p_type (TEXT) + => out schema: {(4) default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.part.p_size (INT4), default.part.p_type (TEXT)} + => in schema: {(9) default.part.p_brand (TEXT), default.part.p_comment (TEXT), default.part.p_container (TEXT), default.part.p_mfgr (TEXT), default.part.p_name (TEXT), default.part.p_partkey (INT4), default.part.p_retailprice (FLOAT8), default.part.p_size (INT4), default.part.p_type (TEXT)} + JOIN(11)(INNER) + => Join Cond: default.supplier.s_suppkey (INT4) = default.partsupp.ps_suppkey (INT4) + => target list: default.partsupp.ps_partkey (INT4), default.partsupp.ps_supplycost (FLOAT8), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT) + => out schema: {(8) default.partsupp.ps_partkey (INT4), default.partsupp.ps_supplycost (FLOAT8), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT)} + => in schema: {(10) default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4), default.partsupp.ps_supplycost (FLOAT8), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} + SCAN(5) on default.partsupp + => target list: default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4), default.partsupp.ps_supplycost (FLOAT8) + => out schema: {(3) default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4), default.partsupp.ps_supplycost (FLOAT8)} + => in schema: {(5) default.partsupp.ps_availqty (INT4), default.partsupp.ps_comment (TEXT), default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4), default.partsupp.ps_supplycost (FLOAT8)} + SCAN(3) on default.supplier + => target list: default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4) + => out schema: {(7) default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} + => in schema: {(7) default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} +explain +------------------------------- +------------------------------------------------------------------------------- +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000010) +------------------------------------------------------------------------------- +|-eb_0000000000000_0000_000010 + |-eb_0000000000000_0000_000009 + |-eb_0000000000000_0000_000008 + |-eb_0000000000000_0000_000007 + |-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000005 + |-eb_0000000000000_0000_000004 + |-eb_0000000000000_0000_000003 + |-eb_0000000000000_0000_000002 + |-eb_0000000000000_0000_000001 +------------------------------------------------------------------------------- +Order of Execution +------------------------------------------------------------------------------- +1: eb_0000000000000_0000_000001 +2: eb_0000000000000_0000_000002 +3: eb_0000000000000_0000_000003 +4: eb_0000000000000_0000_000004 +5: eb_0000000000000_0000_000006 +6: eb_0000000000000_0000_000007 +7: eb_0000000000000_0000_000005 +8: eb_0000000000000_0000_000008 +9: eb_0000000000000_0000_000009 +10: eb_0000000000000_0000_000010 +------------------------------------------------------------------------------- + +======================================================= +Block Id: eb_0000000000000_0000_000001 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.supplier.s_suppkey (INT4), num=32) + +SCAN(3) on default.supplier + => target list: default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4) + => out schema: {(7) default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} + => in schema: {(7) default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000002 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.partsupp.ps_suppkey (INT4), num=32) + +SCAN(5) on default.partsupp + => target list: default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4), default.partsupp.ps_supplycost (FLOAT8) + => out schema: {(3) default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4), default.partsupp.ps_supplycost (FLOAT8)} + => in schema: {(5) default.partsupp.ps_availqty (INT4), default.partsupp.ps_comment (TEXT), default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4), default.partsupp.ps_supplycost (FLOAT8)} + +======================================================= +Block Id: eb_0000000000000_0000_000003 [INTERMEDIATE] +======================================================= + +[Incoming] +[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.supplier.s_suppkey (INT4), num=32) +[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.partsupp.ps_suppkey (INT4), num=32) + +[Outgoing] +[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.partsupp.ps_partkey (INT4), num=32) + +JOIN(11)(INNER) + => Join Cond: default.supplier.s_suppkey (INT4) = default.partsupp.ps_suppkey (INT4) + => target list: default.partsupp.ps_partkey (INT4), default.partsupp.ps_supplycost (FLOAT8), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT) + => out schema: {(8) default.partsupp.ps_partkey (INT4), default.partsupp.ps_supplycost (FLOAT8), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT)} + => in schema: {(10) default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4), default.partsupp.ps_supplycost (FLOAT8), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} + SCAN(17) on eb_0000000000000_0000_000002 + => out schema: {(3) default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4), default.partsupp.ps_supplycost (FLOAT8)} + => in schema: {(3) default.partsupp.ps_partkey (INT4), default.partsupp.ps_suppkey (INT4), default.partsupp.ps_supplycost (FLOAT8)} + SCAN(16) on eb_0000000000000_0000_000001 + => out schema: {(7) default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} + => in schema: {(7) default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000004 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.part.p_partkey (INT4), num=32) + +SCAN(7) on default.part + => filter: default.part.p_size (INT4) = 15 AND default.part.p_type (TEXT)LIKE'%BRASS' + => target list: default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.part.p_size (INT4), default.part.p_type (TEXT) + => out schema: {(4) default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.part.p_size (INT4), default.part.p_type (TEXT)} + => in schema: {(9) default.part.p_brand (TEXT), default.part.p_comment (TEXT), default.part.p_container (TEXT), default.part.p_mfgr (TEXT), default.part.p_name (TEXT), default.part.p_partkey (INT4), default.part.p_retailprice (FLOAT8), default.part.p_size (INT4), default.part.p_type (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000005 [INTERMEDIATE] +======================================================= + +[Incoming] +[q_0000000000000_0000] 3 => 5 (type=HASH_SHUFFLE, key=default.partsupp.ps_partkey (INT4), num=32) +[q_0000000000000_0000] 4 => 5 (type=HASH_SHUFFLE, key=default.part.p_partkey (INT4), num=32) + +[Outgoing] +[q_0000000000000_0000] 5 => 9 (type=HASH_SHUFFLE, key=default.supplier.s_nationkey (INT4), num=32) + +JOIN(12)(INNER) + => Join Cond: default.part.p_partkey (INT4) = default.partsupp.ps_partkey (INT4) + => target list: default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.part.p_size (INT4), default.part.p_type (TEXT), default.partsupp.ps_supplycost (FLOAT8), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT) + => out schema: {(11) default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.part.p_size (INT4), default.part.p_type (TEXT), default.partsupp.ps_supplycost (FLOAT8), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT)} + => in schema: {(12) default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.part.p_size (INT4), default.part.p_type (TEXT), default.partsupp.ps_partkey (INT4), default.partsupp.ps_supplycost (FLOAT8), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT)} + SCAN(19) on eb_0000000000000_0000_000004 + => out schema: {(4) default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.part.p_size (INT4), default.part.p_type (TEXT)} + => in schema: {(4) default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.part.p_size (INT4), default.part.p_type (TEXT)} + SCAN(18) on eb_0000000000000_0000_000003 + => out schema: {(8) default.partsupp.ps_partkey (INT4), default.partsupp.ps_supplycost (FLOAT8), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT)} + => in schema: {(8) default.partsupp.ps_partkey (INT4), default.partsupp.ps_supplycost (FLOAT8), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000006 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 6 => 8 (type=HASH_SHUFFLE, key=default.nation.n_regionkey (INT4), num=32) + +SCAN(1) on default.nation + => target list: default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4) + => out schema: {(3) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000007 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 7 => 8 (type=HASH_SHUFFLE, key=default.region.r_regionkey (INT4), num=32) + +SCAN(0) on default.region + => filter: default.region.r_name (TEXT) = AMERICA + => target list: default.region.r_name (TEXT), default.region.r_regionkey (INT4) + => out schema: {(2) default.region.r_name (TEXT), default.region.r_regionkey (INT4)} + => in schema: {(3) default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000008 [INTERMEDIATE] +======================================================= + +[Incoming] +[q_0000000000000_0000] 6 => 8 (type=HASH_SHUFFLE, key=default.nation.n_regionkey (INT4), num=32) +[q_0000000000000_0000] 7 => 8 (type=HASH_SHUFFLE, key=default.region.r_regionkey (INT4), num=32) + +[Outgoing] +[q_0000000000000_0000] 8 => 9 (type=HASH_SHUFFLE, key=default.nation.n_nationkey (INT4), num=32) + +JOIN(13)(INNER) + => Join Cond: default.nation.n_regionkey (INT4) = default.region.r_regionkey (INT4) + => target list: default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.region.r_name (TEXT) + => out schema: {(3) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.region.r_name (TEXT)} + => in schema: {(5) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} + SCAN(21) on eb_0000000000000_0000_000007 + => out schema: {(2) default.region.r_name (TEXT), default.region.r_regionkey (INT4)} + => in schema: {(2) default.region.r_name (TEXT), default.region.r_regionkey (INT4)} + SCAN(20) on eb_0000000000000_0000_000006 + => out schema: {(3) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + => in schema: {(3) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000009 [ROOT] +======================================================= + +[Incoming] +[q_0000000000000_0000] 5 => 9 (type=HASH_SHUFFLE, key=default.supplier.s_nationkey (INT4), num=32) +[q_0000000000000_0000] 8 => 9 (type=HASH_SHUFFLE, key=default.nation.n_nationkey (INT4), num=32) + +JOIN(14)(INNER) + => Join Cond: default.supplier.s_nationkey (INT4) = default.nation.n_nationkey (INT4) + => target list: default.nation.n_name (TEXT), default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.part.p_size (INT4), default.part.p_type (TEXT), default.partsupp.ps_supplycost (FLOAT8), default.region.r_name (TEXT), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_phone (TEXT) + => out schema: {(12) default.nation.n_name (TEXT), default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.part.p_size (INT4), default.part.p_type (TEXT), default.partsupp.ps_supplycost (FLOAT8), default.region.r_name (TEXT), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_phone (TEXT)} + => in schema: {(14) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.part.p_size (INT4), default.part.p_type (TEXT), default.partsupp.ps_supplycost (FLOAT8), default.region.r_name (TEXT), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT)} + SCAN(23) on eb_0000000000000_0000_000008 + => out schema: {(3) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.region.r_name (TEXT)} + => in schema: {(3) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.region.r_name (TEXT)} + SCAN(22) on eb_0000000000000_0000_000005 + => out schema: {(11) default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.part.p_size (INT4), default.part.p_type (TEXT), default.partsupp.ps_supplycost (FLOAT8), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT)} + => in schema: {(11) default.part.p_mfgr (TEXT), default.part.p_partkey (INT4), default.part.p_size (INT4), default.part.p_type (TEXT), default.partsupp.ps_supplycost (FLOAT8), default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000010 [TERMINAL] +======================================================= diff --git a/tajo-core/src/test/resources/results/TestTPCH/testQ2FourJoins.result b/tajo-core/src/test/resources/results/TestTPCH/testQ2FourJoins.result index 7c81978c0b..64d8e12272 100644 --- a/tajo-core/src/test/resources/results/TestTPCH/testQ2FourJoins.result +++ b/tajo-core/src/test/resources/results/TestTPCH/testQ2FourJoins.result @@ -1,3 +1,3 @@ s_acctbal,s_name,n_name,p_partkey,p_mfgr,s_address,s_phone,s_comment,ps_supplycost,r_name,p_type,p_size ------------------------------- -4192.4,Supplier#000000003,ARGENTINA,2,Manufacturer#1,q1,G3Pj6OjIuUYfUoH18BFTKP5aU9bEV3,11-383-516-1199,blithely silent requests after the express dependencies are sl,1.01,AMERICA,LARGE BRUSHED BRASS,15 \ No newline at end of file +4192.4,Supplier#000000003,ARGENTINA,2,Manufacturer#1,q1,G3Pj6OjIuUYfUoH18BFTKP5aU9bEV3,11-383-516-1199,blithely silent requests after the express dependencies are sl,1.01,AMERICA,LARGE BRUSHED BRASS,15 diff --git a/tajo-core/src/test/resources/results/TestTPCH/testTPCH14Expr.plan b/tajo-core/src/test/resources/results/TestTPCH/testTPCH14Expr.plan new file mode 100644 index 0000000000..8a716ebc30 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestTPCH/testTPCH14Expr.plan @@ -0,0 +1,119 @@ +explain +------------------------------- +PROJECTION(3) + => Targets: 100.0 * ?sum_1 (FLOAT8) / ?sum_3 (FLOAT8) as promo_revenue + => out schema: {(1) promo_revenue (FLOAT8)} + => in schema: {(2) ?sum_1 (FLOAT8), ?sum_3 (FLOAT8)} + GROUP_BY(5)() + => exprs: (sum(?casewhen (FLOAT8)),sum(?multiply_2 (FLOAT8))) + => target list: ?sum_1 (FLOAT8), ?sum_3 (FLOAT8) + => out schema:{(2) ?sum_1 (FLOAT8), ?sum_3 (FLOAT8)} + => in schema:{(2) ?casewhen (FLOAT8), ?multiply_2 (FLOAT8)} + JOIN(7)(INNER) + => Join Cond: default.lineitem.l_partkey (INT4) = default.part.p_partkey (INT4) + => target list: CASE WHEN default.part.p_type (TEXT)LIKE'PROMO%' THEN default.lineitem.l_extendedprice (FLOAT8) ELSE 0.0 END as ?casewhen, ?multiply_2 (FLOAT8) + => out schema: {(2) ?casewhen (FLOAT8), ?multiply_2 (FLOAT8)} + => in schema: {(5) ?multiply_2 (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_partkey (INT4), default.part.p_partkey (INT4), default.part.p_type (TEXT)} + SCAN(1) on default.part + => target list: default.part.p_partkey (INT4), default.part.p_type (TEXT) + => out schema: {(2) default.part.p_partkey (INT4), default.part.p_type (TEXT)} + => in schema: {(9) default.part.p_brand (TEXT), default.part.p_comment (TEXT), default.part.p_container (TEXT), default.part.p_mfgr (TEXT), default.part.p_name (TEXT), default.part.p_partkey (INT4), default.part.p_retailprice (FLOAT8), default.part.p_size (INT4), default.part.p_type (TEXT)} + SCAN(0) on default.lineitem + => target list: default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_partkey (INT4), default.lineitem.l_extendedprice (FLOAT8) * 1.0 - default.lineitem.l_discount (FLOAT8) as ?multiply_2 + => out schema: {(3) ?multiply_2 (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_partkey (INT4)} + => in schema: {(16) default.lineitem.l_comment (TEXT), default.lineitem.l_commitdate (TEXT), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_linenumber (INT4), default.lineitem.l_linestatus (TEXT), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.lineitem.l_quantity (FLOAT8), default.lineitem.l_receiptdate (TEXT), default.lineitem.l_returnflag (TEXT), default.lineitem.l_shipdate (TEXT), default.lineitem.l_shipinstruct (TEXT), default.lineitem.l_shipmode (TEXT), default.lineitem.l_suppkey (INT4), default.lineitem.l_tax (FLOAT8)} +explain +------------------------------- +------------------------------------------------------------------------------- +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000005) +------------------------------------------------------------------------------- +|-eb_0000000000000_0000_000005 + |-eb_0000000000000_0000_000004 + |-eb_0000000000000_0000_000003 + |-eb_0000000000000_0000_000002 + |-eb_0000000000000_0000_000001 +------------------------------------------------------------------------------- +Order of Execution +------------------------------------------------------------------------------- +1: eb_0000000000000_0000_000001 +2: eb_0000000000000_0000_000002 +3: eb_0000000000000_0000_000003 +4: eb_0000000000000_0000_000004 +5: eb_0000000000000_0000_000005 +------------------------------------------------------------------------------- + +======================================================= +Block Id: eb_0000000000000_0000_000001 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.lineitem.l_partkey (INT4), num=32) + +SCAN(0) on default.lineitem + => target list: default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_partkey (INT4), default.lineitem.l_extendedprice (FLOAT8) * 1.0 - default.lineitem.l_discount (FLOAT8) as ?multiply_2 + => out schema: {(3) ?multiply_2 (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_partkey (INT4)} + => in schema: {(16) default.lineitem.l_comment (TEXT), default.lineitem.l_commitdate (TEXT), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_linenumber (INT4), default.lineitem.l_linestatus (TEXT), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.lineitem.l_quantity (FLOAT8), default.lineitem.l_receiptdate (TEXT), default.lineitem.l_returnflag (TEXT), default.lineitem.l_shipdate (TEXT), default.lineitem.l_shipinstruct (TEXT), default.lineitem.l_shipmode (TEXT), default.lineitem.l_suppkey (INT4), default.lineitem.l_tax (FLOAT8)} + +======================================================= +Block Id: eb_0000000000000_0000_000002 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.part.p_partkey (INT4), num=32) + +SCAN(1) on default.part + => target list: default.part.p_partkey (INT4), default.part.p_type (TEXT) + => out schema: {(2) default.part.p_partkey (INT4), default.part.p_type (TEXT)} + => in schema: {(9) default.part.p_brand (TEXT), default.part.p_comment (TEXT), default.part.p_container (TEXT), default.part.p_mfgr (TEXT), default.part.p_name (TEXT), default.part.p_partkey (INT4), default.part.p_retailprice (FLOAT8), default.part.p_size (INT4), default.part.p_type (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000003 [INTERMEDIATE] +======================================================= + +[Incoming] +[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.lineitem.l_partkey (INT4), num=32) +[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.part.p_partkey (INT4), num=32) + +[Outgoing] +[q_0000000000000_0000] 3 => 4 (type=HASH_SHUFFLE, key=, num=1) + +GROUP_BY(11)() + => exprs: (sum(?casewhen (FLOAT8)),sum(?multiply_2 (FLOAT8))) + => target list: ?sum_6 (FLOAT8), ?sum_7 (FLOAT8) + => out schema:{(2) ?sum_6 (FLOAT8), ?sum_7 (FLOAT8)} + => in schema:{(2) ?casewhen (FLOAT8), ?multiply_2 (FLOAT8)} + JOIN(7)(INNER) + => Join Cond: default.lineitem.l_partkey (INT4) = default.part.p_partkey (INT4) + => target list: CASE WHEN default.part.p_type (TEXT)LIKE'PROMO%' THEN default.lineitem.l_extendedprice (FLOAT8) ELSE 0.0 END as ?casewhen, ?multiply_2 (FLOAT8) + => out schema: {(2) ?casewhen (FLOAT8), ?multiply_2 (FLOAT8)} + => in schema: {(5) ?multiply_2 (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_partkey (INT4), default.part.p_partkey (INT4), default.part.p_type (TEXT)} + SCAN(10) on eb_0000000000000_0000_000002 + => out schema: {(2) default.part.p_partkey (INT4), default.part.p_type (TEXT)} + => in schema: {(2) default.part.p_partkey (INT4), default.part.p_type (TEXT)} + SCAN(9) on eb_0000000000000_0000_000001 + => out schema: {(3) ?multiply_2 (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_partkey (INT4)} + => in schema: {(3) ?multiply_2 (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_partkey (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000004 [ROOT] +======================================================= + +[Incoming] +[q_0000000000000_0000] 3 => 4 (type=HASH_SHUFFLE, key=, num=1) + +PROJECTION(3) + => Targets: 100.0 * ?sum_1 (FLOAT8) / ?sum_3 (FLOAT8) as promo_revenue + => out schema: {(1) promo_revenue (FLOAT8)} + => in schema: {(2) ?sum_1 (FLOAT8), ?sum_3 (FLOAT8)} + GROUP_BY(5)() + => exprs: (sum(?sum_6 (FLOAT8)),sum(?sum_7 (FLOAT8))) + => target list: ?sum_1 (FLOAT8), ?sum_3 (FLOAT8) + => out schema:{(2) ?sum_1 (FLOAT8), ?sum_3 (FLOAT8)} + => in schema:{(2) ?sum_6 (FLOAT8), ?sum_7 (FLOAT8)} + SCAN(12) on eb_0000000000000_0000_000003 + => out schema: {(2) ?sum_6 (FLOAT8), ?sum_7 (FLOAT8)} + => in schema: {(2) ?sum_6 (FLOAT8), ?sum_7 (FLOAT8)} + +======================================================= +Block Id: eb_0000000000000_0000_000005 [TERMINAL] +======================================================= diff --git a/tajo-core/src/test/resources/results/TestTPCH/testTPCH14Expr.result b/tajo-core/src/test/resources/results/TestTPCH/testTPCH14Expr.result index 6bc69cc274..ace2a070df 100644 --- a/tajo-core/src/test/resources/results/TestTPCH/testTPCH14Expr.result +++ b/tajo-core/src/test/resources/results/TestTPCH/testTPCH14Expr.result @@ -1,3 +1,3 @@ promo_revenue ------------------------------- -33.610645634855025 \ No newline at end of file +33.610645634855025 diff --git a/tajo-core/src/test/resources/results/TestTPCH/testTPCHQ5.plan b/tajo-core/src/test/resources/results/TestTPCH/testTPCHQ5.plan new file mode 100644 index 0000000000..95abaf1cb6 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestTPCH/testTPCHQ5.plan @@ -0,0 +1,332 @@ +explain +------------------------------- +SORT(8) + => Sort Keys: revenue (FLOAT8) (desc) + GROUP_BY(7)(n_name) + => exprs: (sum(?multiply (FLOAT8))) + => target list: default.nation.n_name (TEXT), revenue (FLOAT8) + => out schema:{(2) default.nation.n_name (TEXT), revenue (FLOAT8)} + => in schema:{(2) ?multiply (FLOAT8), default.nation.n_name (TEXT)} + JOIN(20)(INNER) + => Join Cond: default.supplier.s_nationkey (INT4) = default.nation.n_nationkey (INT4) + => target list: ?multiply (FLOAT8), default.nation.n_name (TEXT) + => out schema: {(2) ?multiply (FLOAT8), default.nation.n_name (TEXT)} + => in schema: {(6) ?multiply (FLOAT8), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.supplier.s_nationkey (INT4)} + JOIN(19)(INNER) + => Join Cond: default.nation.n_regionkey (INT4) = default.region.r_regionkey (INT4) + => target list: default.nation.n_name (TEXT), default.nation.n_nationkey (INT4) + => out schema: {(2) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4)} + => in schema: {(4) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4)} + SCAN(5) on default.region + => filter: default.region.r_name (TEXT) = ASIA + => target list: default.region.r_regionkey (INT4) + => out schema: {(1) default.region.r_regionkey (INT4)} + => in schema: {(3) default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} + SCAN(4) on default.nation + => target list: default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4) + => out schema: {(3) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + JOIN(18)(INNER) + => Join Cond: default.lineitem.l_suppkey (INT4) = default.supplier.s_suppkey (INT4) + => target list: ?multiply (FLOAT8), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.supplier.s_nationkey (INT4) + => out schema: {(4) ?multiply (FLOAT8), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.supplier.s_nationkey (INT4)} + => in schema: {(6) ?multiply (FLOAT8), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_suppkey (INT4), default.supplier.s_nationkey (INT4), default.supplier.s_suppkey (INT4)} + JOIN(17)(INNER) + => Join Cond: default.customer.c_nationkey (INT4) = default.supplier.s_nationkey (INT4) + => target list: default.supplier.s_nationkey (INT4), default.supplier.s_suppkey (INT4) + => out schema: {(2) default.supplier.s_nationkey (INT4), default.supplier.s_suppkey (INT4)} + => in schema: {(3) default.customer.c_nationkey (INT4), default.supplier.s_nationkey (INT4), default.supplier.s_suppkey (INT4)} + SCAN(0) on default.customer + => target list: default.customer.c_nationkey (INT4) + => out schema: {(1) default.customer.c_nationkey (INT4)} + => in schema: {(8) default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT)} + SCAN(3) on default.supplier + => target list: default.supplier.s_nationkey (INT4), default.supplier.s_suppkey (INT4) + => out schema: {(2) default.supplier.s_nationkey (INT4), default.supplier.s_suppkey (INT4)} + => in schema: {(7) default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} + JOIN(16)(INNER) + => Join Cond: default.lineitem.l_orderkey (INT4) = default.orders.o_orderkey (INT4) + => target list: ?multiply (FLOAT8), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_suppkey (INT4) + => out schema: {(4) ?multiply (FLOAT8), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_suppkey (INT4)} + => in schema: {(6) ?multiply (FLOAT8), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_orderkey (INT4), default.lineitem.l_suppkey (INT4), default.orders.o_orderkey (INT4)} + SCAN(1) on default.orders + => filter: default.orders.o_orderdate (TEXT) >= 1994-01-01 AND default.orders.o_orderdate (TEXT) < 1995-01-01 + => target list: default.orders.o_orderkey (INT4) + => out schema: {(1) default.orders.o_orderkey (INT4)} + => in schema: {(9) default.orders.o_clerk (TEXT), default.orders.o_comment (TEXT), default.orders.o_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4), default.orders.o_orderpriority (TEXT), default.orders.o_orderstatus (TEXT), default.orders.o_shippriority (INT4), default.orders.o_totalprice (FLOAT8)} + SCAN(2) on default.lineitem + => target list: default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_orderkey (INT4), default.lineitem.l_suppkey (INT4), default.lineitem.l_extendedprice (FLOAT8) * 1.0 - default.lineitem.l_discount (FLOAT8) as ?multiply + => out schema: {(5) ?multiply (FLOAT8), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_orderkey (INT4), default.lineitem.l_suppkey (INT4)} + => in schema: {(16) default.lineitem.l_comment (TEXT), default.lineitem.l_commitdate (TEXT), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_linenumber (INT4), default.lineitem.l_linestatus (TEXT), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.lineitem.l_quantity (FLOAT8), default.lineitem.l_receiptdate (TEXT), default.lineitem.l_returnflag (TEXT), default.lineitem.l_shipdate (TEXT), default.lineitem.l_shipinstruct (TEXT), default.lineitem.l_shipmode (TEXT), default.lineitem.l_suppkey (INT4), default.lineitem.l_tax (FLOAT8)} +explain +------------------------------- +------------------------------------------------------------------------------- +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000014) +------------------------------------------------------------------------------- +|-eb_0000000000000_0000_000014 + |-eb_0000000000000_0000_000013 + |-eb_0000000000000_0000_000012 + |-eb_0000000000000_0000_000011 + |-eb_0000000000000_0000_000010 + |-eb_0000000000000_0000_000009 + |-eb_0000000000000_0000_000008 + |-eb_0000000000000_0000_000007 + |-eb_0000000000000_0000_000006 + |-eb_0000000000000_0000_000005 + |-eb_0000000000000_0000_000004 + |-eb_0000000000000_0000_000003 + |-eb_0000000000000_0000_000002 + |-eb_0000000000000_0000_000001 +------------------------------------------------------------------------------- +Order of Execution +------------------------------------------------------------------------------- +1: eb_0000000000000_0000_000001 +2: eb_0000000000000_0000_000002 +3: eb_0000000000000_0000_000004 +4: eb_0000000000000_0000_000005 +5: eb_0000000000000_0000_000003 +6: eb_0000000000000_0000_000006 +7: eb_0000000000000_0000_000008 +8: eb_0000000000000_0000_000009 +9: eb_0000000000000_0000_000007 +10: eb_0000000000000_0000_000010 +11: eb_0000000000000_0000_000011 +12: eb_0000000000000_0000_000012 +13: eb_0000000000000_0000_000013 +14: eb_0000000000000_0000_000014 +------------------------------------------------------------------------------- + +======================================================= +Block Id: eb_0000000000000_0000_000001 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.lineitem.l_orderkey (INT4), num=32) + +SCAN(2) on default.lineitem + => target list: default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_orderkey (INT4), default.lineitem.l_suppkey (INT4), default.lineitem.l_extendedprice (FLOAT8) * 1.0 - default.lineitem.l_discount (FLOAT8) as ?multiply + => out schema: {(5) ?multiply (FLOAT8), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_orderkey (INT4), default.lineitem.l_suppkey (INT4)} + => in schema: {(16) default.lineitem.l_comment (TEXT), default.lineitem.l_commitdate (TEXT), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_linenumber (INT4), default.lineitem.l_linestatus (TEXT), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.lineitem.l_quantity (FLOAT8), default.lineitem.l_receiptdate (TEXT), default.lineitem.l_returnflag (TEXT), default.lineitem.l_shipdate (TEXT), default.lineitem.l_shipinstruct (TEXT), default.lineitem.l_shipmode (TEXT), default.lineitem.l_suppkey (INT4), default.lineitem.l_tax (FLOAT8)} + +======================================================= +Block Id: eb_0000000000000_0000_000002 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.orders.o_orderkey (INT4), num=32) + +SCAN(1) on default.orders + => filter: default.orders.o_orderdate (TEXT) >= 1994-01-01 AND default.orders.o_orderdate (TEXT) < 1995-01-01 + => target list: default.orders.o_orderkey (INT4) + => out schema: {(1) default.orders.o_orderkey (INT4)} + => in schema: {(9) default.orders.o_clerk (TEXT), default.orders.o_comment (TEXT), default.orders.o_custkey (INT4), default.orders.o_orderdate (TEXT), default.orders.o_orderkey (INT4), default.orders.o_orderpriority (TEXT), default.orders.o_orderstatus (TEXT), default.orders.o_shippriority (INT4), default.orders.o_totalprice (FLOAT8)} + +======================================================= +Block Id: eb_0000000000000_0000_000003 [INTERMEDIATE] +======================================================= + +[Incoming] +[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.lineitem.l_orderkey (INT4), num=32) +[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.orders.o_orderkey (INT4), num=32) + +[Outgoing] +[q_0000000000000_0000] 3 => 7 (type=HASH_SHUFFLE, key=default.lineitem.l_suppkey (INT4), num=32) + +JOIN(16)(INNER) + => Join Cond: default.lineitem.l_orderkey (INT4) = default.orders.o_orderkey (INT4) + => target list: ?multiply (FLOAT8), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_suppkey (INT4) + => out schema: {(4) ?multiply (FLOAT8), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_suppkey (INT4)} + => in schema: {(6) ?multiply (FLOAT8), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_orderkey (INT4), default.lineitem.l_suppkey (INT4), default.orders.o_orderkey (INT4)} + SCAN(23) on eb_0000000000000_0000_000002 + => out schema: {(1) default.orders.o_orderkey (INT4)} + => in schema: {(1) default.orders.o_orderkey (INT4)} + SCAN(22) on eb_0000000000000_0000_000001 + => out schema: {(5) ?multiply (FLOAT8), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_orderkey (INT4), default.lineitem.l_suppkey (INT4)} + => in schema: {(5) ?multiply (FLOAT8), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_orderkey (INT4), default.lineitem.l_suppkey (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000004 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 4 => 6 (type=HASH_SHUFFLE, key=default.supplier.s_nationkey (INT4), num=32) + +SCAN(3) on default.supplier + => target list: default.supplier.s_nationkey (INT4), default.supplier.s_suppkey (INT4) + => out schema: {(2) default.supplier.s_nationkey (INT4), default.supplier.s_suppkey (INT4)} + => in schema: {(7) default.supplier.s_acctbal (FLOAT8), default.supplier.s_address (TEXT), default.supplier.s_comment (TEXT), default.supplier.s_name (TEXT), default.supplier.s_nationkey (INT4), default.supplier.s_phone (TEXT), default.supplier.s_suppkey (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000005 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 5 => 6 (type=HASH_SHUFFLE, key=default.customer.c_nationkey (INT4), num=32) + +SCAN(0) on default.customer + => target list: default.customer.c_nationkey (INT4) + => out schema: {(1) default.customer.c_nationkey (INT4)} + => in schema: {(8) default.customer.c_acctbal (FLOAT8), default.customer.c_address (TEXT), default.customer.c_comment (TEXT), default.customer.c_custkey (INT4), default.customer.c_mktsegment (TEXT), default.customer.c_name (TEXT), default.customer.c_nationkey (INT4), default.customer.c_phone (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000006 [INTERMEDIATE] +======================================================= + +[Incoming] +[q_0000000000000_0000] 4 => 6 (type=HASH_SHUFFLE, key=default.supplier.s_nationkey (INT4), num=32) +[q_0000000000000_0000] 5 => 6 (type=HASH_SHUFFLE, key=default.customer.c_nationkey (INT4), num=32) + +[Outgoing] +[q_0000000000000_0000] 6 => 7 (type=HASH_SHUFFLE, key=default.supplier.s_suppkey (INT4), num=32) + +JOIN(17)(INNER) + => Join Cond: default.customer.c_nationkey (INT4) = default.supplier.s_nationkey (INT4) + => target list: default.supplier.s_nationkey (INT4), default.supplier.s_suppkey (INT4) + => out schema: {(2) default.supplier.s_nationkey (INT4), default.supplier.s_suppkey (INT4)} + => in schema: {(3) default.customer.c_nationkey (INT4), default.supplier.s_nationkey (INT4), default.supplier.s_suppkey (INT4)} + SCAN(25) on eb_0000000000000_0000_000005 + => out schema: {(1) default.customer.c_nationkey (INT4)} + => in schema: {(1) default.customer.c_nationkey (INT4)} + SCAN(24) on eb_0000000000000_0000_000004 + => out schema: {(2) default.supplier.s_nationkey (INT4), default.supplier.s_suppkey (INT4)} + => in schema: {(2) default.supplier.s_nationkey (INT4), default.supplier.s_suppkey (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000007 [INTERMEDIATE] +======================================================= + +[Incoming] +[q_0000000000000_0000] 3 => 7 (type=HASH_SHUFFLE, key=default.lineitem.l_suppkey (INT4), num=32) +[q_0000000000000_0000] 6 => 7 (type=HASH_SHUFFLE, key=default.supplier.s_suppkey (INT4), num=32) + +[Outgoing] +[q_0000000000000_0000] 7 => 11 (type=HASH_SHUFFLE, key=default.supplier.s_nationkey (INT4), num=32) + +JOIN(18)(INNER) + => Join Cond: default.lineitem.l_suppkey (INT4) = default.supplier.s_suppkey (INT4) + => target list: ?multiply (FLOAT8), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.supplier.s_nationkey (INT4) + => out schema: {(4) ?multiply (FLOAT8), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.supplier.s_nationkey (INT4)} + => in schema: {(6) ?multiply (FLOAT8), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_suppkey (INT4), default.supplier.s_nationkey (INT4), default.supplier.s_suppkey (INT4)} + SCAN(27) on eb_0000000000000_0000_000006 + => out schema: {(2) default.supplier.s_nationkey (INT4), default.supplier.s_suppkey (INT4)} + => in schema: {(2) default.supplier.s_nationkey (INT4), default.supplier.s_suppkey (INT4)} + SCAN(26) on eb_0000000000000_0000_000003 + => out schema: {(4) ?multiply (FLOAT8), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_suppkey (INT4)} + => in schema: {(4) ?multiply (FLOAT8), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_suppkey (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000008 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 8 => 10 (type=HASH_SHUFFLE, key=default.nation.n_regionkey (INT4), num=32) + +SCAN(4) on default.nation + => target list: default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4) + => out schema: {(3) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + => in schema: {(4) default.nation.n_comment (TEXT), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000009 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 9 => 10 (type=HASH_SHUFFLE, key=default.region.r_regionkey (INT4), num=32) + +SCAN(5) on default.region + => filter: default.region.r_name (TEXT) = ASIA + => target list: default.region.r_regionkey (INT4) + => out schema: {(1) default.region.r_regionkey (INT4)} + => in schema: {(3) default.region.r_comment (TEXT), default.region.r_name (TEXT), default.region.r_regionkey (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000010 [INTERMEDIATE] +======================================================= + +[Incoming] +[q_0000000000000_0000] 8 => 10 (type=HASH_SHUFFLE, key=default.nation.n_regionkey (INT4), num=32) +[q_0000000000000_0000] 9 => 10 (type=HASH_SHUFFLE, key=default.region.r_regionkey (INT4), num=32) + +[Outgoing] +[q_0000000000000_0000] 10 => 11 (type=HASH_SHUFFLE, key=default.nation.n_nationkey (INT4), num=32) + +JOIN(19)(INNER) + => Join Cond: default.nation.n_regionkey (INT4) = default.region.r_regionkey (INT4) + => target list: default.nation.n_name (TEXT), default.nation.n_nationkey (INT4) + => out schema: {(2) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4)} + => in schema: {(4) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4), default.region.r_regionkey (INT4)} + SCAN(29) on eb_0000000000000_0000_000009 + => out schema: {(1) default.region.r_regionkey (INT4)} + => in schema: {(1) default.region.r_regionkey (INT4)} + SCAN(28) on eb_0000000000000_0000_000008 + => out schema: {(3) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + => in schema: {(3) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.nation.n_regionkey (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000011 [INTERMEDIATE] +======================================================= + +[Incoming] +[q_0000000000000_0000] 7 => 11 (type=HASH_SHUFFLE, key=default.supplier.s_nationkey (INT4), num=32) +[q_0000000000000_0000] 10 => 11 (type=HASH_SHUFFLE, key=default.nation.n_nationkey (INT4), num=32) + +[Outgoing] +[q_0000000000000_0000] 11 => 12 (type=HASH_SHUFFLE, key=default.nation.n_name (TEXT), num=32) + +GROUP_BY(32)(n_name) + => exprs: (sum(?multiply (FLOAT8))) + => target list: default.nation.n_name (TEXT), ?sum_7 (FLOAT8) + => out schema:{(2) default.nation.n_name (TEXT), ?sum_7 (FLOAT8)} + => in schema:{(2) ?multiply (FLOAT8), default.nation.n_name (TEXT)} + JOIN(20)(INNER) + => Join Cond: default.supplier.s_nationkey (INT4) = default.nation.n_nationkey (INT4) + => target list: ?multiply (FLOAT8), default.nation.n_name (TEXT) + => out schema: {(2) ?multiply (FLOAT8), default.nation.n_name (TEXT)} + => in schema: {(6) ?multiply (FLOAT8), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.nation.n_name (TEXT), default.nation.n_nationkey (INT4), default.supplier.s_nationkey (INT4)} + SCAN(31) on eb_0000000000000_0000_000010 + => out schema: {(2) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4)} + => in schema: {(2) default.nation.n_name (TEXT), default.nation.n_nationkey (INT4)} + SCAN(30) on eb_0000000000000_0000_000007 + => out schema: {(4) ?multiply (FLOAT8), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.supplier.s_nationkey (INT4)} + => in schema: {(4) ?multiply (FLOAT8), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.supplier.s_nationkey (INT4)} + +======================================================= +Block Id: eb_0000000000000_0000_000012 [INTERMEDIATE] +======================================================= + +[Incoming] +[q_0000000000000_0000] 11 => 12 (type=HASH_SHUFFLE, key=default.nation.n_name (TEXT), num=32) + +[Outgoing] +[q_0000000000000_0000] 12 => 13 (type=RANGE_SHUFFLE, key=revenue (FLOAT8), num=32) + +SORT(34) + => Sort Keys: revenue (FLOAT8) (desc) + GROUP_BY(7)(n_name) + => exprs: (sum(?sum_7 (FLOAT8))) + => target list: default.nation.n_name (TEXT), revenue (FLOAT8) + => out schema:{(2) default.nation.n_name (TEXT), revenue (FLOAT8)} + => in schema:{(2) default.nation.n_name (TEXT), ?sum_7 (FLOAT8)} + SCAN(33) on eb_0000000000000_0000_000011 + => out schema: {(2) default.nation.n_name (TEXT), ?sum_7 (FLOAT8)} + => in schema: {(2) default.nation.n_name (TEXT), ?sum_7 (FLOAT8)} + +======================================================= +Block Id: eb_0000000000000_0000_000013 [ROOT] +======================================================= + +[Incoming] +[q_0000000000000_0000] 12 => 13 (type=RANGE_SHUFFLE, key=revenue (FLOAT8), num=32) + +[Enforcers] + 0: sorted input=eb_0000000000000_0000_000012 + +SORT(8) + => Sort Keys: revenue (FLOAT8) (desc) + SCAN(35) on eb_0000000000000_0000_000012 + => out schema: {(2) default.nation.n_name (TEXT), revenue (FLOAT8)} + => in schema: {(2) default.nation.n_name (TEXT), revenue (FLOAT8)} + +======================================================= +Block Id: eb_0000000000000_0000_000014 [TERMINAL] +======================================================= diff --git a/tajo-core/src/test/resources/results/TestTPCH/testTPCHQ5.result b/tajo-core/src/test/resources/results/TestTPCH/testTPCHQ5.result new file mode 100644 index 0000000000..f15c366bbf --- /dev/null +++ b/tajo-core/src/test/resources/results/TestTPCH/testTPCHQ5.result @@ -0,0 +1,2 @@ +n_name,revenue +------------------------------- diff --git a/tajo-core/src/test/resources/results/TestUnionQuery/testComplexUnion1.plan b/tajo-core/src/test/resources/results/TestUnionQuery/testComplexUnion1.plan new file mode 100644 index 0000000000..9d713aac84 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestUnionQuery/testComplexUnion1.plan @@ -0,0 +1,80 @@ +explain +------------------------------- +TABLE_SUBQUERY(7) as default.result + => Targets: default.result.l_orderkey (INT4), default.result.l_partkey (INT4), default.result.query (TEXT) + => out schema: {(3) default.result.l_orderkey (INT4), default.result.l_partkey (INT4), default.result.query (TEXT)} + => in schema: {(3) default.result.l_orderkey (INT4), default.result.l_partkey (INT4), default.result.query (TEXT)} + UNION(6) + PROJECTION(5) + => Targets: default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), bbc as query + => out schema: {(3) default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), query (TEXT)} + => in schema: {(2) default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4)} + SCAN(3) on default.lineitem + => filter: default.lineitem.l_orderkey (INT4) = 1 + => target list: default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4) + => out schema: {(2) default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4)} + => in schema: {(16) default.lineitem.l_comment (TEXT), default.lineitem.l_commitdate (TEXT), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_linenumber (INT4), default.lineitem.l_linestatus (TEXT), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.lineitem.l_quantity (FLOAT8), default.lineitem.l_receiptdate (TEXT), default.lineitem.l_returnflag (TEXT), default.lineitem.l_shipdate (TEXT), default.lineitem.l_shipinstruct (TEXT), default.lineitem.l_shipmode (TEXT), default.lineitem.l_suppkey (INT4), default.lineitem.l_tax (FLOAT8)} + PROJECTION(2) + => Targets: default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), abc as query + => out schema: {(3) default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), query (TEXT)} + => in schema: {(2) default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4)} + SCAN(0) on default.lineitem + => filter: default.lineitem.l_orderkey (INT4) = 1 + => target list: default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4) + => out schema: {(2) default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4)} + => in schema: {(16) default.lineitem.l_comment (TEXT), default.lineitem.l_commitdate (TEXT), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_linenumber (INT4), default.lineitem.l_linestatus (TEXT), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.lineitem.l_quantity (FLOAT8), default.lineitem.l_receiptdate (TEXT), default.lineitem.l_returnflag (TEXT), default.lineitem.l_shipdate (TEXT), default.lineitem.l_shipinstruct (TEXT), default.lineitem.l_shipmode (TEXT), default.lineitem.l_suppkey (INT4), default.lineitem.l_tax (FLOAT8)} +explain +------------------------------- +------------------------------------------------------------------------------- +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000003) +------------------------------------------------------------------------------- +|-eb_0000000000000_0000_000003 + |-eb_0000000000000_0000_000002 + |-eb_0000000000000_0000_000001 +------------------------------------------------------------------------------- +Order of Execution +------------------------------------------------------------------------------- +1: eb_0000000000000_0000_000001 +2: eb_0000000000000_0000_000002 +3: eb_0000000000000_0000_000003 +------------------------------------------------------------------------------- + +======================================================= +Block Id: eb_0000000000000_0000_000001 [ROOT] +======================================================= + +TABLE_SUBQUERY(11) as default.result + => Targets: default.result.l_orderkey (INT4) as default.result.l_orderkey, default.result.l_partkey (INT4) as default.result.l_partkey, default.result.query (TEXT) as default.result.query + => out schema: {(3) default.result.l_orderkey (INT4), default.result.l_partkey (INT4), default.result.query (TEXT)} + => in schema: {(3) default.result.l_orderkey (INT4), default.result.l_partkey (INT4), default.result.query (TEXT)} + PROJECTION(2) + => Targets: default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), abc as query + => out schema: {(3) default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), query (TEXT)} + => in schema: {(2) default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4)} + SCAN(0) on default.lineitem + => filter: default.lineitem.l_orderkey (INT4) = 1 + => target list: default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4) + => out schema: {(2) default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4)} + => in schema: {(16) default.lineitem.l_comment (TEXT), default.lineitem.l_commitdate (TEXT), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_linenumber (INT4), default.lineitem.l_linestatus (TEXT), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.lineitem.l_quantity (FLOAT8), default.lineitem.l_receiptdate (TEXT), default.lineitem.l_returnflag (TEXT), default.lineitem.l_shipdate (TEXT), default.lineitem.l_shipinstruct (TEXT), default.lineitem.l_shipmode (TEXT), default.lineitem.l_suppkey (INT4), default.lineitem.l_tax (FLOAT8)} + +======================================================= +Block Id: eb_0000000000000_0000_000002 [ROOT] +======================================================= + +TABLE_SUBQUERY(12) as default.result + => Targets: default.result.l_orderkey (INT4), default.result.l_partkey (INT4), default.result.query (TEXT) + => out schema: {(3) default.result.l_orderkey (INT4), default.result.l_partkey (INT4), default.result.query (TEXT)} + => in schema: {(3) default.result.l_orderkey (INT4), default.result.l_partkey (INT4), default.result.query (TEXT)} + PROJECTION(5) + => Targets: default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), bbc as query + => out schema: {(3) default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), query (TEXT)} + => in schema: {(2) default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4)} + SCAN(3) on default.lineitem + => filter: default.lineitem.l_orderkey (INT4) = 1 + => target list: default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4) + => out schema: {(2) default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4)} + => in schema: {(16) default.lineitem.l_comment (TEXT), default.lineitem.l_commitdate (TEXT), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_linenumber (INT4), default.lineitem.l_linestatus (TEXT), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.lineitem.l_quantity (FLOAT8), default.lineitem.l_receiptdate (TEXT), default.lineitem.l_returnflag (TEXT), default.lineitem.l_shipdate (TEXT), default.lineitem.l_shipinstruct (TEXT), default.lineitem.l_shipmode (TEXT), default.lineitem.l_suppkey (INT4), default.lineitem.l_tax (FLOAT8)} + +======================================================= +Block Id: eb_0000000000000_0000_000003 [TERMINAL] +======================================================= diff --git a/tajo-core/src/test/resources/results/TestUnionQuery/testComplexUnion1.result b/tajo-core/src/test/resources/results/TestUnionQuery/testComplexUnion1.result new file mode 100644 index 0000000000..cf07e63268 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestUnionQuery/testComplexUnion1.result @@ -0,0 +1,6 @@ +l_orderkey,l_partkey,query +------------------------------- +1,1,abc +1,1,abc +1,1,bbc +1,1,bbc diff --git a/tajo-core/src/test/resources/results/TestUnionQuery/testComplexUnion2.plan b/tajo-core/src/test/resources/results/TestUnionQuery/testComplexUnion2.plan new file mode 100644 index 0000000000..2e2280d4a1 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestUnionQuery/testComplexUnion2.plan @@ -0,0 +1,116 @@ +explain +------------------------------- +TABLE_SUBQUERY(8) as default.result + => Targets: default.result.l_orderkey (INT4), default.result.l_partkey (INT4), default.result.url (TEXT) + => out schema: {(3) default.result.l_orderkey (INT4), default.result.l_partkey (INT4), default.result.url (TEXT)} + => in schema: {(3) default.result.l_orderkey (INT4), default.result.l_partkey (INT4), default.result.url (TEXT)} + JOIN(11)(INNER) + => Join Cond: default.res1.l_partkey (INT4) = default.res2.p_partkey (INT4) + => target list: default.res1.l_orderkey (INT4), default.res1.l_partkey (INT4), default.res1.url (TEXT) + => out schema: {(3) default.res1.l_orderkey (INT4), default.res1.l_partkey (INT4), default.res1.url (TEXT)} + => in schema: {(4) default.res1.l_orderkey (INT4), default.res1.l_partkey (INT4), default.res1.url (TEXT), default.res2.p_partkey (INT4)} + TABLE_SUBQUERY(5) as default.res2 + => Targets: default.res2.p_partkey (INT4) + => out schema: {(1) default.res2.p_partkey (INT4)} + => in schema: {(9) default.res2.p_brand (TEXT), default.res2.p_comment (TEXT), default.res2.p_container (TEXT), default.res2.p_mfgr (TEXT), default.res2.p_name (TEXT), default.res2.p_partkey (INT4), default.res2.p_retailprice (FLOAT8), default.res2.p_size (INT4), default.res2.p_type (TEXT)} + SCAN(3) on default.part + => target list: default.part.p_brand (TEXT), default.part.p_comment (TEXT), default.part.p_container (TEXT), default.part.p_mfgr (TEXT), default.part.p_name (TEXT), default.part.p_partkey (INT4), default.part.p_retailprice (FLOAT8), default.part.p_size (INT4), default.part.p_type (TEXT) + => out schema: {(9) default.part.p_brand (TEXT), default.part.p_comment (TEXT), default.part.p_container (TEXT), default.part.p_mfgr (TEXT), default.part.p_name (TEXT), default.part.p_partkey (INT4), default.part.p_retailprice (FLOAT8), default.part.p_size (INT4), default.part.p_type (TEXT)} + => in schema: {(9) default.part.p_brand (TEXT), default.part.p_comment (TEXT), default.part.p_container (TEXT), default.part.p_mfgr (TEXT), default.part.p_name (TEXT), default.part.p_partkey (INT4), default.part.p_retailprice (FLOAT8), default.part.p_size (INT4), default.part.p_type (TEXT)} + TABLE_SUBQUERY(2) as default.res1 + => Targets: default.res1.l_orderkey (INT4), default.res1.l_partkey (INT4), default.res1.url (TEXT) + => out schema: {(3) default.res1.l_orderkey (INT4), default.res1.l_partkey (INT4), default.res1.url (TEXT)} + => in schema: {(3) default.res1.l_orderkey (INT4), default.res1.l_partkey (INT4), default.res1.url (TEXT)} + SCAN(0) on default.lineitem + => target list: CASE WHEN default.lineitem.l_partkey (INT4) IS NOT NULL THEN WHEN default.lineitem.l_orderkey (INT4) = 1 THEN 1 ELSE 2 END as url, default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4) + => out schema: {(3) default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), url (TEXT)} + => in schema: {(16) default.lineitem.l_comment (TEXT), default.lineitem.l_commitdate (TEXT), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_linenumber (INT4), default.lineitem.l_linestatus (TEXT), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.lineitem.l_quantity (FLOAT8), default.lineitem.l_receiptdate (TEXT), default.lineitem.l_returnflag (TEXT), default.lineitem.l_shipdate (TEXT), default.lineitem.l_shipinstruct (TEXT), default.lineitem.l_shipmode (TEXT), default.lineitem.l_suppkey (INT4), default.lineitem.l_tax (FLOAT8)} +explain +------------------------------- +------------------------------------------------------------------------------- +Execution Block Graph (TERMINAL - eb_0000000000000_0000_000004) +------------------------------------------------------------------------------- +|-eb_0000000000000_0000_000004 + |-eb_0000000000000_0000_000003 + |-eb_0000000000000_0000_000002 + |-eb_0000000000000_0000_000001 +------------------------------------------------------------------------------- +Order of Execution +------------------------------------------------------------------------------- +1: eb_0000000000000_0000_000001 +2: eb_0000000000000_0000_000002 +3: eb_0000000000000_0000_000003 +4: eb_0000000000000_0000_000004 +------------------------------------------------------------------------------- + +======================================================= +Block Id: eb_0000000000000_0000_000001 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.res1.l_partkey (INT4), num=32) + +TABLE_SUBQUERY(2) as default.res1 + => Targets: default.res1.l_orderkey (INT4), default.res1.l_partkey (INT4), default.res1.url (TEXT) + => out schema: {(3) default.res1.l_orderkey (INT4), default.res1.l_partkey (INT4), default.res1.url (TEXT)} + => in schema: {(3) default.res1.l_orderkey (INT4), default.res1.l_partkey (INT4), default.res1.url (TEXT)} + PROJECTION(1) + => Targets: default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), url (TEXT) + => out schema: {(3) default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), url (TEXT)} + => in schema: {(3) default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), url (TEXT)} + SCAN(0) on default.lineitem + => target list: CASE WHEN default.lineitem.l_partkey (INT4) IS NOT NULL THEN WHEN default.lineitem.l_orderkey (INT4) = 1 THEN 1 ELSE 2 END as url, default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4) + => out schema: {(3) default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), url (TEXT)} + => in schema: {(16) default.lineitem.l_comment (TEXT), default.lineitem.l_commitdate (TEXT), default.lineitem.l_discount (FLOAT8), default.lineitem.l_extendedprice (FLOAT8), default.lineitem.l_linenumber (INT4), default.lineitem.l_linestatus (TEXT), default.lineitem.l_orderkey (INT4), default.lineitem.l_partkey (INT4), default.lineitem.l_quantity (FLOAT8), default.lineitem.l_receiptdate (TEXT), default.lineitem.l_returnflag (TEXT), default.lineitem.l_shipdate (TEXT), default.lineitem.l_shipinstruct (TEXT), default.lineitem.l_shipmode (TEXT), default.lineitem.l_suppkey (INT4), default.lineitem.l_tax (FLOAT8)} + +======================================================= +Block Id: eb_0000000000000_0000_000002 [LEAF] +======================================================= + +[Outgoing] +[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.res2.p_partkey (INT4), num=32) + +TABLE_SUBQUERY(5) as default.res2 + => Targets: default.res2.p_partkey (INT4) + => out schema: {(1) default.res2.p_partkey (INT4)} + => in schema: {(9) default.res2.p_brand (TEXT), default.res2.p_comment (TEXT), default.res2.p_container (TEXT), default.res2.p_mfgr (TEXT), default.res2.p_name (TEXT), default.res2.p_partkey (INT4), default.res2.p_retailprice (FLOAT8), default.res2.p_size (INT4), default.res2.p_type (TEXT)} + PROJECTION(4) + => Targets: default.part.p_partkey (INT4), default.part.p_name (TEXT), default.part.p_mfgr (TEXT), default.part.p_brand (TEXT), default.part.p_type (TEXT), default.part.p_size (INT4), default.part.p_container (TEXT), default.part.p_retailprice (FLOAT8), default.part.p_comment (TEXT) + => out schema: {(9) default.part.p_brand (TEXT), default.part.p_comment (TEXT), default.part.p_container (TEXT), default.part.p_mfgr (TEXT), default.part.p_name (TEXT), default.part.p_partkey (INT4), default.part.p_retailprice (FLOAT8), default.part.p_size (INT4), default.part.p_type (TEXT)} + => in schema: {(9) default.part.p_brand (TEXT), default.part.p_comment (TEXT), default.part.p_container (TEXT), default.part.p_mfgr (TEXT), default.part.p_name (TEXT), default.part.p_partkey (INT4), default.part.p_retailprice (FLOAT8), default.part.p_size (INT4), default.part.p_type (TEXT)} + SCAN(3) on default.part + => target list: default.part.p_brand (TEXT), default.part.p_comment (TEXT), default.part.p_container (TEXT), default.part.p_mfgr (TEXT), default.part.p_name (TEXT), default.part.p_partkey (INT4), default.part.p_retailprice (FLOAT8), default.part.p_size (INT4), default.part.p_type (TEXT) + => out schema: {(9) default.part.p_brand (TEXT), default.part.p_comment (TEXT), default.part.p_container (TEXT), default.part.p_mfgr (TEXT), default.part.p_name (TEXT), default.part.p_partkey (INT4), default.part.p_retailprice (FLOAT8), default.part.p_size (INT4), default.part.p_type (TEXT)} + => in schema: {(9) default.part.p_brand (TEXT), default.part.p_comment (TEXT), default.part.p_container (TEXT), default.part.p_mfgr (TEXT), default.part.p_name (TEXT), default.part.p_partkey (INT4), default.part.p_retailprice (FLOAT8), default.part.p_size (INT4), default.part.p_type (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000003 [ROOT] +======================================================= + +[Incoming] +[q_0000000000000_0000] 1 => 3 (type=HASH_SHUFFLE, key=default.res1.l_partkey (INT4), num=32) +[q_0000000000000_0000] 2 => 3 (type=HASH_SHUFFLE, key=default.res2.p_partkey (INT4), num=32) + +TABLE_SUBQUERY(8) as default.result + => Targets: default.result.l_orderkey (INT4), default.result.l_partkey (INT4), default.result.url (TEXT) + => out schema: {(3) default.result.l_orderkey (INT4), default.result.l_partkey (INT4), default.result.url (TEXT)} + => in schema: {(3) default.result.l_orderkey (INT4), default.result.l_partkey (INT4), default.result.url (TEXT)} + PROJECTION(7) + => Targets: default.res1.l_orderkey (INT4), default.res1.l_partkey (INT4), default.res1.url (TEXT) + => out schema: {(3) default.res1.l_orderkey (INT4), default.res1.l_partkey (INT4), default.res1.url (TEXT)} + => in schema: {(3) default.res1.l_orderkey (INT4), default.res1.l_partkey (INT4), default.res1.url (TEXT)} + JOIN(11)(INNER) + => Join Cond: default.res1.l_partkey (INT4) = default.res2.p_partkey (INT4) + => target list: default.res1.l_orderkey (INT4), default.res1.l_partkey (INT4), default.res1.url (TEXT) + => out schema: {(3) default.res1.l_orderkey (INT4), default.res1.l_partkey (INT4), default.res1.url (TEXT)} + => in schema: {(4) default.res1.l_orderkey (INT4), default.res1.l_partkey (INT4), default.res1.url (TEXT), default.res2.p_partkey (INT4)} + SCAN(14) on eb_0000000000000_0000_000002 + => out schema: {(1) default.res2.p_partkey (INT4)} + => in schema: {(1) default.res2.p_partkey (INT4)} + SCAN(13) on eb_0000000000000_0000_000001 + => out schema: {(3) default.res1.l_orderkey (INT4), default.res1.l_partkey (INT4), default.res1.url (TEXT)} + => in schema: {(3) default.res1.l_orderkey (INT4), default.res1.l_partkey (INT4), default.res1.url (TEXT)} + +======================================================= +Block Id: eb_0000000000000_0000_000004 [TERMINAL] +======================================================= diff --git a/tajo-core/src/test/resources/results/TestUnionQuery/testComplexUnion2.result b/tajo-core/src/test/resources/results/TestUnionQuery/testComplexUnion2.result new file mode 100644 index 0000000000..da92d8c675 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestUnionQuery/testComplexUnion2.result @@ -0,0 +1,7 @@ +l_orderkey,l_partkey,url +------------------------------- +1,1, +1,1, +2,2, +3,2, +3,3, 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 4a3d087816..1c29a6da2e 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 @@ -34,7 +34,6 @@ public class AggregationFunctionCallEval extends FunctionEval implements Cloneable { -// private static final Log LOG = LogFactory.getLog(AggregationFunctionCallEval.class); // Both firstPhase and lastPhase flags should be true before global planning. @Expose private boolean firstPhase = true; @Expose private boolean lastPhase = true; @@ -87,7 +86,6 @@ public void merge(FunctionContext context, Tuple tuple) { } protected void mergeParam(FunctionContext context, Tuple params) { -// if (!intermediatePhase && !lastPhase) { if (firstPhase) { // firstPhase functionInvoke.eval(context, params); @@ -114,7 +112,6 @@ public Datum terminate(FunctionContext context) { @Override public DataType getValueType() { -// LOG.info("at getValueType, " + funcDesc.getFunctionName() + " firstPhase: " + firstPhase + ", lastPhase: " + lastPhase); if (!lastPhase) { return functionInvoke.getPartialResultType(); } else { diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/logical/JoinNode.java b/tajo-plan/src/main/java/org/apache/tajo/plan/logical/JoinNode.java index 8d65cf55e5..a0d81255ad 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/logical/JoinNode.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/logical/JoinNode.java @@ -22,28 +22,21 @@ package org.apache.tajo.plan.logical; import com.google.gson.annotations.Expose; - import org.apache.tajo.algebra.JoinType; import org.apache.tajo.plan.PlanString; -import org.apache.tajo.plan.util.PlannerUtil; import org.apache.tajo.plan.Target; import org.apache.tajo.plan.expr.BinaryEval; import org.apache.tajo.plan.expr.EvalNode; +import org.apache.tajo.plan.util.PlannerUtil; import org.apache.tajo.util.TUtil; -import java.util.ArrayList; import java.util.Arrays; -import java.util.List; public class JoinNode extends BinaryNode implements Projectable, Cloneable { @Expose private JoinType joinType; @Expose private EvalNode joinQual; @Expose private Target[] targets; - // transition states -// private boolean candidateBroadcast = false; -// private List broadcastCandidateTargets = new ArrayList(); - public JoinNode(int pid) { super(pid, NodeType.JOIN); } @@ -54,18 +47,6 @@ public void init(JoinType joinType, LogicalNode left, LogicalNode right) { setRightChild(right); } -// public boolean isCandidateBroadcast() { -// return candidateBroadcast; -// } - -// public void setCandidateBroadcast(boolean candidateBroadcast) { -// this.candidateBroadcast = candidateBroadcast; -// } - -// public List getBroadcastCandidateTargets() { -// return broadcastCandidateTargets; -// } - public JoinType getJoinType() { return this.joinType; } @@ -131,7 +112,6 @@ public PlanString getPlanString() { public int hashCode() { final int prime = 31; int result = 1; -// result = prime * result + (candidateBroadcast ? 1231 : 1237); result = prime * result + ((joinQual == null) ? 0 : joinQual.hashCode()); result = prime * result + ((joinType == null) ? 0 : joinType.hashCode()); result = prime * result + Arrays.hashCode(targets); diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeSerializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeSerializer.java index 11de936d75..88d831e223 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeSerializer.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeSerializer.java @@ -76,12 +76,6 @@ public static LogicalNodeTree serialize(LogicalNode node) { private static PlanProto.LogicalNode.Builder createNodeBuilder(SerializeContext context, LogicalNode node) { int selfId; -// if (context.idMap.containsKey(node)) { -// selfId = context.idMap.get(node); -// } else { -// selfId = context.seqId++; -// context.idMap.put(node, selfId); -// } if (context.idMap.containsKey(node.getPID())) { selfId = context.idMap.get(node.getPID()); } else { @@ -727,8 +721,6 @@ public static PlanProto.Target convertTarget(Target target) { private int [] registerGetChildIds(SerializeContext context, LogicalNode node) { int [] childIds = new int[node.childNum()]; for (int i = 0; i < node.childNum(); i++) { -// if (context.idMap.containsKey(node.getChild(i))) { -// childIds[i] = context.idMap.get(node.getChild(i)); if (node.getChild(i) != null && context.idMap.containsKey(node.getChild(i).getPID())) { childIds[i] = context.idMap.get(node.getChild(i).getPID()); } else { From 913d22dea02cf3983430cbb141322fcddd7728d1 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 25 May 2015 21:19:53 +0900 Subject: [PATCH 33/37] Add comments --- .../engine/planner/global/ExecutionBlock.java | 26 ++++- .../rewriter/rules/BroadcastJoinRule.java | 108 +++++++++++------- .../rewriter/rules/GlobalPlanRewriteUtil.java | 24 ++++ 3 files changed, 117 insertions(+), 41 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java index 2d02443d14..87238961ad 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java @@ -45,7 +45,30 @@ public class ExecutionBlock { private Map broadcastRelations = TUtil.newHashMap(); /* - * The nullSupplying and preservedRow flags are used for finding which relations will be broadcasted. + * An execution block is null-supplying or preserved-row when its output is used as an input for outer join. + * These flags are set according to the type of outer join. + * Here are brief descriptions for these flags. + * + * 1) left outer join + * + * left outer join + * / \ + * preserved-row null-supplying + * + * 2) right outer join + * + * right outer join + * / \ + * null-supplying preserved-row + * + * 3) full outer join + * + * full outer join + * / \ + * null-supplying preserved-row + * preserved-row null-supplying + * + * The null-supplying and preserved-row flags are used to find which relations will be broadcasted. */ protected transient boolean nullSuppllying = false; protected transient boolean preservedRow = false; @@ -74,6 +97,7 @@ public void setPlan(LogicalNode plan) { s.add(node); while (!s.isEmpty()) { node = s.remove(s.size()-1); + // TODO: the below code should be improved to handle every case if (isUnionOnly && node.getType() != NodeType.ROOT && node.getType() != NodeType.TABLE_SUBQUERY && node.getType() != NodeType.SCAN && node.getType() != NodeType.PARTITIONS_SCAN && node.getType() != NodeType.UNION && node.getType() != NodeType.PROJECTION) { diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java index 576f2d60d9..1c0fcc032e 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java @@ -18,8 +18,6 @@ package org.apache.tajo.engine.planner.global.rewriter.rules; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.tajo.ExecutionBlockId; import org.apache.tajo.OverridableConf; import org.apache.tajo.SessionVars; @@ -37,14 +35,34 @@ import java.util.*; /** - * Broadcast rules are as follows. + * {@link BroadcastJoinRule} converts repartition join plan into broadcast join plan. + * To describe the broadcast join rules, we have to define the broadcastable property for a relation as follows. + * + *

Broadcastable relation

* A relation is broadcastable when its size is smaller than a given threshold. + * + *

Assumetion

+ * If every input of an execution block is broadcastable, the output of the execution block is also broadcastable. + * + *

Rules to convert repartition join into broadcast join

+ *
    + *
  • Given an EB containing a join and its child EBs, those EBs can be merged into a single EB if at least one child EB's output is broadcastable.
  • + *
  • Given a user-defined threshold, the total size of broadcast relations of an EB cannot exceed such threshold.
  • + *
      + *
    • After merging EBs according to the first rule, the result EB may not satisfy the second rule. In this case, enforce repartition join for large relations to satisfy the second rule.
    • + *
    + *
  • Preserved-row relations cannot be broadcasted to avoid duplicated results. That is, full outer join cannot be executed with broadcast join.
  • + *
      + *
    • Here is brief backgrounds for this rule. Data of preserved-row relations will be appeared in the join result regardless of join conditions. If multiple tasks execute outer join with broadcasted preserved-row relations, they emit duplicates results.
    • + *
    • Even though a single task can execute outer join when every input is broadcastable, broadcast join is not allowed if one of input relation consists of multiple files.
    • + *
    + *
+ * */ public class BroadcastJoinRule implements GlobalPlanRewriteRule { - private final static Log LOG = LogFactory.getLog(BroadcastJoinRule.class); - private BroadcastJoinPlanBuilder planOptimizer; - private BroadcastJoinOptimizeFinalizer optimizeFinalizer; + private BroadcastJoinPlanBuilder planBuilder; + private BroadcastJoinPlanFinalizer planFinalizer; @Override public String getName() { @@ -60,8 +78,8 @@ public boolean isEligible(OverridableConf queryContext, MasterPlan plan) { if (broadcastSizeThreshold > 0) { GlobalPlanRewriteUtil.ParentFinder parentFinder = new GlobalPlanRewriteUtil.ParentFinder(); RelationSizeComparator relSizeComparator = new RelationSizeComparator(); - planOptimizer = new BroadcastJoinPlanBuilder(plan, relSizeComparator, parentFinder, broadcastSizeThreshold); - optimizeFinalizer = new BroadcastJoinOptimizeFinalizer(plan, relSizeComparator); + planBuilder = new BroadcastJoinPlanBuilder(plan, relSizeComparator, parentFinder, broadcastSizeThreshold); + planFinalizer = new BroadcastJoinPlanFinalizer(plan, relSizeComparator); return true; } } @@ -72,8 +90,8 @@ public boolean isEligible(OverridableConf queryContext, MasterPlan plan) { @Override public MasterPlan rewrite(MasterPlan plan) throws PlanningException{ - plan.accept(plan.getRoot().getId(), planOptimizer); - plan.accept(plan.getRoot().getId(), optimizeFinalizer); + plan.accept(plan.getRoot().getId(), planBuilder); + plan.accept(plan.getRoot().getId(), planFinalizer); return plan; } @@ -86,13 +104,15 @@ public int compare(ScanNode o1, ScanNode o2) { } /** - * + * If a plan contains only broadcast relations, it will be executed at multiple workers who store any broadcast relations. + * {@Link BroadcastJoinPlanFinalizer} checks whether every input is the broadcast candidate or not. + * If so, it removes the broadcast property from the largest relation. */ - private static class BroadcastJoinOptimizeFinalizer implements DirectedGraphVisitor { + private static class BroadcastJoinPlanFinalizer implements DirectedGraphVisitor { private final MasterPlan plan; private final RelationSizeComparator relSizeComparator; - public BroadcastJoinOptimizeFinalizer(MasterPlan plan, RelationSizeComparator relationSizeComparator) { + public BroadcastJoinPlanFinalizer(MasterPlan plan, RelationSizeComparator relationSizeComparator) { this.plan = plan; this.relSizeComparator = relationSizeComparator; } @@ -168,33 +188,6 @@ private void visitNonLeafNode(ExecutionBlock current) { List childs = plan.getChilds(current); Map unionScanMap = current.getUnionScanMap(); - // When the total size of broadcastable relations exceeds the threshold, enforce two-phase join for large ones - // in order to broadcast as many relations as possible. - List broadcastCandidates = TUtil.newList(); - for (ScanNode scanNode : current.getScanNodes()) { - long estimatedRelationSize = GlobalPlanRewriteUtil.getTableVolume(scanNode); - if (estimatedRelationSize > 0 && estimatedRelationSize <= broadcastSizeThreshold) { - broadcastCandidates.add(scanNode); - } - } - Collections.sort(broadcastCandidates, relSizeComparator); - - // Enforce broadcast for candidates in ascending order of relation size - long totalBroadcastVolume = 0; - int i; - for (i = 0; i < broadcastCandidates.size(); i++) { - long volumeOfCandidate = GlobalPlanRewriteUtil.getTableVolume(broadcastCandidates.get(i)); - if (totalBroadcastVolume + volumeOfCandidate > broadcastSizeThreshold) { - break; - } - totalBroadcastVolume += volumeOfCandidate; - } - - for (; i < broadcastCandidates.size(); ) { - ScanNode nonBroadcast = broadcastCandidates.remove(i); - broadcastCandidates.remove(nonBroadcast); - } - if (current.hasBroadcastRelation()) { // The current execution block and its every child are able to be merged. for (ExecutionBlock child : childs) { @@ -206,6 +199,8 @@ private void visitNonLeafNode(ExecutionBlock current) { } } + checkTotalSizeOfBroadcastableRelations(current); + // We assume that if every input of an execution block is broadcastable, // the output of the execution block is also broadcastable. if (!current.isPreservedRow() && isFullyBroadcastable(current)) { @@ -225,6 +220,39 @@ private void visitNonLeafNode(ExecutionBlock current) { } } + /** + * When the total size of broadcastable relations exceeds the threshold, enforce repartition join for large ones + * in order to broadcast as many relations as possible. + * + * @param block + */ + private void checkTotalSizeOfBroadcastableRelations(ExecutionBlock block) { + List broadcastCandidates = TUtil.newList(); + for (ScanNode scanNode : block.getScanNodes()) { + long estimatedRelationSize = GlobalPlanRewriteUtil.getTableVolume(scanNode); + if (estimatedRelationSize > 0 && estimatedRelationSize <= broadcastSizeThreshold) { + broadcastCandidates.add(scanNode); + } + } + Collections.sort(broadcastCandidates, relSizeComparator); + + // Enforce broadcast for candidates in ascending order of relation size + long totalBroadcastVolume = 0; + int i; + for (i = 0; i < broadcastCandidates.size(); i++) { + long volumeOfCandidate = GlobalPlanRewriteUtil.getTableVolume(broadcastCandidates.get(i)); + if (totalBroadcastVolume + volumeOfCandidate > broadcastSizeThreshold) { + break; + } + totalBroadcastVolume += volumeOfCandidate; + } + + for (; i < broadcastCandidates.size(); ) { + ScanNode nonBroadcast = broadcastCandidates.remove(i); + block.removeBroadcastRelation(nonBroadcast); + } + } + private void updateScanOfParentAsBroadcastable(MasterPlan plan, ExecutionBlock current) throws PlanningException { ExecutionBlock parent = plan.getParent(current); if (parent != null && !plan.isTerminal(parent)) { diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanRewriteUtil.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanRewriteUtil.java index 4b22c12b23..d58f920f2a 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanRewriteUtil.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanRewriteUtil.java @@ -27,6 +27,14 @@ import java.util.List; public class GlobalPlanRewriteUtil { + /** + * Merge the parent EB with the child EB. + * + * @param plan + * @param child + * @param parent + * @return + */ public static ExecutionBlock mergeExecutionBlocks(MasterPlan plan, ExecutionBlock child, ExecutionBlock parent) { for (ScanNode broadcastable : child.getBroadcastRelations()) { parent.addBroadcastRelation(broadcastable); @@ -50,6 +58,14 @@ public static ExecutionBlock mergeExecutionBlocks(MasterPlan plan, ExecutionBloc return parent; } + /** + * Replace a child of the given parent logical node with the new one. + * + * @param newChild + * @param originalChild + * @param parent + * @throws PlanningException + */ public static void replaceChild(LogicalNode newChild, ScanNode originalChild, LogicalNode parent) throws PlanningException { if (parent instanceof UnaryNode) { @@ -68,6 +84,14 @@ public static void replaceChild(LogicalNode newChild, ScanNode originalChild, Lo } } + /** + * Find a scan node in the plan of the parent EB corresponding to the output of the child EB. + * + * @param child + * @param parent + * @return + * @throws PlanningException + */ public static ScanNode findScanForChildEb(ExecutionBlock child, ExecutionBlock parent) throws PlanningException { ScanNode scanForChild = null; for (ScanNode scanNode : parent.getScanNodes()) { From 0dfb4dc8f4b121fc21e4d7edf21d598a00ccff42 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 25 May 2015 21:26:29 +0900 Subject: [PATCH 34/37] Modify comments --- .../engine/planner/global/rewriter/rules/BroadcastJoinRule.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java index 1c0fcc032e..4023d81296 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java @@ -36,7 +36,7 @@ /** * {@link BroadcastJoinRule} converts repartition join plan into broadcast join plan. - * To describe the broadcast join rules, we have to define the broadcastable property for a relation as follows. + * Broadcast join rules can be defined as follows. * *

Broadcastable relation

* A relation is broadcastable when its size is smaller than a given threshold. From 3e60819c05e2a14521e8d5b1b48ba685130f6091 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 25 May 2015 23:59:43 +0900 Subject: [PATCH 35/37] Fix test failure --- .../testComplexJoinCondition6.Hash.plan | 3 +-- .../testComplexJoinCondition6.Sort.plan | 3 +-- .../testComplexJoinCondition7.Hash.plan | 3 +-- .../testComplexJoinCondition7.Sort.plan | 3 +-- 4 files changed, 4 insertions(+), 8 deletions(-) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition6.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition6.Hash.plan index 2a6177e9f4..2402db7154 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition6.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition6.Hash.plan @@ -57,8 +57,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.nation - 1: type=Broadcast, tables=default.nation - 2: type=Broadcast, tables=default.n1 + 1: type=Broadcast, tables=default.n1 SORT(18) => Sort Keys: default.n1.n_nationkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition6.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition6.Sort.plan index 2a6177e9f4..2402db7154 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition6.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition6.Sort.plan @@ -57,8 +57,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.nation - 1: type=Broadcast, tables=default.nation - 2: type=Broadcast, tables=default.n1 + 1: type=Broadcast, tables=default.n1 SORT(18) => Sort Keys: default.n1.n_nationkey (INT4) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition7.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition7.Hash.plan index e093471766..8e6d32bd82 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition7.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition7.Hash.plan @@ -57,8 +57,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.nation - 1: type=Broadcast, tables=default.nation - 2: type=Broadcast, tables=default.n1 + 1: type=Broadcast, tables=default.n1 SORT(18) => Sort Keys: default.n1.n_nationkey (INT4) (asc),default.n2.n_name (TEXT) (asc) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition7.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition7.Sort.plan index e093471766..8e6d32bd82 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition7.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testComplexJoinCondition7.Sort.plan @@ -57,8 +57,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] [Enforcers] 0: type=Broadcast, tables=default.nation - 1: type=Broadcast, tables=default.nation - 2: type=Broadcast, tables=default.n1 + 1: type=Broadcast, tables=default.n1 SORT(18) => Sort Keys: default.n1.n_nationkey (INT4) (asc),default.n2.n_name (TEXT) (asc) From ddf0389375d72abb77517720cb8bb5560da4bdcb Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 27 May 2015 11:02:10 +0900 Subject: [PATCH 36/37] Address comments --- .../tajo/engine/planner/enforce/Enforcer.java | 17 ------- .../engine/planner/global/ExecutionBlock.java | 48 +++++++++++++------ .../rewriter/rules/BroadcastJoinRule.java | 9 +++- 3 files changed, 41 insertions(+), 33 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/enforce/Enforcer.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/enforce/Enforcer.java index 0d2ae3c8a6..8128390933 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/enforce/Enforcer.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/enforce/Enforcer.java @@ -86,23 +86,6 @@ public void addSortedInput(String tableName, SortSpec[] sortSpecs) { TUtil.putToNestedList(properties, builder.getType(), builder.build()); } - public void removeSortedInput(String tableName) { - EnforceProperty willBeRemoved = null; - List sortedInputProperties = properties.get(EnforceType.SORTED_INPUT); - // assume that there is only a single sorted input property for each table - for (EnforceProperty property : sortedInputProperties) { - if (property.hasSortedInput()) { - if (property.getSortedInput().getTableName().equals(tableName)) { - willBeRemoved = property; - break; - } - } - } - if (willBeRemoved != null) { - sortedInputProperties.remove(willBeRemoved); - } - } - public void addOutputDistinct() { EnforceProperty.Builder builder = newProperty(); OutputDistinctEnforce.Builder enforce = OutputDistinctEnforce.newBuilder(); diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java index 87238961ad..4f352c1c9d 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java @@ -46,32 +46,50 @@ public class ExecutionBlock { /* * An execution block is null-supplying or preserved-row when its output is used as an input for outer join. - * These flags are set according to the type of outer join. - * Here are brief descriptions for these flags. + * These properties are decided based on the type of parent execution block's outer join. + * Here are brief descriptions for these properties. * * 1) left outer join * - * left outer join - * / \ - * preserved-row null-supplying + * parent eb + * ------------------- + * | left outer join | + * ------------------- + * / \ + * left child eb right child eb + * ----------------- ------------------ + * | preserved-row | | null-supplying | + * ----------------- ------------------ * * 2) right outer join * - * right outer join - * / \ - * null-supplying preserved-row + * parent eb + * -------------------- + * | right outer join | + * -------------------- + * / \ + * left child eb right child eb + * ------------------ ----------------- + * | null-supplying | | preserved-row | + * ------------------ ----------------- * * 3) full outer join * - * full outer join - * / \ - * null-supplying preserved-row - * preserved-row null-supplying + * parent eb + * ------------------- + * | full outer join | + * ------------------- + * / \ + * left child eb right child eb + * ------------------ ------------------ + * | null-supplying | | preserved-row | + * | preserved-row | | null-supplying | + * ------------------ ------------------ * - * The null-supplying and preserved-row flags are used to find which relations will be broadcasted. + * The null-supplying and preserved-row properties are used to find which relations will be broadcasted. */ - protected transient boolean nullSuppllying = false; - protected transient boolean preservedRow = false; + protected boolean nullSuppllying = false; + protected boolean preservedRow = false; public ExecutionBlock(ExecutionBlockId executionBlockId) { this.executionBlockId = executionBlockId; diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java index 4023d81296..85b5e109db 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/BroadcastJoinRule.java @@ -99,7 +99,14 @@ private static class RelationSizeComparator implements Comparator { @Override public int compare(ScanNode o1, ScanNode o2) { - return (int) (GlobalPlanRewriteUtil.getTableVolume(o1) - GlobalPlanRewriteUtil.getTableVolume(o2)); + long compare = GlobalPlanRewriteUtil.getTableVolume(o1) - GlobalPlanRewriteUtil.getTableVolume(o2); + if (compare == 0) { + return 0; + } else if (compare > 0) { + return 1; + } else { + return -1; + } } } From 1bd562d253e4f573aaa0f0e374fe84d8c3db349d Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 27 May 2015 18:56:51 +0900 Subject: [PATCH 37/37] Fix bug --- .../global/rewriter/rules/GlobalPlanRewriteUtil.java | 9 ++++++++- .../testBroadcastSubquery2.Hash.plan | 4 ++-- .../testBroadcastSubquery2.Sort.plan | 4 ++-- .../testThetaJoinKeyPairs.Hash.plan | 4 ++-- .../testThetaJoinKeyPairs.Sort.plan | 4 ++-- 5 files changed, 16 insertions(+), 9 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanRewriteUtil.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanRewriteUtil.java index d58f920f2a..cc98300b50 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanRewriteUtil.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanRewriteUtil.java @@ -43,7 +43,14 @@ public static ExecutionBlock mergeExecutionBlocks(MasterPlan plan, ExecutionBloc // connect parent and grand children List grandChilds = plan.getChilds(child); for (ExecutionBlock eachGrandChild : grandChilds) { - plan.addConnect(eachGrandChild, parent, plan.getChannel(eachGrandChild, child).getShuffleType()); + DataChannel originalChannel = plan.getChannel(eachGrandChild, child); + DataChannel newChannel = new DataChannel(eachGrandChild, parent, originalChannel.getShuffleType(), + originalChannel.getShuffleOutputNum()); + newChannel.setSchema(originalChannel.getSchema()); + newChannel.setShuffleKeys(originalChannel.getShuffleKeys()); + newChannel.setStoreType(originalChannel.getStoreType()); + newChannel.setTransmitType(originalChannel.getTransmitType()); + plan.addConnect(newChannel); plan.disconnect(eachGrandChild, child); } diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery2.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery2.Hash.plan index 56c6ac137a..839afaf636 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery2.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery2.Hash.plan @@ -88,7 +88,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] ======================================================= [Outgoing] -[q_0000000000000_0000] 3 => 10 (type=HASH_SHUFFLE) +[q_0000000000000_0000] 3 => 10 (type=HASH_SHUFFLE, key=default.e.l_orderkey (INT4), num=32) [Enforcers] 0: type=Broadcast, tables=default.part @@ -125,7 +125,7 @@ Block Id: eb_0000000000000_0000_000010 [INTERMEDIATE] ======================================================= [Incoming] -[q_0000000000000_0000] 3 => 10 (type=HASH_SHUFFLE) +[q_0000000000000_0000] 3 => 10 (type=HASH_SHUFFLE, key=default.e.l_orderkey (INT4), num=32) [Outgoing] [q_0000000000000_0000] 10 => 11 (type=HASH_SHUFFLE, key=, num=1) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery2.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery2.Sort.plan index 56c6ac137a..839afaf636 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery2.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testBroadcastSubquery2.Sort.plan @@ -88,7 +88,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] ======================================================= [Outgoing] -[q_0000000000000_0000] 3 => 10 (type=HASH_SHUFFLE) +[q_0000000000000_0000] 3 => 10 (type=HASH_SHUFFLE, key=default.e.l_orderkey (INT4), num=32) [Enforcers] 0: type=Broadcast, tables=default.part @@ -125,7 +125,7 @@ Block Id: eb_0000000000000_0000_000010 [INTERMEDIATE] ======================================================= [Incoming] -[q_0000000000000_0000] 3 => 10 (type=HASH_SHUFFLE) +[q_0000000000000_0000] 3 => 10 (type=HASH_SHUFFLE, key=default.e.l_orderkey (INT4), num=32) [Outgoing] [q_0000000000000_0000] 10 => 11 (type=HASH_SHUFFLE, key=, num=1) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testThetaJoinKeyPairs.Hash.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testThetaJoinKeyPairs.Hash.plan index 8ca7e29eb2..c0b6f82ce6 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testThetaJoinKeyPairs.Hash.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testThetaJoinKeyPairs.Hash.plan @@ -56,7 +56,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] ======================================================= [Outgoing] -[q_0000000000000_0000] 3 => 6 (type=HASH_SHUFFLE) +[q_0000000000000_0000] 3 => 6 (type=HASH_SHUFFLE, key=default.r.r_regionkey (INT4), num=32) [Enforcers] 0: type=Broadcast, tables=default.r @@ -85,7 +85,7 @@ Block Id: eb_0000000000000_0000_000006 [INTERMEDIATE] ======================================================= [Incoming] -[q_0000000000000_0000] 3 => 6 (type=HASH_SHUFFLE) +[q_0000000000000_0000] 3 => 6 (type=HASH_SHUFFLE, key=default.r.r_regionkey (INT4), num=32) [Outgoing] [q_0000000000000_0000] 6 => 7 (type=RANGE_SHUFFLE, key=default.n.n_nationkey (INT4), num=32) diff --git a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testThetaJoinKeyPairs.Sort.plan b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testThetaJoinKeyPairs.Sort.plan index 8ca7e29eb2..c0b6f82ce6 100644 --- a/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testThetaJoinKeyPairs.Sort.plan +++ b/tajo-core/src/test/resources/results/TestInnerJoinWithSubQuery/testThetaJoinKeyPairs.Sort.plan @@ -56,7 +56,7 @@ Block Id: eb_0000000000000_0000_000003 [LEAF] ======================================================= [Outgoing] -[q_0000000000000_0000] 3 => 6 (type=HASH_SHUFFLE) +[q_0000000000000_0000] 3 => 6 (type=HASH_SHUFFLE, key=default.r.r_regionkey (INT4), num=32) [Enforcers] 0: type=Broadcast, tables=default.r @@ -85,7 +85,7 @@ Block Id: eb_0000000000000_0000_000006 [INTERMEDIATE] ======================================================= [Incoming] -[q_0000000000000_0000] 3 => 6 (type=HASH_SHUFFLE) +[q_0000000000000_0000] 3 => 6 (type=HASH_SHUFFLE, key=default.r.r_regionkey (INT4), num=32) [Outgoing] [q_0000000000000_0000] 6 => 7 (type=RANGE_SHUFFLE, key=default.n.n_nationkey (INT4), num=32)