private static JobTaskVertex createMapper(
      JobGraph jobGraph, int numSubTasks, TypeSerializerFactory<?> serializer) {
    JobTaskVertex pointsInput =
        JobGraphUtils.createTask(
            RegularPactTask.class, "Map[DotProducts]", jobGraph, numSubTasks, numSubTasks);

    {
      TaskConfig taskConfig = new TaskConfig(pointsInput.getConfiguration());

      taskConfig.setStubWrapper(new UserCodeClassWrapper<DotProducts>(DotProducts.class));
      taskConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
      taskConfig.setOutputSerializer(serializer);
      taskConfig.setDriver(CollectorMapDriver.class);
      taskConfig.setDriverStrategy(DriverStrategy.COLLECTOR_MAP);

      taskConfig.addInputToGroup(0);
      taskConfig.setInputLocalStrategy(0, LocalStrategy.NONE);
      taskConfig.setInputSerializer(serializer, 0);

      taskConfig.setBroadcastInputName("models", 0);
      taskConfig.addBroadcastInputToGroup(0);
      taskConfig.setBroadcastInputSerializer(serializer, 0);
    }

    return pointsInput;
  }
  private JobGraph createJobGraphV1(
      String pointsPath, String centersPath, String resultPath, int numSubTasks)
      throws JobGraphDefinitionException {

    // -- init
    // -------------------------------------------------------------------------------------------------
    final TypeSerializerFactory<?> serializer = RecordSerializerFactory.get();

    JobGraph jobGraph = new JobGraph("Distance Builder");

    // -- vertices
    // ---------------------------------------------------------------------------------------------
    JobInputVertex points = createPointsInput(jobGraph, pointsPath, numSubTasks, serializer);
    JobInputVertex models = createModelsInput(jobGraph, centersPath, numSubTasks, serializer);
    JobTaskVertex mapper = createMapper(jobGraph, numSubTasks, serializer);
    JobOutputVertex output = createOutput(jobGraph, resultPath, numSubTasks, serializer);

    // -- edges
    // ------------------------------------------------------------------------------------------------
    JobGraphUtils.connect(points, mapper, ChannelType.NETWORK, DistributionPattern.POINTWISE);
    JobGraphUtils.connect(models, mapper, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
    JobGraphUtils.connect(mapper, output, ChannelType.NETWORK, DistributionPattern.POINTWISE);

    // -- instance sharing
    // -------------------------------------------------------------------------------------
    points.setVertexToShareInstancesWith(output);
    models.setVertexToShareInstancesWith(output);
    mapper.setVertexToShareInstancesWith(output);

    return jobGraph;
  }
  public static void main(String[] args)
      throws JobGraphDefinitionException, IOException, JobExecutionException {
    if (args.length != 5) {
      System.out.println("Not correct parameters");
      System.exit(-1);
    }

    final int dop = Integer.valueOf(args[0]);
    final String input = args[1];
    final String output = args[2];
    final int spi = Integer.valueOf(args[3]);
    final int baseMemory = Integer.valueOf(args[4]);

    JobGraph graph = new JobGraph("Connected Components");

    // Create tasks
    JobInputVertex sourceVertex = createInput(AdjacencyListInput.class, input, graph, dop, spi);

    JobTaskVertex convert = createTask(ConvertToTransitiveClosureTypes.class, graph, dop, spi);
    convert.setVertexToShareInstancesWith(sourceVertex);

    JobTaskVertex initialState = createTask(InitialStateComponents.class, graph, dop, spi);
    initialState.setVertexToShareInstancesWith(sourceVertex);

    JobTaskVertex initialUpdateAssigner = createTask(InitialUpdates.class, graph, dop, spi);
    initialUpdateAssigner.setVertexToShareInstancesWith(sourceVertex);

    JobTaskVertex tmpTask = createTask(UpdateTempTask.class, graph, dop);
    tmpTask.setVertexToShareInstancesWith(sourceVertex);
    setMemorySize(tmpTask, baseMemory * 1 / 9);

    // Inner iteration loop tasks -- END
    JobTaskVertex updatesMatch =
        createTask(UpdateableMatchingOptimizedCombined.class, graph, dop, spi);
    updatesMatch.setVertexToShareInstancesWith(sourceVertex);
    setMemorySize(updatesMatch, baseMemory * 8 / 9);
    // Inner iteration loop tasks -- END

    JobOutputVertex sinkVertex = createOutput(NullOutput.class, output, graph, dop, spi);
    sinkVertex.setVertexToShareInstancesWith(sourceVertex);

    // Connect tasks
    connectJobVertices(ShipStrategy.FORWARD, sourceVertex, convert, null, null);
    connectJobVertices(ShipStrategy.PARTITION_HASH, convert, initialState, null, null);

    connectJobVertices(ShipStrategy.FORWARD, initialState, initialUpdateAssigner, null, null);
    connectJobVertices(ShipStrategy.PARTITION_HASH, initialUpdateAssigner, tmpTask, null, null);

    NepheleUtil.connectBoundedRoundsIterationLoop(
        tmpTask, sinkVertex, null, null, updatesMatch, ShipStrategy.PARTITION_HASH, 14, graph);

    connectJobVertices(ShipStrategy.FORWARD, initialState, updatesMatch, null, null);

    // Submit job
    submit(graph, getConfiguration());
  }
  private static JobTaskVertex createReducer(
      JobGraph jobGraph,
      int numSubTasks,
      TypeSerializerFactory<?> inputSerializer,
      TypeComparatorFactory<?> inputComparator,
      TypeSerializerFactory<?> outputSerializer) {
    // ---------------- the tail (co group) --------------------

    JobTaskVertex tail =
        JobGraphUtils.createTask(
            IterationTailPactTask.class,
            "Reduce / Iteration Tail",
            jobGraph,
            numSubTasks,
            numSubTasks);

    TaskConfig tailConfig = new TaskConfig(tail.getConfiguration());
    tailConfig.setIterationId(ITERATION_ID);
    tailConfig.setIsWorksetUpdate();

    // inputs and driver
    tailConfig.setDriver(ReduceDriver.class);
    tailConfig.setDriverStrategy(DriverStrategy.SORTED_GROUP);
    tailConfig.addInputToGroup(0);
    tailConfig.setInputSerializer(inputSerializer, 0);
    tailConfig.setDriverComparator(inputComparator, 0);

    tailConfig.setInputLocalStrategy(0, LocalStrategy.SORT);
    tailConfig.setInputComparator(inputComparator, 0);
    tailConfig.setMemoryInput(0, MEMORY_PER_CONSUMER * JobGraphUtils.MEGABYTE);
    tailConfig.setFilehandlesInput(0, 128);
    tailConfig.setSpillingThresholdInput(0, 0.9f);

    // output
    tailConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
    tailConfig.setOutputSerializer(outputSerializer);

    // the udf
    tailConfig.setStubWrapper(
        new UserCodeObjectWrapper<RecomputeClusterCenter>(new RecomputeClusterCenter()));

    return tail;
  }
  private static JobTaskVertex createMapper(
      JobGraph jobGraph,
      int numSubTasks,
      TypeSerializerFactory<?> inputSerializer,
      TypeSerializerFactory<?> broadcastVarSerializer,
      TypeSerializerFactory<?> outputSerializer,
      TypeComparatorFactory<?> outputComparator) {
    JobTaskVertex mapper =
        JobGraphUtils.createTask(
            IterationIntermediatePactTask.class,
            "Map (Select nearest center)",
            jobGraph,
            numSubTasks,
            numSubTasks);

    TaskConfig intermediateConfig = new TaskConfig(mapper.getConfiguration());
    intermediateConfig.setIterationId(ITERATION_ID);

    intermediateConfig.setDriver(CollectorMapDriver.class);
    intermediateConfig.setDriverStrategy(DriverStrategy.COLLECTOR_MAP);
    intermediateConfig.addInputToGroup(0);
    intermediateConfig.setInputSerializer(inputSerializer, 0);

    intermediateConfig.setOutputSerializer(outputSerializer);
    intermediateConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
    intermediateConfig.setOutputComparator(outputComparator, 0);

    intermediateConfig.setBroadcastInputName("centers", 0);
    intermediateConfig.addBroadcastInputToGroup(0);
    intermediateConfig.setBroadcastInputSerializer(broadcastVarSerializer, 0);

    // the udf
    intermediateConfig.setStubWrapper(
        new UserCodeObjectWrapper<SelectNearestCenter>(new SelectNearestCenter()));

    return mapper;
  }
  private static JobTaskVertex createIterationHead(
      JobGraph jobGraph, int numSubTasks, TypeSerializerFactory<?> serializer) {
    JobTaskVertex head =
        JobGraphUtils.createTask(
            IterationHeadPactTask.class, "Iteration Head", jobGraph, numSubTasks, numSubTasks);

    TaskConfig headConfig = new TaskConfig(head.getConfiguration());
    headConfig.setIterationId(ITERATION_ID);

    // initial input / partial solution
    headConfig.addInputToGroup(0);
    headConfig.setIterationHeadPartialSolutionOrWorksetInputIndex(0);
    headConfig.setInputSerializer(serializer, 0);

    // back channel / iterations
    headConfig.setBackChannelMemory(MEMORY_PER_CONSUMER * JobGraphUtils.MEGABYTE);

    // output into iteration. broadcasting the centers
    headConfig.setOutputSerializer(serializer);
    headConfig.addOutputShipStrategy(ShipStrategyType.BROADCAST);

    // final output
    TaskConfig headFinalOutConfig = new TaskConfig(new Configuration());
    headFinalOutConfig.setOutputSerializer(serializer);
    headFinalOutConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
    headConfig.setIterationHeadFinalOutputConfig(headFinalOutConfig);

    // the sync
    headConfig.setIterationHeadIndexOfSyncOutput(2);

    // the driver
    headConfig.setDriver(NoOpDriver.class);
    headConfig.setDriverStrategy(DriverStrategy.UNARY_NO_OP);

    return head;
  }
  private static JobGraph createJobGraph(
      String pointsPath, String centersPath, String resultPath, int numSubTasks, int numIterations)
      throws JobGraphDefinitionException {

    // -- init
    // -------------------------------------------------------------------------------------------------
    final TypeSerializerFactory<?> serializer = RecordSerializerFactory.get();
    @SuppressWarnings("unchecked")
    final TypeComparatorFactory<?> int0Comparator =
        new RecordComparatorFactory(new int[] {0}, new Class[] {IntValue.class});

    JobGraph jobGraph = new JobGraph("KMeans Iterative");

    // -- vertices
    // ---------------------------------------------------------------------------------------------
    JobInputVertex points = createPointsInput(jobGraph, pointsPath, numSubTasks, serializer);
    JobInputVertex centers = createCentersInput(jobGraph, centersPath, numSubTasks, serializer);

    JobTaskVertex head = createIterationHead(jobGraph, numSubTasks, serializer);
    JobTaskVertex mapper =
        createMapper(jobGraph, numSubTasks, serializer, serializer, serializer, int0Comparator);

    JobTaskVertex reducer =
        createReducer(jobGraph, numSubTasks, serializer, int0Comparator, serializer);

    JobOutputVertex fakeTailOutput =
        JobGraphUtils.createFakeOutput(jobGraph, "FakeTailOutput", numSubTasks, numSubTasks);

    JobOutputVertex sync = createSync(jobGraph, numIterations, numSubTasks);

    JobOutputVertex output = createOutput(jobGraph, resultPath, numSubTasks, serializer);

    // -- edges
    // ------------------------------------------------------------------------------------------------
    JobGraphUtils.connect(points, mapper, ChannelType.NETWORK, DistributionPattern.POINTWISE);

    JobGraphUtils.connect(centers, head, ChannelType.NETWORK, DistributionPattern.POINTWISE);

    JobGraphUtils.connect(head, mapper, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
    new TaskConfig(mapper.getConfiguration())
        .setBroadcastGateIterativeWithNumberOfEventsUntilInterrupt(0, numSubTasks);
    new TaskConfig(mapper.getConfiguration()).setInputCached(0, true);
    new TaskConfig(mapper.getConfiguration())
        .setInputMaterializationMemory(0, MEMORY_PER_CONSUMER * JobGraphUtils.MEGABYTE);

    JobGraphUtils.connect(mapper, reducer, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
    new TaskConfig(reducer.getConfiguration())
        .setGateIterativeWithNumberOfEventsUntilInterrupt(0, numSubTasks);

    JobGraphUtils.connect(
        reducer, fakeTailOutput, ChannelType.NETWORK, DistributionPattern.POINTWISE);

    JobGraphUtils.connect(head, output, ChannelType.NETWORK, DistributionPattern.POINTWISE);

    JobGraphUtils.connect(head, sync, ChannelType.NETWORK, DistributionPattern.BIPARTITE);

    // -- instance sharing
    // -------------------------------------------------------------------------------------
    points.setVertexToShareInstancesWith(output);
    centers.setVertexToShareInstancesWith(output);
    head.setVertexToShareInstancesWith(output);
    mapper.setVertexToShareInstancesWith(output);
    reducer.setVertexToShareInstancesWith(output);
    fakeTailOutput.setVertexToShareInstancesWith(output);
    sync.setVertexToShareInstancesWith(output);

    return jobGraph;
  }