/**
   * Constructs a sample execution graph consisting of two vertices connected by a channel of the
   * given type.
   *
   * @param channelType the channel type to connect the vertices with
   * @param instanceManager the instance manager that shall be used during the creation of the
   *     execution graph
   * @return a sample execution graph
   */
  private ExecutionGraph createExecutionGraph(
      final ChannelType channelType, final InstanceManager instanceManager) {

    final JobGraph jobGraph = new JobGraph("Job Graph");

    final JobInputVertex inputVertex = new JobInputVertex("Input 1", jobGraph);
    inputVertex.setInputClass(InputTask.class);
    inputVertex.setNumberOfSubtasks(1);

    final JobOutputVertex outputVertex = new JobOutputVertex("Output 1", jobGraph);
    outputVertex.setOutputClass(OutputTask.class);
    outputVertex.setNumberOfSubtasks(1);

    try {
      inputVertex.connectTo(outputVertex, channelType);
    } catch (JobGraphDefinitionException e) {
      fail(StringUtils.stringifyException(e));
    }

    try {
      LibraryCacheManager.register(jobGraph.getJobID(), new String[0]);
      return new ExecutionGraph(jobGraph, instanceManager);

    } catch (GraphConversionException e) {
      fail(StringUtils.stringifyException(e));
    } catch (IOException e) {
      fail(StringUtils.stringifyException(e));
    }

    return null;
  }
  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;
  }
  private static JobInputVertex createPointsInput(
      JobGraph jobGraph, String pointsPath, int numSubTasks, TypeSerializerFactory<?> serializer) {
    @SuppressWarnings("unchecked")
    CsvInputFormat pointsInFormat =
        new CsvInputFormat(
            '|', IntValue.class, DoubleValue.class, DoubleValue.class, DoubleValue.class);
    JobInputVertex pointsInput =
        JobGraphUtils.createInput(
            pointsInFormat, pointsPath, "[Points]", jobGraph, numSubTasks, numSubTasks);
    {
      TaskConfig taskConfig = new TaskConfig(pointsInput.getConfiguration());
      taskConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
      taskConfig.setOutputSerializer(serializer);

      TaskConfig chainedMapper = new TaskConfig(new Configuration());
      chainedMapper.setDriverStrategy(DriverStrategy.COLLECTOR_MAP);
      chainedMapper.setStubWrapper(new UserCodeObjectWrapper<PointBuilder>(new PointBuilder()));
      chainedMapper.addOutputShipStrategy(ShipStrategyType.FORWARD);
      chainedMapper.setOutputSerializer(serializer);

      taskConfig.addChainedTask(ChainedCollectorMapDriver.class, chainedMapper, "Build points");
    }

    return pointsInput;
  }
  @SuppressWarnings("unchecked")
  private static JobInputVertex createModelsInput(
      JobGraph jobGraph, String pointsPath, int numSubTasks, TypeSerializerFactory<?> serializer) {
    CsvInputFormat modelsInFormat =
        new CsvInputFormat(' ', LongValue.class, LongValue.class, LongValue.class, LongValue.class);
    JobInputVertex modelsInput =
        JobGraphUtils.createInput(
            modelsInFormat, pointsPath, "Input[Models]", jobGraph, numSubTasks, numSubTasks);

    {
      TaskConfig taskConfig = new TaskConfig(modelsInput.getConfiguration());
      taskConfig.addOutputShipStrategy(ShipStrategyType.BROADCAST);
      taskConfig.setOutputSerializer(serializer);
    }

    return modelsInput;
  }
  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;
  }