/** * Gets the program operator that this node represents in the plan. * * @return The program operator this node represents in the plan. */ public Operator<?> getProgramOperator() { return this.template.getOperator(); }
@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(); }
@Override public String toString() { return "(" + this.branchingNode.getOperator() + ") [" + this.joinedPathsVector + "]"; } }
public String toString() { StringBuilder buf = new StringBuilder(50); buf.append("Connection: "); if (this.source == null) { buf.append("null"); } else { buf.append(this.source.getOperator().getName()); buf.append('(').append(this.source.getOperatorName()).append(')'); } buf.append(" -> "); if (this.shipStrategy != null) { buf.append('['); buf.append(this.shipStrategy.name()); buf.append(']').append(' '); } if (this.target == null) { buf.append("null"); } else { buf.append(this.target.getOperator().getName()); buf.append('(').append(this.target.getOperatorName()).append(')'); } return buf.toString(); } }
/** * Gets the operator for which this optimizer sink node was created. * * @return The node's underlying operator. */ @Override public GenericDataSinkBase<?> getOperator() { return (GenericDataSinkBase<?>) super.getOperator(); }
if (n instanceof DataSinkNode) { type = "sink"; contents = n.getOperator().toString(); } else if (n instanceof DataSourceNode) { type = "source"; contents = n.getOperator().toString(); contents = n.getOperator().getName(); contents = n.getOperator().getName(); contents = n.getOperator().getName(); if (inputNum == 0) { child1name += child1name.length() > 0 ? ", " : ""; child1name += source.getOptimizerNode().getOperator().getName() + " (id: " + this.nodeIds.get(source) + ")"; } else if (inputNum == 1) { child2name += child2name.length() > 0 ? ", " : ""; child2name += source.getOptimizerNode().getOperator().getName() + " (id: " + this.nodeIds.get(source) + ")"; if (n.getOperator().getCompilerHints() != null) { CompilerHints hints = n.getOperator().getCompilerHints(); CompilerHints defaults = new CompilerHints();
@Override public DualInputOperator<?, ?, ?, ?> getOperator() { return (DualInputOperator<?, ?, ?, ?>) super.getOperator(); }
/** * Gets the contract object for this data source node. * * @return The contract. */ @Override public GenericDataSourceBase<?, ?> getOperator() { return (GenericDataSourceBase<?, ?>) super.getOperator(); }
@Override public SingleInputOperator<?, ?, ?> getOperator() { return (SingleInputOperator<?, ?, ?>) super.getOperator(); }
public ResourceSpec getPreferredResources() { return this.template.getOperator().getPreferredResources(); }
public ResourceSpec getMinResources() { return this.template.getOperator().getMinResources(); }
@SuppressWarnings("unchecked") private static <T> TypeInformation<T> getTypeInfoFromSource(SourcePlanNode node) { Operator<?> op = node.getOptimizerNode().getOperator(); if (op instanceof GenericDataSourceBase) { return ((GenericDataSourceBase<T, ?>) op).getOperatorInfo().getOutputType(); } else { throw new RuntimeException("Wrong operator type found in post pass."); } }
/** * This function connects the operators that produce the broadcast inputs to this operator. * * @param operatorToNode The map from program operators to optimizer nodes. * @param defaultExchangeMode The data exchange mode to use, if the operator does not * specify one. * * @throws CompilerException */ public void setBroadcastInputs(Map<Operator<?>, OptimizerNode> operatorToNode, ExecutionMode defaultExchangeMode) { // skip for Operators that don't support broadcast variables if (!(getOperator() instanceof AbstractUdfOperator<?, ?>)) { return; } // get all broadcast inputs AbstractUdfOperator<?, ?> operator = ((AbstractUdfOperator<?, ?>) getOperator()); // create connections and add them for (Map.Entry<String, Operator<?>> input : operator.getBroadcastInputs().entrySet()) { OptimizerNode predecessor = operatorToNode.get(input.getValue()); DagConnection connection = new DagConnection(predecessor, this, ShipStrategyType.BROADCAST, defaultExchangeMode); addBroadcastConnection(input.getKey(), connection); predecessor.addOutgoingConnection(connection); } }
private SinkPlanNode findSink(Collection<SinkPlanNode> collection, String name) { for (SinkPlanNode node : collection) { String nodeName = node.getOptimizerNode().getOperator().getName(); if (nodeName != null && nodeName.equals(name)) { return node; } } throw new IllegalArgumentException("No node with that name was found."); } }
private SinkPlanNode findSink(Collection<SinkPlanNode> collection, String name) { for (SinkPlanNode node : collection) { String nodeName = node.getOptimizerNode().getOperator().getName(); if (nodeName != null && nodeName.equals(name)) { return node; } } throw new IllegalArgumentException("No node with that name was found."); } }
Operator<?> c = n.getOriginalOptimizerNode().getOperator(); String name = c.getName(); for (Iterator<PlanNode> iter = list.iterator(); iter.hasNext();) { PlanNode in = iter.next(); if (in.getOriginalOptimizerNode().getOperator() == c) {
if (getOperator() == null || getOperator().getCompilerHints() == null) { return ; CompilerHints hints = getOperator().getCompilerHints(); if (hints.getOutputSize() >= 0) { this.estimatedOutputSize = hints.getOutputSize();
SingleInputPlanNode sn = (SingleInputPlanNode) node; if (!(sn.getOptimizerNode().getOperator() instanceof SingleInputOperator)) { if(sn.getOptimizerNode().getOperator() instanceof NoOpUnaryUdfOp) { traverseChannel(sn.getInput()); return; SingleInputOperator<?, ?, ?> singleInputOperator = (SingleInputOperator<?, ?, ?>) sn.getOptimizerNode().getOperator(); DualInputPlanNode dn = (DualInputPlanNode) node; if (!(dn.getOptimizerNode().getOperator() instanceof DualInputOperator)) { throw new RuntimeException("Wrong operator type found in post pass."); DualInputOperator<?, ?, ?, ?> dualInputOperator = (DualInputOperator<?, ?, ?, ?>) dn.getOptimizerNode().getOperator();
if (optNode.getOperator().getCompilerHints() != null) { CompilerHints hints = optNode.getOperator().getCompilerHints(); CompilerHints defaults = new CompilerHints();
final TypeInformation<?> sourceOutputType = sourceNode.getOptimizerNode().getOperator().getOperatorInfo().getOutputType(); final TypeInformation<IntermediateSampleData> isdTypeInformation = TypeExtractor.getForClass(IntermediateSampleData.class); final UnaryOperatorInformation sipOperatorInformation = new UnaryOperatorInformation(sourceOutputType, isdTypeInformation);