/**
   * Verifies that a robust repartitioning plan with a hash join is created in the absence of
   * statistics.
   */
  @Test
  public void testQueryNoStatistics() {
    try {
      TPCHQuery3 query = new TPCHQuery3();
      Plan p = query.getPlan(DEFAULT_PARALLELISM_STRING, IN_FILE, IN_FILE, OUT_FILE);
      p.setExecutionConfig(defaultExecutionConfig);
      // compile
      final OptimizedPlan plan = compileNoStats(p);

      final OptimizerPlanNodeResolver or = getOptimizerPlanNodeResolver(plan);

      // get the nodes from the final plan
      final SinkPlanNode sink = or.getNode("Output");
      final SingleInputPlanNode reducer = or.getNode("AggLio");
      final SingleInputPlanNode combiner =
          reducer.getPredecessor() instanceof SingleInputPlanNode
              ? (SingleInputPlanNode) reducer.getPredecessor()
              : null;
      final DualInputPlanNode join = or.getNode("JoinLiO");
      final SingleInputPlanNode filteringMapper = or.getNode("FilterO");

      // verify the optimizer choices
      checkStandardStrategies(filteringMapper, join, combiner, reducer, sink);
      Assert.assertTrue(checkRepartitionShipStrategies(join, reducer, combiner));
      Assert.assertTrue(
          checkHashJoinStrategies(join, reducer, true)
              || checkHashJoinStrategies(join, reducer, false));
    } catch (Exception e) {
      e.printStackTrace();
      Assert.fail(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 void testQueryGeneric(
     long orderSize,
     long lineItemSize,
     float ordersFilterFactor,
     float joinFilterFactor,
     boolean broadcastOkay,
     boolean partitionedOkay,
     boolean hashJoinFirstOkay,
     boolean hashJoinSecondOkay,
     boolean mergeJoinOkay) {
   TPCHQuery3 query = new TPCHQuery3();
   Plan p = query.getPlan(DEFAULT_PARALLELISM_STRING, IN_FILE, IN_FILE, OUT_FILE);
   p.setExecutionConfig(defaultExecutionConfig);
   testQueryGeneric(
       p,
       orderSize,
       lineItemSize,
       ordersFilterFactor,
       joinFilterFactor,
       broadcastOkay,
       partitionedOkay,
       hashJoinFirstOkay,
       hashJoinSecondOkay,
       mergeJoinOkay);
 }
  @SuppressWarnings("unchecked")
  public static Plan getPlan(
      int numSubTasks, String verticesInput, String edgeInput, String output, int maxIterations) {

    // data source for initial vertices
    FileDataSource initialVertices =
        new FileDataSource(new CsvInputFormat(' ', LongValue.class), verticesInput, "Vertices");

    MapOperator verticesWithId =
        MapOperator.builder(DuplicateLongMap.class)
            .input(initialVertices)
            .name("Assign Vertex Ids")
            .build();

    DeltaIteration iteration = new DeltaIteration(0, "Connected Components Iteration");
    iteration.setInitialSolutionSet(verticesWithId);
    iteration.setInitialWorkset(verticesWithId);
    iteration.setMaximumNumberOfIterations(maxIterations);

    // create DataSourceContract for the edges
    FileDataSource edges =
        new FileDataSource(
            new CsvInputFormat(' ', LongValue.class, LongValue.class), edgeInput, "Edges");

    // create CrossOperator for distance computation
    JoinOperator joinWithNeighbors =
        JoinOperator.builder(new NeighborWithComponentIDJoin(), LongValue.class, 0, 0)
            .input1(iteration.getWorkset())
            .input2(edges)
            .name("Join Candidate Id With Neighbor")
            .build();

    CoGroupOperator minAndUpdate =
        CoGroupOperator.builder(new MinIdAndUpdate(), LongValue.class, 0, 0)
            .input1(joinWithNeighbors)
            .input2(iteration.getSolutionSet())
            .name("Min Id and Update")
            .build();

    iteration.setNextWorkset(minAndUpdate);
    iteration.setSolutionSetDelta(minAndUpdate);

    // create DataSinkContract for writing the new cluster positions
    FileDataSink result = new FileDataSink(new CsvOutputFormat(), output, iteration, "Result");
    CsvOutputFormat.configureRecordFormat(result)
        .recordDelimiter('\n')
        .fieldDelimiter(' ')
        .field(LongValue.class, 0)
        .field(LongValue.class, 1);

    // return the PACT plan
    Plan plan = new Plan(result, "Workset Connected Components");
    plan.setDefaultParallelism(numSubTasks);
    return plan;
  }
  //	@Test
  public void testMapCancelling() throws Exception {
    GenericDataSource<InfiniteIntegerInputFormat> source =
        new GenericDataSource<InfiniteIntegerInputFormat>(
            new InfiniteIntegerInputFormat(), "Source");
    MapOperator mapper =
        MapOperator.builder(IdentityMapper.class).input(source).name("Identity Mapper").build();
    GenericDataSink sink = new GenericDataSink(new DiscardingOutputFormat(), mapper, "Sink");

    Plan p = new Plan(sink);
    p.setDefaultParallelism(DOP);

    runAndCancelJob(p, 5 * 1000, 10 * 1000);
  }
Beispiel #6
0
  public JobExecutionResult execute(Plan program) throws Exception {
    long startTime = System.currentTimeMillis();

    initCache(program.getCachedFiles());
    Collection<? extends GenericDataSinkBase<?>> sinks = program.getDataSinks();
    for (Operator<?> sink : sinks) {
      execute(sink);
    }

    long endTime = System.currentTimeMillis();
    Map<String, Object> accumulatorResults = AccumulatorHelper.toResultMap(accumulators);
    return new JobExecutionResult(null, endTime - startTime, accumulatorResults);
  }
  /**
   * Statistics that push towards a repartition merge join. If the join blows the data volume up
   * significantly, re-exploiting the sorted order is cheaper.
   */
  @Test
  public void testQueryWithStatsForRepartitionMerge() {
    TPCHQuery3 query = new TPCHQuery3();
    Plan p = query.getPlan(DEFAULT_PARALLELISM_STRING, IN_FILE, IN_FILE, OUT_FILE);
    p.setExecutionConfig(defaultExecutionConfig);
    // set compiler hints
    OperatorResolver cr = getContractResolver(p);
    JoinOperator match = cr.getNode("JoinLiO");
    match.getCompilerHints().setFilterFactor(100f);

    testQueryGeneric(
        100l * 1024 * 1024 * 1024 * 1024,
        100l * 1024 * 1024 * 1024 * 1024,
        0.05f,
        100f,
        false,
        true,
        false,
        false,
        true);
  }
  /**
   * compiles jars from files in the shell virtual directory on the fly, sends and executes it in
   * the remote environment
   *
   * @param jobName name of the job as string
   * @return Result of the computation
   * @throws Exception
   */
  @Override
  public JobExecutionResult execute(String jobName) throws Exception {
    Plan p = createProgramPlan(jobName);

    String jarFile = flinkILoop.writeFilesToDisk().getAbsolutePath();

    // get "external jars, and add the shell command jar, pass to executor
    List<String> alljars = new ArrayList<String>();
    // get external (library) jars
    String[] extJars = this.flinkILoop.getExternalJars();

    if (!ArrayUtils.isEmpty(extJars)) {
      alljars.addAll(Arrays.asList(extJars));
    }
    // add shell commands
    alljars.add(jarFile);
    String[] alljarsArr = new String[alljars.size()];
    alljarsArr = alljars.toArray(alljarsArr);
    PlanExecutor executor = PlanExecutor.createRemoteExecutor(host, port, alljarsArr);

    executor.setPrintStatusDuringExecution(p.getExecutionConfig().isSysoutLoggingEnabled());
    return executor.executePlan(p);
  }
  @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());
    }
  }