final Set<RequestedGlobalProperties> intGlobal = this.inConn.getInterestingProperties().getGlobalProperties(); DagConnection broadcastConnection = broadcastConnections.get(i); String broadcastConnectionName = broadcastConnectionNames.get(i); List<PlanNode> broadcastPlanCandidates = broadcastConnection.getSource().getAlternativePlans(estimator); for (PlanNode plan: broadcastPlanCandidates) { NamedChannel c = new NamedChannel(broadcastConnectionName, plan); DataExchangeMode exMode = DataExchangeMode.select(broadcastConnection.getDataExchangeMode(), ShipStrategyType.BROADCAST, broadcastConnection.isBreakingPipeline()); c.setShipStrategy(ShipStrategyType.BROADCAST, exMode); broadcastChannels.add(c); final ExecutionMode executionMode = this.inConn.getDataExchangeMode(); final boolean breaksPipeline = this.inConn.isBreakingPipeline(); continue; } else { this.inConn.setShipStrategy(ShipStrategyType.FORWARD); if (this.inConn.getShipStrategy() == null) { final Channel c = new Channel(child, this.inConn.getMaterializationMode()); igps.parameterizeChannel(c, parallelismChange, executionMode, breaksPipeline); final Channel c = new Channel(child, this.inConn.getMaterializationMode()); final ShipStrategyType shipStrategy = this.inConn.getShipStrategy(); final DataExchangeMode exMode = DataExchangeMode.select(executionMode, shipStrategy, breaksPipeline);
public SinkJoiner(OptimizerNode input1, OptimizerNode input2) { super(new NoOpBinaryUdfOp<Nothing>(new NothingTypeInfo())); DagConnection conn1 = new DagConnection(input1, this, null, ExecutionMode.PIPELINED); DagConnection conn2 = new DagConnection(input2, this, null, ExecutionMode.PIPELINED); this.input1 = conn1; this.input2 = conn2; setParallelism(1); }
public void clearInterestingProperties() { this.intProps = null; for (DagConnection conn : getIncomingConnections()) { conn.clearInterestingProperties(); } for (DagConnection conn : getBroadcastConnections()) { conn.clearInterestingProperties(); } }
@Override public boolean preVisit(OptimizerNode node) { // if the current node is a union if (node instanceof BinaryUnionNode) { int parallelism = -1; // set ship strategy of all outgoing connections to FORWARD. for (DagConnection conn : node.getOutgoingConnections()) { parallelism = conn.getTarget().getParallelism(); conn.setShipStrategy(ShipStrategyType.FORWARD); } // adjust parallelism to be same as successor node.setParallelism(parallelism); } // traverse the whole plan return true; }
boolean dynamicIn = conn.isOnDynamicPath(); anyDynamic |= dynamicIn; allDynamic &= dynamicIn; boolean dynamicIn = conn.isOnDynamicPath(); anyDynamic |= dynamicIn; allDynamic &= dynamicIn; if (!conn.getSource().isOnDynamicPath()) { conn.setMaterializationMode(conn.getMaterializationMode().makeCached());
} else { pred = contractToNode.get(children); conn = new DagConnection(pred, this, defaultExchangeMode); if (preSet != null) { conn.setShipStrategy(preSet);
final ExecutionMode input1Mode = this.input1.getDataExchangeMode(); final ExecutionMode input2Mode = this.input2.getDataExchangeMode(); final boolean input1breakPipeline = this.input1.isBreakingPipeline(); final boolean input2breakPipeline = this.input2.isBreakingPipeline(); continue; } else { this.input1.setShipStrategy(ShipStrategyType.FORWARD); continue; } else { this.input2.setShipStrategy(ShipStrategyType.FORWARD); Channel c1 = new Channel(child1, this.input1.getMaterializationMode()); if (this.input1.getShipStrategy() == null) { ShipStrategyType shipStrategy = this.input1.getShipStrategy(); DataExchangeMode exMode = DataExchangeMode.select(input1Mode, shipStrategy, input1breakPipeline); if (this.keys1 != null) { c1.setShipStrategy(this.input1.getShipStrategy(), this.keys1.toFieldList(), exMode); } else { c1.setShipStrategy(this.input1.getShipStrategy(), exMode); Channel c2 = new Channel(child2, this.input2.getMaterializationMode()); if (this.input2.getShipStrategy() == null) { ShipStrategyType shipStrategy = this.input2.getShipStrategy(); DataExchangeMode exMode = DataExchangeMode.select(input2Mode, shipStrategy, input2breakPipeline);
final int inDop = getPredecessorNode().getParallelism(); final ExecutionMode executionMode = this.input.getDataExchangeMode(); final boolean dopChange = parallelism != inDop; final boolean breakPipeline = this.input.isBreakingPipeline(); InterestingProperties ips = this.input.getInterestingProperties(); for (PlanNode p : subPlans) { for (RequestedGlobalProperties gp : ips.getGlobalProperties()) {
/** * Checks, if all outgoing connections have their interesting properties set from their target nodes. * * @return True, if on all outgoing connections, the interesting properties are set. False otherwise. */ public boolean haveAllOutputConnectionInterestingProperties() { for (DagConnection conn : getOutgoingConnections()) { if (conn.getInterestingProperties() == null) { return false; } } return true; }
public OptimizerNode getSecondPredecessorNode() { if(this.input2 != null) { return this.input2.getSource(); } else { return null; } }
this.terminationCriterionRootConnection.setInterestingProperties(new InterestingProperties()); this.terminationCriterion.accept(new InterestingPropertyVisitor(estimator)); this.rootConnection.setInterestingProperties(intProps); this.nextPartialSolution.accept(new InterestingPropertyVisitor(estimator)); this.rootConnection.clearInterestingProperties(); this.nextPartialSolution.accept(InterestingPropertiesClearer.INSTANCE); this.rootConnection.setInterestingProperties(intProps); this.nextPartialSolution.accept(new InterestingPropertyVisitor(estimator)); inProps.addGlobalProperties(new RequestedGlobalProperties()); inProps.addLocalProperties(new RequestedLocalProperties()); this.inConn.setInterestingProperties(inProps);
public int getMaxDepth() { int maxDepth = 0; for (DagConnection conn : getIncomingConnections()) { maxDepth = Math.max(maxDepth, conn.getMaxDepth()); } for (DagConnection conn : getBroadcastConnections()) { maxDepth = Math.max(maxDepth, conn.getMaxDepth()); } return maxDepth; }
@Override public String toString() { StringBuilder bld = new StringBuilder(); bld.append(getOperatorName()); bld.append(" (").append(getOperator().getName()).append(") "); int i = 1; for (DagConnection conn : getIncomingConnections()) { String shipStrategyName = conn.getShipStrategy() == null ? "null" : conn.getShipStrategy().name(); bld.append('(').append(i++).append(":").append(shipStrategyName).append(')'); } return bld.toString(); }
globPropsReq.parameterizeChannel(toNoOp, false, rootConnection.getDataExchangeMode(), false); locPropsReq.parameterizeChannel(toNoOp);
OptimizerNode successor = conn.getTarget();
assertFalse(sinkAfterFilter.getInputConnection().isBreakingPipeline()); assertFalse(sinkAfterJoin.getInputConnection().isBreakingPipeline()); assertFalse(sinkDirect.getInputConnection().isBreakingPipeline()); assertFalse(filterNode.getIncomingConnection().isBreakingPipeline()); assertFalse(mapNode.getIncomingConnection().isBreakingPipeline()); assertFalse(joinNode.getFirstIncomingConnection().isBreakingPipeline()); assertFalse(joinNode.getSecondIncomingConnection().isBreakingPipeline()); assertFalse(joinInput.getIncomingConnection().isBreakingPipeline());
final ExecutionMode input1Mode = this.input1.getDataExchangeMode(); final ExecutionMode input2Mode = this.input2.getDataExchangeMode(); final boolean input1breakPipeline = this.input1.isBreakingPipeline(); final boolean input2breakPipeline = this.input2.isBreakingPipeline(); continue; } else { this.input1.setShipStrategy(ShipStrategyType.FORWARD); continue; } else { this.input2.setShipStrategy(ShipStrategyType.FORWARD); Channel c1 = new Channel(child1, this.input1.getMaterializationMode()); if (this.input1.getShipStrategy() == null) { ShipStrategyType shipStrategy = this.input1.getShipStrategy(); DataExchangeMode exMode = DataExchangeMode.select(input1Mode, shipStrategy, input1breakPipeline); if (this.keys1 != null) { c1.setShipStrategy(this.input1.getShipStrategy(), this.keys1.toFieldList(), exMode); } else { c1.setShipStrategy(this.input1.getShipStrategy(), exMode); Channel c2 = new Channel(child2, this.input2.getMaterializationMode()); if (this.input2.getShipStrategy() == null) { ShipStrategyType shipStrategy = this.input2.getShipStrategy(); DataExchangeMode exMode = DataExchangeMode.select(input2Mode, shipStrategy, input2breakPipeline);
boolean dynamicIn = conn.isOnDynamicPath(); anyDynamic |= dynamicIn; allDynamic &= dynamicIn; boolean dynamicIn = conn.isOnDynamicPath(); anyDynamic |= dynamicIn; allDynamic &= dynamicIn; if (!conn.getSource().isOnDynamicPath()) { conn.setMaterializationMode(conn.getMaterializationMode().makeCached());
final int inDop = getPredecessorNode().getParallelism(); final ExecutionMode executionMode = this.input.getDataExchangeMode(); final boolean dopChange = parallelism != inDop; final boolean breakPipeline = this.input.isBreakingPipeline(); InterestingProperties ips = this.input.getInterestingProperties(); for (PlanNode p : subPlans) { for (RequestedGlobalProperties gp : ips.getGlobalProperties()) {
@Override public boolean preVisit(OptimizerNode node) { // if the current node is a union if (node instanceof BinaryUnionNode) { int parallelism = -1; // set ship strategy of all outgoing connections to FORWARD. for (DagConnection conn : node.getOutgoingConnections()) { parallelism = conn.getTarget().getParallelism(); conn.setShipStrategy(ShipStrategyType.FORWARD); } // adjust parallelism to be same as successor node.setParallelism(parallelism); } // traverse the whole plan return true; }