@Test
  public void testCrossLambda() {
    try {
      final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();

      DataSet<Tuple2<Integer, String>> left =
          env.fromElements(
              new Tuple2<Integer, String>(1, "hello"),
              new Tuple2<Integer, String>(2, "what's"),
              new Tuple2<Integer, String>(2, "up"));
      DataSet<Tuple2<Integer, String>> right =
          env.fromElements(
              new Tuple2<Integer, String>(1, "not"),
              new Tuple2<Integer, String>(1, "much"),
              new Tuple2<Integer, String>(2, "really"));
      DataSet<Tuple2<Integer, String>> joined =
          left.cross(right)
              .with((t, s) -> new Tuple2<Integer, String>(t.f0 + s.f0, t.f1 + " " + s.f1));

    } catch (UnsupportedLambdaExpressionException e) {
      // Success
      return;
    } catch (Exception e) {
      Assert.fail();
    }
  }
  @Test
  public void testJoinWithTuples() {
    try {
      final Partitioner<Long> partitioner = new TestPartitionerLong();

      ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();

      DataSet<Tuple2<Long, Long>> input1 = env.fromElements(new Tuple2<Long, Long>(0L, 0L));
      DataSet<Tuple3<Long, Long, Long>> input2 =
          env.fromElements(new Tuple3<Long, Long, Long>(0L, 0L, 0L));

      input1
          .join(input2, JoinHint.REPARTITION_HASH_FIRST)
          .where(1)
          .equalTo(0)
          .withPartitioner(partitioner)
          .print();

      Plan p = env.createProgramPlan();
      OptimizedPlan op = compileNoStats(p);

      SinkPlanNode sink = op.getDataSinks().iterator().next();
      DualInputPlanNode join = (DualInputPlanNode) sink.getInput().getSource();

      assertEquals(ShipStrategyType.PARTITION_CUSTOM, join.getInput1().getShipStrategy());
      assertEquals(ShipStrategyType.PARTITION_CUSTOM, join.getInput2().getShipStrategy());
      assertEquals(partitioner, join.getInput1().getPartitioner());
      assertEquals(partitioner, join.getInput2().getPartitioner());
    } catch (Exception e) {
      e.printStackTrace();
      fail(e.getMessage());
    }
  }
  @Test
  public void testRejectWhenSolutionSetKeysDontMatchJoin() {
    try {
      ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();

      @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, 10, 1);

      try {
        iteration.getWorkset().join(iteration.getSolutionSet()).where(1).equalTo(2);
        fail("Accepted invalid program.");
      } catch (InvalidProgramException e) {
        // all good!
      }

      try {
        iteration.getSolutionSet().join(iteration.getWorkset()).where(2).equalTo(1);
        fail("Accepted invalid program.");
      } catch (InvalidProgramException e) {
        // all good!
      }
    } catch (Exception e) {
      System.err.println(e.getMessage());
      e.printStackTrace();
      fail(e.getMessage());
    }
  }
  @Test
  public void testJoinWithTuplesWrongType() {
    try {
      final Partitioner<Integer> partitioner = new TestPartitionerInt();

      ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();

      DataSet<Tuple2<Long, Long>> input1 = env.fromElements(new Tuple2<Long, Long>(0L, 0L));
      DataSet<Tuple3<Long, Long, Long>> input2 =
          env.fromElements(new Tuple3<Long, Long, Long>(0L, 0L, 0L));

      try {
        input1
            .join(input2, JoinHint.REPARTITION_HASH_FIRST)
            .where(1)
            .equalTo(0)
            .withPartitioner(partitioner);

        fail("should throw an exception");
      } catch (InvalidProgramException e) {
        // expected
      }
    } catch (Exception e) {
      e.printStackTrace();
      fail(e.getMessage());
    }
  }
  @Test
  public void testJoinWithKeySelectorsWrongType() {
    try {
      final Partitioner<Long> partitioner = new TestPartitionerLong();

      ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();

      DataSet<Pojo2> input1 = env.fromElements(new Pojo2());
      DataSet<Pojo3> input2 = env.fromElements(new Pojo3());

      try {
        input1
            .join(input2, JoinHint.REPARTITION_HASH_FIRST)
            .where(new Pojo2KeySelector())
            .equalTo(new Pojo3KeySelector())
            .withPartitioner(partitioner);

        fail("should throw an exception");
      } catch (InvalidProgramException e) {
        // expected
      }
    } catch (Exception e) {
      e.printStackTrace();
      fail(e.getMessage());
    }
  }
  @Test
  public void testDistinctPreservesPartitioningOfDistinctFields() {
    try {
      ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
      env.setParallelism(4);

      @SuppressWarnings("unchecked")
      DataSet<Tuple2<Long, Long>> data =
          env.fromElements(new Tuple2<Long, Long>(0L, 0L), new Tuple2<Long, Long>(1L, 1L))
              .map(new IdentityMapper<Tuple2<Long, Long>>())
              .setParallelism(4);

      data.distinct(0).groupBy(0).sum(1).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 distinctReducer = (SingleInputPlanNode) reducer.getInput().getSource();

      assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy());

      // reducer can be forward, reuses partitioning from distinct
      assertEquals(ShipStrategyType.FORWARD, reducer.getInput().getShipStrategy());

      // distinct reducer is partitioned
      assertEquals(ShipStrategyType.PARTITION_HASH, distinctReducer.getInput().getShipStrategy());
    } catch (Exception e) {
      e.printStackTrace();
      fail(e.getMessage());
    }
  }
  @Test
  public void testCustomPartitioningNotReused() {
    try {
      ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();

      @SuppressWarnings("unchecked")
      DataSet<Tuple3<Long, Long, Long>> input =
          env.fromElements(new Tuple3<Long, Long, Long>(0L, 0L, 0L));

      input
          .partitionCustom(
              new Partitioner<Long>() {
                @Override
                public int partition(Long key, int numPartitions) {
                  return 0;
                }
              },
              0)
          .map(new IdentityMapper<Tuple3<Long, Long, Long>>())
          .withForwardedFields("0", "1", "2")
          .groupBy(0, 1)
          .reduceGroup(new IdentityGroupReducerCombinable<Tuple3<Long, Long, Long>>())
          .withForwardedFields("0", "1", "2")
          .groupBy(1)
          .reduceGroup(new IdentityGroupReducerCombinable<Tuple3<Long, Long, Long>>())
          .output(new DiscardingOutputFormat<Tuple3<Long, Long, Long>>());

      Plan p = env.createProgramPlan();
      OptimizedPlan op = compileNoStats(p);

      SinkPlanNode sink = op.getDataSinks().iterator().next();
      SingleInputPlanNode reducer2 = (SingleInputPlanNode) sink.getInput().getSource();
      SingleInputPlanNode combiner = (SingleInputPlanNode) reducer2.getInput().getSource();
      SingleInputPlanNode reducer1 = (SingleInputPlanNode) combiner.getInput().getSource();

      assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy());

      // should be locally forwarding, reusing sort and partitioning
      assertEquals(ShipStrategyType.PARTITION_HASH, reducer2.getInput().getShipStrategy());
      assertEquals(LocalStrategy.COMBININGSORT, reducer2.getInput().getLocalStrategy());

      assertEquals(ShipStrategyType.FORWARD, combiner.getInput().getShipStrategy());
      assertEquals(LocalStrategy.NONE, combiner.getInput().getLocalStrategy());

      assertEquals(ShipStrategyType.FORWARD, reducer1.getInput().getShipStrategy());
      assertEquals(LocalStrategy.COMBININGSORT, reducer1.getInput().getLocalStrategy());
    } catch (Exception e) {
      e.printStackTrace();
      fail(e.getMessage());
    }
  }
  @Test
  public void testIncompatibleHashAndCustomPartitioning() {
    try {
      ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();

      DataSet<Tuple3<Long, Long, Long>> input =
          env.fromElements(new Tuple3<Long, Long, Long>(0L, 0L, 0L));

      DataSet<Tuple3<Long, Long, Long>> partitioned =
          input
              .partitionCustom(
                  new Partitioner<Long>() {
                    @Override
                    public int partition(Long key, int numPartitions) {
                      return 0;
                    }
                  },
                  0)
              .map(new IdentityMapper<Tuple3<Long, Long, Long>>())
              .withForwardedFields("0", "1", "2");

      DataSet<Tuple3<Long, Long, Long>> grouped =
          partitioned
              .distinct(0, 1)
              .groupBy(1)
              .sortGroup(0, Order.ASCENDING)
              .reduceGroup(new IdentityGroupReducer<Tuple3<Long, Long, Long>>())
              .withForwardedFields("0", "1");

      grouped
          .join(partitioned, JoinHint.REPARTITION_HASH_FIRST)
          .where(0)
          .equalTo(0)
          .with(new DummyFlatJoinFunction<Tuple3<Long, Long, Long>>())
          .print();

      Plan p = env.createProgramPlan();
      OptimizedPlan op = compileNoStats(p);

      SinkPlanNode sink = op.getDataSinks().iterator().next();
      DualInputPlanNode coGroup = (DualInputPlanNode) sink.getInput().getSource();

      assertEquals(ShipStrategyType.PARTITION_HASH, coGroup.getInput1().getShipStrategy());
      assertTrue(
          coGroup.getInput2().getShipStrategy() == ShipStrategyType.PARTITION_HASH
              || coGroup.getInput2().getShipStrategy() == ShipStrategyType.FORWARD);
    } catch (Exception e) {
      e.printStackTrace();
      fail(e.getMessage());
    }
  }
  private Plan getTestPlanLeftStatic(String strategy) {

    ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
    env.setParallelism(DEFAULT_PARALLELISM);

    @SuppressWarnings("unchecked")
    DataSet<Tuple3<Long, Long, Long>> bigInput =
        env.fromElements(
                new Tuple3<Long, Long, Long>(1L, 2L, 3L),
                new Tuple3<Long, Long, Long>(1L, 2L, 3L),
                new Tuple3<Long, Long, Long>(1L, 2L, 3L))
            .name("Big");

    @SuppressWarnings("unchecked")
    DataSet<Tuple3<Long, Long, Long>> smallInput =
        env.fromElements(new Tuple3<Long, Long, Long>(1L, 2L, 3L)).name("Small");

    IterativeDataSet<Tuple3<Long, Long, Long>> iteration = bigInput.iterate(10);

    Configuration joinStrategy = new Configuration();
    joinStrategy.setString(Optimizer.HINT_LOCAL_STRATEGY, strategy);

    DataSet<Tuple3<Long, Long, Long>> inner =
        smallInput
            .join(iteration)
            .where(0)
            .equalTo(0)
            .with(new DummyJoiner())
            .name("DummyJoiner")
            .withParameters(joinStrategy);

    DataSet<Tuple3<Long, Long, Long>> output = iteration.closeWith(inner);

    output.output(new DiscardingOutputFormat<Tuple3<Long, Long, Long>>());

    return env.createProgramPlan();
  }
  @Test(expected = ExpressionException.class)
  public void testNonWorkingSubstring2() throws Exception {
    ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
    TableEnvironment tableEnv = new TableEnvironment();

    DataSet<Tuple2<String, String>> ds =
        env.fromElements(new Tuple2<>("ABCD", "a"), new Tuple2<>("ABCD", "b"));

    Table in = tableEnv.fromDataSet(ds, "a, b");

    Table result = in.select("a.substring(b, 15)");

    DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
    List<Row> results = resultSet.collect();
    String expected = "";
    compareResultAsText(results, expected);
  }
  @Test
  public void testSubstringWithMaxEnd() throws Exception {
    ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
    TableEnvironment tableEnv = new TableEnvironment();

    DataSet<Tuple2<String, Integer>> ds =
        env.fromElements(new Tuple2<>("ABCD", 2), new Tuple2<>("ABCD", 1));

    Table in = tableEnv.fromDataSet(ds, "a, b");

    Table result = in.select("a.substring(b)");

    DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
    List<Row> results = resultSet.collect();
    String expected = "CD\nBCD";
    compareResultAsText(results, expected);
  }
  @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 testTranslation() {
    try {
      final String ITERATION_NAME = "Test Name";

      final String AGGREGATOR_NAME = "AggregatorName";

      final String BC_SET_GATHER_NAME = "gather messages";

      final String BC_SET_SUM_NAME = "sum updates";

      final String BC_SET_APLLY_NAME = "apply updates";

      final int NUM_ITERATIONS = 13;

      final int ITERATION_parallelism = 77;

      ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();

      DataSet<Long> bcGather = env.fromElements(1L);
      DataSet<Long> bcSum = env.fromElements(1L);
      DataSet<Long> bcApply = env.fromElements(1L);

      DataSet<Vertex<Long, Long>> result;

      // ------------ construct the test program ------------------
      {
        DataSet<Edge<Long, NullValue>> edges =
            env.fromElements(new Tuple3<Long, Long, NullValue>(1L, 2L, NullValue.getInstance()))
                .map(new Tuple3ToEdgeMap<Long, NullValue>());

        Graph<Long, Long, NullValue> graph = Graph.fromDataSet(edges, new InitVertices(), env);

        GSAConfiguration parameters = new GSAConfiguration();

        parameters.registerAggregator(AGGREGATOR_NAME, new LongSumAggregator());
        parameters.setName(ITERATION_NAME);
        parameters.setParallelism(ITERATION_parallelism);
        parameters.addBroadcastSetForGatherFunction(BC_SET_GATHER_NAME, bcGather);
        parameters.addBroadcastSetForSumFunction(BC_SET_SUM_NAME, bcSum);
        parameters.addBroadcastSetForApplyFunction(BC_SET_APLLY_NAME, bcApply);

        result =
            graph
                .runGatherSumApplyIteration(
                    new GatherNeighborIds(),
                    new SelectMinId(),
                    new UpdateComponentId(),
                    NUM_ITERATIONS,
                    parameters)
                .getVertices();

        result.output(new DiscardingOutputFormat<Vertex<Long, Long>>());
      }

      // ------------- validate the java program ----------------

      assertTrue(result instanceof DeltaIterationResultSet);

      DeltaIterationResultSet<?, ?> resultSet = (DeltaIterationResultSet<?, ?>) result;
      DeltaIteration<?, ?> iteration = (DeltaIteration<?, ?>) resultSet.getIterationHead();

      // check the basic iteration properties
      assertEquals(NUM_ITERATIONS, resultSet.getMaxIterations());
      assertArrayEquals(new int[] {0}, resultSet.getKeyPositions());
      assertEquals(ITERATION_parallelism, iteration.getParallelism());
      assertEquals(ITERATION_NAME, iteration.getName());

      assertEquals(
          AGGREGATOR_NAME,
          iteration.getAggregators().getAllRegisteredAggregators().iterator().next().getName());

      // validate that the semantic properties are set as they should
      TwoInputUdfOperator<?, ?, ?, ?> solutionSetJoin =
          (TwoInputUdfOperator<?, ?, ?, ?>) resultSet.getNextWorkset();
      assertTrue(
          solutionSetJoin.getSemanticProperties().getForwardingTargetFields(0, 0).contains(0));
      assertTrue(
          solutionSetJoin.getSemanticProperties().getForwardingTargetFields(1, 0).contains(0));

      SingleInputUdfOperator<?, ?, ?> sumReduce =
          (SingleInputUdfOperator<?, ?, ?>) solutionSetJoin.getInput1();
      SingleInputUdfOperator<?, ?, ?> gatherMap =
          (SingleInputUdfOperator<?, ?, ?>) sumReduce.getInput();

      // validate that the broadcast sets are forwarded
      assertEquals(bcGather, gatherMap.getBroadcastSets().get(BC_SET_GATHER_NAME));
      assertEquals(bcSum, sumReduce.getBroadcastSets().get(BC_SET_SUM_NAME));
      assertEquals(bcApply, solutionSetJoin.getBroadcastSets().get(BC_SET_APLLY_NAME));
    } catch (Exception e) {
      System.err.println(e.getMessage());
      e.printStackTrace();
      fail(e.getMessage());
    }
  }
  @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());
    }
  }
  private Plan getJavaTestPlan(boolean joinPreservesSolutionSet, boolean mapBeforeSolutionDelta) {

    ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
    env.setParallelism(DEFAULT_PARALLELISM);

    @SuppressWarnings("unchecked")
    DataSet<Tuple3<Long, Long, Long>> solutionSetInput =
        env.fromElements(new Tuple3<Long, Long, Long>(1L, 2L, 3L)).name("Solution Set");
    @SuppressWarnings("unchecked")
    DataSet<Tuple3<Long, Long, Long>> worksetInput =
        env.fromElements(new Tuple3<Long, Long, Long>(1L, 2L, 3L)).name("Workset");
    @SuppressWarnings("unchecked")
    DataSet<Tuple3<Long, Long, Long>> invariantInput =
        env.fromElements(new Tuple3<Long, Long, Long>(1L, 2L, 3L)).name("Invariant Input");

    DeltaIteration<Tuple3<Long, Long, Long>, Tuple3<Long, Long, Long>> iter =
        solutionSetInput.iterateDelta(worksetInput, 100, 1, 2);

    DataSet<Tuple3<Long, Long, Long>> joinedWithSolutionSet =
        iter.getWorkset()
            .join(invariantInput)
            .where(1, 2)
            .equalTo(1, 2)
            .with(
                new RichJoinFunction<
                    Tuple3<Long, Long, Long>,
                    Tuple3<Long, Long, Long>,
                    Tuple3<Long, Long, Long>>() {
                  public Tuple3<Long, Long, Long> join(
                      Tuple3<Long, Long, Long> first, Tuple3<Long, Long, Long> second) {
                    return first;
                  }
                })
            .name(JOIN_WITH_INVARIANT_NAME)
            .join(iter.getSolutionSet())
            .where(1, 0)
            .equalTo(1, 2)
            .with(
                new RichJoinFunction<
                    Tuple3<Long, Long, Long>,
                    Tuple3<Long, Long, Long>,
                    Tuple3<Long, Long, Long>>() {
                  public Tuple3<Long, Long, Long> join(
                      Tuple3<Long, Long, Long> first, Tuple3<Long, Long, Long> second) {
                    return second;
                  }
                })
            .name(JOIN_WITH_SOLUTION_SET)
            .withForwardedFieldsSecond(
                joinPreservesSolutionSet ? new String[] {"0->0", "1->1", "2->2"} : null);

    DataSet<Tuple3<Long, Long, Long>> nextWorkset =
        joinedWithSolutionSet
            .groupBy(1, 2)
            .reduceGroup(
                new RichGroupReduceFunction<Tuple3<Long, Long, Long>, Tuple3<Long, Long, Long>>() {
                  public void reduce(
                      Iterable<Tuple3<Long, Long, Long>> values,
                      Collector<Tuple3<Long, Long, Long>> out) {}
                })
            .name(NEXT_WORKSET_REDUCER_NAME)
            .withForwardedFields("1->1", "2->2", "0->0");

    DataSet<Tuple3<Long, Long, Long>> nextSolutionSet =
        mapBeforeSolutionDelta
            ? joinedWithSolutionSet
                .map(
                    new RichMapFunction<Tuple3<Long, Long, Long>, Tuple3<Long, Long, Long>>() {
                      public Tuple3<Long, Long, Long> map(Tuple3<Long, Long, Long> value) {
                        return value;
                      }
                    })
                .name(SOLUTION_DELTA_MAPPER_NAME)
                .withForwardedFields("0->0", "1->1", "2->2")
            : joinedWithSolutionSet;

    iter.closeWith(nextSolutionSet, nextWorkset)
        .output(new DiscardingOutputFormat<Tuple3<Long, Long, Long>>());

    return env.createProgramPlan();
  }
  @Test
  public void testRejectPlanIfSolutionSetKeysAndJoinKeysDontMatch() {
    try {
      ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
      env.setParallelism(DEFAULT_PARALLELISM);

      @SuppressWarnings("unchecked")
      DataSet<Tuple3<Long, Long, Long>> solutionSetInput =
          env.fromElements(new Tuple3<Long, Long, Long>(1L, 2L, 3L)).name("Solution Set");
      @SuppressWarnings("unchecked")
      DataSet<Tuple3<Long, Long, Long>> worksetInput =
          env.fromElements(new Tuple3<Long, Long, Long>(1L, 2L, 3L)).name("Workset");
      @SuppressWarnings("unchecked")
      DataSet<Tuple3<Long, Long, Long>> invariantInput =
          env.fromElements(new Tuple3<Long, Long, Long>(1L, 2L, 3L)).name("Invariant Input");

      DeltaIteration<Tuple3<Long, Long, Long>, Tuple3<Long, Long, Long>> iter =
          solutionSetInput.iterateDelta(worksetInput, 100, 1, 2);

      DataSet<Tuple3<Long, Long, Long>> result =
          iter.getWorkset()
              .join(invariantInput)
              .where(1, 2)
              .equalTo(1, 2)
              .with(
                  new JoinFunction<
                      Tuple3<Long, Long, Long>,
                      Tuple3<Long, Long, Long>,
                      Tuple3<Long, Long, Long>>() {
                    public Tuple3<Long, Long, Long> join(
                        Tuple3<Long, Long, Long> first, Tuple3<Long, Long, Long> second) {
                      return first;
                    }
                  });

      try {
        result
            .join(iter.getSolutionSet())
            .where(1, 0)
            .equalTo(0, 2)
            .with(
                new JoinFunction<
                    Tuple3<Long, Long, Long>,
                    Tuple3<Long, Long, Long>,
                    Tuple3<Long, Long, Long>>() {
                  public Tuple3<Long, Long, Long> join(
                      Tuple3<Long, Long, Long> first, Tuple3<Long, Long, Long> second) {
                    return second;
                  }
                });
        fail("The join should be rejected with key type mismatches.");
      } catch (InvalidProgramException e) {
        // expected!
      }

    } catch (Exception e) {
      System.err.println(e.getMessage());
      e.printStackTrace();
      fail("Test errored: " + e.getMessage());
    }
  }