@Override
  public void fetchFailed(
      String host, InputAttemptIdentifier srcAttemptIdentifier, boolean connectFailed) {
    // TODO NEWTEZ. Implement logic to report fetch failures after a threshold.
    // For now, reporting immediately.
    LOG.info(
        "Fetch failed for src: "
            + srcAttemptIdentifier
            + "InputIdentifier: "
            + srcAttemptIdentifier
            + ", connectFailed: "
            + connectFailed);
    failedShufflesCounter.increment(1);
    if (srcAttemptIdentifier == null) {
      String message = "Received fetchFailure for an unknown src (null)";
      LOG.fatal(message);
      inputContext.fatalError(null, message);
    } else {
      InputReadErrorEvent readError =
          new InputReadErrorEvent(
              "Fetch failure while fetching from "
                  + TezRuntimeUtils.getTaskAttemptIdentifier(
                      inputContext.getSourceVertexName(),
                      srcAttemptIdentifier.getInputIdentifier().getInputIndex(),
                      srcAttemptIdentifier.getAttemptNumber()),
              srcAttemptIdentifier.getInputIdentifier().getInputIndex(),
              srcAttemptIdentifier.getAttemptNumber());

      List<Event> failedEvents = Lists.newArrayListWithCapacity(1);
      failedEvents.add(readError);
      inputContext.sendEvents(failedEvents);
    }
  }
 private void registerCompletedInput(FetchedInput fetchedInput) {
   lock.lock();
   try {
     completedInputSet.add(fetchedInput.getInputAttemptIdentifier().getInputIdentifier());
     completedInputs.add(fetchedInput);
     if (!inputReadyNotificationSent.getAndSet(true)) {
       // TODO Should eventually be controlled by Inputs which are processing the data.
       inputContext.inputIsReady();
     }
     int numComplete = numCompletedInputs.incrementAndGet();
     if (numComplete == numInputs) {
       LOG.info("All inputs fetched for input vertex : " + inputContext.getSourceVertexName());
     }
   } finally {
     lock.unlock();
   }
 }
  public ShuffleManager(
      TezInputContext inputContext,
      Configuration conf,
      int numInputs,
      int bufferSize,
      boolean ifileReadAheadEnabled,
      int ifileReadAheadLength,
      CompressionCodec codec,
      FetchedInputAllocator inputAllocator)
      throws IOException {
    this.inputContext = inputContext;
    this.numInputs = numInputs;

    this.shuffledInputsCounter =
        inputContext.getCounters().findCounter(TaskCounter.NUM_SHUFFLED_INPUTS);
    this.failedShufflesCounter =
        inputContext.getCounters().findCounter(TaskCounter.NUM_FAILED_SHUFFLE_INPUTS);
    this.bytesShuffledCounter = inputContext.getCounters().findCounter(TaskCounter.SHUFFLE_BYTES);
    this.decompressedDataSizeCounter =
        inputContext.getCounters().findCounter(TaskCounter.SHUFFLE_BYTES_DECOMPRESSED);
    this.bytesShuffledToDiskCounter =
        inputContext.getCounters().findCounter(TaskCounter.SHUFFLE_BYTES_TO_DISK);
    this.bytesShuffledToMemCounter =
        inputContext.getCounters().findCounter(TaskCounter.SHUFFLE_BYTES_TO_MEM);

    this.ifileBufferSize = bufferSize;
    this.ifileReadAhead = ifileReadAheadEnabled;
    this.ifileReadAheadLength = ifileReadAheadLength;
    this.codec = codec;
    this.inputManager = inputAllocator;

    this.srcNameTrimmed = TezUtils.cleanVertexName(inputContext.getSourceVertexName());

    completedInputSet =
        Collections.newSetFromMap(new ConcurrentHashMap<InputIdentifier, Boolean>(numInputs));
    completedInputs = new LinkedBlockingQueue<FetchedInput>(numInputs);
    knownSrcHosts = new ConcurrentHashMap<String, InputHost>();
    pendingHosts = new LinkedBlockingQueue<InputHost>();
    obsoletedInputs =
        Collections.newSetFromMap(new ConcurrentHashMap<InputAttemptIdentifier, Boolean>());
    runningFetchers = Collections.newSetFromMap(new ConcurrentHashMap<Fetcher, Boolean>());

    int maxConfiguredFetchers =
        conf.getInt(
            TezJobConfig.TEZ_RUNTIME_SHUFFLE_PARALLEL_COPIES,
            TezJobConfig.TEZ_RUNTIME_SHUFFLE_PARALLEL_COPIES_DEFAULT);

    this.numFetchers = Math.min(maxConfiguredFetchers, numInputs);

    ExecutorService fetcherRawExecutor =
        Executors.newFixedThreadPool(
            numFetchers,
            new ThreadFactoryBuilder()
                .setDaemon(true)
                .setNameFormat("Fetcher [" + srcNameTrimmed + "] #%d")
                .build());
    this.fetcherExecutor = MoreExecutors.listeningDecorator(fetcherRawExecutor);

    ExecutorService schedulerRawExecutor =
        Executors.newFixedThreadPool(
            1,
            new ThreadFactoryBuilder()
                .setDaemon(true)
                .setNameFormat("ShuffleRunner [" + srcNameTrimmed + "]")
                .build());
    this.schedulerExecutor = MoreExecutors.listeningDecorator(schedulerRawExecutor);

    this.startTime = System.currentTimeMillis();
    this.lastProgressTime = startTime;

    this.shuffleSecret =
        ShuffleUtils.getJobTokenSecretFromTokenBytes(
            inputContext.getServiceConsumerMetaData(
                TezConfiguration.TEZ_SHUFFLE_HANDLER_SERVICE_ID));
    httpConnectionParams = ShuffleUtils.constructHttpShuffleConnectionParams(conf);
    LOG.info(
        this.getClass().getSimpleName()
            + " : numInputs="
            + numInputs
            + ", compressionCodec="
            + (codec == null ? "NoCompressionCodec" : codec.getClass().getName())
            + ", numFetchers="
            + numFetchers
            + ", ifileBufferSize="
            + ifileBufferSize
            + ", ifileReadAheadEnabled="
            + ifileReadAhead
            + ", ifileReadAheadLength="
            + ifileReadAheadLength
            + ", "
            + httpConnectionParams.toString());
  }