@Test public void testBroadcastHashFirstTest() { try { DualInputPlanNode node = createPlanAndGetJoinNode(JoinHint.BROADCAST_HASH_FIRST); assertEquals(ShipStrategyType.BROADCAST, node.getInput1().getShipStrategy()); assertEquals(ShipStrategyType.FORWARD, node.getInput2().getShipStrategy()); assertEquals(DriverStrategy.HYBRIDHASH_BUILD_FIRST, node.getDriverStrategy()); } catch (Exception e) { e.printStackTrace(); fail(e.getClass().getSimpleName() + ": " + e.getMessage()); } }
@Test public void testPartitionHashFirstTest() { try { DualInputPlanNode node = createPlanAndGetJoinNode(JoinHint.REPARTITION_HASH_FIRST); assertEquals(ShipStrategyType.PARTITION_HASH, node.getInput1().getShipStrategy()); assertEquals(ShipStrategyType.PARTITION_HASH, node.getInput2().getShipStrategy()); assertEquals(DriverStrategy.HYBRIDHASH_BUILD_FIRST, node.getDriverStrategy()); } catch (Exception e) { e.printStackTrace(); fail(e.getClass().getSimpleName() + ": " + e.getMessage()); } }
@Test public void testPartitionHashSecondTest() { try { DualInputPlanNode node = createPlanAndGetJoinNode(JoinHint.REPARTITION_HASH_SECOND); assertEquals(ShipStrategyType.PARTITION_HASH, node.getInput1().getShipStrategy()); assertEquals(ShipStrategyType.PARTITION_HASH, node.getInput2().getShipStrategy()); assertEquals(DriverStrategy.HYBRIDHASH_BUILD_SECOND, node.getDriverStrategy()); } catch (Exception e) { e.printStackTrace(); fail(e.getClass().getSimpleName() + ": " + e.getMessage()); } }
@Test public void testBroadcastHashSecondTest() { try { DualInputPlanNode node = createPlanAndGetJoinNode(JoinHint.BROADCAST_HASH_SECOND); assertEquals(ShipStrategyType.FORWARD, node.getInput1().getShipStrategy()); assertEquals(ShipStrategyType.BROADCAST, node.getInput2().getShipStrategy()); assertEquals(DriverStrategy.HYBRIDHASH_BUILD_SECOND, node.getDriverStrategy()); } catch (Exception e) { e.printStackTrace(); fail(e.getClass().getSimpleName() + ": " + e.getMessage()); } }
@Test public void testPartitionSortMergeTest() { try { DualInputPlanNode node = createPlanAndGetJoinNode(JoinHint.REPARTITION_SORT_MERGE); assertEquals(ShipStrategyType.PARTITION_HASH, node.getInput1().getShipStrategy()); assertEquals(ShipStrategyType.PARTITION_HASH, node.getInput2().getShipStrategy()); assertEquals(DriverStrategy.INNER_MERGE, node.getDriverStrategy()); } catch (Exception e) { e.printStackTrace(); fail(e.getClass().getSimpleName() + ": " + e.getMessage()); } }
@Test public void testOptimizerChoosesTest() { try { DualInputPlanNode node = createPlanAndGetJoinNode(JoinHint.OPTIMIZER_CHOOSES); assertEquals(ShipStrategyType.PARTITION_HASH, node.getInput1().getShipStrategy()); assertEquals(ShipStrategyType.PARTITION_HASH, node.getInput2().getShipStrategy()); assertTrue(DriverStrategy.HYBRIDHASH_BUILD_FIRST == node.getDriverStrategy() || DriverStrategy.HYBRIDHASH_BUILD_SECOND == node.getDriverStrategy()); } catch (Exception e) { e.printStackTrace(); fail(e.getClass().getSimpleName() + ": " + e.getMessage()); } }
@Override public boolean preVisit(PlanNode visitable) { if (visitable instanceof DualInputPlanNode) { DualInputPlanNode node = (DualInputPlanNode) visitable; Channel c1 = node.getInput1(); Channel c2 = node.getInput2(); Assert.assertEquals("Incompatible shipping strategy chosen for match", ShipStrategyType.FORWARD, c1.getShipStrategy()); Assert.assertEquals("Incompatible shipping strategy chosen for match", ShipStrategyType.PARTITION_HASH, c2.getShipStrategy()); return false; } return true; }
private boolean checkRepartitionShipStrategies(DualInputPlanNode join, SingleInputPlanNode reducer, SingleInputPlanNode combiner) { if (ShipStrategyType.PARTITION_HASH == join.getInput1().getShipStrategy() && ShipStrategyType.PARTITION_HASH == join.getInput2().getShipStrategy() && ShipStrategyType.FORWARD == reducer.getInput().getShipStrategy()) { // check combiner Assert.assertNull("Plan should not have a combiner", combiner); return true; } else { return false; } }
@Override public boolean preVisit(PlanNode visitable) { if (visitable instanceof DualInputPlanNode && visitable.getProgramOperator() instanceof InnerJoinOperatorBase) { DualInputPlanNode node = ((DualInputPlanNode) visitable); final Channel inConn1 = node.getInput1(); final Channel inConn2 = node.getInput2(); Assert.assertTrue("Join should just forward the input if it is already partitioned", inConn1.getShipStrategy() == ShipStrategyType.FORWARD); Assert.assertTrue("Join should just forward the input if it is already partitioned", inConn2.getShipStrategy() == ShipStrategyType.FORWARD); return false; } return true; }
/** * 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()); } }
private boolean checkBroadcastShipStrategies(DualInputPlanNode join, SingleInputPlanNode reducer, SingleInputPlanNode combiner) { if (ShipStrategyType.BROADCAST == join.getInput1().getShipStrategy() && ShipStrategyType.FORWARD == join.getInput2().getShipStrategy() && ShipStrategyType.PARTITION_HASH == reducer.getInput().getShipStrategy()) { // check combiner Assert.assertNotNull("Plan should have a combiner", combiner); Assert.assertEquals(ShipStrategyType.FORWARD, combiner.getInput().getShipStrategy()); return true; } else { return false; } }
private boolean checkRepartitionMergeJoin(DualInputPlanNode join, SingleInputPlanNode reducer) { if (DriverStrategy.INNER_MERGE == join.getDriverStrategy()) { // driver keys Assert.assertEquals(set0, join.getKeysForInput1()); Assert.assertEquals(set0, join.getKeysForInput2()); // local strategies Assert.assertEquals(LocalStrategy.SORT, join.getInput1().getLocalStrategy()); Assert.assertEquals(LocalStrategy.SORT, join.getInput2().getLocalStrategy()); Assert.assertEquals(LocalStrategy.NONE, reducer.getInput().getLocalStrategy()); // local strategy keys Assert.assertEquals(set01, join.getInput1().getLocalStrategyKeys()); Assert.assertEquals(set0, join.getInput2().getLocalStrategyKeys()); Assert.assertTrue(join.getInput1().getLocalStrategySortOrder()[0] == join.getInput2().getLocalStrategySortOrder()[0]); Assert.assertEquals(set01, reducer.getKeys(0)); Assert.assertTrue(Arrays.equals(join.getInput1().getLocalStrategySortOrder(), reducer.getSortOrders(0))); return true; } else { return false; } }
private boolean checkBroadcastMergeJoin(DualInputPlanNode join, SingleInputPlanNode reducer) { if (DriverStrategy.INNER_MERGE == join.getDriverStrategy()) { // driver keys Assert.assertEquals(set0, join.getKeysForInput1()); Assert.assertEquals(set0, join.getKeysForInput2()); // local strategies Assert.assertEquals(LocalStrategy.SORT, join.getInput1().getLocalStrategy()); Assert.assertEquals(LocalStrategy.SORT, join.getInput2().getLocalStrategy()); Assert.assertEquals(LocalStrategy.COMBININGSORT, reducer.getInput().getLocalStrategy()); // local strategy keys Assert.assertEquals(set0, join.getInput1().getLocalStrategyKeys()); Assert.assertEquals(set0, join.getInput2().getLocalStrategyKeys()); Assert.assertTrue(Arrays.equals(join.getInput1().getLocalStrategySortOrder(), join.getInput2().getLocalStrategySortOrder())); Assert.assertEquals(set01, reducer.getInput().getLocalStrategyKeys()); Assert.assertEquals(set01, reducer.getKeys(0)); Assert.assertTrue(Arrays.equals(reducer.getInput().getLocalStrategySortOrder(), reducer.getSortOrders(0))); return true; } else { return false; } }
@Override public boolean preVisit(PlanNode visitable) { if (visitable instanceof WorksetIterationPlanNode) { PlanNode deltaNode = ((WorksetIterationPlanNode) visitable).getSolutionSetDeltaPlanNode(); //get the CoGroup DualInputPlanNode dpn = (DualInputPlanNode) deltaNode.getInputs().iterator().next().getSource(); Channel in1 = dpn.getInput1(); Channel in2 = dpn.getInput2(); Assert.assertTrue(in1.getLocalProperties().getOrdering() == null); Assert.assertTrue(in2.getLocalProperties().getOrdering() != null); Assert.assertTrue(in2.getLocalProperties().getOrdering().getInvolvedIndexes().contains(0)); Assert.assertTrue(in1.getShipStrategy() == ShipStrategyType.FORWARD); Assert.assertTrue(in2.getShipStrategy() == ShipStrategyType.PARTITION_HASH); return false; } return true; }
@Test public void CoGroupWithSameDistributionTest() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); DataSet<Tuple3<Integer, Integer, Integer>> set1 = env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); DataSet<Tuple3<Integer, Integer, Integer>> set2 = env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); TestDistribution testDistribution1 = new TestDistribution(3); TestDistribution testDistribution2 = new TestDistribution(3); DataSet<Tuple3<Integer, Integer, Integer>> coGrouped = DataSetUtils.partitionByRange(set1, testDistribution1, 0) .coGroup(DataSetUtils.partitionByRange(set2, testDistribution2, 0)) .where(0).equalTo(0).with(new CoGroupFunc()); coGrouped.output(new DiscardingOutputFormat<Tuple3<Integer, Integer, Integer>>()); Plan plan = env.createProgramPlan(); OptimizedPlan oPlan = compileWithStats(plan); SinkPlanNode sink = oPlan.getDataSinks().iterator().next(); DualInputPlanNode coGroup= (DualInputPlanNode)sink.getInput().getSource(); Channel input1 = coGroup.getInput1(); Channel input2 = coGroup.getInput2(); assertEquals(ShipStrategyType.FORWARD, input1.getShipStrategy()); assertEquals(ShipStrategyType.FORWARD, input2.getShipStrategy()); }
@Test public void JoinWithSameDistributionTest() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); DataSet<Tuple3<Integer, Integer, Integer>> set1 = env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); DataSet<Tuple3<Integer, Integer, Integer>> set2 = env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); TestDistribution dist1 = new TestDistribution(3); TestDistribution dist2 = new TestDistribution(3); DataSet<Tuple3<Integer, Integer, Integer>> coGrouped = DataSetUtils.partitionByRange(set1, dist1, 0) .join(DataSetUtils.partitionByRange(set2, dist2, 0)) .where(0).equalTo(0).with(new JoinFunc()); coGrouped.output(new DiscardingOutputFormat<Tuple3<Integer, Integer, Integer>>()); Plan plan = env.createProgramPlan(); OptimizedPlan oPlan = compileWithStats(plan); SinkPlanNode sink = oPlan.getDataSinks().iterator().next(); DualInputPlanNode join = (DualInputPlanNode)sink.getInput().getSource(); Channel input1 = join.getInput1(); Channel input2 = join.getInput2(); assertEquals(ShipStrategyType.FORWARD, input1.getShipStrategy()); assertEquals(ShipStrategyType.FORWARD, input2.getShipStrategy()); }
@Test public void JoinWithDifferentDistributionTest() throws Exception{ ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); DataSet<Tuple3<Integer, Integer, Integer>> set1 = env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); DataSet<Tuple3<Integer, Integer, Integer>> set2 = env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); TestDistribution dist1 = new TestDistribution(3); TestDistribution dist2 = new TestDistribution(4); DataSet<Tuple3<Integer, Integer, Integer>> coGrouped = DataSetUtils.partitionByRange(set1, dist1, 0) .join(DataSetUtils.partitionByRange(set2, dist2, 0)) .where(0).equalTo(0).with(new JoinFunc()); coGrouped.output(new DiscardingOutputFormat<Tuple3<Integer, Integer, Integer>>()); Plan plan = env.createProgramPlan(); OptimizedPlan oPlan = compileWithStats(plan); SinkPlanNode sink = oPlan.getDataSinks().iterator().next(); DualInputPlanNode join = (DualInputPlanNode)sink.getInput().getSource(); Channel input1 = join.getInput1(); Channel input2 = join.getInput2(); assertEquals(ShipStrategyType.PARTITION_HASH, input1.getShipStrategy()); assertEquals(ShipStrategyType.PARTITION_HASH, input2.getShipStrategy()); }
@Test public void CoGroupWithDifferentDistributionTest() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); DataSet<Tuple3<Integer, Integer, Integer>> set1 = env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); DataSet<Tuple3<Integer, Integer, Integer>> set2 = env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); TestDistribution testDistribution1 = new TestDistribution(3); TestDistribution testDistribution2 = new TestDistribution(2); DataSet<Tuple3<Integer, Integer, Integer>> coGrouped = DataSetUtils.partitionByRange(set1, testDistribution1, 0) .coGroup(DataSetUtils.partitionByRange(set2, testDistribution2, 0)) .where(0).equalTo(0).with(new CoGroupFunc()); coGrouped.output(new DiscardingOutputFormat<Tuple3<Integer, Integer, Integer>>()); Plan plan = env.createProgramPlan(); OptimizedPlan oPlan = compileWithStats(plan); SinkPlanNode sink = oPlan.getDataSinks().iterator().next(); DualInputPlanNode coGroup= (DualInputPlanNode)sink.getInput().getSource(); Channel input1 = coGroup.getInput1(); Channel input2 = coGroup.getInput2(); assertEquals(ShipStrategyType.PARTITION_HASH, input1.getShipStrategy()); assertEquals(ShipStrategyType.PARTITION_HASH, input2.getShipStrategy()); }
private boolean checkHashJoinStrategies(DualInputPlanNode join, SingleInputPlanNode reducer, boolean buildFirst) { if ((buildFirst && DriverStrategy.HYBRIDHASH_BUILD_FIRST == join.getDriverStrategy()) || (!buildFirst && DriverStrategy.HYBRIDHASH_BUILD_SECOND == join.getDriverStrategy())) { // driver keys Assert.assertEquals(set0, join.getKeysForInput1()); Assert.assertEquals(set0, join.getKeysForInput2()); // local strategies Assert.assertEquals(LocalStrategy.NONE, join.getInput1().getLocalStrategy()); Assert.assertEquals(LocalStrategy.NONE, join.getInput2().getLocalStrategy()); Assert.assertEquals(LocalStrategy.COMBININGSORT, reducer.getInput().getLocalStrategy()); // local strategy keys Assert.assertEquals(set01, reducer.getInput().getLocalStrategyKeys()); Assert.assertEquals(set01, reducer.getKeys(0)); Assert.assertTrue(Arrays.equals(reducer.getInput().getLocalStrategySortOrder(), reducer.getSortOrders(0))); return true; } else { return false; } }
@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."); } }