/** * 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()); } }
/** * This tests whether a HYBRIDHASH_BUILD_SECOND is correctly transformed to a HYBRIDHASH_BUILD_SECOND_CACHED * when inside of an iteration an on the static path */ @Test public void testRightSide() { try { Plan plan = getTestPlanRightStatic(Optimizer.HINT_LOCAL_STRATEGY_HASH_BUILD_SECOND); 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()); } }
/** * This test makes sure that only a HYBRIDHASH on the static path is transformed to the cached variant */ @Test public void testLeftSideCountercheck() { try { Plan plan = getTestPlanLeftStatic(Optimizer.HINT_LOCAL_STRATEGY_HASH_BUILD_SECOND); OptimizedPlan oPlan = compileNoStats(plan); OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(oPlan); DualInputPlanNode innerJoin = resolver.getNode("DummyJoiner"); // verify correct join strategy assertEquals(DriverStrategy.HYBRIDHASH_BUILD_SECOND, innerJoin.getDriverStrategy()); assertEquals(TempMode.CACHED, 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()); } }
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);
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);
DualInputPlanNode innerJoin = resolver.getNode("DummyJoiner");
@Test public void testMultiSolutionSetJoinPlan() { try { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); @SuppressWarnings("unchecked") DataSet<Tuple2<Long, Double>> inputData = env.fromElements(new Tuple2<Long, Double>(1L, 1.0)); DataSet<Tuple2<Long, Double>> result = constructPlan(inputData, 10); // add two sinks, to test the case of branching after an iteration result.output(new DiscardingOutputFormat<Tuple2<Long, Double>>()); result.output(new DiscardingOutputFormat<Tuple2<Long, Double>>()); Plan p = env.createProgramPlan(); OptimizedPlan optPlan = compileNoStats(p); OptimizerPlanNodeResolver or = getOptimizerPlanNodeResolver(optPlan); DualInputPlanNode join1 = or.getNode(JOIN_1); DualInputPlanNode join2 = or.getNode(JOIN_2); assertEquals(DriverStrategy.HYBRIDHASH_BUILD_FIRST, join1.getDriverStrategy()); assertEquals(DriverStrategy.HYBRIDHASH_BUILD_SECOND, join2.getDriverStrategy()); assertEquals(ShipStrategyType.PARTITION_HASH, join1.getInput2().getShipStrategy()); assertEquals(ShipStrategyType.PARTITION_HASH, join2.getInput1().getShipStrategy()); assertEquals(SolutionSetPlanNode.class, join1.getInput1().getSource().getClass()); assertEquals(SolutionSetPlanNode.class, join2.getInput2().getSource().getClass()); new JobGraphGenerator().compileJobGraph(optPlan); } catch (Exception e) { System.err.println(e.getMessage()); e.printStackTrace(); fail("Test erroneous: " + e.getMessage()); } }
/** * 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 testCrossWithLarge() { // construct the plan ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(DEFAULT_PARALLELISM); DataSet<Long> set1 = env.generateSequence(0,1); DataSet<Long> set2 = env.generateSequence(0,1); set1.crossWithHuge(set2).name("Cross") .output(new DiscardingOutputFormat<Tuple2<Long, Long>>()); try { Plan plan = env.createProgramPlan(); 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."); } } }
/** * This tests whether a HYBRIDHASH_BUILD_FIRST is correctly transformed to a HYBRIDHASH_BUILD_FIRST_CACHED * when inside of an iteration an on the static path */ @Test public void testLeftSide() { try { Plan plan = getTestPlanLeftStatic(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_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 testCrossWithSmall() { // construct the plan ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(DEFAULT_PARALLELISM); DataSet<Long> set1 = env.generateSequence(0,1); DataSet<Long> set2 = env.generateSequence(0,1); set1.crossWithTiny(set2).name("Cross") .output(new DiscardingOutputFormat<Tuple2<Long, Long>>()); try { Plan plan = env.createProgramPlan(); OptimizedPlan oPlan = compileWithStats(plan); OptimizerPlanNodeResolver resolver = new OptimizerPlanNodeResolver(oPlan); DualInputPlanNode crossPlanNode = resolver.getNode("Cross"); Channel in1 = crossPlanNode.getInput1(); Channel in2 = crossPlanNode.getInput2(); assertEquals(ShipStrategyType.FORWARD, in1.getShipStrategy()); assertEquals(ShipStrategyType.BROADCAST, in2.getShipStrategy()); } catch(CompilerException ce) { ce.printStackTrace(); fail("The Flink optimizer is unable to compile this plan correctly."); } }