/** * Verifies that a robust repartitioning plan with a hash join is created in the absence of * statistics. */ @Test public void testQueryNoStatistics() { try { TPCHQuery3 query = new TPCHQuery3(); Plan p = query.getPlan(DEFAULT_PARALLELISM_STRING, IN_FILE, IN_FILE, OUT_FILE); 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("Output"); final SingleInputPlanNode reducer = or.getNode("AggLio"); final SingleInputPlanNode combiner = reducer.getPredecessor() instanceof SingleInputPlanNode ? (SingleInputPlanNode) reducer.getPredecessor() : null; final DualInputPlanNode join = or.getNode("JoinLiO"); final SingleInputPlanNode filteringMapper = or.getNode("FilterO"); // 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()); } }
/** * This test simulates a join of a big left side with a small right side inside of an iteration, * where the small side is on a static path. Currently the best execution plan is a * HYBRIDHASH_BUILD_SECOND_CACHED, where the small side is hashed and cached. This test also makes * sure that all relevant plans are correctly enumerated by the optimizer. */ @Test public void testCorrectChoosing() { try { Plan plan = getTestPlanRightStatic(""); SourceCollectorVisitor sourceCollector = new SourceCollectorVisitor(); plan.accept(sourceCollector); for (GenericDataSourceBase<?, ?> s : sourceCollector.getSources()) { if (s.getName().equals("bigFile")) { this.setSourceStatistics(s, 10000000, 1000); } else if (s.getName().equals("smallFile")) { this.setSourceStatistics(s, 100, 100); } } OptimizedPlan oPlan = compileNoStats(plan); OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(oPlan); DualInputPlanNode innerJoin = resolver.getNode("DummyJoiner"); // verify correct join strategy assertEquals(DriverStrategy.HYBRIDHASH_BUILD_SECOND_CACHED, innerJoin.getDriverStrategy()); assertEquals(TempMode.NONE, innerJoin.getInput1().getTempMode()); assertEquals(TempMode.NONE, innerJoin.getInput2().getTempMode()); new JobGraphGenerator().compileJobGraph(oPlan); } catch (Exception e) { System.err.println(e.getMessage()); e.printStackTrace(); fail("Test errored: " + e.getMessage()); } }
@Test public void testCrossWithLarge() { // construct the plan FileDataSource source1 = new FileDataSource(new DummyInputFormat(), IN_FILE, "Source 1"); FileDataSource source2 = new FileDataSource(new DummyInputFormat(), IN_FILE, "Source 2"); CrossOperator cross = CrossWithLargeOperator.builder(new DummyCrossStub()) .input1(source1) .input2(source2) .name("Cross") .build(); FileDataSink sink = new FileDataSink(new DummyOutputFormat(), OUT_FILE, cross, "Sink"); Plan plan = new Plan(sink); plan.setDefaultParallelism(DEFAULT_PARALLELISM); try { OptimizedPlan oPlan = compileNoStats(plan); OptimizerPlanNodeResolver resolver = new OptimizerPlanNodeResolver(oPlan); DualInputPlanNode crossPlanNode = resolver.getNode("Cross"); Channel in1 = crossPlanNode.getInput1(); Channel in2 = crossPlanNode.getInput2(); assertEquals(ShipStrategyType.BROADCAST, in1.getShipStrategy()); assertEquals(ShipStrategyType.FORWARD, in2.getShipStrategy()); } catch (CompilerException ce) { ce.printStackTrace(); fail("The pact compiler is unable to compile this plan correctly."); } }
@Test public void testJavaApiWithDeferredSoltionSetUpdateWithMapper() { try { Plan plan = getJavaTestPlan(false, true); OptimizedPlan oPlan = compileNoStats(plan); OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(oPlan); DualInputPlanNode joinWithInvariantNode = resolver.getNode(JOIN_WITH_INVARIANT_NAME); DualInputPlanNode joinWithSolutionSetNode = resolver.getNode(JOIN_WITH_SOLUTION_SET); SingleInputPlanNode worksetReducer = resolver.getNode(NEXT_WORKSET_REDUCER_NAME); SingleInputPlanNode deltaMapper = resolver.getNode(SOLUTION_DELTA_MAPPER_NAME); // iteration preserves partitioning in reducer, so the first partitioning is out of the loop, // the in-loop partitioning is before the final reducer // verify joinWithInvariant assertEquals(ShipStrategyType.FORWARD, joinWithInvariantNode.getInput1().getShipStrategy()); assertEquals( ShipStrategyType.PARTITION_HASH, joinWithInvariantNode.getInput2().getShipStrategy()); assertEquals(new FieldList(1, 2), joinWithInvariantNode.getKeysForInput1()); assertEquals(new FieldList(1, 2), joinWithInvariantNode.getKeysForInput2()); // verify joinWithSolutionSet assertEquals( ShipStrategyType.PARTITION_HASH, joinWithSolutionSetNode.getInput1().getShipStrategy()); assertEquals(ShipStrategyType.FORWARD, joinWithSolutionSetNode.getInput2().getShipStrategy()); assertEquals(new FieldList(1, 0), joinWithSolutionSetNode.getKeysForInput1()); // verify reducer assertEquals(ShipStrategyType.PARTITION_HASH, worksetReducer.getInput().getShipStrategy()); assertEquals(new FieldList(1, 2), worksetReducer.getKeys(0)); // currently, the system may partition before or after the mapper ShipStrategyType ss1 = deltaMapper.getInput().getShipStrategy(); ShipStrategyType ss2 = deltaMapper.getOutgoingChannels().get(0).getShipStrategy(); assertTrue( (ss1 == ShipStrategyType.FORWARD && ss2 == ShipStrategyType.PARTITION_HASH) || (ss2 == ShipStrategyType.FORWARD && ss1 == ShipStrategyType.PARTITION_HASH)); new JobGraphGenerator().compileJobGraph(oPlan); } catch (Exception e) { System.err.println(e.getMessage()); e.printStackTrace(); fail("Test errored: " + e.getMessage()); } }
/** * This test makes sure that only a HYBRIDHASH on the static path is transformed to the cached * variant */ @Test public void testRightSideCountercheck() { try { Plan plan = getTestPlanRightStatic(Optimizer.HINT_LOCAL_STRATEGY_HASH_BUILD_FIRST); OptimizedPlan oPlan = compileNoStats(plan); OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(oPlan); DualInputPlanNode innerJoin = resolver.getNode("DummyJoiner"); // verify correct join strategy assertEquals(DriverStrategy.HYBRIDHASH_BUILD_FIRST, innerJoin.getDriverStrategy()); assertEquals(TempMode.NONE, innerJoin.getInput1().getTempMode()); assertEquals(TempMode.CACHED, innerJoin.getInput2().getTempMode()); new JobGraphGenerator().compileJobGraph(oPlan); } catch (Exception e) { System.err.println(e.getMessage()); e.printStackTrace(); fail("Test errored: " + e.getMessage()); } }
private void testQueryGeneric( Plan p, long orderSize, long lineitemSize, float orderSelectivity, float joinSelectivity, boolean broadcastOkay, boolean partitionedOkay, boolean hashJoinFirstOkay, boolean hashJoinSecondOkay, boolean mergeJoinOkay) { try { // set statistics OperatorResolver cr = getContractResolver(p); FileDataSource ordersSource = cr.getNode(ORDERS); FileDataSource lineItemSource = cr.getNode(LINEITEM); MapOperator mapper = cr.getNode(MAPPER_NAME); JoinOperator joiner = cr.getNode(JOIN_NAME); setSourceStatistics(ordersSource, orderSize, 100f); setSourceStatistics(lineItemSource, lineitemSize, 140f); mapper.getCompilerHints().setAvgOutputRecordSize(16f); mapper.getCompilerHints().setFilterFactor(orderSelectivity); joiner.getCompilerHints().setFilterFactor(joinSelectivity); // compile final OptimizedPlan plan = compileWithStats(p); final OptimizerPlanNodeResolver or = getOptimizerPlanNodeResolver(plan); // get the nodes from the final plan final SinkPlanNode sink = or.getNode("Output"); final SingleInputPlanNode reducer = or.getNode("AggLio"); final SingleInputPlanNode combiner = reducer.getPredecessor() instanceof SingleInputPlanNode ? (SingleInputPlanNode) reducer.getPredecessor() : null; final DualInputPlanNode join = or.getNode("JoinLiO"); final SingleInputPlanNode filteringMapper = or.getNode("FilterO"); checkStandardStrategies(filteringMapper, join, combiner, reducer, sink); // check the possible variants and that the variant ia allowed in this specific setting if (checkBroadcastShipStrategies(join, reducer, combiner)) { Assert.assertTrue("Broadcast join incorrectly chosen.", broadcastOkay); if (checkHashJoinStrategies(join, reducer, true)) { Assert.assertTrue("Hash join (build orders) incorrectly chosen", hashJoinFirstOkay); } else if (checkHashJoinStrategies(join, reducer, false)) { Assert.assertTrue("Hash join (build lineitem) incorrectly chosen", hashJoinSecondOkay); } else if (checkBroadcastMergeJoin(join, reducer)) { Assert.assertTrue("Merge join incorrectly chosen", mergeJoinOkay); } else { Assert.fail("Plan has no correct hash join or merge join strategies."); } } else if (checkRepartitionShipStrategies(join, reducer, combiner)) { Assert.assertTrue("Partitioned join incorrectly chosen.", partitionedOkay); if (checkHashJoinStrategies(join, reducer, true)) { Assert.assertTrue("Hash join (build orders) incorrectly chosen", hashJoinFirstOkay); } else if (checkHashJoinStrategies(join, reducer, false)) { Assert.assertTrue("Hash join (build lineitem) incorrectly chosen", hashJoinSecondOkay); } else if (checkRepartitionMergeJoin(join, reducer)) { Assert.assertTrue("Merge join incorrectly chosen", mergeJoinOkay); } else { Assert.fail("Plan has no correct hash join or merge join strategies."); } } else { Assert.fail("Plan has neither correct BC join or partitioned join configuration."); } } catch (Exception e) { e.printStackTrace(); Assert.fail(e.getMessage()); } }