/**
  * Kill the task and clean up ProcessTreeInfo
  *
  * @param tid task attempt ID of the task to be killed.
  * @param msg diagonostic message
  * @param wasFailure if true, fail the task
  */
 private void killTask(TaskAttemptID tid, String msg, boolean wasFailure) {
   // Kill the task and mark it as killed.
   taskTracker.cleanUpOverMemoryTask(tid, wasFailure, msg);
   // Now destroy the ProcessTree, remove it from monitoring map.
   ProcessTreeInfo ptInfo = processTreeInfoMap.get(tid);
   ProcfsBasedProcessTree pTree = ptInfo.getProcessTree();
   try {
     LinuxSystemCall.killProcessGroup(Integer.parseInt(ptInfo.getPID()));
   } catch (java.io.IOException e) {
     LOG.error("Could not kill process group " + ptInfo.getPID(), e);
   }
   processTreeInfoMap.remove(tid);
   LOG.info("Removed ProcessTree with root " + ptInfo.getPID());
 }
  @Override
  public void run() {

    LOG.info("Starting thread: " + this.getClass());

    while (running) {
      try {
        // Print the processTrees for debugging.
        if (LOG.isDebugEnabled()) {
          StringBuffer tmp = new StringBuffer("[ ");
          for (ProcessTreeInfo p : processTreeInfoMap.values()) {
            tmp.append(p.getPID());
            tmp.append(" ");
          }
          LOG.debug("Current ProcessTree list : " + tmp.substring(0, tmp.length()) + "]");
        }

        // Add new Tasks
        synchronized (tasksToBeAdded) {
          processTreeInfoMap.putAll(tasksToBeAdded);
          tasksToBeAdded.clear();
        }

        // Remove finished Tasks
        synchronized (tasksToBeRemoved) {
          for (TaskAttemptID tid : tasksToBeRemoved) {
            processTreeInfoMap.remove(tid);
          }
          tasksToBeRemoved.clear();
        }

        long memoryStillInUsage = 0;
        long rssMemoryStillInUsage = 0;
        // Now, check memory usage and kill any overflowing tasks
        for (Iterator<Map.Entry<TaskAttemptID, ProcessTreeInfo>> it =
                processTreeInfoMap.entrySet().iterator();
            it.hasNext(); ) {
          Map.Entry<TaskAttemptID, ProcessTreeInfo> entry = it.next();
          TaskAttemptID tid = entry.getKey();
          ProcessTreeInfo ptInfo = entry.getValue();
          try {
            String pId = ptInfo.getPID();

            // Initialize any uninitialized processTrees
            if (pId == null) {
              // get pid from taskAttemptId
              pId = taskTracker.getPid(ptInfo.getTID());
              if (pId != null) {
                // PID will be null, either if the pid file is yet to be created
                // or if the tip is finished and we removed pidFile, but the TIP
                // itself is still retained in runningTasks till successful
                // transmission to JT

                // create process tree object
                long sleeptimeBeforeSigkill =
                    taskTracker
                        .getJobConf()
                        .getLong(
                            "mapred.tasktracker.tasks.sleeptime-before-sigkill",
                            ProcessTree.DEFAULT_SLEEPTIME_BEFORE_SIGKILL);

                ProcfsBasedProcessTree pt =
                    new ProcfsBasedProcessTree(
                        pId, ProcessTree.isSetsidAvailable, sleeptimeBeforeSigkill);
                LOG.debug("Tracking ProcessTree " + pId + " for the first time");

                ptInfo.setPid(pId);
                ptInfo.setProcessTree(pt);
              }
            }
            // End of initializing any uninitialized processTrees

            if (pId == null) {
              continue; // processTree cannot be tracked
            }

            LOG.debug("Constructing ProcessTree for : PID = " + pId + " TID = " + tid);
            ProcfsBasedProcessTree pTree = ptInfo.getProcessTree();
            pTree = pTree.getProcessTree(); // get the updated process-tree
            ptInfo.setProcessTree(pTree); // update ptInfo with process-tree of
            // updated state
            long currentMemUsage = pTree.getCumulativeVmem();
            long currentRssMemUsage = pTree.getCumulativeRssmem();
            // as processes begin with an age 1, we want to see if there
            // are processes more than 1 iteration old.
            long curMemUsageOfAgedProcesses = pTree.getCumulativeVmem(1);
            long limit = ptInfo.getMemLimit();
            String user = taskTracker.getUserName(ptInfo.tid);
            if (user == null) {
              // If user is null the task is deleted from the TT memory
              continue;
            }
            // Log RSS and virtual memory usage of all tasks
            LOG.debug(
                (String.format(
                    "Memory usage of ProcessTree %s : "
                        + "[USER,TID,RSS,VMEM,VLimit,TotalRSSLimit]"
                        + "=[%s,%s,%s,%s,%s,%s]",
                    pId,
                    user,
                    ptInfo.tid,
                    currentRssMemUsage,
                    currentMemUsage,
                    limit,
                    maxRssMemoryAllowedForAllTasks)));

            if (doCheckVirtualMemory()
                && isProcessTreeOverLimit(
                    tid.toString(), currentMemUsage, curMemUsageOfAgedProcesses, limit)) {
              // Task (the root process) is still alive and overflowing memory.
              // Dump the process-tree and then clean it up.
              String msg =
                  "TaskTree [pid="
                      + pId
                      + ",tipID="
                      + tid
                      + "] is running beyond memory-limits. Current usage : "
                      + currentMemUsage
                      + "bytes. Limit : "
                      + limit
                      + "bytes. Killing task. \nDump of the process-tree for "
                      + tid
                      + " : \n"
                      + pTree.getProcessTreeDump();
              LOG.warn(msg);
              taskTracker.cleanUpOverMemoryTask(tid, true, msg);

              LinuxSystemCall.killProcessGroup(Integer.parseInt(pId));
              it.remove();
              LOG.info("Removed ProcessTree with root " + pId);
            } else {
              // Accounting the total memory in usage for all tasks that are still
              // alive and within limits.
              memoryStillInUsage += currentMemUsage;
              rssMemoryStillInUsage += currentRssMemUsage;
            }
          } catch (Exception e) {
            // Log the exception and proceed to the next task.
            LOG.warn(
                "Uncaught exception in TaskMemoryManager " + "while managing memory of " + tid, e);
          }
        }
        long availableRssMemory = resourceCalculator.getAvailablePhysicalMemorySize();

        LOG.info(
            "vMemory:"
                + memoryStillInUsage
                + " rssMemory:"
                + rssMemoryStillInUsage
                + " rssMemoryLimit:"
                + maxRssMemoryAllowedForAllTasks
                + " rssMemoryAvailable:"
                + availableRssMemory
                + " rssMemoryReserved:"
                + reservedRssMemory
                + " totalTasks:"
                + processTreeInfoMap.size());

        if (doCheckVirtualMemory() && memoryStillInUsage > maxMemoryAllowedForAllTasks) {
          LOG.warn(
              "The total memory in usage "
                  + memoryStillInUsage
                  + " is overflowing TTs limits "
                  + maxMemoryAllowedForAllTasks
                  + ". Trying to kill a few tasks with the least progress.");
          killTasksWithLeastProgress(memoryStillInUsage);
        }

        updateMaxRssMemory();
        if (doCheckPhysicalMemory()
            && (rssMemoryStillInUsage > maxRssMemoryAllowedForAllTasks
                || availableRssMemory < reservedRssMemory)) {
          LOG.warn(
              "The total physical memory in usage "
                  + rssMemoryStillInUsage
                  + " is overflowing TTs limits "
                  + maxRssMemoryAllowedForAllTasks
                  + ". Trying to kill a few tasks with the highest memory.");
          failTasksWithMaxRssMemory(rssMemoryStillInUsage, availableRssMemory);
        }

        // Sleep for some time before beginning next cycle
        LOG.debug(this.getClass() + " : Sleeping for " + monitoringInterval + " ms");
        Thread.sleep(monitoringInterval);
      } catch (InterruptedException iex) {
        if (running) {
          LOG.error("Class " + this.getClass() + " was interrupted", iex);
        }
      } catch (Throwable t) {
        LOG.error("Class " + this.getClass() + " encountered error", t);
      }
    }
  }