public static OperatorResolver getContractResolver(Plan plan) { return new OperatorResolver(plan); }
public static OptimizerPlanNodeResolver getOptimizerPlanNodeResolver(OptimizedPlan plan) { return new OptimizerPlanNodeResolver(plan); }
@Override protected List<LocalPropertiesPair> createPossibleLocalProperties() { RequestedLocalProperties sort1 = new RequestedLocalProperties(Utils.createOrdering(this.keys1)); RequestedLocalProperties sort2 = new RequestedLocalProperties(Utils.createOrdering(this.keys2)); return Collections.singletonList(new LocalPropertiesPair(sort1, sort2)); }
public static TypeComparatorFactory<?> getShipComparator(Channel channel, ExecutionConfig executionConfig) { PlanNode source = channel.getSource(); Operator<?> javaOp = source.getProgramOperator(); TypeInformation<?> type = javaOp.getOperatorInfo().getOutputType(); return createComparator(type, channel.getShipStrategyKeys(), getSortOrders(channel.getShipStrategyKeys(), channel.getShipStrategySortOrder()), executionConfig); }
SingleRootJoiner() { super(new NoOpBinaryUdfOp<Nothing>(new NothingTypeInfo())); setParallelism(1); }
/** * Parametrizes the local strategy fields of a channel such that the channel produces the desired local properties. * * @param channel The channel to parametrize. */ public void parameterizeChannel(Channel channel) { LocalProperties current = channel.getLocalProperties(); if (isMetBy(current)) { // we are met, all is good channel.setLocalStrategy(LocalStrategy.NONE); } else if (this.ordering != null) { channel.setLocalStrategy(LocalStrategy.SORT, this.ordering.getInvolvedIndexes(), this.ordering.getFieldSortDirections()); } else if (this.groupedFields != null) { boolean[] dirs = new boolean[this.groupedFields.size()]; Arrays.fill(dirs, true); channel.setLocalStrategy(LocalStrategy.SORT, Utils.createOrderedFromSet(this.groupedFields), dirs); } else { channel.setLocalStrategy(LocalStrategy.NONE); } }
SourceCollectorVisitor sourceCollector = new SourceCollectorVisitor(); plan.accept(sourceCollector); for(GenericDataSourceBase<?, ?> s : sourceCollector.getSources()) { if(s.getName().equals("bigFile")) { this.setSourceStatistics(s, 10000000, 1000); DualInputPlanNode innerJoin = resolver.getNode("DummyJoiner");
GenericDataSourceBase<?, ?> ordersSource = cr.getNode(ORDERS); GenericDataSourceBase<?, ?> lineItemSource = cr.getNode(LINEITEM); SingleInputOperator<?, ?, ?> mapper = cr.getNode(MAPPER_NAME); DualInputOperator<?, ?, ?, ?> joiner = cr.getNode(JOIN_NAME); setSourceStatistics(ordersSource, orderSize, 100f); setSourceStatistics(lineItemSource, lineitemSize, 140f); final SinkPlanNode sink = or.getNode(SINK); final SingleInputPlanNode reducer = or.getNode(REDUCE_NAME); final SingleInputPlanNode combiner = reducer.getPredecessor() instanceof SingleInputPlanNode ? (SingleInputPlanNode) reducer.getPredecessor() : null; final DualInputPlanNode join = or.getNode(JOIN_NAME); final SingleInputPlanNode filteringMapper = or.getNode(MAPPER_NAME);
/** * Verifies that a robust repartitioning plan with a hash join is created in the absence of statistics. */ @Test public void testQueryNoStatistics() { try { Plan p = getTPCH3Plan(); p.setExecutionConfig(defaultExecutionConfig); // compile final OptimizedPlan plan = compileNoStats(p); final OptimizerPlanNodeResolver or = getOptimizerPlanNodeResolver(plan); // get the nodes from the final plan final SinkPlanNode sink = or.getNode(SINK); final SingleInputPlanNode reducer = or.getNode(REDUCE_NAME); final SingleInputPlanNode combiner = reducer.getPredecessor() instanceof SingleInputPlanNode ? (SingleInputPlanNode) reducer.getPredecessor() : null; final DualInputPlanNode join = or.getNode(JOIN_NAME); final SingleInputPlanNode filteringMapper = or.getNode(MAPPER_NAME); // verify the optimizer choices checkStandardStrategies(filteringMapper, join, combiner, reducer, sink); Assert.assertTrue(checkRepartitionShipStrategies(join, reducer, combiner)); Assert.assertTrue(checkHashJoinStrategies(join, reducer, true) || checkHashJoinStrategies(join, reducer, false)); } catch (Exception e) { e.printStackTrace(); Assert.fail(e.getMessage()); } }
@Test public void testCompileKMeansSingleStepWithStats() { Plan p = getKMeansPlan(); p.setExecutionConfig(new ExecutionConfig()); // set the statistics OperatorResolver cr = getContractResolver(p); GenericDataSourceBase<?, ?> pointsSource = cr.getNode(DATAPOINTS); GenericDataSourceBase<?, ?> centersSource = cr.getNode(CENTERS); setSourceStatistics(pointsSource, 100L * 1024 * 1024 * 1024, 32f); setSourceStatistics(centersSource, 1024 * 1024, 32f); OptimizedPlan plan = compileWithStats(p); checkPlan(plan); }
public void setSourceStatistics(GenericDataSourceBase<?, ?> source, long size, float recordWidth) { setSourceStatistics(source, new FileBaseStatistics(Long.MAX_VALUE, size, recordWidth)); }
@Override protected List<LocalPropertiesPair> createPossibleLocalProperties() { RequestedLocalProperties sort = new RequestedLocalProperties(Utils.createOrdering(this.keys1)); RequestedLocalProperties none = new RequestedLocalProperties(); return Collections.singletonList(new LocalPropertiesPair(sort, none)); }
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); }
case HASH_PARTITIONED: shipType = ShipStrategyType.PARTITION_HASH; partitionKeys = Utils.createOrderedFromSet(this.partitioningFields); sortDirection = null; partitioner = null;
/** * Statistics that push towards a repartition merge join. If the join blows the data volume up significantly, * re-exploiting the sorted order is cheaper. */ @Test public void testQueryWithStatsForRepartitionMerge() { Plan p = getTPCH3Plan(); p.setExecutionConfig(defaultExecutionConfig); // set compiler hints OperatorResolver cr = getContractResolver(p); DualInputOperator<?, ?, ?, ?> match = cr.getNode(JOIN_NAME); match.getCompilerHints().setFilterFactor(100f); testQueryGeneric(100L * 1024 * 1024 * 1024 * 1024, 100L * 1024 * 1024 * 1024 * 1024, 0.01f, 100f, false, true, false, false, true); }
@Override protected List<LocalPropertiesPair> createPossibleLocalProperties() { RequestedLocalProperties none = new RequestedLocalProperties(); RequestedLocalProperties sort = new RequestedLocalProperties(Utils.createOrdering(this.keys2)); return Collections.singletonList(new LocalPropertiesPair(none, sort)); }
public LocalProperties getLocalProperties() { if (this.localProps == null) { computeLocalPropertiesAfterShippingOnly(); switch (this.localStrategy) { case NONE: break; case SORT: case COMBININGSORT: this.localProps = LocalProperties.forOrdering(Utils.createOrdering(this.localKeys, this.localSortOrder)); break; default: throw new CompilerException("Unsupported local strategy for channel."); } } return this.localProps; }
this.ordering1 = Utils.createOrdering(this.keys1); this.ordering2 = Utils.createOrdering(this.keys2);
this.ordering1 = Utils.createOrdering(this.keys1); this.ordering2 = Utils.createOrdering(this.keys2);
break; case PARTITION_RANGE: this.globalProps.setRangePartitioned(Utils.createOrdering(this.shipKeys, this.shipSortOrder), this.dataDistribution); break; case FORWARD: