@Override protected void testProgram() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); env.setDegreeOfParallelism(4); DataSet<Integer> data = env.fromElements(1, 2, 3, 4, 5, 6, 7, 8); IterativeDataSet<Integer> iteration = data.iterate(10); DataSet<Integer> result = data.reduceGroup(new PickOneAllReduce()).withBroadcastSet(iteration, "bc"); final List<Integer> resultList = new ArrayList<Integer>(); iteration.closeWith(result).output(new LocalCollectionOutputFormat<Integer>(resultList)); env.execute(); Assert.assertEquals(8, resultList.get(0).intValue()); }
@Test public void testCorrectTranslation() { try { final String JOB_NAME = "Test JobName"; final String ITERATION_NAME = "Test Name"; final String BEFORE_NEXT_WORKSET_MAP = "Some Mapper"; final String AGGREGATOR_NAME = "AggregatorName"; final int[] ITERATION_KEYS = new int[] {2}; final int NUM_ITERATIONS = 13; final int DEFAULT_DOP = 133; final int ITERATION_DOP = 77; ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); // ------------ construct the test program ------------------ { env.setDegreeOfParallelism(DEFAULT_DOP); @SuppressWarnings("unchecked") DataSet<Tuple3<Double, Long, String>> initialSolutionSet = env.fromElements(new Tuple3<Double, Long, String>(3.44, 5L, "abc")); @SuppressWarnings("unchecked") DataSet<Tuple2<Double, String>> initialWorkSet = env.fromElements(new Tuple2<Double, String>(1.23, "abc")); DeltaIteration<Tuple3<Double, Long, String>, Tuple2<Double, String>> iteration = initialSolutionSet.iterateDelta(initialWorkSet, NUM_ITERATIONS, ITERATION_KEYS); iteration.name(ITERATION_NAME).parallelism(ITERATION_DOP); iteration.registerAggregator(AGGREGATOR_NAME, new LongSumAggregator()); // test that multiple workset consumers are supported DataSet<Tuple2<Double, String>> worksetSelfJoin = iteration .getWorkset() .map(new IdentityMapper<Tuple2<Double, String>>()) .join(iteration.getWorkset()) .where(1) .equalTo(1) .projectFirst(0, 1) .types(Double.class, String.class); DataSet<Tuple3<Double, Long, String>> joined = worksetSelfJoin .join(iteration.getSolutionSet()) .where(1) .equalTo(2) .with(new SolutionWorksetJoin()); DataSet<Tuple3<Double, Long, String>> result = iteration.closeWith( joined, joined.map(new NextWorksetMapper()).name(BEFORE_NEXT_WORKSET_MAP)); result.print(); result.writeAsText("/dev/null"); } Plan p = env.createProgramPlan(JOB_NAME); // ------------- validate the plan ---------------- assertEquals(JOB_NAME, p.getJobName()); assertEquals(DEFAULT_DOP, p.getDefaultParallelism()); // validate the iteration GenericDataSinkBase<?> sink1, sink2; { Iterator<? extends GenericDataSinkBase<?>> sinks = p.getDataSinks().iterator(); sink1 = sinks.next(); sink2 = sinks.next(); } DeltaIterationBase<?, ?> iteration = (DeltaIterationBase<?, ?>) sink1.getInput(); // check that multi consumer translation works for iterations assertEquals(iteration, sink2.getInput()); // check the basic iteration properties assertEquals(NUM_ITERATIONS, iteration.getMaximumNumberOfIterations()); assertArrayEquals(ITERATION_KEYS, iteration.getSolutionSetKeyFields()); assertEquals(ITERATION_DOP, iteration.getDegreeOfParallelism()); assertEquals(ITERATION_NAME, iteration.getName()); MapOperatorBase<?, ?, ?> nextWorksetMapper = (MapOperatorBase<?, ?, ?>) iteration.getNextWorkset(); JoinOperatorBase<?, ?, ?, ?> solutionSetJoin = (JoinOperatorBase<?, ?, ?, ?>) iteration.getSolutionSetDelta(); JoinOperatorBase<?, ?, ?, ?> worksetSelfJoin = (JoinOperatorBase<?, ?, ?, ?>) solutionSetJoin.getFirstInput(); MapOperatorBase<?, ?, ?> worksetMapper = (MapOperatorBase<?, ?, ?>) worksetSelfJoin.getFirstInput(); assertEquals(IdentityMapper.class, worksetMapper.getUserCodeWrapper().getUserCodeClass()); assertEquals( NextWorksetMapper.class, nextWorksetMapper.getUserCodeWrapper().getUserCodeClass()); if (solutionSetJoin.getUserCodeWrapper().getUserCodeObject() instanceof WrappingFunction) { WrappingFunction<?> wf = (WrappingFunction<?>) solutionSetJoin.getUserCodeWrapper().getUserCodeObject(); assertEquals(SolutionWorksetJoin.class, wf.getWrappedFunction().getClass()); } else { assertEquals( SolutionWorksetJoin.class, solutionSetJoin.getUserCodeWrapper().getUserCodeClass()); } assertEquals(BEFORE_NEXT_WORKSET_MAP, nextWorksetMapper.getName()); assertEquals( AGGREGATOR_NAME, iteration.getAggregators().getAllRegisteredAggregators().iterator().next().getName()); } catch (Exception e) { System.err.println(e.getMessage()); e.printStackTrace(); fail(e.getMessage()); } }