private boolean checkRepartitionMergeJoin(DualInputPlanNode join, SingleInputPlanNode reducer) {
    if (DriverStrategy.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;
    }
  }
  @Test
  public void testJavaApiWithDeferredSoltionSetUpdateWithMapper() {
    try {
      Plan plan = getJavaTestPlan(false, true);

      OptimizedPlan oPlan = compileNoStats(plan);

      OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(oPlan);
      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);

      // iteration preserves partitioning in reducer, so the first partitioning is out of the loop,
      // the in-loop partitioning is before the final reducer

      // verify joinWithInvariant
      assertEquals(ShipStrategyType.FORWARD, joinWithInvariantNode.getInput1().getShipStrategy());
      assertEquals(
          ShipStrategyType.PARTITION_HASH, joinWithInvariantNode.getInput2().getShipStrategy());
      assertEquals(new FieldList(1, 2), joinWithInvariantNode.getKeysForInput1());
      assertEquals(new FieldList(1, 2), joinWithInvariantNode.getKeysForInput2());

      // verify joinWithSolutionSet
      assertEquals(
          ShipStrategyType.PARTITION_HASH, joinWithSolutionSetNode.getInput1().getShipStrategy());
      assertEquals(ShipStrategyType.FORWARD, joinWithSolutionSetNode.getInput2().getShipStrategy());
      assertEquals(new FieldList(1, 0), joinWithSolutionSetNode.getKeysForInput1());

      // verify reducer
      assertEquals(ShipStrategyType.PARTITION_HASH, worksetReducer.getInput().getShipStrategy());
      assertEquals(new FieldList(1, 2), worksetReducer.getKeys(0));

      // currently, the system may partition before or after the mapper
      ShipStrategyType ss1 = deltaMapper.getInput().getShipStrategy();
      ShipStrategyType ss2 = deltaMapper.getOutgoingChannels().get(0).getShipStrategy();

      assertTrue(
          (ss1 == ShipStrategyType.FORWARD && ss2 == ShipStrategyType.PARTITION_HASH)
              || (ss2 == ShipStrategyType.FORWARD && ss1 == ShipStrategyType.PARTITION_HASH));

      new JobGraphGenerator().compileJobGraph(oPlan);
    } catch (Exception e) {
      System.err.println(e.getMessage());
      e.printStackTrace();
      fail("Test errored: " + e.getMessage());
    }
  }
  /**
   * This test simulates a join of a big left side with a small right side inside of an iteration,
   * where the small side is on a static path. Currently the best execution plan is a
   * HYBRIDHASH_BUILD_SECOND_CACHED, where the small side is hashed and cached. This test also makes
   * sure that all relevant plans are correctly enumerated by the optimizer.
   */
  @Test
  public void testCorrectChoosing() {
    try {

      Plan plan = getTestPlanRightStatic("");

      SourceCollectorVisitor sourceCollector = new SourceCollectorVisitor();
      plan.accept(sourceCollector);

      for (GenericDataSourceBase<?, ?> s : sourceCollector.getSources()) {
        if (s.getName().equals("bigFile")) {
          this.setSourceStatistics(s, 10000000, 1000);
        } else if (s.getName().equals("smallFile")) {
          this.setSourceStatistics(s, 100, 100);
        }
      }

      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());
    }
  }
 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;
   }
 }
 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 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;
    }
  }
  /**
   * 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());
    }
  }