@Override
  public void setup(OperatorContext context) {
    operatorId = context.getId();
    globalProcessedFileCount.setValue(processedFiles.size());
    LOG.debug("Setup processed file count: {}", globalProcessedFileCount);
    this.context = context;

    try {
      filePath = new Path(directory);
      configuration = new Configuration();
      fs = getFSInstance();
    } catch (IOException ex) {
      failureHandling(ex);
    }

    fileCounters.setCounter(FileCounters.GLOBAL_PROCESSED_FILES, globalProcessedFileCount);
    fileCounters.setCounter(FileCounters.LOCAL_PROCESSED_FILES, localProcessedFileCount);
    fileCounters.setCounter(FileCounters.GLOBAL_NUMBER_OF_FAILURES, globalNumberOfFailures);
    fileCounters.setCounter(FileCounters.LOCAL_NUMBER_OF_FAILURES, localNumberOfFailures);
    fileCounters.setCounter(FileCounters.GLOBAL_NUMBER_OF_RETRIES, globalNumberOfRetries);
    fileCounters.setCounter(FileCounters.LOCAL_NUMBER_OF_RETRIES, localNumberOfRetries);
    fileCounters.setCounter(FileCounters.PENDING_FILES, pendingFileCount);

    idempotentStorageManager.setup(context);
    if (context.getValue(OperatorContext.ACTIVATION_WINDOW_ID)
        < idempotentStorageManager.getLargestRecoveryWindow()) {
      // reset current file and offset in case of replay
      currentFile = null;
      offset = 0;
    }
  }
 @Override
 public void beginWindow(long windowId) {
   currentWindowId = windowId;
   if (windowId <= idempotentStorageManager.getLargestRecoveryWindow()) {
     replay(windowId);
   }
 }
 @Override
 public void committed(long windowId) {
   try {
     idempotentStorageManager.deleteUpTo(operatorId, windowId);
   } catch (IOException e) {
     throw new RuntimeException(e);
   }
 }
  @Override
  public void endWindow() {
    if (currentWindowId > idempotentStorageManager.getLargestRecoveryWindow()) {
      try {
        idempotentStorageManager.save(currentWindowRecoveryState, operatorId, currentWindowId);
      } catch (IOException e) {
        throw new RuntimeException("saving recovery", e);
      }
    }
    currentWindowRecoveryState.clear();
    if (context != null) {
      pendingFileCount.setValue(pendingFiles.size() + failedFiles.size() + unfinishedFiles.size());

      if (currentFile != null) {
        pendingFileCount.increment();
      }

      context.setCounters(fileCounters);
    }
  }
  @Override
  public void teardown() {
    IOException savedException = null;
    boolean fileFailed = false;

    try {
      if (inputStream != null) {
        inputStream.close();
      }
    } catch (IOException ex) {
      savedException = ex;
      fileFailed = true;
    }

    boolean fsFailed = false;

    try {
      fs.close();
    } catch (IOException ex) {
      savedException = ex;
      fsFailed = true;
    }

    if (savedException != null) {
      String errorMessage = "";

      if (fileFailed) {
        errorMessage += "Failed to close " + currentFile + ". ";
      }

      if (fsFailed) {
        errorMessage += "Failed to close filesystem.";
      }

      throw new RuntimeException(errorMessage, savedException);
    }
    idempotentStorageManager.teardown();
  }
  @Override
  public Collection<Partition<AbstractFileInputOperator<T>>> definePartitions(
      Collection<Partition<AbstractFileInputOperator<T>>> partitions, PartitioningContext context) {
    lastRepartition = System.currentTimeMillis();

    int totalCount = getNewPartitionCount(partitions, context);

    LOG.debug("Computed new partitions: {}", totalCount);

    if (totalCount == partitions.size()) {
      return partitions;
    }

    AbstractFileInputOperator<T> tempOperator =
        partitions.iterator().next().getPartitionedInstance();

    MutableLong tempGlobalNumberOfRetries = tempOperator.globalNumberOfRetries;
    MutableLong tempGlobalNumberOfFailures = tempOperator.globalNumberOfRetries;

    /*
     * Build collective state from all instances of the operator.
     */
    Set<String> totalProcessedFiles = Sets.newHashSet();
    Set<FailedFile> currentFiles = Sets.newHashSet();
    List<DirectoryScanner> oldscanners = Lists.newLinkedList();
    List<FailedFile> totalFailedFiles = Lists.newLinkedList();
    List<String> totalPendingFiles = Lists.newLinkedList();
    Set<Integer> deletedOperators = Sets.newHashSet();

    for (Partition<AbstractFileInputOperator<T>> partition : partitions) {
      AbstractFileInputOperator<T> oper = partition.getPartitionedInstance();
      totalProcessedFiles.addAll(oper.processedFiles);
      totalFailedFiles.addAll(oper.failedFiles);
      totalPendingFiles.addAll(oper.pendingFiles);
      currentFiles.addAll(unfinishedFiles);
      tempGlobalNumberOfRetries.add(oper.localNumberOfRetries);
      tempGlobalNumberOfFailures.add(oper.localNumberOfFailures);
      if (oper.currentFile != null) {
        currentFiles.add(new FailedFile(oper.currentFile, oper.offset));
      }
      oldscanners.add(oper.getScanner());
      deletedOperators.add(oper.operatorId);
    }

    /*
     * Create partitions of scanners, scanner's partition method will do state
     * transfer for DirectoryScanner objects.
     */
    List<DirectoryScanner> scanners = scanner.partition(totalCount, oldscanners);

    Kryo kryo = new Kryo();
    Collection<Partition<AbstractFileInputOperator<T>>> newPartitions =
        Lists.newArrayListWithExpectedSize(totalCount);
    Collection<IdempotentStorageManager> newManagers =
        Lists.newArrayListWithExpectedSize(totalCount);

    for (int i = 0; i < scanners.size(); i++) {

      // Kryo.copy fails as it attempts to clone transient fields
      ByteArrayOutputStream bos = new ByteArrayOutputStream();
      Output loutput = new Output(bos);
      kryo.writeObject(loutput, this);
      loutput.close();
      Input lInput = new Input(bos.toByteArray());
      @SuppressWarnings("unchecked")
      AbstractFileInputOperator<T> oper = kryo.readObject(lInput, this.getClass());
      lInput.close();

      DirectoryScanner scn = scanners.get(i);
      oper.setScanner(scn);

      // Do state transfer for processed files.
      oper.processedFiles.addAll(totalProcessedFiles);
      oper.globalNumberOfFailures = tempGlobalNumberOfRetries;
      oper.localNumberOfFailures.setValue(0);
      oper.globalNumberOfRetries = tempGlobalNumberOfFailures;
      oper.localNumberOfRetries.setValue(0);

      /* redistribute unfinished files properly */
      oper.unfinishedFiles.clear();
      oper.currentFile = null;
      oper.offset = 0;
      Iterator<FailedFile> unfinishedIter = currentFiles.iterator();
      while (unfinishedIter.hasNext()) {
        FailedFile unfinishedFile = unfinishedIter.next();
        if (scn.acceptFile(unfinishedFile.path)) {
          oper.unfinishedFiles.add(unfinishedFile);
          unfinishedIter.remove();
        }
      }

      /* transfer failed files */
      oper.failedFiles.clear();
      Iterator<FailedFile> iter = totalFailedFiles.iterator();
      while (iter.hasNext()) {
        FailedFile ff = iter.next();
        if (scn.acceptFile(ff.path)) {
          oper.failedFiles.add(ff);
          iter.remove();
        }
      }

      /* redistribute pending files properly */
      oper.pendingFiles.clear();
      Iterator<String> pendingFilesIterator = totalPendingFiles.iterator();
      while (pendingFilesIterator.hasNext()) {
        String pathString = pendingFilesIterator.next();
        if (scn.acceptFile(pathString)) {
          oper.pendingFiles.add(pathString);
          pendingFilesIterator.remove();
        }
      }
      newPartitions.add(new DefaultPartition<AbstractFileInputOperator<T>>(oper));
      newManagers.add(oper.idempotentStorageManager);
    }

    idempotentStorageManager.partitioned(newManagers, deletedOperators);
    LOG.info("definePartitions called returning {} partitions", newPartitions.size());
    return newPartitions;
  }
  @Override
  public void emitTuples() {
    if (currentWindowId <= idempotentStorageManager.getLargestRecoveryWindow()) {
      return;
    }

    if (inputStream == null) {
      try {
        if (currentFile != null && offset > 0) {
          // open file resets offset to 0 so this a way around it.
          int tmpOffset = offset;
          if (fs.exists(new Path(currentFile))) {
            this.inputStream = openFile(new Path(currentFile));
            offset = tmpOffset;
            skipCount = tmpOffset;
          } else {
            currentFile = null;
            offset = 0;
            skipCount = 0;
          }
        } else if (!unfinishedFiles.isEmpty()) {
          retryFailedFile(unfinishedFiles.poll());
        } else if (!pendingFiles.isEmpty()) {
          String newPathString = pendingFiles.iterator().next();
          pendingFiles.remove(newPathString);
          if (fs.exists(new Path(newPathString)))
            this.inputStream = openFile(new Path(newPathString));
        } else if (!failedFiles.isEmpty()) {
          retryFailedFile(failedFiles.poll());
        } else {
          scanDirectory();
        }
      } catch (IOException ex) {
        failureHandling(ex);
      }
    }
    if (inputStream != null) {
      int startOffset = offset;
      String file = currentFile; // current file is reset to null when closed.

      try {
        int counterForTuple = 0;
        while (counterForTuple++ < emitBatchSize) {
          T line = readEntity();
          if (line == null) {
            LOG.info("done reading file ({} entries).", offset);
            closeFile(inputStream);
            break;
          }

          // If skipCount is non zero, then failed file recovery is going on, skipCount is
          // used to prevent already emitted records from being emitted again during recovery.
          // When failed file is open, skipCount is set to the last read offset for that file.
          //
          if (skipCount == 0) {
            offset++;
            emit(line);
          } else {
            skipCount--;
          }
        }
      } catch (IOException e) {
        failureHandling(e);
      }
      // Only when something was emitted from the file then we record it for entry.
      if (offset > startOffset) {
        currentWindowRecoveryState.add(new RecoveryEntry(file, startOffset, offset));
      }
    }
  }
  protected void replay(long windowId) {
    // This operator can partition itself dynamically. When that happens a file can be re-hashed
    // to a different partition than the previous one. In order to handle this, the partition loads
    // all the recovery data for a window and then processes only those files which would be hashed
    // to it in the current run.
    try {
      Map<Integer, Object> recoveryDataPerOperator = idempotentStorageManager.load(windowId);

      for (Object recovery : recoveryDataPerOperator.values()) {
        @SuppressWarnings("unchecked")
        LinkedList<RecoveryEntry> recoveryData = (LinkedList<RecoveryEntry>) recovery;

        for (RecoveryEntry recoveryEntry : recoveryData) {
          if (scanner.acceptFile(recoveryEntry.file)) {
            // The operator may have continued processing the same file in multiple windows.
            // So the recovery states of subsequent windows will have an entry for that file however
            // the offset changes.
            // In this case we continue reading from previously opened stream.
            if (currentFile == null
                || !(currentFile.equals(recoveryEntry.file)
                    && offset == recoveryEntry.startOffset)) {
              if (inputStream != null) {
                closeFile(inputStream);
              }
              processedFiles.add(recoveryEntry.file);
              // removing the file from failed and unfinished queues and pending set
              Iterator<FailedFile> failedFileIterator = failedFiles.iterator();
              while (failedFileIterator.hasNext()) {
                FailedFile ff = failedFileIterator.next();
                if (ff.path.equals(recoveryEntry.file) && ff.offset == recoveryEntry.startOffset) {
                  failedFileIterator.remove();
                  break;
                }
              }

              Iterator<FailedFile> unfinishedFileIterator = unfinishedFiles.iterator();
              while (unfinishedFileIterator.hasNext()) {
                FailedFile ff = unfinishedFileIterator.next();
                if (ff.path.equals(recoveryEntry.file) && ff.offset == recoveryEntry.startOffset) {
                  unfinishedFileIterator.remove();
                  break;
                }
              }
              if (pendingFiles.contains(recoveryEntry.file)) {
                pendingFiles.remove(recoveryEntry.file);
              }
              inputStream =
                  retryFailedFile(new FailedFile(recoveryEntry.file, recoveryEntry.startOffset));
              while (offset < recoveryEntry.endOffset) {
                T line = readEntity();
                offset++;
                emit(line);
              }
            } else {
              while (offset < recoveryEntry.endOffset) {
                T line = readEntity();
                offset++;
                emit(line);
              }
            }
          }
        }
      }
    } catch (IOException e) {
      throw new RuntimeException("replay", e);
    }
  }