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 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; } }
@Test public void testPartitionCustomOperatorPreservesFields() { try { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); DataSet<Tuple2<Long, Long>> data = env.fromCollection(Collections.singleton(new Tuple2<>(0L, 0L))); data.partitionCustom(new Partitioner<Long>() { public int partition(Long key, int numPartitions) { return key.intValue(); } }, 1) .groupBy(1) .reduceGroup(new IdentityGroupReducerCombinable<Tuple2<Long, Long>>()) .output(new DiscardingOutputFormat<Tuple2<Long, Long>>()); Plan p = env.createProgramPlan(); OptimizedPlan op = compileNoStats(p); SinkPlanNode sink = op.getDataSinks().iterator().next(); SingleInputPlanNode reducer = (SingleInputPlanNode) sink.getInput().getSource(); SingleInputPlanNode partitioner = (SingleInputPlanNode) reducer.getInput().getSource(); assertEquals(ShipStrategyType.FORWARD, reducer.getInput().getShipStrategy()); assertEquals(ShipStrategyType.PARTITION_CUSTOM, partitioner.getInput().getShipStrategy()); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); } }
SingleInputPlanNode beforeMapper = (SingleInputPlanNode) mapper.getInput().getSource(); assertEquals(TempMode.NONE, mapper.getInput().getTempMode()); assertEquals(TempMode.NONE, beforeMapper.getInput().getTempMode()); assertEquals(TempMode.NONE, mapper.getBroadcastInputs().get(0).getTempMode()); assertEquals(DataExchangeMode.PIPELINED, mapper.getInput().getDataExchangeMode()); assertEquals(DataExchangeMode.BATCH, beforeMapper.getInput().getDataExchangeMode()); assertEquals(DataExchangeMode.BATCH, mapper.getBroadcastInputs().get(0).getDataExchangeMode());
SingleInputPlanNode reducer = (SingleInputPlanNode) keyRemovingMapper.getInput().getSource(); SingleInputPlanNode combiner = (SingleInputPlanNode) reducer.getInput().getSource(); assertEquals(ShipStrategyType.FORWARD, keyRemovingMapper.getInput().getShipStrategy()); assertEquals(ShipStrategyType.PARTITION_CUSTOM, reducer.getInput().getShipStrategy()); assertEquals(ShipStrategyType.FORWARD, combiner.getInput().getShipStrategy());
@Test public void testNoBreakerForIndependentVariable() { try { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); DataSet<String> source1 = env.fromElements("test"); DataSet<String> source2 = env.fromElements("test"); source1.map(new IdentityMapper<String>()).withBroadcastSet(source2, "some name") .output(new DiscardingOutputFormat<String>()); Plan p = env.createProgramPlan(); OptimizedPlan op = compileNoStats(p); SinkPlanNode sink = op.getDataSinks().iterator().next(); SingleInputPlanNode mapper = (SingleInputPlanNode) sink.getInput().getSource(); assertEquals(TempMode.NONE, mapper.getInput().getTempMode()); assertEquals(TempMode.NONE, mapper.getBroadcastInputs().get(0).getTempMode()); assertEquals(DataExchangeMode.PIPELINED, mapper.getInput().getDataExchangeMode()); assertEquals(DataExchangeMode.PIPELINED, mapper.getBroadcastInputs().get(0).getDataExchangeMode()); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); } }
@Test public void testCustomPartitioningTupleAgg() { try { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); DataSet<Tuple2<Integer, Integer>> data = env.fromElements(new Tuple2<Integer, Integer>(0, 0)) .rebalance().setParallelism(4); data.groupBy(0).withPartitioner(new TestPartitionerInt()) .sum(1) .output(new DiscardingOutputFormat<Tuple2<Integer, Integer>>()); Plan p = env.createProgramPlan(); OptimizedPlan op = compileNoStats(p); SinkPlanNode sink = op.getDataSinks().iterator().next(); SingleInputPlanNode reducer = (SingleInputPlanNode) sink.getInput().getSource(); SingleInputPlanNode combiner = (SingleInputPlanNode) reducer.getInput().getSource(); assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy()); assertEquals(ShipStrategyType.PARTITION_CUSTOM, reducer.getInput().getShipStrategy()); assertEquals(ShipStrategyType.FORWARD, combiner.getInput().getShipStrategy()); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); } }
@Test public void testCustomPartitioningTupleReduce() { try { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); DataSet<Tuple2<Integer, Integer>> data = env.fromElements(new Tuple2<Integer, Integer>(0, 0)) .rebalance().setParallelism(4); data.groupBy(0).withPartitioner(new TestPartitionerInt()) .reduce(new SelectOneReducer<Tuple2<Integer,Integer>>()) .output(new DiscardingOutputFormat<Tuple2<Integer, Integer>>()); Plan p = env.createProgramPlan(); OptimizedPlan op = compileNoStats(p); SinkPlanNode sink = op.getDataSinks().iterator().next(); SingleInputPlanNode reducer = (SingleInputPlanNode) sink.getInput().getSource(); SingleInputPlanNode combiner = (SingleInputPlanNode) reducer.getInput().getSource(); assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy()); assertEquals(ShipStrategyType.PARTITION_CUSTOM, reducer.getInput().getShipStrategy()); assertEquals(ShipStrategyType.FORWARD, combiner.getInput().getShipStrategy()); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); } }
@Test public void testCustomPartitioningTupleReduce() { try { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); DataSet<Pojo2> data = env.fromElements(new Pojo2()) .rebalance().setParallelism(4); data.groupBy("a").withPartitioner(new TestPartitionerInt()) .reduce(new SelectOneReducer<Pojo2>()) .output(new DiscardingOutputFormat<Pojo2>()); Plan p = env.createProgramPlan(); OptimizedPlan op = compileNoStats(p); SinkPlanNode sink = op.getDataSinks().iterator().next(); SingleInputPlanNode reducer = (SingleInputPlanNode) sink.getInput().getSource(); SingleInputPlanNode combiner = (SingleInputPlanNode) reducer.getInput().getSource(); assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy()); assertEquals(ShipStrategyType.PARTITION_CUSTOM, reducer.getInput().getShipStrategy()); assertEquals(ShipStrategyType.FORWARD, combiner.getInput().getShipStrategy()); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); } }
@Test public void testCustomPartitioningTupleGroupReduce() { try { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); DataSet<Tuple2<Integer, Integer>> data = env.fromElements(new Tuple2<Integer, Integer>(0, 0)) .rebalance().setParallelism(4); data.groupBy(0).withPartitioner(new TestPartitionerInt()) .reduceGroup(new IdentityGroupReducerCombinable<Tuple2<Integer,Integer>>()) .output(new DiscardingOutputFormat<Tuple2<Integer, Integer>>()); Plan p = env.createProgramPlan(); OptimizedPlan op = compileNoStats(p); SinkPlanNode sink = op.getDataSinks().iterator().next(); SingleInputPlanNode reducer = (SingleInputPlanNode) sink.getInput().getSource(); SingleInputPlanNode combiner = (SingleInputPlanNode) reducer.getInput().getSource(); assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy()); assertEquals(ShipStrategyType.PARTITION_CUSTOM, reducer.getInput().getShipStrategy()); assertEquals(ShipStrategyType.FORWARD, combiner.getInput().getShipStrategy()); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); } }
@Test public void testCustomPartitioningTupleGroupReduce() { try { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); DataSet<Pojo2> data = env.fromElements(new Pojo2()) .rebalance().setParallelism(4); data.groupBy("a").withPartitioner(new TestPartitionerInt()) .reduceGroup(new IdentityGroupReducerCombinable<Pojo2>()) .output(new DiscardingOutputFormat<Pojo2>()); Plan p = env.createProgramPlan(); OptimizedPlan op = compileNoStats(p); SinkPlanNode sink = op.getDataSinks().iterator().next(); SingleInputPlanNode reducer = (SingleInputPlanNode) sink.getInput().getSource(); SingleInputPlanNode combiner = (SingleInputPlanNode) reducer.getInput().getSource(); assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy()); assertEquals(ShipStrategyType.PARTITION_CUSTOM, reducer.getInput().getShipStrategy()); assertEquals(ShipStrategyType.FORWARD, combiner.getInput().getShipStrategy()); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); } }
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 testCustomPartitioningTupleGroupReduceSorted() { try { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); DataSet<Tuple3<Integer, Integer, Integer>> data = env.fromElements(new Tuple3<Integer, Integer, Integer>(0, 0, 0)) .rebalance().setParallelism(4); data.groupBy(0).withPartitioner(new TestPartitionerInt()) .sortGroup(1, Order.ASCENDING) .reduceGroup(new IdentityGroupReducerCombinable<Tuple3<Integer,Integer,Integer>>()) .output(new DiscardingOutputFormat<Tuple3<Integer, Integer, Integer>>()); Plan p = env.createProgramPlan(); OptimizedPlan op = compileNoStats(p); SinkPlanNode sink = op.getDataSinks().iterator().next(); SingleInputPlanNode reducer = (SingleInputPlanNode) sink.getInput().getSource(); SingleInputPlanNode combiner = (SingleInputPlanNode) reducer.getInput().getSource(); assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy()); assertEquals(ShipStrategyType.PARTITION_CUSTOM, reducer.getInput().getShipStrategy()); assertEquals(ShipStrategyType.FORWARD, combiner.getInput().getShipStrategy()); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); } }
SingleInputPlanNode mapperInput = (SingleInputPlanNode) mapper.getInput().getSource(); assertEquals(TempMode.NONE, mapper.getInput().getTempMode()); assertEquals(TempMode.NONE, mapper.getBroadcastInputs().get(0).getTempMode()); assertEquals(DataExchangeMode.BATCH, mapperInput.getInput().getDataExchangeMode()); assertEquals(DataExchangeMode.BATCH, mapper.getBroadcastInputs().get(0).getDataExchangeMode());
@Test public void testCustomPartitioningKeySelectorGroupReduce() { try { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); DataSet<Tuple2<Integer, Integer>> data = env.fromElements(new Tuple2<Integer, Integer>(0, 0)) .rebalance().setParallelism(4); data.groupBy(new TestKeySelector<Tuple2<Integer,Integer>>()) .withPartitioner(new TestPartitionerInt()) .reduceGroup(new IdentityGroupReducerCombinable<Tuple2<Integer,Integer>>()) .output(new DiscardingOutputFormat<Tuple2<Integer, Integer>>()); Plan p = env.createProgramPlan(); OptimizedPlan op = compileNoStats(p); SinkPlanNode sink = op.getDataSinks().iterator().next(); SingleInputPlanNode reducer = (SingleInputPlanNode) sink.getInput().getSource(); SingleInputPlanNode combiner = (SingleInputPlanNode) reducer.getInput().getSource(); assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy()); assertEquals(ShipStrategyType.PARTITION_CUSTOM, reducer.getInput().getShipStrategy()); assertEquals(ShipStrategyType.FORWARD, combiner.getInput().getShipStrategy()); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); } }
private void checkStandardStrategies(SingleInputPlanNode map, DualInputPlanNode join, SingleInputPlanNode combiner, SingleInputPlanNode reducer, SinkPlanNode sink) { // check ship strategies that are always fix Assert.assertEquals(ShipStrategyType.FORWARD, map.getInput().getShipStrategy()); Assert.assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy()); // check the driver strategies that are always fix Assert.assertEquals(DriverStrategy.FLAT_MAP, map.getDriverStrategy()); Assert.assertEquals(DriverStrategy.SORTED_GROUP_REDUCE, reducer.getDriverStrategy()); Assert.assertEquals(DriverStrategy.NONE, sink.getDriverStrategy()); if (combiner != null) { Assert.assertEquals(DriverStrategy.SORTED_GROUP_COMBINE, combiner.getDriverStrategy()); Assert.assertEquals(LocalStrategy.NONE, combiner.getInput().getLocalStrategy()); } }
@Test public void testCustomPartitioningTupleGroupReduceSorted() { try { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); DataSet<Pojo3> data = env.fromElements(new Pojo3()) .rebalance().setParallelism(4); data.groupBy("a").withPartitioner(new TestPartitionerInt()) .sortGroup("b", Order.ASCENDING) .reduceGroup(new IdentityGroupReducerCombinable<Pojo3>()) .output(new DiscardingOutputFormat<Pojo3>()); Plan p = env.createProgramPlan(); OptimizedPlan op = compileNoStats(p); SinkPlanNode sink = op.getDataSinks().iterator().next(); SingleInputPlanNode reducer = (SingleInputPlanNode) sink.getInput().getSource(); SingleInputPlanNode combiner = (SingleInputPlanNode) reducer.getInput().getSource(); assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy()); assertEquals(ShipStrategyType.PARTITION_CUSTOM, reducer.getInput().getShipStrategy()); assertEquals(ShipStrategyType.FORWARD, combiner.getInput().getShipStrategy()); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); } }
@Test public void testCustomPartitioningTupleGroupReduceSorted2() { try { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); DataSet<Pojo4> data = env.fromElements(new Pojo4()) .rebalance().setParallelism(4); data.groupBy("a").withPartitioner(new TestPartitionerInt()) .sortGroup("b", Order.ASCENDING) .sortGroup("c", Order.DESCENDING) .reduceGroup(new IdentityGroupReducerCombinable<Pojo4>()) .output(new DiscardingOutputFormat<Pojo4>()); Plan p = env.createProgramPlan(); OptimizedPlan op = compileNoStats(p); SinkPlanNode sink = op.getDataSinks().iterator().next(); SingleInputPlanNode reducer = (SingleInputPlanNode) sink.getInput().getSource(); SingleInputPlanNode combiner = (SingleInputPlanNode) reducer.getInput().getSource(); assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy()); assertEquals(ShipStrategyType.PARTITION_CUSTOM, reducer.getInput().getShipStrategy()); assertEquals(ShipStrategyType.FORWARD, combiner.getInput().getShipStrategy()); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); } }
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; } }
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; } }