You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ae...@apache.org on 2018/01/30 19:28:40 UTC

[01/37] hadoop git commit: MAPREDUCE-7015. Possible race condition in JHS if the job is not loaded. Contributed by Peter Bacsko

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-7240 7955ee404 -> 7de498c3a


MAPREDUCE-7015. Possible race condition in JHS if the job is not loaded. Contributed by Peter Bacsko


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/cff9edd4
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/cff9edd4
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/cff9edd4

Branch: refs/heads/HDFS-7240
Commit: cff9edd4b514bdcfe22cd49964e3707fb78ab876
Parents: 55c3277
Author: Jason Lowe <jl...@apache.org>
Authored: Wed Jan 24 14:44:07 2018 -0600
Committer: Jason Lowe <jl...@apache.org>
Committed: Wed Jan 24 14:44:07 2018 -0600

----------------------------------------------------------------------
 .../mapreduce/v2/hs/CachedHistoryStorage.java   |  8 +++++-
 .../mapreduce/v2/hs/HistoryFileManager.java     | 30 ++++++++++++++++----
 .../hadoop/mapreduce/v2/hs/TestJobHistory.java  | 26 +++++++++++++++++
 3 files changed, 57 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cff9edd4/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CachedHistoryStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CachedHistoryStorage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CachedHistoryStorage.java
index b001ae4..69f4831 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CachedHistoryStorage.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CachedHistoryStorage.java
@@ -173,9 +173,14 @@ public class CachedHistoryStorage extends AbstractService implements
     HistoryFileInfo fileInfo;
 
     fileInfo = hsManager.getFileInfo(jobId);
+
     if (fileInfo == null) {
       throw new HSFileRuntimeException("Unable to find job " + jobId);
-    } else if (fileInfo.isDeleted()) {
+    }
+
+    fileInfo.waitUntilMoved();
+
+    if (fileInfo.isDeleted()) {
       throw new HSFileRuntimeException("Cannot load deleted job " + jobId);
     } else {
       return fileInfo.loadJob();
@@ -211,6 +216,7 @@ public class CachedHistoryStorage extends AbstractService implements
       for (HistoryFileInfo mi : hsManager.getAllFileInfo()) {
         if (mi != null) {
           JobId id = mi.getJobId();
+          mi.waitUntilMoved();
           result.put(id, new PartialJob(mi.getJobIndexInfo(), id));
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cff9edd4/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryFileManager.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryFileManager.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryFileManager.java
index b418db7..a07ca26 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryFileManager.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryFileManager.java
@@ -452,6 +452,8 @@ public class HistoryFileManager extends AbstractService {
       } catch (Throwable t) {
         LOG.error("Error while trying to move a job to done", t);
         this.state = HistoryInfoState.MOVE_FAILED;
+      } finally {
+        notifyAll();
       }
     }
 
@@ -485,12 +487,16 @@ public class HistoryFileManager extends AbstractService {
     }
     
     protected synchronized void delete() throws IOException {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("deleting " + historyFile + " and " + confFile);
+      try {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("deleting " + historyFile + " and " + confFile);
+        }
+        state = HistoryInfoState.DELETED;
+        doneDirFc.delete(doneDirFc.makeQualified(historyFile), false);
+        doneDirFc.delete(doneDirFc.makeQualified(confFile), false);
+      } finally {
+        notifyAll();
       }
-      state = HistoryInfoState.DELETED;
-      doneDirFc.delete(doneDirFc.makeQualified(historyFile), false);
-      doneDirFc.delete(doneDirFc.makeQualified(confFile), false);
     }
 
     public JobIndexInfo getJobIndexInfo() {
@@ -517,6 +523,17 @@ public class HistoryFileManager extends AbstractService {
           jobIndexInfo.getNumMaps();
       return (maxTasksForLoadedJob > 0) && (totalTasks > maxTasksForLoadedJob);
     }
+
+    public synchronized void waitUntilMoved() {
+      while (isMovePending() && !didMoveFail()) {
+        try {
+          wait();
+        } catch (InterruptedException e) {
+          LOG.warn("Waiting has been interrupted");
+          throw new RuntimeException(e);
+        }
+      }
+    }
   }
 
   private SerialNumberIndex serialNumberIndex = null;
@@ -956,6 +973,7 @@ public class HistoryFileManager extends AbstractService {
           if (LOG.isDebugEnabled()) {
             LOG.debug("Scheduling move to done of " +found);
           }
+
           moveToDoneExecutor.execute(new Runnable() {
             @Override
             public void run() {
@@ -1193,5 +1211,5 @@ public class HistoryFileManager extends AbstractService {
   @VisibleForTesting
   void setMaxHistoryAge(long newValue){
     maxHistoryAge=newValue;
-  } 
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cff9edd4/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistory.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistory.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistory.java
index 936c772..9f36477 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistory.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistory.java
@@ -446,6 +446,32 @@ public class TestJobHistory {
   }
 
   @Test
+  public void testCachedStorageWaitsForFileMove() throws IOException {
+    HistoryFileManager historyManager = mock(HistoryFileManager.class);
+    jobHistory = spy(new JobHistory());
+    doReturn(historyManager).when(jobHistory).createHistoryFileManager();
+
+    Configuration conf = new Configuration();
+    jobHistory.init(conf);
+    jobHistory.start();
+
+    CachedHistoryStorage storage = spy((CachedHistoryStorage) jobHistory
+        .getHistoryStorage());
+
+    Job job  = mock(Job.class);
+    JobId jobId  = mock(JobId.class);
+    when(job.getID()).thenReturn(jobId);
+    when(job.getTotalMaps()).thenReturn(10);
+    when(job.getTotalReduces()).thenReturn(2);
+    HistoryFileInfo fileInfo = mock(HistoryFileInfo.class);
+    when(historyManager.getFileInfo(eq(jobId))).thenReturn(fileInfo);
+    when(fileInfo.loadJob()).thenReturn(job);
+
+    storage.getFullJob(jobId);
+    verify(fileInfo).waitUntilMoved();
+  }
+
+  @Test
   public void testRefreshLoadedJobCacheUnSupportedOperation() {
     jobHistory = spy(new JobHistory());
     HistoryStorage storage = new HistoryStorage() {


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[22/37] hadoop git commit: MAPREDUCE-7020. Task timeout in uber mode can crash AM. Contributed by Peter Bacsko

Posted by ae...@apache.org.
MAPREDUCE-7020. Task timeout in uber mode can crash AM. Contributed by Peter Bacsko


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/6eef3d7f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/6eef3d7f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/6eef3d7f

Branch: refs/heads/HDFS-7240
Commit: 6eef3d7f1a1e5e3f27fb3bf7596663640d786181
Parents: e990904
Author: Jason Lowe <jl...@apache.org>
Authored: Fri Jan 26 15:31:43 2018 -0600
Committer: Jason Lowe <jl...@apache.org>
Committed: Fri Jan 26 15:31:43 2018 -0600

----------------------------------------------------------------------
 .../apache/hadoop/mapred/TaskAttemptListenerImpl.java |  8 +++++---
 .../hadoop/mapred/TestTaskAttemptListenerImpl.java    |  6 ++++--
 .../src/main/java/org/apache/hadoop/mapred/Task.java  | 14 +++++++++++---
 3 files changed, 20 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6eef3d7f/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java
index b155af22..668d8ed 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java
@@ -369,14 +369,16 @@ public class TaskAttemptListenerImpl extends CompositeService
     org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId yarnAttemptID =
         TypeConverter.toYarn(taskAttemptID);
 
+    AMFeedback feedback = new AMFeedback();
     AtomicReference<TaskAttemptStatus> lastStatusRef =
         attemptIdToStatus.get(yarnAttemptID);
     if (lastStatusRef == null) {
-      throw new IllegalStateException("Status update was called"
-          + " with illegal TaskAttemptId: " + yarnAttemptID);
+      LOG.error("Status update was called with illegal TaskAttemptId: "
+          + yarnAttemptID);
+      feedback.setTaskFound(false);
+      return feedback;
     }
 
-    AMFeedback feedback = new AMFeedback();
     feedback.setTaskFound(true);
 
     // Propagating preemption to the task if TASK_PREEMPTION is enabled

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6eef3d7f/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptListenerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptListenerImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptListenerImpl.java
index 4ff6fb2..da7421b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptListenerImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptListenerImpl.java
@@ -487,13 +487,15 @@ public class TestTaskAttemptListenerImpl {
     assertEquals(Phase.REDUCE, status.phase);
   }
 
-  @Test(expected = IllegalStateException.class)
+  @Test
   public void testStatusUpdateFromUnregisteredTask()
       throws IOException, InterruptedException{
     configureMocks();
     startListener(false);
 
-    listener.statusUpdate(attemptID, firstReduceStatus);
+    AMFeedback feedback = listener.statusUpdate(attemptID, firstReduceStatus);
+
+    assertFalse(feedback.getTaskFound());
   }
 
   private void configureMocks() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6eef3d7f/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Task.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Task.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Task.java
index 87c9e16..5b98b35 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Task.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Task.java
@@ -855,6 +855,9 @@ abstract public class Task implements Writable, Configurable {
       long taskProgressInterval = MRJobConfUtil.
           getTaskProgressReportInterval(conf);
 
+      boolean uberized = conf.getBoolean("mapreduce.task.uberized",
+          false);
+
       while (!taskDone.get()) {
         synchronized (lock) {
           done = false;
@@ -893,9 +896,14 @@ abstract public class Task implements Writable, Configurable {
           // if Task Tracker is not aware of our task ID (probably because it died and 
           // came back up), kill ourselves
           if (!taskFound) {
-            LOG.warn("Parent died.  Exiting "+taskId);
-            resetDoneFlag();
-            System.exit(66);
+            if (uberized) {
+              taskDone.set(true);
+              break;
+            } else {
+              LOG.warn("Parent died.  Exiting "+taskId);
+              resetDoneFlag();
+              System.exit(66);
+            }
           }
 
           // Set a flag that says we should preempt this is read by


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[35/37] hadoop git commit: Revert "YARN-2185. Use pipes when localizing archives. Contributed by Miklos Szegedi"

Posted by ae...@apache.org.
Revert "YARN-2185. Use pipes when localizing archives. Contributed by Miklos Szegedi"

This reverts commit 1b0f265db1a5bfccf1d870912237ea9618bd9c34.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/901d15a3
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/901d15a3
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/901d15a3

Branch: refs/heads/HDFS-7240
Commit: 901d15a30b9fc6c7015f4e2e2c06e6ee42a39662
Parents: 6463e10
Author: Jason Lowe <jl...@apache.org>
Authored: Tue Jan 30 08:34:39 2018 -0600
Committer: Jason Lowe <jl...@apache.org>
Committed: Tue Jan 30 08:34:39 2018 -0600

----------------------------------------------------------------------
 .../java/org/apache/hadoop/fs/FileUtil.java     | 251 +------------------
 .../java/org/apache/hadoop/util/RunJar.java     |  65 -----
 .../org/apache/hadoop/yarn/util/FSDownload.java | 215 ++++++----------
 .../apache/hadoop/yarn/util/TestFSDownload.java |  30 +--
 4 files changed, 99 insertions(+), 462 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/901d15a3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
index bf9b146..4d971aa 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
@@ -20,35 +20,27 @@ package org.apache.hadoop.fs;
 
 import java.io.BufferedInputStream;
 import java.io.BufferedOutputStream;
-import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
-import java.io.InputStreamReader;
 import java.io.OutputStream;
 import java.net.InetAddress;
 import java.net.URI;
 import java.net.UnknownHostException;
-import java.nio.charset.Charset;
 import java.nio.file.AccessDeniedException;
 import java.util.ArrayList;
 import java.util.Enumeration;
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
 import java.util.jar.Attributes;
 import java.util.jar.JarOutputStream;
 import java.util.jar.Manifest;
 import java.util.zip.GZIPInputStream;
 import java.util.zip.ZipEntry;
 import java.util.zip.ZipFile;
-import java.util.zip.ZipInputStream;
 
 import org.apache.commons.collections.map.CaseInsensitiveMap;
 import org.apache.commons.compress.archivers.tar.TarArchiveEntry;
@@ -83,11 +75,6 @@ public class FileUtil {
   public static final int SYMLINK_NO_PRIVILEGE = 2;
 
   /**
-   * Buffer size for copy the content of compressed file to new file.
-   */
-  private static final int BUFFER_SIZE = 8_192;
-
-  /**
    * convert an array of FileStatus to an array of Path
    *
    * @param stats
@@ -539,22 +526,6 @@ public class FileUtil {
   }
 
   /**
-   * Convert a os-native filename to a path that works for the shell
-   * and avoids script injection attacks.
-   * @param file The filename to convert
-   * @return The unix pathname
-   * @throws IOException on windows, there can be problems with the subprocess
-   */
-  public static String makeSecureShellPath(File file) throws IOException {
-    if (Shell.WINDOWS) {
-      // Currently it is never called, but it might be helpful in the future.
-      throw new UnsupportedOperationException("Not implemented for Windows");
-    } else {
-      return makeShellPath(file, false).replace("'", "'\\''");
-    }
-  }
-
-  /**
    * Convert a os-native filename to a path that works for the shell.
    * @param file The filename to convert
    * @param makeCanonicalPath
@@ -605,48 +576,11 @@ public class FileUtil {
   }
 
   /**
-   * Given a stream input it will unzip the it in the unzip directory.
-   * passed as the second parameter
-   * @param inputStream The zip file as input
-   * @param toDir The unzip directory where to unzip the zip file.
-   * @throws IOException an exception occurred
-   */
-  public static void unZip(InputStream inputStream, File toDir)
-      throws IOException {
-    try (ZipInputStream zip = new ZipInputStream(inputStream)) {
-      int numOfFailedLastModifiedSet = 0;
-      for(ZipEntry entry = zip.getNextEntry();
-          entry != null;
-          entry = zip.getNextEntry()) {
-        if (!entry.isDirectory()) {
-          File file = new File(toDir, entry.getName());
-          File parent = file.getParentFile();
-          if (!parent.mkdirs() &&
-              !parent.isDirectory()) {
-            throw new IOException("Mkdirs failed to create " +
-                parent.getAbsolutePath());
-          }
-          try (OutputStream out = new FileOutputStream(file)) {
-            IOUtils.copyBytes(zip, out, BUFFER_SIZE);
-          }
-          if (!file.setLastModified(entry.getTime())) {
-            numOfFailedLastModifiedSet++;
-          }
-        }
-      }
-      if (numOfFailedLastModifiedSet > 0) {
-        LOG.warn("Could not set last modfied time for {} file(s)",
-            numOfFailedLastModifiedSet);
-      }
-    }
-  }
-
-  /**
-   * Given a File input it will unzip it in the unzip directory.
+   * Given a File input it will unzip the file in a the unzip directory
    * passed as the second parameter
    * @param inFile The zip file as input
    * @param unzipDir The unzip directory where to unzip the zip file.
-   * @throws IOException An I/O exception has occurred
+   * @throws IOException
    */
   public static void unZip(File inFile, File unzipDir) throws IOException {
     Enumeration<? extends ZipEntry> entries;
@@ -687,138 +621,6 @@ public class FileUtil {
   }
 
   /**
-   * Run a command and send the contents of an input stream to it.
-   * @param inputStream Input stream to forward to the shell command
-   * @param command shell command to run
-   * @throws IOException read or write failed
-   * @throws InterruptedException command interrupted
-   * @throws ExecutionException task submit failed
-   */
-  private static void runCommandOnStream(
-      InputStream inputStream, String command)
-      throws IOException, InterruptedException, ExecutionException {
-    ExecutorService executor = null;
-    ProcessBuilder builder = new ProcessBuilder();
-    builder.command(
-        Shell.WINDOWS ? "cmd" : "bash",
-        Shell.WINDOWS ? "/c" : "-c",
-        command);
-    Process process = builder.start();
-    int exitCode;
-    try {
-      // Consume stdout and stderr, to avoid blocking the command
-      executor = Executors.newFixedThreadPool(2);
-      Future output = executor.submit(() -> {
-        try {
-          // Read until the output stream receives an EOF and closed.
-          if (LOG.isDebugEnabled()) {
-            // Log directly to avoid out of memory errors
-            try (BufferedReader reader =
-                     new BufferedReader(
-                         new InputStreamReader(process.getInputStream(),
-                             Charset.forName("UTF-8")))) {
-              String line;
-              while((line = reader.readLine()) != null) {
-                LOG.debug(line);
-              }
-            }
-          } else {
-            org.apache.commons.io.IOUtils.copy(
-                process.getInputStream(),
-                new IOUtils.NullOutputStream());
-          }
-        } catch (IOException e) {
-          LOG.debug(e.getMessage());
-        }
-      });
-      Future error = executor.submit(() -> {
-        try {
-          // Read until the error stream receives an EOF and closed.
-          if (LOG.isDebugEnabled()) {
-            // Log directly to avoid out of memory errors
-            try (BufferedReader reader =
-                     new BufferedReader(
-                         new InputStreamReader(process.getErrorStream(),
-                             Charset.forName("UTF-8")))) {
-              String line;
-              while((line = reader.readLine()) != null) {
-                LOG.debug(line);
-              }
-            }
-          } else {
-            org.apache.commons.io.IOUtils.copy(
-                process.getErrorStream(),
-                new IOUtils.NullOutputStream());
-          }
-        } catch (IOException e) {
-          LOG.debug(e.getMessage());
-        }
-      });
-
-      // Pass the input stream to the command to process
-      try {
-        org.apache.commons.io.IOUtils.copy(
-            inputStream, process.getOutputStream());
-      } finally {
-        process.getOutputStream().close();
-      }
-
-      // Wait for both stdout and stderr futures to finish
-      error.get();
-      output.get();
-    } finally {
-      // Clean up the threads
-      if (executor != null) {
-        executor.shutdown();
-      }
-      // Wait to avoid leaking the child process
-      exitCode = process.waitFor();
-    }
-
-    if (exitCode != 0) {
-      throw new IOException(
-          String.format(
-              "Error executing command. %s " +
-                  "Process exited with exit code %d.",
-              command, exitCode));
-    }
-  }
-
-  /**
-   * Given a Tar File as input it will untar the file in a the untar directory
-   * passed as the second parameter
-   *
-   * This utility will untar ".tar" files and ".tar.gz","tgz" files.
-   *
-   * @param inputStream The tar file as input.
-   * @param untarDir The untar directory where to untar the tar file.
-   * @param gzipped The input stream is gzipped
-   *                TODO Use magic number and PusbackInputStream to identify
-   * @throws IOException an exception occurred
-   * @throws InterruptedException command interrupted
-   * @throws ExecutionException task submit failed
-   */
-  public static void unTar(InputStream inputStream, File untarDir,
-                           boolean gzipped)
-      throws IOException, InterruptedException, ExecutionException {
-    if (!untarDir.mkdirs()) {
-      if (!untarDir.isDirectory()) {
-        throw new IOException("Mkdirs failed to create " + untarDir);
-      }
-    }
-
-    if(Shell.WINDOWS) {
-      // Tar is not native to Windows. Use simple Java based implementation for
-      // tests and simple tar archives
-      unTarUsingJava(inputStream, untarDir, gzipped);
-    } else {
-      // spawn tar utility to untar archive for full fledged unix behavior such
-      // as resolving symlinks in tar archives
-      unTarUsingTar(inputStream, untarDir, gzipped);
-    }
-  }
-
-  /**
    * Given a Tar File as input it will untar the file in a the untar directory
    * passed as the second parameter
    *
@@ -848,41 +650,23 @@ public class FileUtil {
     }
   }
 
-  private static void unTarUsingTar(InputStream inputStream, File untarDir,
-                                    boolean gzipped)
-      throws IOException, InterruptedException, ExecutionException {
-    StringBuilder untarCommand = new StringBuilder();
-    if (gzipped) {
-      untarCommand.append("gzip -dc | (");
-    }
-    untarCommand.append("cd '");
-    untarCommand.append(FileUtil.makeSecureShellPath(untarDir));
-    untarCommand.append("' && ");
-    untarCommand.append("tar -x ");
-
-    if (gzipped) {
-      untarCommand.append(")");
-    }
-    runCommandOnStream(inputStream, untarCommand.toString());
-  }
-
   private static void unTarUsingTar(File inFile, File untarDir,
       boolean gzipped) throws IOException {
     StringBuffer untarCommand = new StringBuffer();
     if (gzipped) {
       untarCommand.append(" gzip -dc '");
-      untarCommand.append(FileUtil.makeSecureShellPath(inFile));
+      untarCommand.append(FileUtil.makeShellPath(inFile));
       untarCommand.append("' | (");
     }
     untarCommand.append("cd '");
-    untarCommand.append(FileUtil.makeSecureShellPath(untarDir));
-    untarCommand.append("' && ");
+    untarCommand.append(FileUtil.makeShellPath(untarDir));
+    untarCommand.append("' ; ");
     untarCommand.append("tar -xf ");
 
     if (gzipped) {
       untarCommand.append(" -)");
     } else {
-      untarCommand.append(FileUtil.makeSecureShellPath(inFile));
+      untarCommand.append(FileUtil.makeShellPath(inFile));
     }
     String[] shellCmd = { "bash", "-c", untarCommand.toString() };
     ShellCommandExecutor shexec = new ShellCommandExecutor(shellCmd);
@@ -917,29 +701,6 @@ public class FileUtil {
     }
   }
 
-  private static void unTarUsingJava(InputStream inputStream, File untarDir,
-                                     boolean gzipped) throws IOException {
-    TarArchiveInputStream tis = null;
-    try {
-      if (gzipped) {
-        inputStream = new BufferedInputStream(new GZIPInputStream(
-            inputStream));
-      } else {
-        inputStream =
-            new BufferedInputStream(inputStream);
-      }
-
-      tis = new TarArchiveInputStream(inputStream);
-
-      for (TarArchiveEntry entry = tis.getNextTarEntry(); entry != null;) {
-        unpackEntries(tis, entry, untarDir);
-        entry = tis.getNextTarEntry();
-      }
-    } finally {
-      IOUtils.cleanupWithLogger(LOG, tis, inputStream);
-    }
-  }
-
   private static void unpackEntries(TarArchiveInputStream tis,
       TarArchiveEntry entry, File outputDir) throws IOException {
     if (entry.isDirectory()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/901d15a3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java
index 89b7d76..19b51ad 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java
@@ -34,11 +34,9 @@ import java.util.Enumeration;
 import java.util.List;
 import java.util.jar.JarEntry;
 import java.util.jar.JarFile;
-import java.util.jar.JarInputStream;
 import java.util.jar.Manifest;
 import java.util.regex.Pattern;
 
-import org.apache.commons.io.input.TeeInputStream;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.FileUtil;
@@ -100,69 +98,6 @@ public class RunJar {
    * Unpack matching files from a jar. Entries inside the jar that do
    * not match the given pattern will be skipped.
    *
-   * @param inputStream the jar stream to unpack
-   * @param toDir the destination directory into which to unpack the jar
-   * @param unpackRegex the pattern to match jar entries against
-   *
-   * @throws IOException if an I/O error has occurred or toDir
-   * cannot be created and does not already exist
-   */
-  public static void unJar(InputStream inputStream, File toDir,
-                           Pattern unpackRegex)
-      throws IOException {
-    try (JarInputStream jar = new JarInputStream(inputStream)) {
-      int numOfFailedLastModifiedSet = 0;
-      for (JarEntry entry = jar.getNextJarEntry();
-           entry != null;
-           entry = jar.getNextJarEntry()) {
-        if (!entry.isDirectory() &&
-            unpackRegex.matcher(entry.getName()).matches()) {
-          File file = new File(toDir, entry.getName());
-          ensureDirectory(file.getParentFile());
-          try (OutputStream out = new FileOutputStream(file)) {
-            IOUtils.copyBytes(jar, out, BUFFER_SIZE);
-          }
-          if (!file.setLastModified(entry.getTime())) {
-            numOfFailedLastModifiedSet++;
-          }
-        }
-      }
-      if (numOfFailedLastModifiedSet > 0) {
-        LOG.warn("Could not set last modfied time for {} file(s)",
-            numOfFailedLastModifiedSet);
-      }
-    }
-  }
-
-  /**
-   * Unpack matching files from a jar. Entries inside the jar that do
-   * not match the given pattern will be skipped. Keep also a copy
-   * of the entire jar in the same directory for backward compatibility.
-   * TODO remove this feature in a new release and do only unJar
-   *
-   * @param inputStream the jar stream to unpack
-   * @param toDir the destination directory into which to unpack the jar
-   * @param unpackRegex the pattern to match jar entries against
-   *
-   * @throws IOException if an I/O error has occurred or toDir
-   * cannot be created and does not already exist
-   */
-  @Deprecated
-  public static void unJarAndSave(InputStream inputStream, File toDir,
-                           String name, Pattern unpackRegex)
-      throws IOException{
-    File file = new File(toDir, name);
-    ensureDirectory(toDir);
-    try (OutputStream jar = new FileOutputStream(file);
-         TeeInputStream teeInputStream = new TeeInputStream(inputStream, jar)) {
-      unJar(teeInputStream, toDir, unpackRegex);
-    }
-  }
-
-  /**
-   * Unpack matching files from a jar. Entries inside the jar that do
-   * not match the given pattern will be skipped.
-   *
    * @param jarFile the .jar file to unpack
    * @param toDir the destination directory into which to unpack the jar
    * @param unpackRegex the pattern to match jar entries against

http://git-wip-us.apache.org/repos/asf/hadoop/blob/901d15a3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/FSDownload.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/FSDownload.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/FSDownload.java
index 1a60948..6e59574 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/FSDownload.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/FSDownload.java
@@ -21,8 +21,6 @@ package org.apache.hadoop.yarn.util;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
 import java.net.URISyntaxException;
 import java.security.PrivilegedExceptionAction;
 import java.util.concurrent.Callable;
@@ -31,7 +29,6 @@ import java.util.concurrent.Future;
 import java.util.regex.Pattern;
 
 import org.apache.commons.io.FileUtils;
-import org.apache.commons.io.IOUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
@@ -57,7 +54,6 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
 import com.google.common.util.concurrent.Futures;
-import org.apache.hadoop.yarn.exceptions.YarnException;
 
 /**
  * Download a single URL to the local disk.
@@ -251,21 +247,9 @@ public class FSDownload implements Callable<Path> {
     }
   }
 
-  /**
-   * Localize files.
-   * @param destination destination directory
-   * @throws IOException cannot read or write file
-   * @throws YarnException subcommand returned an error
-   */
-  private void verifyAndCopy(Path destination)
-      throws IOException, YarnException {
-    final Path sCopy;
-    try {
-      sCopy = resource.getResource().toPath();
-    } catch (URISyntaxException e) {
-      throw new IOException("Invalid resource", e);
-    }
+  private Path copy(Path sCopy, Path dstdir) throws IOException {
     FileSystem sourceFs = sCopy.getFileSystem(conf);
+    Path dCopy = new Path(dstdir, "tmp_"+sCopy.getName());
     FileStatus sStat = sourceFs.getFileStatus(sCopy);
     if (sStat.getModificationTime() != resource.getTimestamp()) {
       throw new IOException("Resource " + sCopy +
@@ -280,108 +264,82 @@ public class FSDownload implements Callable<Path> {
       }
     }
 
-    downloadAndUnpack(sCopy, destination);
+    FileUtil.copy(sourceFs, sStat, FileSystem.getLocal(conf), dCopy, false,
+        true, conf);
+    return dCopy;
   }
 
-  /**
-   * Copy source path to destination with localization rules.
-   * @param source source path to copy. Typically HDFS
-   * @param destination destination path. Typically local filesystem
-   * @exception YarnException Any error has occurred
-   */
-  private void downloadAndUnpack(Path source, Path destination)
-      throws YarnException {
-    try {
-      FileSystem sourceFileSystem = source.getFileSystem(conf);
-      FileSystem destinationFileSystem = destination.getFileSystem(conf);
-      if (sourceFileSystem.getFileStatus(source).isDirectory()) {
-        FileUtil.copy(
-            sourceFileSystem, source,
-            destinationFileSystem, destination, false,
-            true, conf);
+  private long unpack(File localrsrc, File dst) throws IOException {
+    switch (resource.getType()) {
+    case ARCHIVE: {
+      String lowerDst = StringUtils.toLowerCase(dst.getName());
+      if (lowerDst.endsWith(".jar")) {
+        RunJar.unJar(localrsrc, dst);
+      } else if (lowerDst.endsWith(".zip")) {
+        FileUtil.unZip(localrsrc, dst);
+      } else if (lowerDst.endsWith(".tar.gz") ||
+                 lowerDst.endsWith(".tgz") ||
+                 lowerDst.endsWith(".tar")) {
+        FileUtil.unTar(localrsrc, dst);
       } else {
-        unpack(source, destination, sourceFileSystem, destinationFileSystem);
+        LOG.warn("Cannot unpack " + localrsrc);
+        if (!localrsrc.renameTo(dst)) {
+            throw new IOException("Unable to rename file: [" + localrsrc
+              + "] to [" + dst + "]");
+        }
       }
-    } catch (Exception e) {
-      throw new YarnException("Download and unpack failed", e);
     }
-  }
-
-  /**
-   * Do the localization action on the input stream.
-   * We use the deprecated method RunJar.unJarAndSave for compatibility reasons.
-   * We should use the more efficient RunJar.unJar in the future.
-   * @param source Source path
-   * @param destination Destination pth
-   * @param sourceFileSystem Source filesystem
-   * @param destinationFileSystem Destination filesystem
-   * @throws IOException Could not read or write stream
-   * @throws InterruptedException Operation interrupted by caller
-   * @throws ExecutionException Could not create thread pool execution
-   */
-  @SuppressWarnings("deprecation")
-  private void unpack(Path source, Path destination,
-                      FileSystem sourceFileSystem,
-                      FileSystem destinationFileSystem)
-      throws IOException, InterruptedException, ExecutionException {
-    try (InputStream inputStream = sourceFileSystem.open(source)) {
-      File dst = new File(destination.toUri());
+    break;
+    case PATTERN: {
       String lowerDst = StringUtils.toLowerCase(dst.getName());
-      switch (resource.getType()) {
-      case ARCHIVE:
-        if (lowerDst.endsWith(".jar")) {
-          RunJar.unJar(inputStream, dst, RunJar.MATCH_ANY);
-        } else if (lowerDst.endsWith(".zip")) {
-          FileUtil.unZip(inputStream, dst);
-        } else if (lowerDst.endsWith(".tar.gz") ||
-            lowerDst.endsWith(".tgz") ||
-            lowerDst.endsWith(".tar")) {
-          FileUtil.unTar(inputStream, dst, lowerDst.endsWith("gz"));
-        } else {
-          LOG.warn("Cannot unpack " + source);
-          try (OutputStream outputStream =
-                   destinationFileSystem.create(destination, true)) {
-            IOUtils.copy(inputStream, outputStream);
-          }
+      if (lowerDst.endsWith(".jar")) {
+        String p = resource.getPattern();
+        RunJar.unJar(localrsrc, dst,
+            p == null ? RunJar.MATCH_ANY : Pattern.compile(p));
+        File newDst = new File(dst, dst.getName());
+        if (!dst.exists() && !dst.mkdir()) {
+          throw new IOException("Unable to create directory: [" + dst + "]");
         }
-        break;
-      case PATTERN:
-        if (lowerDst.endsWith(".jar")) {
-          String p = resource.getPattern();
-          if (!dst.exists() && !dst.mkdir()) {
-            throw new IOException("Unable to create directory: [" + dst + "]");
-          }
-          RunJar.unJarAndSave(inputStream, dst, source.getName(),
-              p == null ? RunJar.MATCH_ANY : Pattern.compile(p));
-        } else if (lowerDst.endsWith(".zip")) {
-          LOG.warn("Treating [" + source + "] as an archive even though it " +
-              "was specified as PATTERN");
-          FileUtil.unZip(inputStream, dst);
-        } else if (lowerDst.endsWith(".tar.gz") ||
-            lowerDst.endsWith(".tgz") ||
-            lowerDst.endsWith(".tar")) {
-          LOG.warn("Treating [" + source + "] as an archive even though it " +
-              "was specified as PATTERN");
-          FileUtil.unTar(inputStream, dst, lowerDst.endsWith("gz"));
-        } else {
-          LOG.warn("Cannot unpack " + source);
-          try (OutputStream outputStream =
-                   destinationFileSystem.create(destination, true)) {
-            IOUtils.copy(inputStream, outputStream);
-          }
+        if (!localrsrc.renameTo(newDst)) {
+          throw new IOException("Unable to rename file: [" + localrsrc
+              + "] to [" + newDst + "]");
         }
-        break;
-      case FILE:
-      default:
-        try (OutputStream outputStream =
-                 destinationFileSystem.create(destination, true)) {
-          IOUtils.copy(inputStream, outputStream);
+      } else if (lowerDst.endsWith(".zip")) {
+        LOG.warn("Treating [" + localrsrc + "] as an archive even though it " +
+        		"was specified as PATTERN");
+        FileUtil.unZip(localrsrc, dst);
+      } else if (lowerDst.endsWith(".tar.gz") ||
+                 lowerDst.endsWith(".tgz") ||
+                 lowerDst.endsWith(".tar")) {
+        LOG.warn("Treating [" + localrsrc + "] as an archive even though it " +
+        "was specified as PATTERN");
+        FileUtil.unTar(localrsrc, dst);
+      } else {
+        LOG.warn("Cannot unpack " + localrsrc);
+        if (!localrsrc.renameTo(dst)) {
+          throw new IOException("Unable to rename file: [" + localrsrc
+              + "] to [" + dst + "]");
         }
-        break;
       }
-      // TODO Should calculate here before returning
-      //return FileUtil.getDU(destDir);
     }
+    break;
+    case FILE:
+    default:
+      if (!localrsrc.renameTo(dst)) {
+        throw new IOException("Unable to rename file: [" + localrsrc
+          + "] to [" + dst + "]");
+      }
+      break;
+    }
+    if(localrsrc.isFile()){
+      try {
+        files.delete(new Path(localrsrc.toString()), false);
+      } catch (IOException ignore) {
+      }
+    }
+    return 0;
+    // TODO Should calculate here before returning
+    //return FileUtil.getDU(destDir);
   }
 
   @Override
@@ -394,34 +352,27 @@ public class FSDownload implements Callable<Path> {
     }
 
     if (LOG.isDebugEnabled()) {
-      LOG.debug(String.format("Starting to download %s %s %s",
-          sCopy,
-          resource.getType(),
-          resource.getPattern()));
+      LOG.debug("Starting to download " + sCopy);
     }
 
-    final Path destinationTmp = new Path(destDirPath + "_tmp");
-    createDir(destinationTmp, PRIVATE_DIR_PERMS);
-    Path dFinal =
-        files.makeQualified(new Path(destinationTmp, sCopy.getName()));
+    createDir(destDirPath, cachePerms);
+    final Path dst_work = new Path(destDirPath + "_tmp");
+    createDir(dst_work, cachePerms);
+    Path dFinal = files.makeQualified(new Path(dst_work, sCopy.getName()));
     try {
-      if (userUgi == null) {
-        verifyAndCopy(dFinal);
-      } else {
-        userUgi.doAs(new PrivilegedExceptionAction<Void>() {
-          @Override
-          public Void run() throws Exception {
-            verifyAndCopy(dFinal);
-            return null;
-          }
-        });
-      }
+      Path dTmp = null == userUgi ? files.makeQualified(copy(sCopy, dst_work))
+          : userUgi.doAs(new PrivilegedExceptionAction<Path>() {
+            public Path run() throws Exception {
+              return files.makeQualified(copy(sCopy, dst_work));
+            };
+          });
+      unpack(new File(dTmp.toUri()), new File(dFinal.toUri()));
       changePermissions(dFinal.getFileSystem(conf), dFinal);
-      files.rename(destinationTmp, destDirPath, Rename.OVERWRITE);
+      files.rename(dst_work, destDirPath, Rename.OVERWRITE);
 
       if (LOG.isDebugEnabled()) {
-        LOG.debug(String.format("File has been downloaded to %s from %s",
-            new Path(destDirPath, sCopy.getName()), sCopy));
+        LOG.debug("File has been downloaded to " +
+            new Path(destDirPath, sCopy.getName()));
       }
     } catch (Exception e) {
       try {
@@ -431,7 +382,7 @@ public class FSDownload implements Callable<Path> {
       throw e;
     } finally {
       try {
-        files.delete(destinationTmp, true);
+        files.delete(dst_work, true);
       } catch (FileNotFoundException ignore) {
       }
       conf = null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/901d15a3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestFSDownload.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestFSDownload.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestFSDownload.java
index fa8c039..877dd08 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestFSDownload.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestFSDownload.java
@@ -82,9 +82,6 @@ import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
 
-/**
- * Unit test for the FSDownload class.
- */
 public class TestFSDownload {
 
   private static final Log LOG = LogFactory.getLog(TestFSDownload.class);
@@ -93,8 +90,7 @@ public class TestFSDownload {
   private enum TEST_FILE_TYPE {
     TAR, JAR, ZIP, TGZ
   };
-  private Configuration conf = new Configuration();
-
+  
   @AfterClass
   public static void deleteTestDir() throws IOException {
     FileContext fs = FileContext.getLocalFSFileContext();
@@ -136,18 +132,6 @@ public class TestFSDownload {
     FileOutputStream stream = new FileOutputStream(jarFile);
     LOG.info("Create jar out stream ");
     JarOutputStream out = new JarOutputStream(stream, new Manifest());
-    ZipEntry entry = new ZipEntry("classes/1.class");
-    out.putNextEntry(entry);
-    out.write(1);
-    out.write(2);
-    out.write(3);
-    out.closeEntry();
-    ZipEntry entry2 = new ZipEntry("classes/2.class");
-    out.putNextEntry(entry2);
-    out.write(1);
-    out.write(2);
-    out.write(3);
-    out.closeEntry();
     LOG.info("Done writing jar stream ");
     out.close();
     LocalResource ret = recordFactory.newRecordInstance(LocalResource.class);
@@ -272,6 +256,7 @@ public class TestFSDownload {
   @Test (timeout=10000)
   public void testDownloadBadPublic() throws IOException, URISyntaxException,
       InterruptedException {
+    Configuration conf = new Configuration();
     conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "077");
     FileContext files = FileContext.getLocalFSFileContext(conf);
     final Path basedir = files.makeQualified(new Path("target",
@@ -322,6 +307,7 @@ public class TestFSDownload {
   @Test (timeout=60000)
   public void testDownloadPublicWithStatCache() throws IOException,
       URISyntaxException, InterruptedException, ExecutionException {
+    final Configuration conf = new Configuration();
     FileContext files = FileContext.getLocalFSFileContext(conf);
     Path basedir = files.makeQualified(new Path("target",
       TestFSDownload.class.getSimpleName()));
@@ -396,6 +382,7 @@ public class TestFSDownload {
   @Test (timeout=10000)
   public void testDownload() throws IOException, URISyntaxException,
       InterruptedException {
+    Configuration conf = new Configuration();
     conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "077");
     FileContext files = FileContext.getLocalFSFileContext(conf);
     final Path basedir = files.makeQualified(new Path("target",
@@ -451,7 +438,7 @@ public class TestFSDownload {
         FileStatus status = files.getFileStatus(localized.getParent());
         FsPermission perm = status.getPermission();
         assertEquals("Cache directory permissions are incorrect",
-            new FsPermission((short)0700), perm);
+            new FsPermission((short)0755), perm);
 
         status = files.getFileStatus(localized);
         perm = status.getPermission();
@@ -468,6 +455,7 @@ public class TestFSDownload {
 
   private void downloadWithFileType(TEST_FILE_TYPE fileType) throws IOException, 
       URISyntaxException, InterruptedException{
+    Configuration conf = new Configuration();
     conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "077");
     FileContext files = FileContext.getLocalFSFileContext(conf);
     final Path basedir = files.makeQualified(new Path("target",
@@ -542,7 +530,7 @@ public class TestFSDownload {
     }
   }
 
-  @Test (timeout=10000)
+  @Test (timeout=10000) 
   public void testDownloadArchive() throws IOException, URISyntaxException,
       InterruptedException {
     downloadWithFileType(TEST_FILE_TYPE.TAR);
@@ -554,7 +542,7 @@ public class TestFSDownload {
     downloadWithFileType(TEST_FILE_TYPE.JAR);
   }
 
-  @Test (timeout=10000)
+  @Test (timeout=10000) 
   public void testDownloadArchiveZip() throws IOException, URISyntaxException,
       InterruptedException {
     downloadWithFileType(TEST_FILE_TYPE.ZIP);
@@ -615,6 +603,7 @@ public class TestFSDownload {
 
   @Test (timeout=10000)
   public void testDirDownload() throws IOException, InterruptedException {
+    Configuration conf = new Configuration();
     FileContext files = FileContext.getLocalFSFileContext(conf);
     final Path basedir = files.makeQualified(new Path("target",
       TestFSDownload.class.getSimpleName()));
@@ -679,6 +668,7 @@ public class TestFSDownload {
 
   @Test (timeout=10000)
   public void testUniqueDestinationPath() throws Exception {
+    Configuration conf = new Configuration();
     FileContext files = FileContext.getLocalFSFileContext(conf);
     final Path basedir = files.makeQualified(new Path("target",
         TestFSDownload.class.getSimpleName()));


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[30/37] hadoop git commit: HADOOP-15186. Allow Azure Data Lake SDK dependency version to be set on the command line. Contributed by Vishwajeet Dusane.

Posted by ae...@apache.org.
HADOOP-15186. Allow Azure Data Lake SDK dependency version to be set on the command line.
Contributed by Vishwajeet Dusane.

(cherry picked from commit 95a96b13e2a54e01ea6c6933045d912998477da3)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/7fd287b4
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/7fd287b4
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/7fd287b4

Branch: refs/heads/HDFS-7240
Commit: 7fd287b4af5a191f18ea92850b7d904e4b4fb693
Parents: 56feaa4
Author: Steve Loughran <st...@apache.org>
Authored: Mon Jan 29 09:48:14 2018 -0800
Committer: Steve Loughran <st...@apache.org>
Committed: Mon Jan 29 09:48:14 2018 -0800

----------------------------------------------------------------------
 hadoop-tools/hadoop-azure-datalake/pom.xml | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7fd287b4/hadoop-tools/hadoop-azure-datalake/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/pom.xml b/hadoop-tools/hadoop-azure-datalake/pom.xml
index 3ac84ff..70c8f6c 100644
--- a/hadoop-tools/hadoop-azure-datalake/pom.xml
+++ b/hadoop-tools/hadoop-azure-datalake/pom.xml
@@ -33,6 +33,7 @@
     <minimalJsonVersion>0.9.1</minimalJsonVersion>
     <file.encoding>UTF-8</file.encoding>
     <downloadSources>true</downloadSources>
+    <azure.data.lake.store.sdk.version>2.2.5</azure.data.lake.store.sdk.version>
   </properties>
   <build>
     <plugins>
@@ -109,7 +110,7 @@
     <dependency>
       <groupId>com.microsoft.azure</groupId>
       <artifactId>azure-data-lake-store-sdk</artifactId>
-      <version>2.2.5</version>
+      <version>${azure.data.lake.store.sdk.version}</version>
     </dependency>
     <!--  ENDS HERE-->
     <dependency>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[02/37] hadoop git commit: YARN-7737. prelaunch.err file not found exception on container failure. Contributed by Keqiu Hu.

Posted by ae...@apache.org.
YARN-7737. prelaunch.err file not found exception on container failure. Contributed by Keqiu Hu.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/fa8cf4d1
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/fa8cf4d1
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/fa8cf4d1

Branch: refs/heads/HDFS-7240
Commit: fa8cf4d1b4896a602dc383d5e266768392a9790c
Parents: cff9edd
Author: Zhe Zhang <zh...@apache.org>
Authored: Wed Jan 24 12:54:34 2018 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Wed Jan 24 12:54:54 2018 -0800

----------------------------------------------------------------------
 .../nodemanager/containermanager/launcher/ContainerLaunch.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa8cf4d1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java
index 95ced22..7f43458 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java
@@ -270,7 +270,7 @@ public class ContainerLaunch implements Callable<Integer> {
         // Write out the environment
         exec.writeLaunchEnv(containerScriptOutStream, environment,
             localResources, launchContext.getCommands(),
-            new Path(containerLogDirs.get(0)), user);
+            containerLogDir, user);
       }
       // /////////// End of writing out container-script
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[11/37] hadoop git commit: YARN-7102. NM heartbeat stuck when responseId overflows MAX_INT. Contributed by Botong Huang

Posted by ae...@apache.org.
YARN-7102. NM heartbeat stuck when responseId overflows MAX_INT. Contributed by Botong Huang


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/ff8378eb
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ff8378eb
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ff8378eb

Branch: refs/heads/HDFS-7240
Commit: ff8378eb1b960c72d18a984c7e5d145b407ca11a
Parents: 16be42d
Author: Jason Lowe <jl...@apache.org>
Authored: Thu Jan 25 17:47:19 2018 -0600
Committer: Jason Lowe <jl...@apache.org>
Committed: Thu Jan 25 17:47:19 2018 -0600

----------------------------------------------------------------------
 .../yarn/sls/nodemanager/NMSimulator.java       |  4 +-
 .../hadoop/yarn/sls/nodemanager/NodeInfo.java   | 11 +---
 .../yarn/sls/scheduler/RMNodeWrapper.java       | 12 +---
 .../resourcemanager/ResourceTrackerService.java | 67 ++++++++++++++------
 .../server/resourcemanager/rmnode/RMNode.java   | 13 ++--
 .../resourcemanager/rmnode/RMNodeImpl.java      | 47 ++++----------
 .../rmnode/RMNodeStatusEvent.java               | 13 +---
 .../yarn/server/resourcemanager/MockNM.java     | 18 ++++--
 .../yarn/server/resourcemanager/MockNodes.java  |  9 +--
 .../resourcemanager/TestRMNodeTransitions.java  | 14 +---
 .../TestResourceTrackerService.java             | 35 ++++++++--
 .../TestRMAppLogAggregationStatus.java          | 10 +--
 12 files changed, 125 insertions(+), 128 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff8378eb/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NMSimulator.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NMSimulator.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NMSimulator.java
index 6b19128..ba0fd56 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NMSimulator.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NMSimulator.java
@@ -71,7 +71,7 @@ public class NMSimulator extends TaskRunner.Task {
   // resource manager
   private ResourceManager rm;
   // heart beat response id
-  private int RESPONSE_ID = 1;
+  private int responseId = 0;
   private final static Logger LOG = LoggerFactory.getLogger(NMSimulator.class);
   
   public void init(String nodeIdStr, Resource nodeResource,
@@ -131,7 +131,7 @@ public class NMSimulator extends TaskRunner.Task {
     ns.setContainersStatuses(generateContainerStatusList());
     ns.setNodeId(node.getNodeID());
     ns.setKeepAliveApplications(new ArrayList<ApplicationId>());
-    ns.setResponseId(RESPONSE_ID ++);
+    ns.setResponseId(responseId++);
     ns.setNodeHealthStatus(NodeHealthStatus.newInstance(true, "", 0));
     beatRequest.setNodeStatus(ns);
     NodeHeartbeatResponse beatResponse =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff8378eb/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NodeInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NodeInfo.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NodeInfo.java
index e71ddff..1016ce1 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NodeInfo.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NodeInfo.java
@@ -144,8 +144,8 @@ public class NodeInfo {
       return runningApplications;
     }
 
-    public void updateNodeHeartbeatResponseForCleanup(
-            NodeHeartbeatResponse response) {
+    public void setAndUpdateNodeHeartbeatResponse(
+        NodeHeartbeatResponse response) {
     }
 
     public NodeHeartbeatResponse getLastNodeHeartBeatResponse() {
@@ -179,13 +179,6 @@ public class NodeInfo {
     }
 
     @Override
-    public void updateNodeHeartbeatResponseForUpdatedContainers(
-        NodeHeartbeatResponse response) {
-      // TODO Auto-generated method stub
-      
-    }
-
-    @Override
     public List<Container> pullNewlyIncreasedContainers() {
       // TODO Auto-generated method stub
       return null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff8378eb/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/RMNodeWrapper.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/RMNodeWrapper.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/RMNodeWrapper.java
index 6b7ac3c..fdad826 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/RMNodeWrapper.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/RMNodeWrapper.java
@@ -127,9 +127,9 @@ public class RMNodeWrapper implements RMNode {
   }
 
   @Override
-  public void updateNodeHeartbeatResponseForCleanup(
-          NodeHeartbeatResponse nodeHeartbeatResponse) {
-    node.updateNodeHeartbeatResponseForCleanup(nodeHeartbeatResponse);
+  public void setAndUpdateNodeHeartbeatResponse(
+      NodeHeartbeatResponse nodeHeartbeatResponse) {
+    node.setAndUpdateNodeHeartbeatResponse(nodeHeartbeatResponse);
   }
 
   @Override
@@ -167,12 +167,6 @@ public class RMNodeWrapper implements RMNode {
     return RMNodeLabelsManager.EMPTY_STRING_SET;
   }
 
-  @Override
-  public void updateNodeHeartbeatResponseForUpdatedContainers(
-      NodeHeartbeatResponse response) {
-    // TODO Auto-generated method stub
-  }
-
   @SuppressWarnings("unchecked")
   @Override
   public List<Container> pullNewlyIncreasedContainers() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff8378eb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
index a42d053..9d95f63 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
@@ -32,6 +32,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
 
+import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -81,6 +82,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeReconnectEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeStartedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeStatusEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.authorize.RMPolicyProvider;
@@ -403,14 +405,37 @@ public class ResourceTrackerService extends AbstractService implements
     } else {
       LOG.info("Reconnect from the node at: " + host);
       this.nmLivelinessMonitor.unregister(nodeId);
-      // Reset heartbeat ID since node just restarted.
-      oldNode.resetLastNodeHeartBeatResponse();
-      this.rmContext
-          .getDispatcher()
-          .getEventHandler()
-          .handle(
-              new RMNodeReconnectEvent(nodeId, rmNode, request
-                  .getRunningApplications(), request.getNMContainerStatuses()));
+
+      if (CollectionUtils.isEmpty(request.getRunningApplications())
+          && rmNode.getState() != NodeState.DECOMMISSIONING
+          && rmNode.getHttpPort() != oldNode.getHttpPort()) {
+        // Reconnected node differs, so replace old node and start new node
+        switch (rmNode.getState()) {
+        case RUNNING:
+          ClusterMetrics.getMetrics().decrNumActiveNodes();
+          break;
+        case UNHEALTHY:
+          ClusterMetrics.getMetrics().decrNumUnhealthyNMs();
+          break;
+        default:
+          LOG.debug("Unexpected Rmnode state");
+        }
+        this.rmContext.getDispatcher().getEventHandler()
+            .handle(new NodeRemovedSchedulerEvent(rmNode));
+
+        this.rmContext.getRMNodes().put(nodeId, rmNode);
+        this.rmContext.getDispatcher().getEventHandler()
+            .handle(new RMNodeStartedEvent(nodeId, null, null));
+
+      } else {
+        // Reset heartbeat ID since node just restarted.
+        oldNode.resetLastNodeHeartBeatResponse();
+
+        this.rmContext.getDispatcher().getEventHandler()
+            .handle(new RMNodeReconnectEvent(nodeId, rmNode,
+                request.getRunningApplications(),
+                request.getNMContainerStatuses()));
+      }
     }
     // On every node manager register we will be clearing NMToken keys if
     // present for any running application.
@@ -508,12 +533,13 @@ public class ResourceTrackerService extends AbstractService implements
 
     // 3. Check if it's a 'fresh' heartbeat i.e. not duplicate heartbeat
     NodeHeartbeatResponse lastNodeHeartbeatResponse = rmNode.getLastNodeHeartBeatResponse();
-    if (remoteNodeStatus.getResponseId() + 1 == lastNodeHeartbeatResponse
-        .getResponseId()) {
+    if (getNextResponseId(
+        remoteNodeStatus.getResponseId()) == lastNodeHeartbeatResponse
+            .getResponseId()) {
       LOG.info("Received duplicate heartbeat from node "
           + rmNode.getNodeAddress()+ " responseId=" + remoteNodeStatus.getResponseId());
       return lastNodeHeartbeatResponse;
-    } else if (remoteNodeStatus.getResponseId() + 1 < lastNodeHeartbeatResponse
+    } else if (remoteNodeStatus.getResponseId() != lastNodeHeartbeatResponse
         .getResponseId()) {
       String message =
           "Too far behind rm response id:"
@@ -549,13 +575,11 @@ public class ResourceTrackerService extends AbstractService implements
     }
 
     // Heartbeat response
-    NodeHeartbeatResponse nodeHeartBeatResponse = YarnServerBuilderUtils
-        .newNodeHeartbeatResponse(lastNodeHeartbeatResponse.
-            getResponseId() + 1, NodeAction.NORMAL, null, null, null, null,
-            nextHeartBeatInterval);
-    rmNode.updateNodeHeartbeatResponseForCleanup(nodeHeartBeatResponse);
-    rmNode.updateNodeHeartbeatResponseForUpdatedContainers(
-        nodeHeartBeatResponse);
+    NodeHeartbeatResponse nodeHeartBeatResponse =
+        YarnServerBuilderUtils.newNodeHeartbeatResponse(
+            getNextResponseId(lastNodeHeartbeatResponse.getResponseId()),
+            NodeAction.NORMAL, null, null, null, null, nextHeartBeatInterval);
+    rmNode.setAndUpdateNodeHeartbeatResponse(nodeHeartBeatResponse);
 
     populateKeys(request, nodeHeartBeatResponse);
 
@@ -573,7 +597,7 @@ public class ResourceTrackerService extends AbstractService implements
 
     // 4. Send status to RMNode, saving the latest response.
     RMNodeStatusEvent nodeStatusEvent =
-        new RMNodeStatusEvent(nodeId, remoteNodeStatus, nodeHeartBeatResponse);
+        new RMNodeStatusEvent(nodeId, remoteNodeStatus);
     if (request.getLogAggregationReportsForApps() != null
         && !request.getLogAggregationReportsForApps().isEmpty()) {
       nodeStatusEvent.setLogAggregationReportsForApps(request
@@ -614,6 +638,11 @@ public class ResourceTrackerService extends AbstractService implements
     return nodeHeartBeatResponse;
   }
 
+  private int getNextResponseId(int responseId) {
+    // Loop between 0 and Integer.MAX_VALUE
+    return (responseId + 1) & Integer.MAX_VALUE;
+  }
+
   private void setAppCollectorsMapToResponse(
       List<ApplicationId> runningApps, NodeHeartbeatResponse response) {
     Map<ApplicationId, AppCollectorData> liveAppCollectorsMap = new

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff8378eb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNode.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNode.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNode.java
index 328c040..a5615ef 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNode.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNode.java
@@ -141,10 +141,11 @@ public interface RMNode {
 
   /**
    * Update a {@link NodeHeartbeatResponse} with the list of containers and
-   * applications to clean up for this node.
+   * applications to clean up for this node, and the containers to be updated.
+   *
    * @param response the {@link NodeHeartbeatResponse} to update
    */
-  void updateNodeHeartbeatResponseForCleanup(NodeHeartbeatResponse response);
+  void setAndUpdateNodeHeartbeatResponse(NodeHeartbeatResponse response);
 
   public NodeHeartbeatResponse getLastNodeHeartBeatResponse();
 
@@ -167,13 +168,7 @@ public interface RMNode {
    * @return labels in this node
    */
   public Set<String> getNodeLabels();
-  
-  /**
-   * Update containers to be updated
-   */
-  void updateNodeHeartbeatResponseForUpdatedContainers(
-      NodeHeartbeatResponse response);
-  
+
   public List<Container> pullNewlyIncreasedContainers();
 
   OpportunisticContainersStatus getOpportunisticContainersStatus();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff8378eb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
index 2b013a0..da54eb9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
@@ -598,7 +598,8 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
   };
 
   @Override
-  public void updateNodeHeartbeatResponseForCleanup(NodeHeartbeatResponse response) {
+  public void setAndUpdateNodeHeartbeatResponse(
+      NodeHeartbeatResponse response) {
     this.writeLock.lock();
 
     try {
@@ -613,38 +614,30 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
       this.finishedApplications.clear();
       this.containersToSignal.clear();
       this.containersToBeRemovedFromNM.clear();
-    } finally {
-      this.writeLock.unlock();
-    }
-  };
-  
-  @VisibleForTesting
-  public Collection<Container> getToBeUpdatedContainers() {
-    return toBeUpdatedContainers.values();
-  }
-  
-  @Override
-  public void updateNodeHeartbeatResponseForUpdatedContainers(
-      NodeHeartbeatResponse response) {
-    this.writeLock.lock();
-    
-    try {
+
       response.addAllContainersToUpdate(toBeUpdatedContainers.values());
       toBeUpdatedContainers.clear();
 
       // NOTE: This is required for backward compatibility.
       response.addAllContainersToDecrease(toBeDecreasedContainers.values());
       toBeDecreasedContainers.clear();
+
+      // Synchronously update the last response in rmNode with updated
+      // responseId
+      this.latestNodeHeartBeatResponse = response;
     } finally {
       this.writeLock.unlock();
     }
+  };
+
+  @VisibleForTesting
+  public Collection<Container> getToBeUpdatedContainers() {
+    return toBeUpdatedContainers.values();
   }
 
   @Override
   public NodeHeartbeatResponse getLastNodeHeartBeatResponse() {
-
     this.readLock.lock();
-
     try {
       return this.latestNodeHeartBeatResponse;
     } finally {
@@ -818,7 +811,6 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
   private static NodeHealthStatus updateRMNodeFromStatusEvents(
       RMNodeImpl rmNode, RMNodeStatusEvent statusEvent) {
     // Switch the last heartbeatresponse.
-    rmNode.latestNodeHeartBeatResponse = statusEvent.getLatestResponse();
     NodeHealthStatus remoteNodeHealthStatus = statusEvent.getNodeHealthStatus();
     rmNode.setHealthReport(remoteNodeHealthStatus.getHealthReport());
     rmNode.setLastHealthReportTime(remoteNodeHealthStatus
@@ -912,21 +904,6 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
             rmNode.context.getDispatcher().getEventHandler().handle(
                 new NodeAddedSchedulerEvent(rmNode));
           }
-        } else {
-          // Reconnected node differs, so replace old node and start new node
-          switch (rmNode.getState()) {
-            case RUNNING:
-              ClusterMetrics.getMetrics().decrNumActiveNodes();
-              break;
-            case UNHEALTHY:
-              ClusterMetrics.getMetrics().decrNumUnhealthyNMs();
-              break;
-            default:
-              LOG.debug("Unexpected Rmnode state");
-            }
-            rmNode.context.getRMNodes().put(newNode.getNodeID(), newNode);
-            rmNode.context.getDispatcher().getEventHandler().handle(
-                new RMNodeStartedEvent(newNode.getNodeID(), null, null));
         }
 
       } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff8378eb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeStatusEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeStatusEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeStatusEvent.java
index f9fe159..c79f270 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeStatusEvent.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeStatusEvent.java
@@ -27,7 +27,6 @@ import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.ResourceUtilization;
 import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport;
-import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
 import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
 import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus;
 import org.apache.hadoop.yarn.server.api.records.NodeStatus;
@@ -35,20 +34,16 @@ import org.apache.hadoop.yarn.server.api.records.NodeStatus;
 public class RMNodeStatusEvent extends RMNodeEvent {
 
   private final NodeStatus nodeStatus;
-  private final NodeHeartbeatResponse latestResponse;
   private List<LogAggregationReport> logAggregationReportsForApps;
 
-  public RMNodeStatusEvent(NodeId nodeId, NodeStatus nodeStatus,
-      NodeHeartbeatResponse latestResponse) {
-    this(nodeId, nodeStatus, latestResponse, null);
+  public RMNodeStatusEvent(NodeId nodeId, NodeStatus nodeStatus) {
+    this(nodeId, nodeStatus, null);
   }
 
   public RMNodeStatusEvent(NodeId nodeId, NodeStatus nodeStatus,
-      NodeHeartbeatResponse latestResponse,
       List<LogAggregationReport> logAggregationReportsForApps) {
     super(nodeId, RMNodeEventType.STATUS_UPDATE);
     this.nodeStatus = nodeStatus;
-    this.latestResponse = latestResponse;
     this.logAggregationReportsForApps = logAggregationReportsForApps;
   }
 
@@ -60,10 +55,6 @@ public class RMNodeStatusEvent extends RMNodeEvent {
     return this.nodeStatus.getContainersStatuses();
   }
 
-  public NodeHeartbeatResponse getLatestResponse() {
-    return this.latestResponse;
-  }
-  
   public List<ApplicationId> getKeepAliveAppIds() {
     return this.nodeStatus.getKeepAliveApplications();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff8378eb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java
index 05b51e3..0a06e82 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java
@@ -131,7 +131,7 @@ public class MockNM {
             container.getResource());
     List<Container> increasedConts = Collections.singletonList(container);
     nodeHeartbeat(Collections.singletonList(containerStatus), increasedConts,
-        true, ++responseId);
+        true, responseId);
   }
 
   public void addRegisteringCollector(ApplicationId appId,
@@ -190,12 +190,13 @@ public class MockNM {
         }
       }
     }
+    responseId = 0;
     return registrationResponse;
   }
 
   public NodeHeartbeatResponse nodeHeartbeat(boolean isHealthy) throws Exception {
     return nodeHeartbeat(Collections.<ContainerStatus>emptyList(),
-        Collections.<Container>emptyList(), isHealthy, ++responseId);
+        Collections.<Container>emptyList(), isHealthy, responseId);
   }
 
   public NodeHeartbeatResponse nodeHeartbeat(ApplicationAttemptId attemptId,
@@ -208,12 +209,12 @@ public class MockNM {
     containerStatusList.add(containerStatus);
     Log.getLog().info("ContainerStatus: " + containerStatus);
     return nodeHeartbeat(containerStatusList,
-        Collections.<Container>emptyList(), true, ++responseId);
+        Collections.<Container>emptyList(), true, responseId);
   }
 
   public NodeHeartbeatResponse nodeHeartbeat(Map<ApplicationId,
       List<ContainerStatus>> conts, boolean isHealthy) throws Exception {
-    return nodeHeartbeat(conts, isHealthy, ++responseId);
+    return nodeHeartbeat(conts, isHealthy, responseId);
   }
 
   public NodeHeartbeatResponse nodeHeartbeat(Map<ApplicationId,
@@ -229,7 +230,7 @@ public class MockNM {
   public NodeHeartbeatResponse nodeHeartbeat(
       List<ContainerStatus> updatedStats, boolean isHealthy) throws Exception {
     return nodeHeartbeat(updatedStats, Collections.<Container>emptyList(),
-        isHealthy, ++responseId);
+        isHealthy, responseId);
   }
 
   public NodeHeartbeatResponse nodeHeartbeat(List<ContainerStatus> updatedStats,
@@ -265,7 +266,8 @@ public class MockNM {
 
     NodeHeartbeatResponse heartbeatResponse =
         resourceTracker.nodeHeartbeat(req);
-    
+    responseId = heartbeatResponse.getResponseId();
+
     MasterKey masterKeyFromRM = heartbeatResponse.getContainerTokenMasterKey();
     if (masterKeyFromRM != null
         && masterKeyFromRM.getKeyId() != this.currentContainerTokenMasterKey
@@ -303,4 +305,8 @@ public class MockNM {
   public String getVersion() {
     return version;
   }
+
+  public void setResponseId(int id) {
+    this.responseId = id;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff8378eb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
index 317c648..d6549b9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
@@ -205,7 +205,8 @@ public class MockNodes {
     }
 
     @Override
-    public void updateNodeHeartbeatResponseForCleanup(NodeHeartbeatResponse response) {
+    public void setAndUpdateNodeHeartbeatResponse(
+        NodeHeartbeatResponse response) {
     }
 
     @Override
@@ -246,12 +247,6 @@ public class MockNodes {
     }
 
     @Override
-    public void updateNodeHeartbeatResponseForUpdatedContainers(
-        NodeHeartbeatResponse response) {
-      
-    }
-
-    @Override
     public List<Container> pullNewlyIncreasedContainers() {
       return Collections.emptyList();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff8378eb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java
index 3657123..487d226 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java
@@ -164,15 +164,12 @@ public class TestRMNodeTransitions {
   
   private RMNodeStatusEvent getMockRMNodeStatusEvent(
       List<ContainerStatus> containerStatus) {
-    NodeHeartbeatResponse response = mock(NodeHeartbeatResponse.class);
-
     NodeHealthStatus healthStatus = mock(NodeHealthStatus.class);
     Boolean yes = new Boolean(true);
     doReturn(yes).when(healthStatus).getIsNodeHealthy();
     
     RMNodeStatusEvent event = mock(RMNodeStatusEvent.class);
     doReturn(healthStatus).when(event).getNodeHealthStatus();
-    doReturn(response).when(event).getLatestResponse();
     doReturn(RMNodeEventType.STATUS_UPDATE).when(event).getType();
     if (containerStatus != null) {
       doReturn(containerStatus).when(event).getContainers();
@@ -181,15 +178,12 @@ public class TestRMNodeTransitions {
   }
   
   private RMNodeStatusEvent getMockRMNodeStatusEventWithRunningApps() {
-    NodeHeartbeatResponse response = mock(NodeHeartbeatResponse.class);
-
     NodeHealthStatus healthStatus = mock(NodeHealthStatus.class);
     Boolean yes = new Boolean(true);
     doReturn(yes).when(healthStatus).getIsNodeHealthy();
 
     RMNodeStatusEvent event = mock(RMNodeStatusEvent.class);
     doReturn(healthStatus).when(event).getNodeHealthStatus();
-    doReturn(response).when(event).getLatestResponse();
     doReturn(RMNodeEventType.STATUS_UPDATE).when(event).getType();
     doReturn(getAppIdList()).when(event).getKeepAliveAppIds();
     return event;
@@ -202,15 +196,12 @@ public class TestRMNodeTransitions {
   }
 
   private RMNodeStatusEvent getMockRMNodeStatusEventWithoutRunningApps() {
-    NodeHeartbeatResponse response = mock(NodeHeartbeatResponse.class);
-
     NodeHealthStatus healthStatus = mock(NodeHealthStatus.class);
     Boolean yes = new Boolean(true);
     doReturn(yes).when(healthStatus).getIsNodeHealthy();
 
     RMNodeStatusEvent event = mock(RMNodeStatusEvent.class);
     doReturn(healthStatus).when(event).getNodeHealthStatus();
-    doReturn(response).when(event).getLatestResponse();
     doReturn(RMNodeEventType.STATUS_UPDATE).when(event).getType();
     doReturn(null).when(event).getKeepAliveAppIds();
     return event;
@@ -646,7 +637,7 @@ public class TestRMNodeTransitions {
     Assert.assertEquals(1, node.getContainersToCleanUp().size());
     Assert.assertEquals(1, node.getAppsToCleanup().size());
     NodeHeartbeatResponse hbrsp = Records.newRecord(NodeHeartbeatResponse.class);
-    node.updateNodeHeartbeatResponseForCleanup(hbrsp);
+    node.setAndUpdateNodeHeartbeatResponse(hbrsp);
     Assert.assertEquals(0, node.getContainersToCleanUp().size());
     Assert.assertEquals(0, node.getAppsToCleanup().size());
     Assert.assertEquals(1, hbrsp.getContainersToCleanup().size());
@@ -1108,7 +1099,8 @@ public class TestRMNodeTransitions {
 
     NodeHeartbeatResponse hbrsp =
         Records.newRecord(NodeHeartbeatResponse.class);
-    node.updateNodeHeartbeatResponseForCleanup(hbrsp);
+    node.setAndUpdateNodeHeartbeatResponse(hbrsp);
+
     Assert.assertEquals(1, hbrsp.getContainersToBeRemovedFromNM().size());
     Assert.assertEquals(0, node.getCompletedContainers().size());
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff8378eb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java
index fc6326e..96e4451 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java
@@ -801,7 +801,7 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
         Records.newRecord(NodeHeartbeatRequest.class);
     heartbeatReq.setNodeLabels(null); // Node heartbeat label update
     nodeStatusObject = getNodeStatusObject(nodeId);
-    nodeStatusObject.setResponseId(responseId+2);
+    nodeStatusObject.setResponseId(responseId+1);
     heartbeatReq.setNodeStatus(nodeStatusObject);
     heartbeatReq.setLastKnownNMTokenMasterKey(registerResponse
         .getNMTokenMasterKey());
@@ -1128,8 +1128,7 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
         "", System.currentTimeMillis());
     NodeStatus nodeStatus = NodeStatus.newInstance(nm1.getNodeId(), 0,
         statusList, null, nodeHealth, null, null, null);
-    node1.handle(new RMNodeStatusEvent(nm1.getNodeId(), nodeStatus,
-        nodeHeartbeat1));
+    node1.handle(new RMNodeStatusEvent(nm1.getNodeId(), nodeStatus));
 
     Assert.assertEquals(1, node1.getRunningApps().size());
     Assert.assertEquals(app1.getApplicationId(), node1.getRunningApps().get(0));
@@ -1145,8 +1144,7 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
     statusList.add(status2);
     nodeStatus = NodeStatus.newInstance(nm1.getNodeId(), 0,
         statusList, null, nodeHealth, null, null, null);
-    node2.handle(new RMNodeStatusEvent(nm2.getNodeId(), nodeStatus,
-        nodeHeartbeat2));
+    node2.handle(new RMNodeStatusEvent(nm2.getNodeId(), nodeStatus));
     Assert.assertEquals(1, node2.getRunningApps().size());
     Assert.assertEquals(app2.getApplicationId(), node2.getRunningApps().get(0));
 
@@ -2290,4 +2288,31 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
       }
     }
   }
+
+  @Test
+  public void testResponseIdOverflow() throws Exception {
+    Configuration conf = new Configuration();
+    rm = new MockRM(conf);
+    rm.start();
+
+    MockNM nm1 = rm.registerNode("host1:1234", 5120);
+
+    NodeHeartbeatResponse nodeHeartbeat = nm1.nodeHeartbeat(true);
+    Assert.assertEquals(NodeAction.NORMAL, nodeHeartbeat.getNodeAction());
+
+    // prepare the responseId that's about to overflow
+    RMNode node = rm.getRMContext().getRMNodes().get(nm1.getNodeId());
+    node.getLastNodeHeartBeatResponse().setResponseId(Integer.MAX_VALUE);
+
+    nm1.setResponseId(Integer.MAX_VALUE);
+
+    // heartbeat twice and check responseId
+    nodeHeartbeat = nm1.nodeHeartbeat(true);
+    Assert.assertEquals(NodeAction.NORMAL, nodeHeartbeat.getNodeAction());
+    Assert.assertEquals(0, nodeHeartbeat.getResponseId());
+
+    nodeHeartbeat = nm1.nodeHeartbeat(true);
+    Assert.assertEquals(NodeAction.NORMAL, nodeHeartbeat.getNodeAction());
+    Assert.assertEquals(1, nodeHeartbeat.getResponseId());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff8378eb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/logaggregationstatus/TestRMAppLogAggregationStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/logaggregationstatus/TestRMAppLogAggregationStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/logaggregationstatus/TestRMAppLogAggregationStatus.java
index 677990b..c2bc611 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/logaggregationstatus/TestRMAppLogAggregationStatus.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/logaggregationstatus/TestRMAppLogAggregationStatus.java
@@ -172,7 +172,7 @@ public class TestRMAppLogAggregationStatus {
     NodeStatus nodeStatus1 = NodeStatus.newInstance(node1.getNodeID(), 0,
         new ArrayList<ContainerStatus>(), null,
         NodeHealthStatus.newInstance(true, null, 0), null, null, null);
-    node1.handle(new RMNodeStatusEvent(node1.getNodeID(), nodeStatus1, null,
+    node1.handle(new RMNodeStatusEvent(node1.getNodeID(), nodeStatus1,
         node1ReportForApp));
 
     List<LogAggregationReport> node2ReportForApp =
@@ -186,7 +186,7 @@ public class TestRMAppLogAggregationStatus {
     NodeStatus nodeStatus2 = NodeStatus.newInstance(node2.getNodeID(), 0,
         new ArrayList<ContainerStatus>(), null,
         NodeHealthStatus.newInstance(true, null, 0), null, null, null);
-    node2.handle(new RMNodeStatusEvent(node2.getNodeID(), nodeStatus2, null,
+    node2.handle(new RMNodeStatusEvent(node2.getNodeID(), nodeStatus2,
         node2ReportForApp));
     // node1 and node2 has updated its log aggregation status
     // verify that the log aggregation status for node1, node2
@@ -223,7 +223,7 @@ public class TestRMAppLogAggregationStatus {
         LogAggregationReport.newInstance(appId,
           LogAggregationStatus.RUNNING, messageForNode1_2);
     node1ReportForApp2.add(report1_2);
-    node1.handle(new RMNodeStatusEvent(node1.getNodeID(), nodeStatus1, null,
+    node1.handle(new RMNodeStatusEvent(node1.getNodeID(), nodeStatus1,
         node1ReportForApp2));
 
     // verify that the log aggregation status for node1
@@ -291,7 +291,7 @@ public class TestRMAppLogAggregationStatus {
       LogAggregationStatus.SUCCEEDED, ""));
     // For every logAggregationReport cached in memory, we can only save at most
     // 10 diagnostic messages/failure messages
-    node1.handle(new RMNodeStatusEvent(node1.getNodeID(), nodeStatus1, null,
+    node1.handle(new RMNodeStatusEvent(node1.getNodeID(), nodeStatus1,
         node1ReportForApp3));
 
     logAggregationStatus = rmApp.getLogAggregationReportsForApp();
@@ -335,7 +335,7 @@ public class TestRMAppLogAggregationStatus {
           LogAggregationStatus.FAILED, "");
     node2ReportForApp2.add(report2_2);
     node2ReportForApp2.add(report2_3);
-    node2.handle(new RMNodeStatusEvent(node2.getNodeID(), nodeStatus2, null,
+    node2.handle(new RMNodeStatusEvent(node2.getNodeID(), nodeStatus2,
         node2ReportForApp2));
     Assert.assertEquals(LogAggregationStatus.FAILED,
       rmApp.getLogAggregationStatusForAppReport());


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[13/37] hadoop git commit: YARN-7814. Remove automatic mounting of the cgroups root directory into Docker containers. Contributed by Shane Kumpf.

Posted by ae...@apache.org.
YARN-7814. Remove automatic mounting of the cgroups root directory into Docker containers. Contributed by Shane Kumpf.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/2e586560
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2e586560
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2e586560

Branch: refs/heads/HDFS-7240
Commit: 2e5865606b7701ee79d0d297238ab58a07a9f61f
Parents: cc10852
Author: Miklos Szegedi <sz...@apache.org>
Authored: Thu Jan 25 16:32:01 2018 -0800
Committer: Miklos Szegedi <sz...@apache.org>
Committed: Thu Jan 25 17:37:23 2018 -0800

----------------------------------------------------------------------
 .../runtime/DockerLinuxContainerRuntime.java     |  7 -------
 .../runtime/TestDockerContainerRuntime.java      | 19 -------------------
 2 files changed, 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e586560/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
index f54323c..2868dea 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
@@ -230,7 +230,6 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
   private PrivilegedOperationExecutor privilegedOperationExecutor;
   private Set<String> allowedNetworks = new HashSet<>();
   private String defaultNetwork;
-  private String cgroupsRootDirectory;
   private CGroupsHandler cGroupsHandler;
   private AccessControlList privilegedContainersAcl;
   private boolean enableUserReMapping;
@@ -290,7 +289,6 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
       LOG.info("cGroupsHandler is null - cgroups not in use.");
     } else {
       this.cGroupsHandler = cGroupsHandler;
-      this.cgroupsRootDirectory = cGroupsHandler.getCGroupMountPath();
     }
   }
 
@@ -796,11 +794,6 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
     setHostname(runCommand, containerIdStr, hostname);
     runCommand.setCapabilities(capabilities);
 
-    if(cgroupsRootDirectory != null) {
-      runCommand.addReadOnlyMountLocation(cgroupsRootDirectory,
-          cgroupsRootDirectory, false);
-    }
-
     List<String> allDirs = new ArrayList<>(containerLocalDirs);
     allDirs.addAll(filecacheDirs);
     allDirs.add(containerWorkDir.toString());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e586560/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java
index a825828..48a96e1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java
@@ -37,7 +37,6 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileg
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperationException;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperationExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.CGroupsHandler;
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.ResourceHandlerModule;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.docker.DockerCommandExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.docker.DockerKillCommand;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.docker.DockerRmCommand;
@@ -332,24 +331,6 @@ public class TestDockerContainerRuntime {
     return expectedCapabilitiesString.toString();
   }
 
-  private String getExpectedCGroupsMountString() {
-    CGroupsHandler cgroupsHandler = ResourceHandlerModule.getCGroupsHandler();
-    if(cgroupsHandler == null) {
-      return "";
-    }
-
-    String cgroupMountPath = cgroupsHandler.getCGroupMountPath();
-    boolean cGroupsMountExists = new File(
-        cgroupMountPath).exists();
-
-    if(cGroupsMountExists) {
-      return "-v " + cgroupMountPath
-          + ":" + cgroupMountPath + ":ro ";
-    } else {
-      return "";
-    }
-  }
-
   @Test
   public void testDockerContainerLaunch()
       throws ContainerExecutionException, PrivilegedOperationException,


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[32/37] hadoop git commit: HDFS-13044. RBF: Add a safe mode for the Router. Contributed by Inigo Goiri.

Posted by ae...@apache.org.
HDFS-13044. RBF: Add a safe mode for the Router. Contributed by Inigo Goiri.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/dbb9dded
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/dbb9dded
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/dbb9dded

Branch: refs/heads/HDFS-7240
Commit: dbb9dded33b3cff3b630e98300d30515a9d1eec4
Parents: fde95d4
Author: Yiqun Lin <yq...@apache.org>
Authored: Tue Jan 30 12:12:08 2018 +0800
Committer: Yiqun Lin <yq...@apache.org>
Committed: Tue Jan 30 12:12:08 2018 +0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |  13 ++
 .../hdfs/server/federation/router/Router.java   |  16 +-
 .../federation/router/RouterRpcServer.java      |  43 ++++-
 .../router/RouterSafeModeException.java         |  53 +++++
 .../router/RouterSafemodeService.java           | 150 +++++++++++++++
 .../store/StateStoreCacheUpdateService.java     |   7 +-
 .../src/main/resources/hdfs-default.xml         |  36 +++-
 .../src/site/markdown/HDFSRouterFederation.md   |   4 +
 .../server/federation/RouterConfigBuilder.java  |  13 ++
 .../federation/router/TestRouterSafemode.java   | 192 +++++++++++++++++++
 10 files changed, 515 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/dbb9dded/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index 84215f3f..4589aaa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -1291,6 +1291,19 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final long FEDERATION_STORE_ROUTER_EXPIRATION_MS_DEFAULT =
       TimeUnit.MINUTES.toMillis(5);
 
+  // HDFS Router safe mode
+  public static final String DFS_ROUTER_SAFEMODE_ENABLE =
+      FEDERATION_ROUTER_PREFIX + "safemode.enable";
+  public static final boolean DFS_ROUTER_SAFEMODE_ENABLE_DEFAULT = true;
+  public static final String DFS_ROUTER_SAFEMODE_EXTENSION =
+      FEDERATION_ROUTER_PREFIX + "safemode.extension";
+  public static final long DFS_ROUTER_SAFEMODE_EXTENSION_DEFAULT =
+      TimeUnit.SECONDS.toMillis(30);
+  public static final String DFS_ROUTER_SAFEMODE_EXPIRATION =
+      FEDERATION_ROUTER_PREFIX + "safemode.expiration";
+  public static final long DFS_ROUTER_SAFEMODE_EXPIRATION_DEFAULT =
+      3 * DFS_ROUTER_CACHE_TIME_TO_LIVE_MS_DEFAULT;
+
   // HDFS Router-based federation mount table entries
   /** Maximum number of cache entries to have. */
   public static final String FEDERATION_MOUNT_TABLE_MAX_CACHE_SIZE =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dbb9dded/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
index 1e72c93..79f43bb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
@@ -118,6 +118,8 @@ public class Router extends CompositeService {
   private RouterStore routerStateManager;
   /** Heartbeat our run status to the router state manager. */
   private RouterHeartbeatService routerHeartbeatService;
+  /** Enter/exit safemode. */
+  private RouterSafemodeService safemodeService;
 
   /** The start time of the namesystem. */
   private final long startTime = Time.now();
@@ -232,13 +234,25 @@ public class Router extends CompositeService {
       addService(this.quotaUpdateService);
     }
 
+    // Safemode service to refuse RPC calls when the router is out of sync
+    if (conf.getBoolean(
+        DFSConfigKeys.DFS_ROUTER_SAFEMODE_ENABLE,
+        DFSConfigKeys.DFS_ROUTER_SAFEMODE_ENABLE_DEFAULT)) {
+      // Create safemode monitoring service
+      this.safemodeService = new RouterSafemodeService(this);
+      addService(this.safemodeService);
+    }
+
     super.serviceInit(conf);
   }
 
   @Override
   protected void serviceStart() throws Exception {
 
-    updateRouterState(RouterServiceState.RUNNING);
+    if (this.safemodeService == null) {
+      // Router is running now
+      updateRouterState(RouterServiceState.RUNNING);
+    }
 
     if (this.pauseMonitor != null) {
       this.pauseMonitor.start();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dbb9dded/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
index 9afd441..57125ca 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
@@ -179,6 +179,8 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
   /** Interface to map global name space to HDFS subcluster name spaces. */
   private final FileSubclusterResolver subclusterResolver;
 
+  /** If we are in safe mode, fail requests as if a standby NN. */
+  private volatile boolean safeMode;
 
   /** Category of the operation that a thread is executing. */
   private final ThreadLocal<OperationCategory> opCategory = new ThreadLocal<>();
@@ -370,12 +372,12 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
    * @param op Category of the operation to check.
    * @param supported If the operation is supported or not. If not, it will
    *                  throw an UnsupportedOperationException.
-   * @throws StandbyException If the Router is in safe mode and cannot serve
-   *                          client requests.
+   * @throws SafeModeException If the Router is in safe mode and cannot serve
+   *                           client requests.
    * @throws UnsupportedOperationException If the operation is not supported.
    */
   protected void checkOperation(OperationCategory op, boolean supported)
-      throws StandbyException, UnsupportedOperationException {
+      throws RouterSafeModeException, UnsupportedOperationException {
     checkOperation(op);
 
     if (!supported) {
@@ -393,10 +395,11 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
    * UNCHECKED. This function should be called by all ClientProtocol functions.
    *
    * @param op Category of the operation to check.
-   * @throws StandbyException If the Router is in safe mode and cannot serve
-   *                          client requests.
+   * @throws SafeModeException If the Router is in safe mode and cannot serve
+   *                           client requests.
    */
-  protected void checkOperation(OperationCategory op) throws StandbyException {
+  protected void checkOperation(OperationCategory op)
+      throws RouterSafeModeException {
     // Log the function we are currently calling.
     if (rpcMonitor != null) {
       rpcMonitor.startOp();
@@ -415,7 +418,33 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
       return;
     }
 
-    // TODO check Router safe mode and return Standby exception
+    if (safeMode) {
+      // Throw standby exception, router is not available
+      if (rpcMonitor != null) {
+        rpcMonitor.routerFailureSafemode();
+      }
+      throw new RouterSafeModeException(router.getRouterId(), op);
+    }
+  }
+
+  /**
+   * In safe mode all RPC requests will fail and return a standby exception.
+   * The client will try another Router, similar to the client retry logic for
+   * HA.
+   *
+   * @param mode True if enabled, False if disabled.
+   */
+  public void setSafeMode(boolean mode) {
+    this.safeMode = mode;
+  }
+
+  /**
+   * Check if the Router is in safe mode and cannot serve RPC calls.
+   *
+   * @return If the Router is in safe mode.
+   */
+  public boolean isInSafeMode() {
+    return this.safeMode;
   }
 
   @Override // ClientProtocol

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dbb9dded/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterSafeModeException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterSafeModeException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterSafeModeException.java
new file mode 100644
index 0000000..7a78b5b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterSafeModeException.java
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.federation.router;
+
+import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
+import org.apache.hadoop.ipc.StandbyException;
+
+/**
+ * Exception that the Router throws when it is in safe mode. This extends
+ * {@link StandbyException} for the client to try another Router when it gets
+ * this exception.
+ */
+public class RouterSafeModeException extends StandbyException {
+
+  private static final long serialVersionUID = 453568188334993493L;
+
+  /** Identifier of the Router that generated this exception. */
+  private final String routerId;
+
+  /**
+   * Build a new Router safe mode exception.
+   * @param router Identifier of the Router.
+   * @param op Category of the operation (READ/WRITE).
+   */
+  public RouterSafeModeException(String router, OperationCategory op) {
+    super("Router " + router + " is in safe mode and cannot handle " + op
+        + " requests.");
+    this.routerId = router;
+  }
+
+  /**
+   * Get the id of the Router that generated this exception.
+   * @return Id of the Router that generated this exception.
+   */
+  public String getRouterId() {
+    return this.routerId;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dbb9dded/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterSafemodeService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterSafemodeService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterSafemodeService.java
new file mode 100644
index 0000000..56aab0a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterSafemodeService.java
@@ -0,0 +1,150 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.federation.router;
+
+import static org.apache.hadoop.util.Time.now;
+
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Service to periodically check if the {@link org.apache.hadoop.hdfs.server.
+ * federation.store.StateStoreService StateStoreService} cached information in
+ * the {@link Router} is up to date. This is for performance and removes the
+ * {@link org.apache.hadoop.hdfs.server.federation.store.StateStoreService
+ * StateStoreService} from the critical path in common operations.
+ */
+public class RouterSafemodeService extends PeriodicService {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RouterSafemodeService.class);
+
+  /** Router to manage safe mode. */
+  private final Router router;
+
+  /** Interval in ms to wait post startup before allowing RPC requests. */
+  private long startupInterval;
+  /** Interval in ms after which the State Store cache is too stale. */
+  private long staleInterval;
+  /** Start time in ms of this service. */
+  private long startupTime;
+
+  /** The time the Router enters safe mode in milliseconds. */
+  private long enterSafeModeTime = now();
+
+
+  /**
+   * Create a new Cache update service.
+   *
+   * @param router Router containing the cache.
+   */
+  public RouterSafemodeService(Router router) {
+    super(RouterSafemodeService.class.getSimpleName());
+    this.router = router;
+  }
+
+  /**
+   * Enter safe mode.
+   */
+  private void enter() {
+    LOG.info("Entering safe mode");
+    enterSafeModeTime = now();
+    RouterRpcServer rpcServer = router.getRpcServer();
+    rpcServer.setSafeMode(true);
+    router.updateRouterState(RouterServiceState.SAFEMODE);
+  }
+
+  /**
+   * Leave safe mode.
+   */
+  private void leave() {
+    // Cache recently updated, leave safemode
+    long timeInSafemode = now() - enterSafeModeTime;
+    LOG.info("Leaving safe mode after {} milliseconds", timeInSafemode);
+    RouterMetrics routerMetrics = router.getRouterMetrics();
+    if (routerMetrics == null) {
+      LOG.error("The Router metrics are not enabled");
+    } else {
+      routerMetrics.setSafeModeTime(timeInSafemode);
+    }
+    RouterRpcServer rpcServer = router.getRpcServer();
+    rpcServer.setSafeMode(false);
+    router.updateRouterState(RouterServiceState.RUNNING);
+  }
+
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+
+    // Use same interval as cache update service
+    this.setIntervalMs(conf.getTimeDuration(
+        DFSConfigKeys.DFS_ROUTER_CACHE_TIME_TO_LIVE_MS,
+        DFSConfigKeys.DFS_ROUTER_CACHE_TIME_TO_LIVE_MS_DEFAULT,
+        TimeUnit.MILLISECONDS));
+
+    this.startupInterval = conf.getTimeDuration(
+        DFSConfigKeys.DFS_ROUTER_SAFEMODE_EXTENSION,
+        DFSConfigKeys.DFS_ROUTER_SAFEMODE_EXTENSION_DEFAULT,
+        TimeUnit.MILLISECONDS);
+    LOG.info("Leave startup safe mode after {} ms", this.startupInterval);
+
+    this.staleInterval = conf.getTimeDuration(
+        DFSConfigKeys.DFS_ROUTER_SAFEMODE_EXPIRATION,
+        DFSConfigKeys.DFS_ROUTER_SAFEMODE_EXPIRATION_DEFAULT,
+        TimeUnit.MILLISECONDS);
+    LOG.info("Enter safe mode after {} ms without reaching the State Store",
+        this.staleInterval);
+
+    this.startupTime = Time.now();
+
+    // Initializing the RPC server in safe mode, it will disable it later
+    enter();
+
+    super.serviceInit(conf);
+  }
+
+  @Override
+  public void periodicInvoke() {
+    long now = Time.now();
+    long delta = now - startupTime;
+    if (delta < startupInterval) {
+      LOG.info("Delaying safemode exit for {} milliseconds...",
+          this.startupInterval - delta);
+      return;
+    }
+    RouterRpcServer rpcServer = router.getRpcServer();
+    StateStoreService stateStore = router.getStateStore();
+    long cacheUpdateTime = stateStore.getCacheUpdateTime();
+    boolean isCacheStale = (now - cacheUpdateTime) > this.staleInterval;
+
+    // Always update to indicate our cache was updated
+    if (isCacheStale) {
+      if (!rpcServer.isInSafeMode()) {
+        enter();
+      }
+    } else if (rpcServer.isInSafeMode()) {
+      // Cache recently updated, leave safe mode
+      leave();
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dbb9dded/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreCacheUpdateService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreCacheUpdateService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreCacheUpdateService.java
index bb8cfb0..9bcbc1e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreCacheUpdateService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreCacheUpdateService.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdfs.server.federation.store;
 
+import java.util.concurrent.TimeUnit;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.server.federation.router.PeriodicService;
@@ -52,9 +54,10 @@ public class StateStoreCacheUpdateService extends PeriodicService {
   @Override
   protected void serviceInit(Configuration conf) throws Exception {
 
-    this.setIntervalMs(conf.getLong(
+    this.setIntervalMs(conf.getTimeDuration(
         DFSConfigKeys.DFS_ROUTER_CACHE_TIME_TO_LIVE_MS,
-        DFSConfigKeys.DFS_ROUTER_CACHE_TIME_TO_LIVE_MS_DEFAULT));
+        DFSConfigKeys.DFS_ROUTER_CACHE_TIME_TO_LIVE_MS_DEFAULT,
+        TimeUnit.MILLISECONDS));
 
     super.serviceInit(conf);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dbb9dded/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index d24310e..7446766 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -5080,9 +5080,12 @@
 
   <property>
     <name>dfs.federation.router.cache.ttl</name>
-    <value>60000</value>
+    <value>1m</value>
     <description>
-      How often to refresh the State Store caches in milliseconds.
+      How often to refresh the State Store caches in milliseconds. This setting
+      supports multiple time unit suffixes as described in
+      dfs.heartbeat.interval. If no suffix is specified then milliseconds is
+      assumed.
     </description>
   </property>
 
@@ -5131,6 +5134,35 @@
   </property>
 
   <property>
+    <name>dfs.federation.router.safemode.enable</name>
+    <value>true</value>
+    <description>
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.federation.router.safemode.extension</name>
+    <value>30s</value>
+    <description>
+      Time after startup that the Router is in safe mode. This setting
+      supports multiple time unit suffixes as described in
+      dfs.heartbeat.interval. If no suffix is specified then milliseconds is
+      assumed.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.federation.router.safemode.expiration</name>
+    <value>3m</value>
+    <description>
+      Time without being able to reach the State Store to enter safe mode. This
+      setting supports multiple time unit suffixes as described in
+      dfs.heartbeat.interval. If no suffix is specified then milliseconds is
+      assumed.
+    </description>
+  </property>
+
+  <property>
     <name>dfs.federation.router.monitor.namenode</name>
     <value></value>
     <description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dbb9dded/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSRouterFederation.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSRouterFederation.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSRouterFederation.md
index 75798a1..6b21123 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSRouterFederation.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSRouterFederation.md
@@ -81,6 +81,10 @@ The Routers are stateless and metadata operations are atomic at the NameNodes.
 If a Router becomes unavailable, any Router can take over for it.
 The clients configure their DFS HA client (e.g., ConfiguredFailoverProvider or RequestHedgingProxyProvider) with all the Routers in the federation as endpoints.
 
+* **Unavailable State Store:**
+If a Router cannot contact the State Store, it will enter into a Safe Mode state which disallows it from serving requests.
+Clients will treat Routers in Safe Mode as it was an Standby NameNode and try another Router.
+
 * **NameNode heartbeat HA:**
 For high availability and flexibility, multiple Routers can monitor the same NameNode and heartbeat the information to the State Store.
 This increases clients' resiliency to stale information, should a Router fail.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dbb9dded/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterConfigBuilder.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterConfigBuilder.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterConfigBuilder.java
index 3d8b35c..3659bf9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterConfigBuilder.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterConfigBuilder.java
@@ -35,6 +35,7 @@ public class RouterConfigBuilder {
   private boolean enableStateStore = false;
   private boolean enableMetrics = false;
   private boolean enableQuota = false;
+  private boolean enableSafemode = false;
 
   public RouterConfigBuilder(Configuration configuration) {
     this.conf = configuration;
@@ -52,6 +53,7 @@ public class RouterConfigBuilder {
     this.enableLocalHeartbeat = true;
     this.enableStateStore = true;
     this.enableMetrics = true;
+    this.enableSafemode = true;
     return this;
   }
 
@@ -95,6 +97,11 @@ public class RouterConfigBuilder {
     return this;
   }
 
+  public RouterConfigBuilder safemode(boolean enable) {
+    this.enableSafemode = enable;
+    return this;
+  }
+
   public RouterConfigBuilder rpc() {
     return this.rpc(true);
   }
@@ -123,6 +130,10 @@ public class RouterConfigBuilder {
     return this.quota(true);
   }
 
+  public RouterConfigBuilder safemode() {
+    return this.safemode(true);
+  }
+
   public Configuration build() {
     conf.setBoolean(DFSConfigKeys.DFS_ROUTER_STORE_ENABLE,
         this.enableStateStore);
@@ -139,6 +150,8 @@ public class RouterConfigBuilder {
         this.enableMetrics);
     conf.setBoolean(DFSConfigKeys.DFS_ROUTER_QUOTA_ENABLE,
         this.enableQuota);
+    conf.setBoolean(DFSConfigKeys.DFS_ROUTER_SAFEMODE_ENABLE,
+        this.enableSafemode);
     return conf;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dbb9dded/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterSafemode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterSafemode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterSafemode.java
new file mode 100644
index 0000000..9299f77
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterSafemode.java
@@ -0,0 +1,192 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.federation.router;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_CACHE_TIME_TO_LIVE_MS;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_SAFEMODE_EXPIRATION;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_SAFEMODE_EXTENSION;
+import static org.apache.hadoop.hdfs.server.federation.store.FederationStateStoreTestUtils.deleteStateStore;
+import static org.apache.hadoop.hdfs.server.federation.store.FederationStateStoreTestUtils.getStateStoreConfiguration;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.federation.RouterConfigBuilder;
+import org.apache.hadoop.service.Service.STATE;
+import org.apache.hadoop.util.Time;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Test the safe mode for the {@link Router} controlled by
+ * {@link RouterSafemodeService}.
+ */
+public class TestRouterSafemode {
+
+  private Router router;
+  private static Configuration conf;
+
+  @BeforeClass
+  public static void create() throws IOException {
+    // Wipe state store
+    deleteStateStore();
+    // Configuration that supports the state store
+    conf = getStateStoreConfiguration();
+    // 2 sec startup standby
+    conf.setTimeDuration(DFS_ROUTER_SAFEMODE_EXTENSION,
+        TimeUnit.SECONDS.toMillis(2), TimeUnit.MILLISECONDS);
+    // 1 sec cache refresh
+    conf.setTimeDuration(DFS_ROUTER_CACHE_TIME_TO_LIVE_MS,
+        TimeUnit.SECONDS.toMillis(1), TimeUnit.MILLISECONDS);
+    // 2 sec post cache update before entering safemode (2 intervals)
+    conf.setTimeDuration(DFS_ROUTER_SAFEMODE_EXPIRATION,
+        TimeUnit.SECONDS.toMillis(2), TimeUnit.MILLISECONDS);
+    // RPC + State Store + Safe Mode only
+    conf = new RouterConfigBuilder(conf)
+        .rpc()
+        .safemode()
+        .stateStore()
+        .metrics()
+        .build();
+  }
+
+  @AfterClass
+  public static void destroy() {
+  }
+
+  @Before
+  public void setup() throws IOException, URISyntaxException {
+    router = new Router();
+    router.init(conf);
+    router.start();
+  }
+
+  @After
+  public void cleanup() throws IOException {
+    if (router != null) {
+      router.stop();
+      router = null;
+    }
+  }
+
+  @Test
+  public void testSafemodeService() throws IOException {
+    RouterSafemodeService server = new RouterSafemodeService(router);
+    server.init(conf);
+    assertEquals(STATE.INITED, server.getServiceState());
+    server.start();
+    assertEquals(STATE.STARTED, server.getServiceState());
+    server.stop();
+    assertEquals(STATE.STOPPED, server.getServiceState());
+    server.close();
+  }
+
+  @Test
+  public void testRouterExitSafemode()
+      throws InterruptedException, IllegalStateException, IOException {
+
+    assertTrue(router.getRpcServer().isInSafeMode());
+    verifyRouter(RouterServiceState.SAFEMODE);
+
+    // Wait for initial time in milliseconds
+    long interval =
+        conf.getTimeDuration(DFS_ROUTER_SAFEMODE_EXTENSION,
+            TimeUnit.SECONDS.toMillis(2), TimeUnit.MILLISECONDS) +
+        conf.getTimeDuration(DFS_ROUTER_CACHE_TIME_TO_LIVE_MS,
+            TimeUnit.SECONDS.toMillis(1), TimeUnit.MILLISECONDS);
+    Thread.sleep(interval);
+
+    assertFalse(router.getRpcServer().isInSafeMode());
+    verifyRouter(RouterServiceState.RUNNING);
+  }
+
+  @Test
+  public void testRouterEnterSafemode()
+      throws IllegalStateException, IOException, InterruptedException {
+
+    // Verify starting state
+    assertTrue(router.getRpcServer().isInSafeMode());
+    verifyRouter(RouterServiceState.SAFEMODE);
+
+    // We should be in safe mode for DFS_ROUTER_SAFEMODE_EXTENSION time
+    long interval0 = conf.getTimeDuration(DFS_ROUTER_SAFEMODE_EXTENSION,
+        TimeUnit.SECONDS.toMillis(2), TimeUnit.MILLISECONDS) - 1000;
+    long t0 = Time.now();
+    while (Time.now() - t0 < interval0) {
+      verifyRouter(RouterServiceState.SAFEMODE);
+      Thread.sleep(100);
+    }
+
+    // We wait some time for the state to propagate
+    long interval1 = 1000 + 2 * conf.getTimeDuration(
+        DFS_ROUTER_CACHE_TIME_TO_LIVE_MS, TimeUnit.SECONDS.toMillis(1),
+        TimeUnit.MILLISECONDS);
+    Thread.sleep(interval1);
+
+    // Running
+    assertFalse(router.getRpcServer().isInSafeMode());
+    verifyRouter(RouterServiceState.RUNNING);
+
+    // Disable cache
+    router.getStateStore().stopCacheUpdateService();
+
+    // Wait until the State Store cache is stale in milliseconds
+    long interval2 =
+        conf.getTimeDuration(DFS_ROUTER_SAFEMODE_EXPIRATION,
+            TimeUnit.SECONDS.toMillis(2), TimeUnit.MILLISECONDS) +
+        conf.getTimeDuration(DFS_ROUTER_CACHE_TIME_TO_LIVE_MS,
+            TimeUnit.SECONDS.toMillis(1), TimeUnit.MILLISECONDS);
+    Thread.sleep(interval2);
+
+    // Safemode
+    assertTrue(router.getRpcServer().isInSafeMode());
+    verifyRouter(RouterServiceState.SAFEMODE);
+  }
+
+  @Test
+  public void testRouterRpcSafeMode()
+      throws IllegalStateException, IOException {
+
+    assertTrue(router.getRpcServer().isInSafeMode());
+    verifyRouter(RouterServiceState.SAFEMODE);
+
+    // If the Router is in Safe Mode, we should get a SafeModeException
+    boolean exception = false;
+    try {
+      router.getRpcServer().delete("/testfile.txt", true);
+      fail("We should have thrown a safe mode exception");
+    } catch (RouterSafeModeException sme) {
+      exception = true;
+    }
+    assertTrue("We should have thrown a safe mode exception", exception);
+  }
+
+  private void verifyRouter(RouterServiceState status)
+      throws IllegalStateException, IOException {
+    assertEquals(status, router.getRouterState());
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[14/37] hadoop git commit: YARN-7817. Add Resource reference to RM's NodeInfo object so REST API can get non memory/vcore resource usages. (Sunil G via wangda)

Posted by ae...@apache.org.
YARN-7817. Add Resource reference to RM's NodeInfo object so REST API can get non memory/vcore resource usages. (Sunil G via wangda)

Change-Id: Ia7ceeabd82046645ddeaf487c763288f36cfbdee


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/e0cfb0a3
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e0cfb0a3
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e0cfb0a3

Branch: refs/heads/HDFS-7240
Commit: e0cfb0a31a1c6ad0f9dcf0705b44229593a0915f
Parents: 2e58656
Author: Wangda Tan <wa...@apache.org>
Authored: Fri Jan 26 15:43:27 2018 +0800
Committer: Wangda Tan <wa...@apache.org>
Committed: Fri Jan 26 15:43:27 2018 +0800

----------------------------------------------------------------------
 .../webapp/JAXBContextResolver.java             |  2 +-
 .../webapp/dao/ClusterMetricsInfo.java          | 24 ++++++++++
 .../resourcemanager/webapp/dao/NodeInfo.java    | 20 ++++++++
 .../webapp/dao/ResourceInfo.java                | 18 ++++++--
 .../webapp/dao/ResourceInformationsInfo.java    | 48 ++++++++++++++++++++
 .../main/webapp/app/models/cluster-metric.js    |  4 +-
 .../src/main/webapp/app/models/yarn-rm-node.js  |  4 +-
 .../main/webapp/app/serializers/yarn-rm-node.js |  4 +-
 8 files changed, 114 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e0cfb0a3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/JAXBContextResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/JAXBContextResolver.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/JAXBContextResolver.java
index 2f50a24..2e4204e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/JAXBContextResolver.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/JAXBContextResolver.java
@@ -55,7 +55,7 @@ public class JAXBContextResolver implements ContextResolver<JAXBContext> {
             UsersInfo.class, UserInfo.class, ApplicationStatisticsInfo.class,
             StatisticsItemInfo.class, CapacitySchedulerHealthInfo.class,
             FairSchedulerQueueInfoList.class, AppTimeoutsInfo.class,
-            AppTimeoutInfo.class };
+            AppTimeoutInfo.class, ResourceInformationsInfo.class };
     // these dao classes need root unwrapping
     final Class[] rootUnwrappedTypes =
         { NewApplication.class, ApplicationSubmissionContextInfo.class,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e0cfb0a3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ClusterMetricsInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ClusterMetricsInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ClusterMetricsInfo.java
index 3214cb9..84f70d9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ClusterMetricsInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ClusterMetricsInfo.java
@@ -61,6 +61,12 @@ public class ClusterMetricsInfo {
   private int activeNodes;
   private int shutdownNodes;
 
+  // Total used resource of the cluster, including all partitions
+  private ResourceInfo totalUsedResourcesAcrossPartition;
+
+  // Total registered resources of the cluster, including all partitions
+  private ResourceInfo totalClusterResourcesAcrossPartition;
+
   public ClusterMetricsInfo() {
   } // JAXB needs this
 
@@ -92,9 +98,20 @@ public class ClusterMetricsInfo {
     this.containersReserved = metrics.getReservedContainers();
 
     if (rs instanceof CapacityScheduler) {
+      CapacityScheduler cs = (CapacityScheduler) rs;
       this.totalMB = availableMB + allocatedMB + reservedMB;
       this.totalVirtualCores =
           availableVirtualCores + allocatedVirtualCores + containersReserved;
+      // TODO, add support of other schedulers to get total used resources
+      // across partition.
+      if (cs.getRootQueue() != null
+          && cs.getRootQueue().getQueueResourceUsage() != null
+          && cs.getRootQueue().getQueueResourceUsage().getAllUsed() != null) {
+        totalUsedResourcesAcrossPartition = new ResourceInfo(
+            cs.getRootQueue().getQueueResourceUsage().getAllUsed());
+        totalClusterResourcesAcrossPartition = new ResourceInfo(
+            cs.getClusterResource());
+      }
     } else {
       this.totalMB = availableMB + allocatedMB;
       this.totalVirtualCores = availableVirtualCores + allocatedVirtualCores;
@@ -310,4 +327,11 @@ public class ClusterMetricsInfo {
     this.shutdownNodes = shutdownNodes;
   }
 
+  public ResourceInfo getTotalUsedResourcesAcrossPartition() {
+    return totalUsedResourcesAcrossPartition;
+  }
+
+  public ResourceInfo getTotalClusterResourcesAcrossPartition() {
+    return totalClusterResourcesAcrossPartition;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e0cfb0a3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeInfo.java
index 2530c8e..3cec215 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeInfo.java
@@ -58,6 +58,8 @@ public class NodeInfo {
   private int numQueuedContainers;
   protected ArrayList<String> nodeLabels = new ArrayList<String>();
   protected ResourceUtilizationInfo resourceUtilization;
+  protected ResourceInfo usedResource;
+  protected ResourceInfo availableResource;
 
   public NodeInfo() {
   } // JAXB needs this
@@ -75,6 +77,8 @@ public class NodeInfo {
       this.usedVirtualCores = report.getUsedResource().getVirtualCores();
       this.availableVirtualCores =
           report.getAvailableResource().getVirtualCores();
+      this.usedResource = new ResourceInfo(report.getUsedResource());
+      this.availableResource = new ResourceInfo(report.getAvailableResource());
     }
     this.id = id.toString();
     this.rack = ni.getRackName();
@@ -183,6 +187,22 @@ public class NodeInfo {
     return this.nodeLabels;
   }
 
+  public ResourceInfo getUsedResource() {
+    return usedResource;
+  }
+
+  public void setUsedResource(ResourceInfo used) {
+    this.usedResource = used;
+  }
+
+  public ResourceInfo getAvailableResource() {
+    return availableResource;
+  }
+
+  public void setAvailableResource(ResourceInfo avail) {
+    this.availableResource = avail;
+  }
+
   public ResourceUtilizationInfo getResourceUtilization() {
     return this.resourceUtilization;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e0cfb0a3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ResourceInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ResourceInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ResourceInfo.java
index e13980a..5bed936 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ResourceInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ResourceInfo.java
@@ -26,6 +26,7 @@ import javax.xml.bind.annotation.XmlRootElement;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
+
 @XmlRootElement
 @XmlAccessorType(XmlAccessType.NONE)
 public class ResourceInfo {
@@ -34,6 +35,9 @@ public class ResourceInfo {
   long memory;
   @XmlElement
   int vCores;
+  @XmlElement
+  ResourceInformationsInfo resourceInformations =
+      new ResourceInformationsInfo();
 
   private Resource resources;
 
@@ -41,9 +45,13 @@ public class ResourceInfo {
   }
 
   public ResourceInfo(Resource res) {
-    memory = res.getMemorySize();
-    vCores = res.getVirtualCores();
-    resources = Resources.clone(res);
+    // Make sure no NPE.
+    if (res != null) {
+      memory = res.getMemorySize();
+      vCores = res.getVirtualCores();
+      resources = Resources.clone(res);
+      resourceInformations.addAll(res.getAllResourcesListCopy());
+    }
   }
 
   public long getMemorySize() {
@@ -84,4 +92,8 @@ public class ResourceInfo {
   public Resource getResource() {
     return Resource.newInstance(resources);
   }
+
+  public ResourceInformationsInfo getResourcesInformations() {
+    return resourceInformations;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e0cfb0a3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ResourceInformationsInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ResourceInformationsInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ResourceInformationsInfo.java
new file mode 100644
index 0000000..651d7df
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ResourceInformationsInfo.java
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.resourcemanager.webapp.dao;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.hadoop.yarn.api.records.ResourceInformation;
+
+@XmlRootElement(name = "resourceInformations")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class ResourceInformationsInfo {
+
+  @XmlElement(name = "resourceInformation")
+  protected ArrayList<ResourceInformation> resourceInformation =
+      new ArrayList<ResourceInformation>();
+
+  public ResourceInformationsInfo() {
+  } // JAXB needs this
+
+  public ArrayList<ResourceInformation> getApps() {
+    return resourceInformation;
+  }
+
+  public void addAll(List<ResourceInformation> resourcesInformationsInfo) {
+    resourceInformation.addAll(resourcesInformationsInfo);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e0cfb0a3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/cluster-metric.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/cluster-metric.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/cluster-metric.js
index bbc6f08..764723a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/cluster-metric.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/cluster-metric.js
@@ -169,8 +169,8 @@ export default DS.Model.extend({
       && this.get("totalUsedResourcesAcrossPartition")) {
       var usages = [];
 
-      var clusterResourceInformations = this.get("totalClusterResourcesAcrossPartition").resourcesInformations;
-      var usedResourceInformations = this.get("totalUsedResourcesAcrossPartition").resourcesInformations;
+      var clusterResourceInformations = this.get("totalClusterResourcesAcrossPartition").resourceInformations.resourceInformation;
+      var usedResourceInformations = this.get("totalUsedResourcesAcrossPartition").resourceInformations.resourceInformation;
 
       clusterResourceInformations.forEach(function(cluster) {
         var perResourceTypeUsage = {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e0cfb0a3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-rm-node.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-rm-node.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-rm-node.js
index 3b2385f..64fe7b8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-rm-node.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-rm-node.js
@@ -99,7 +99,7 @@ export default DS.Model.extend({
 
     const usedResource = this.get("usedResource");
     const availableResource = this.get("availableResource");
-    var resourceInformations = usedResource ? usedResource.resourcesInformations : [];
+    var resourceInformations = usedResource ? usedResource.resourceInformations.resourceInformation : [];
     for (var i = 0; i < resourceInformations.length; i++) {
       ri = resourceInformations[i];
       if (ri.name === "yarn.io/gpu") {
@@ -108,7 +108,7 @@ export default DS.Model.extend({
     }
 
     var available = 0;
-    resourceInformations = availableResource ? availableResource.resourcesInformations : [];
+    resourceInformations = availableResource ? availableResource.resourceInformations.resourceInformation : [];
     for (i = 0; i < resourceInformations.length; i++) {
       ri = resourceInformations[i];
       if (ri.name === "yarn.io/gpu") {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e0cfb0a3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-rm-node.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-rm-node.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-rm-node.js
index a3a1d59..c27fd92 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-rm-node.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-rm-node.js
@@ -42,8 +42,8 @@ export default DS.JSONAPISerializer.extend({
         availableVirtualCores: payload.availableVirtualCores,
         version: payload.version,
         nodeLabels: payload.nodeLabels,
-        usedResource: payload.used,
-        availableResource: payload.avail
+        usedResource: payload.usedResource,
+        availableResource: payload.availableResource
       }
     };
     return fixedPayload;


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[21/37] hadoop git commit: HDFS-13054. Handling PathIsNotEmptyDirectoryException in DFSClient delete call. Contributed by Nanda kumar.

Posted by ae...@apache.org.
HDFS-13054. Handling PathIsNotEmptyDirectoryException in DFSClient delete call. Contributed by Nanda kumar.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/e990904d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e990904d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e990904d

Branch: refs/heads/HDFS-7240
Commit: e990904dd568a1d8f98efb55c1dd2d598ae4752b
Parents: a37e7f0
Author: Arpit Agarwal <ar...@apache.org>
Authored: Fri Jan 26 11:42:27 2018 -0800
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Fri Jan 26 13:09:13 2018 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/DFSClient.java     |  4 +++-
 .../hadoop/hdfs/protocol/ClientProtocol.java       |  3 +++
 .../hadoop/hdfs/TestDistributedFileSystem.java     | 17 +++++++++++++++++
 3 files changed, 23 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e990904d/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index f0769c1..92bb99e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -82,6 +82,7 @@ import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Options.ChecksumOpt;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException;
 import org.apache.hadoop.fs.QuotaUsage;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.StorageType;
@@ -1620,7 +1621,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
           FileNotFoundException.class,
           SafeModeException.class,
           UnresolvedPathException.class,
-          SnapshotAccessControlException.class);
+          SnapshotAccessControlException.class,
+          PathIsNotEmptyDirectoryException.class);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e990904d/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
index fbef037..0d77037 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.crypto.CryptoProtocolVersion;
 import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
+import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException;
 import org.apache.hadoop.hdfs.AddBlockFlag;
 import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.ContentSummary;
@@ -625,6 +626,8 @@ public interface ClientProtocol {
    * @throws org.apache.hadoop.fs.UnresolvedLinkException If <code>src</code>
    *           contains a symlink
    * @throws SnapshotAccessControlException if path is in RO snapshot
+   * @throws PathIsNotEmptyDirectoryException if path is a non-empty directory
+   *           and <code>recursive</code> is set to false
    * @throws IOException If an I/O error occurred
    */
   @AtMostOnce

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e990904d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
index 823c747..072ee9f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
@@ -67,6 +67,7 @@ import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
 import org.apache.hadoop.fs.Options.ChecksumOpt;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.StorageStatistics.LongStatistic;
 import org.apache.hadoop.fs.StorageType;
@@ -571,6 +572,22 @@ public class TestDistributedFileSystem {
         in.close();
         fs.close();
       }
+
+      {
+        // Test PathIsNotEmptyDirectoryException while deleting non-empty dir
+        FileSystem fs = cluster.getFileSystem();
+        fs.mkdirs(new Path("/test/nonEmptyDir"));
+        fs.create(new Path("/tmp/nonEmptyDir/emptyFile")).close();
+        try {
+          fs.delete(new Path("/tmp/nonEmptyDir"), false);
+          Assert.fail("Expecting PathIsNotEmptyDirectoryException");
+        } catch (PathIsNotEmptyDirectoryException ex) {
+          // This is the proper exception to catch; move on.
+        }
+        Assert.assertTrue(fs.exists(new Path("/test/nonEmptyDir")));
+        fs.delete(new Path("/tmp/nonEmptyDir"), true);
+      }
+
     }
     finally {
       if (cluster != null) {cluster.shutdown();}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[24/37] hadoop git commit: YARN-7765. Fixed an issue that kerberos tgt not found when NM posting timeline events. Contributed by Rohith Sharma K S

Posted by ae...@apache.org.
YARN-7765. Fixed an issue that kerberos tgt not found when NM posting timeline events. Contributed by Rohith Sharma K S


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/443523f9
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/443523f9
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/443523f9

Branch: refs/heads/HDFS-7240
Commit: 443523f9c0fcc4ba2503791090b1879c6031759b
Parents: 649ef7a
Author: Jian He <ji...@apache.org>
Authored: Sat Jan 27 22:18:51 2018 -0800
Committer: Jian He <ji...@apache.org>
Committed: Sat Jan 27 22:18:51 2018 -0800

----------------------------------------------------------------------
 .../hadoop/yarn/server/nodemanager/NodeManager.java      | 11 ++++-------
 .../nodemanager/timelineservice/NMTimelinePublisher.java |  1 +
 .../timelineservice/storage/HBaseTimelineWriterImpl.java |  5 +++++
 3 files changed, 10 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/443523f9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
index 5cacd20..42b7b5f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
@@ -454,18 +454,15 @@ public class NodeManager extends CompositeService
     ((NMContext) context).setNodeStatusUpdater(nodeStatusUpdater);
     nmStore.setNodeStatusUpdater(nodeStatusUpdater);
 
-    super.serviceInit(conf);
-    // TODO add local dirs to del
-  }
-
-  @Override
-  protected void serviceStart() throws Exception {
+    // Do secure login before calling init for added services.
     try {
       doSecureLogin();
     } catch (IOException e) {
       throw new YarnRuntimeException("Failed NodeManager login", e);
     }
-    super.serviceStart();
+
+    super.serviceInit(conf);
+    // TODO add local dirs to del
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/443523f9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelinePublisher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelinePublisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelinePublisher.java
index b8192ca..2ce3c5e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelinePublisher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelinePublisher.java
@@ -100,6 +100,7 @@ public class NMTimelinePublisher extends CompositeService {
     this.nmLoginUGI =  UserGroupInformation.isSecurityEnabled() ?
         UserGroupInformation.getLoginUser() :
         UserGroupInformation.getCurrentUser();
+    LOG.info("Initialized NMTimelinePublisher UGI to " + nmLoginUGI);
     super.serviceInit(conf);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/443523f9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineWriterImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineWriterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineWriterImpl.java
index 9e9134c..f938185 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineWriterImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineWriterImpl.java
@@ -129,6 +129,11 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
         new FlowActivityTable().getTableMutator(hbaseConf, conn);
     subApplicationTable =
         new SubApplicationTable().getTableMutator(hbaseConf, conn);
+
+    UserGroupInformation ugi = UserGroupInformation.isSecurityEnabled() ?
+        UserGroupInformation.getLoginUser() :
+        UserGroupInformation.getCurrentUser();
+    LOG.info("Initialized HBaseTimelineWriterImpl UGI to " + ugi);
   }
 
   /**


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[16/37] hadoop git commit: YARN-7742. [UI2] Duplicated containers are rendered per attempt. (Vasudevan Skm / Sunil G via wangda)

Posted by ae...@apache.org.
YARN-7742. [UI2] Duplicated containers are rendered per attempt. (Vasudevan Skm / Sunil G via wangda)

Change-Id: Ib2bb3c64d7f09a7c3488f498d0079bcd2e6ac023


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/8b5b045b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8b5b045b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8b5b045b

Branch: refs/heads/HDFS-7240
Commit: 8b5b045bd2b5aaf567c289fd34bab4bda216c835
Parents: 75618c0
Author: Wangda Tan <wa...@apache.org>
Authored: Fri Jan 26 15:46:06 2018 +0800
Committer: Wangda Tan <wa...@apache.org>
Committed: Fri Jan 26 15:46:06 2018 +0800

----------------------------------------------------------------------
 .../src/main/webapp/app/components/timeline-view.js |  5 ++++-
 .../app/serializers/yarn-timeline-container.js      | 16 +++++++---------
 .../src/main/webapp/app/utils/date-utils.js         |  2 +-
 3 files changed, 12 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8b5b045b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/timeline-view.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/timeline-view.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/timeline-view.js
index 420d793..e23d991 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/timeline-view.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/timeline-view.js
@@ -30,6 +30,7 @@ export default Ember.Component.extend({
 
   clusterMetrics: undefined,
   modelArr: [],
+  containerIdArr: [],
   colors: d3.scale.category10().range(),
   _selected: undefined,
   gridColumns: [],
@@ -232,19 +233,21 @@ export default Ember.Component.extend({
     // init tooltip
     this.initTooltip();
     this.modelArr = [];
+    this.containerIdArr = [];
 
     // init model
     if (this.get("rmModel")) {
       this.get("rmModel").forEach(function(o) {
         if(!this.modelArr.contains(o)) {
           this.modelArr.push(o);
+          this.containerIdArr.push(o.id);
         }
       }.bind(this));
     }
 
     if (this.get("tsModel")) {
       this.get("tsModel").forEach(function(o) {
-        if(!this.modelArr.contains(o)) {
+        if(!this.containerIdArr.contains(o.id)) {
           this.modelArr.push(o);
         }
       }.bind(this));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8b5b045b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-timeline-container.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-timeline-container.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-timeline-container.js
index f2c60f6..471f910 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-timeline-container.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-timeline-container.js
@@ -23,12 +23,10 @@ import Converter from 'yarn-ui/utils/converter';
 export default DS.JSONAPISerializer.extend({
   internalNormalizeSingleResponse(store, primaryModelClass, payload) {
     var payloadEvents = payload.events,
-        createdEvent = payloadEvents.filterBy('id', 'YARN_RM_CONTAINER_CREATED')[0],
+        createdEvent = payloadEvents.filterBy('id', 'YARN_CONTAINER_CREATED')[0],
         startedTime = createdEvent? createdEvent.timestamp : Date.now(),
-        finishedEvent = payloadEvents.filterBy('id', 'YARN_RM_CONTAINER_FINISHED')[0],
-        finishedTime = finishedEvent? finishedEvent.timestamp : Date.now(),
-        containerExitStatus = finishedEvent? finishedEvent.info.YARN_CONTAINER_EXIT_STATUS : '',
-        containerState = finishedEvent? finishedEvent.info.YARN_CONTAINER_STATE : '';
+        finishedEvent = payloadEvents.filterBy('id', 'YARN_CONTAINER_FINISHED')[0],
+        finishedTime = finishedEvent? finishedEvent.timestamp : Date.now()
 
     var fixedPayload = {
       id: payload.id,
@@ -38,11 +36,11 @@ export default DS.JSONAPISerializer.extend({
         allocatedVCores: payload.info.YARN_CONTAINER_ALLOCATED_VCORE,
         assignedNodeId: payload.info.YARN_CONTAINER_ALLOCATED_HOST,
         priority: payload.info.YARN_CONTAINER_ALLOCATED_PRIORITY,
-        startedTime:  Converter.timeStampToDate(startedTime),
-        finishedTime: Converter.timeStampToDate(finishedTime),
+        startedTime:  Converter.timeStampToDate(payload.createdtime),
+        finishedTime: Converter.timeStampToDate(payload.info.YARN_CONTAINER_FINISHED_TIME),
         nodeHttpAddress: payload.info.YARN_CONTAINER_ALLOCATED_HOST_HTTP_ADDRESS,
-        containerExitStatus: containerExitStatus,
-        containerState: containerState
+        containerExitStatus: payload.info.YARN_CONTAINER_EXIT_STATUS,
+        containerState: payload.info.YARN_CONTAINER_STATE
       }
     };
     return fixedPayload;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8b5b045b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/utils/date-utils.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/utils/date-utils.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/utils/date-utils.js
index 83dd200..6a9780c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/utils/date-utils.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/utils/date-utils.js
@@ -24,4 +24,4 @@ const getDefaultTimezone = () => {
 };
 
 export const convertTimestampWithTz = (timestamp, format = "YYYY/MM/DD") =>
-  moment.tz(timestamp, getDefaultTimezone()).format(format);
+  moment.tz(parseInt(timestamp), getDefaultTimezone()).format(format);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[06/37] hadoop git commit: HDFS-13063. Fix the incorrect spelling in HDFSHighAvailabilityWithQJM.md

Posted by ae...@apache.org.
HDFS-13063. Fix the incorrect spelling in HDFSHighAvailabilityWithQJM.md

Signed-off-by: Akira Ajisaka <aa...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/eb2dd086
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/eb2dd086
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/eb2dd086

Branch: refs/heads/HDFS-7240
Commit: eb2dd0868864da994612f1dd1452ea38d31f5e85
Parents: 59828be
Author: Jianfei Jiang <ji...@gmail.com>
Authored: Thu Jan 25 15:27:28 2018 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Thu Jan 25 15:30:42 2018 +0900

----------------------------------------------------------------------
 .../hadoop-hdfs/src/site/markdown/HDFSHighAvailabilityWithQJM.md   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb2dd086/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSHighAvailabilityWithQJM.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSHighAvailabilityWithQJM.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSHighAvailabilityWithQJM.md
index b28790f..8df3209 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSHighAvailabilityWithQJM.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSHighAvailabilityWithQJM.md
@@ -49,7 +49,7 @@ Architecture
 
 In a typical HA cluster, two or more separate machines are configured as NameNodes. At any point in time, exactly one of the NameNodes is in an *Active* state, and the others are in a *Standby* state. The Active NameNode is responsible for all client operations in the cluster, while the Standbys are simply acting as workers, maintaining enough state to provide a fast failover if necessary.
 
-In order for the Standby node to keep its state synchronized with the Active node, both nodes communicate with a group of separate daemons called "JournalNodes" (JNs). When any namespace modification is performed by the Active node, it durably logs a record of the modification to a majority of these JNs. The Standby node is capable of reading the edits from the JNs, and is constantly watching them for changes to the edit log. As the Standby Node sees the edits, it applies them to its own namespace. In the event of a failover, the Standby will ensure that it has read all of the edits from the JounalNodes before promoting itself to the Active state. This ensures that the namespace state is fully synchronized before a failover occurs.
+In order for the Standby node to keep its state synchronized with the Active node, both nodes communicate with a group of separate daemons called "JournalNodes" (JNs). When any namespace modification is performed by the Active node, it durably logs a record of the modification to a majority of these JNs. The Standby node is capable of reading the edits from the JNs, and is constantly watching them for changes to the edit log. As the Standby Node sees the edits, it applies them to its own namespace. In the event of a failover, the Standby will ensure that it has read all of the edits from the JournalNodes before promoting itself to the Active state. This ensures that the namespace state is fully synchronized before a failover occurs.
 
 In order to provide a fast failover, it is also necessary that the Standby node have up-to-date information regarding the location of blocks in the cluster. In order to achieve this, the DataNodes are configured with the location of all NameNodes, and send block location information and heartbeats to all.
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[08/37] hadoop git commit: YARN-7728. Expose container preemptions related information in Capacity Scheduler queue metrics. Contributed by Eric Payne.

Posted by ae...@apache.org.
YARN-7728. Expose container preemptions related information in Capacity Scheduler queue metrics. Contributed by Eric Payne.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/82cc6f69
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/82cc6f69
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/82cc6f69

Branch: refs/heads/HDFS-7240
Commit: 82cc6f6968d2897ae006c5ef29eba74023f3483e
Parents: 7721fff
Author: Sunil G <su...@apache.org>
Authored: Thu Jan 25 14:32:54 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Thu Jan 25 14:32:54 2018 +0530

----------------------------------------------------------------------
 .../resourcemanager/scheduler/QueueMetrics.java  | 18 ++++++++++++++++++
 .../scheduler/capacity/CapacityScheduler.java    | 19 +++++++++++++++++++
 .../TestCapacitySchedulerSurgicalPreemption.java |  4 ++++
 3 files changed, 41 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/82cc6f69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/QueueMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/QueueMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/QueueMetrics.java
index eafe8ed..20a5a1f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/QueueMetrics.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/QueueMetrics.java
@@ -68,6 +68,10 @@ public class QueueMetrics implements MetricsSource {
     MutableCounterLong aggregateOffSwitchContainersAllocated;
   @Metric("Aggregate # of preempted containers") MutableCounterLong
       aggregateContainersPreempted;
+  @Metric("Aggregate # of preempted memory seconds") MutableCounterLong
+      aggregateMemoryMBSecondsPreempted;
+  @Metric("Aggregate # of preempted vcore seconds") MutableCounterLong
+      aggregateVcoreSecondsPreempted;
   @Metric("# of active users") MutableGaugeInt activeUsers;
   @Metric("# of active applications") MutableGaugeInt activeApplications;
   @Metric("App Attempt First Container Allocation Delay")
@@ -534,6 +538,20 @@ public class QueueMetrics implements MetricsSource {
     }
   }
 
+  public void updatePreemptedMemoryMBSeconds(long mbSeconds) {
+    aggregateMemoryMBSecondsPreempted.incr(mbSeconds);
+    if (parent != null) {
+      parent.updatePreemptedMemoryMBSeconds(mbSeconds);
+    }
+  }
+
+  public void updatePreemptedVcoreSeconds(long vcoreSeconds) {
+    aggregateVcoreSecondsPreempted.incr(vcoreSeconds);
+    if (parent != null) {
+      parent.updatePreemptedVcoreSeconds(vcoreSeconds);
+    }
+  }
+
   public void reserveResource(String partition, String user, Resource res) {
     if(partition == null || partition.equals(RMNodeLabelsManager.NO_LABEL)) {
       reserveResource(user, res);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/82cc6f69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
index ba2f85a..99f4456 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
@@ -34,6 +34,7 @@ import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang.time.DateUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
@@ -46,6 +47,7 @@ import org.apache.hadoop.util.Time;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
@@ -1821,6 +1823,23 @@ public class CapacityScheduler extends
     LeafQueue queue = (LeafQueue) application.getQueue();
     queue.completedContainer(getClusterResource(), application, node,
         rmContainer, containerStatus, event, null, true);
+    if (ContainerExitStatus.PREEMPTED == containerStatus.getExitStatus()) {
+      updateQueuePreemptionMetrics(queue, rmContainer);
+    }
+  }
+
+  private void updateQueuePreemptionMetrics(
+      CSQueue queue, RMContainer rmc) {
+    QueueMetrics qMetrics = queue.getMetrics();
+    long usedMillis = rmc.getFinishTime() - rmc.getCreationTime();
+    Resource containerResource = rmc.getAllocatedResource();
+    qMetrics.preemptContainer();
+    long mbSeconds = (containerResource.getMemorySize() * usedMillis)
+        / DateUtils.MILLIS_PER_SECOND;
+    long vcSeconds = (containerResource.getVirtualCores() * usedMillis)
+        / DateUtils.MILLIS_PER_SECOND;
+    qMetrics.updatePreemptedMemoryMBSeconds(mbSeconds);
+    qMetrics.updatePreemptedVcoreSeconds(vcSeconds);
   }
 
   @Lock(Lock.NoLock.class)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/82cc6f69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerSurgicalPreemption.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerSurgicalPreemption.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerSurgicalPreemption.java
index c20e091..9b183c0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerSurgicalPreemption.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerSurgicalPreemption.java
@@ -171,6 +171,10 @@ public class TestCapacitySchedulerSurgicalPreemption
     waitNumberOfLiveContainersOnNodeFromApp(cs.getNode(rmNode2.getNodeID()),
         am1.getApplicationAttemptId(), 16);
 
+    // Ensure preemption metrics were recored.
+    Assert.assertEquals("Number of preempted containers incorrectly recorded:",
+        4, cs.getQueue("root").getMetrics().getAggregatePreemptedContainers());
+
     rm1.close();
   }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[31/37] hadoop git commit: HDFS-12574. Add CryptoInputStream to WebHdfsFileSystem read call. Contributed by Rushabh S Shah

Posted by ae...@apache.org.
HDFS-12574. Add CryptoInputStream to WebHdfsFileSystem read call. Contributed by Rushabh S Shah


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/fde95d46
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/fde95d46
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/fde95d46

Branch: refs/heads/HDFS-7240
Commit: fde95d463c3123b315b3d07cb5b7b7dc19f7cb73
Parents: 7fd287b
Author: Kihwal Lee <ki...@apache.org>
Authored: Mon Jan 29 17:22:29 2018 -0600
Committer: Kihwal Lee <ki...@apache.org>
Committed: Mon Jan 29 17:23:29 2018 -0600

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  48 ++---
 .../org/apache/hadoop/hdfs/HdfsKMSUtil.java     |  41 ++++
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      | 101 ++++++++--
 .../hdfs/web/TestWebHdfsContentLength.java      |   2 +
 .../web/resources/NamenodeWebHdfsMethods.java   |  85 ++++++---
 .../apache/hadoop/hdfs/TestEncryptionZones.java | 188 +++++++++++++++++++
 .../web/resources/TestWebHdfsDataLocality.java  |  23 ++-
 .../org/apache/hadoop/hdfs/web/TestWebHDFS.java |   1 -
 .../hadoop/hdfs/web/TestWebHdfsTokens.java      |   4 +-
 9 files changed, 403 insertions(+), 90 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fde95d46/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index 92bb99e..2497c40 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -38,7 +38,6 @@ import java.net.Socket;
 import java.net.SocketAddress;
 import java.net.URI;
 import java.net.UnknownHostException;
-import java.security.GeneralSecurityException;
 import java.util.ArrayList;
 import java.util.EnumSet;
 import java.util.HashMap;
@@ -62,8 +61,6 @@ import org.apache.hadoop.crypto.CryptoInputStream;
 import org.apache.hadoop.crypto.CryptoOutputStream;
 import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.crypto.key.KeyProvider.KeyVersion;
-import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
-import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.ContentSummary;
@@ -911,45 +908,18 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   }
 
   /**
-   * Decrypts a EDEK by consulting the KeyProvider.
-   */
-  private KeyVersion decryptEncryptedDataEncryptionKey(FileEncryptionInfo
-      feInfo) throws IOException {
-    try (TraceScope ignored = tracer.newScope("decryptEDEK")) {
-      KeyProvider provider = getKeyProvider();
-      if (provider == null) {
-        throw new IOException("No KeyProvider is configured, cannot access" +
-            " an encrypted file");
-      }
-      EncryptedKeyVersion ekv = EncryptedKeyVersion.createForDecryption(
-          feInfo.getKeyName(), feInfo.getEzKeyVersionName(), feInfo.getIV(),
-          feInfo.getEncryptedDataEncryptionKey());
-      try {
-        KeyProviderCryptoExtension cryptoProvider = KeyProviderCryptoExtension
-            .createKeyProviderCryptoExtension(provider);
-        return cryptoProvider.decryptEncryptedKey(ekv);
-      } catch (GeneralSecurityException e) {
-        throw new IOException(e);
-      }
-    }
-  }
-
-  /**
    * Wraps the stream in a CryptoInputStream if the underlying file is
    * encrypted.
    */
   public HdfsDataInputStream createWrappedInputStream(DFSInputStream dfsis)
       throws IOException {
-    final FileEncryptionInfo feInfo = dfsis.getFileEncryptionInfo();
+    FileEncryptionInfo feInfo = dfsis.getFileEncryptionInfo();
     if (feInfo != null) {
-      // File is encrypted, wrap the stream in a crypto stream.
-      // Currently only one version, so no special logic based on the version #
-      HdfsKMSUtil.getCryptoProtocolVersion(feInfo);
-      final CryptoCodec codec = HdfsKMSUtil.getCryptoCodec(conf, feInfo);
-      final KeyVersion decrypted = decryptEncryptedDataEncryptionKey(feInfo);
-      final CryptoInputStream cryptoIn =
-          new CryptoInputStream(dfsis, codec, decrypted.getMaterial(),
-              feInfo.getIV());
+      CryptoInputStream cryptoIn;
+      try (TraceScope ignored = getTracer().newScope("decryptEDEK")) {
+        cryptoIn = HdfsKMSUtil.createWrappedInputStream(dfsis,
+            getKeyProvider(), feInfo, getConfiguration());
+      }
       return new HdfsDataInputStream(cryptoIn);
     } else {
       // No FileEncryptionInfo so no encryption.
@@ -978,7 +948,11 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
       // Currently only one version, so no special logic based on the version #
       HdfsKMSUtil.getCryptoProtocolVersion(feInfo);
       final CryptoCodec codec = HdfsKMSUtil.getCryptoCodec(conf, feInfo);
-      KeyVersion decrypted = decryptEncryptedDataEncryptionKey(feInfo);
+      KeyVersion decrypted;
+      try (TraceScope ignored = tracer.newScope("decryptEDEK")) {
+        decrypted = HdfsKMSUtil.decryptEncryptedDataEncryptionKey(feInfo,
+          getKeyProvider());
+      }
       final CryptoOutputStream cryptoOut =
           new CryptoOutputStream(dfsos, codec,
               decrypted.getMaterial(), feInfo.getIV(), startPos);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fde95d46/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/HdfsKMSUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/HdfsKMSUtil.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/HdfsKMSUtil.java
index 71d2972..de27f7e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/HdfsKMSUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/HdfsKMSUtil.java
@@ -20,15 +20,21 @@ package org.apache.hadoop.hdfs;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_KEY_PREFIX;
 
 import java.io.IOException;
+import java.io.InputStream;
 import java.net.URI;
+import java.security.GeneralSecurityException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.CipherSuite;
 import org.apache.hadoop.crypto.CryptoCodec;
+import org.apache.hadoop.crypto.CryptoInputStream;
 import org.apache.hadoop.crypto.CryptoProtocolVersion;
 import org.apache.hadoop.crypto.key.KeyProvider;
+import org.apache.hadoop.crypto.key.KeyProvider.KeyVersion;
+import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
+import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion;
 import org.apache.hadoop.crypto.key.KeyProviderDelegationTokenExtension;
 import org.apache.hadoop.crypto.key.KeyProviderTokenIssuer;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
@@ -187,4 +193,39 @@ public final class HdfsKMSUtil {
     return new Text(DFS_KMS_PREFIX + namenodeUri.getScheme()
         +"://" + namenodeUri.getAuthority());
   }
+
+  public static CryptoInputStream createWrappedInputStream(InputStream is,
+      KeyProvider keyProvider, FileEncryptionInfo fileEncryptionInfo,
+      Configuration conf) throws IOException {
+    // File is encrypted, wrap the stream in a crypto stream.
+    // Currently only one version, so no special logic based on the version#
+    HdfsKMSUtil.getCryptoProtocolVersion(fileEncryptionInfo);
+    final CryptoCodec codec = HdfsKMSUtil.getCryptoCodec(
+        conf, fileEncryptionInfo);
+    final KeyVersion decrypted =
+        decryptEncryptedDataEncryptionKey(fileEncryptionInfo, keyProvider);
+    return new CryptoInputStream(is, codec, decrypted.getMaterial(),
+        fileEncryptionInfo.getIV());
+  }
+
+  /**
+   * Decrypts a EDEK by consulting the KeyProvider.
+   */
+  static KeyVersion decryptEncryptedDataEncryptionKey(FileEncryptionInfo
+      feInfo, KeyProvider keyProvider) throws IOException {
+    if (keyProvider == null) {
+      throw new IOException("No KeyProvider is configured, cannot access" +
+          " an encrypted file");
+    }
+    EncryptedKeyVersion ekv = EncryptedKeyVersion.createForDecryption(
+        feInfo.getKeyName(), feInfo.getEzKeyVersionName(), feInfo.getIV(),
+        feInfo.getEncryptedDataEncryptionKey());
+    try {
+      KeyProviderCryptoExtension cryptoProvider = KeyProviderCryptoExtension
+          .createKeyProviderCryptoExtension(keyProvider);
+      return cryptoProvider.decryptEncryptedKey(ekv);
+    } catch (GeneralSecurityException e) {
+      throw new IOException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fde95d46/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
index 2ab7a83..b006495 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
@@ -37,8 +37,11 @@ import java.net.InetSocketAddress;
 import java.net.MalformedURLException;
 import java.net.URI;
 import java.net.URL;
+import java.nio.charset.StandardCharsets;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
+import java.util.Base64;
+import java.util.Base64.Decoder;
 import java.util.Collection;
 import java.util.EnumSet;
 import java.util.HashSet;
@@ -66,6 +69,7 @@ import org.apache.hadoop.fs.DelegationTokenRenewer;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSInputStream;
+import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsServerDefaults;
@@ -92,6 +96,8 @@ import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.FileEncryptionInfoProto;
+import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.web.resources.*;
 import org.apache.hadoop.hdfs.web.resources.HttpOpParam.Op;
@@ -133,6 +139,8 @@ public class WebHdfsFileSystem extends FileSystem
   /** Http URI: http://namenode:port/{PATH_PREFIX}/path/to/file */
   public static final String PATH_PREFIX = "/" + WebHdfsConstants.WEBHDFS_SCHEME
       + "/v" + VERSION;
+  public static final String EZ_HEADER = "X-Hadoop-Accept-EZ";
+  public static final String FEFINFO_HEADER = "X-Hadoop-feInfo";
 
   /**
    * Default connection factory may be overridden in tests to use smaller
@@ -613,12 +621,19 @@ public class WebHdfsFileSystem extends FileSystem
 
     private boolean checkRetry;
     private String redirectHost;
+    private boolean followRedirect = true;
 
     protected AbstractRunner(final HttpOpParam.Op op, boolean redirected) {
       this.op = op;
       this.redirected = redirected;
     }
 
+    protected AbstractRunner(final HttpOpParam.Op op, boolean redirected,
+        boolean followRedirect) {
+      this(op, redirected);
+      this.followRedirect = followRedirect;
+    }
+
     T run() throws IOException {
       UserGroupInformation connectUgi = ugi.getRealUser();
       if (connectUgi == null) {
@@ -685,9 +700,17 @@ public class WebHdfsFileSystem extends FileSystem
           // See http://tinyurl.com/java7-http-keepalive
           conn.disconnect();
         }
+        if (!followRedirect) {
+          return conn;
+        }
       }
       try {
-        return connect(op, url);
+        final HttpURLConnection conn = connect(op, url);
+        // output streams will validate on close
+        if (!op.getDoOutput()) {
+          validateResponse(op, conn, false);
+        }
+        return conn;
       } catch (IOException ioe) {
         if (redirectHost != null) {
           if (excludeDatanodes.getValue() != null) {
@@ -713,6 +736,7 @@ public class WebHdfsFileSystem extends FileSystem
         // The value of the header is unimportant.  Only its presence matters.
         conn.setRequestProperty(restCsrfCustomHeader, "\"\"");
       }
+      conn.setRequestProperty(EZ_HEADER, "true");
       switch (op.getType()) {
       // if not sending a message body for a POST or PUT operation, need
       // to ensure the server/proxy knows this
@@ -760,10 +784,6 @@ public class WebHdfsFileSystem extends FileSystem
         final URL url = getUrl();
         try {
           final HttpURLConnection conn = connect(url);
-          // output streams will validate on close
-          if (!op.getDoOutput()) {
-            validateResponse(op, conn, false);
-          }
           return getResponse(conn);
         } catch (AccessControlException ace) {
           // no retries for auth failures
@@ -809,7 +829,6 @@ public class WebHdfsFileSystem extends FileSystem
               a.action == RetryPolicy.RetryAction.RetryDecision.RETRY;
           boolean isFailoverAndRetry =
               a.action == RetryPolicy.RetryAction.RetryDecision.FAILOVER_AND_RETRY;
-
           if (isRetry || isFailoverAndRetry) {
             LOG.info("Retrying connect to namenode: {}. Already retried {}"
                     + " time(s); retry policy is {}, delay {}ms.",
@@ -990,16 +1009,16 @@ public class WebHdfsFileSystem extends FileSystem
   /**
    * Used by open() which tracks the resolved url itself
    */
-  final class URLRunner extends AbstractRunner<HttpURLConnection> {
+  class URLRunner extends AbstractRunner<HttpURLConnection> {
     private final URL url;
     @Override
-    protected URL getUrl() {
+    protected URL getUrl() throws IOException {
       return url;
     }
 
     protected URLRunner(final HttpOpParam.Op op, final URL url,
-        boolean redirected) {
-      super(op, redirected);
+        boolean redirected, boolean followRedirect) {
+      super(op, redirected, followRedirect);
       this.url = url;
     }
 
@@ -1412,12 +1431,20 @@ public class WebHdfsFileSystem extends FileSystem
     ).run();
   }
 
+  @SuppressWarnings("resource")
   @Override
   public FSDataInputStream open(final Path f, final int bufferSize
   ) throws IOException {
     statistics.incrementReadOps(1);
     storageStatistics.incrementOpCounter(OpType.OPEN);
-    return new FSDataInputStream(new WebHdfsInputStream(f, bufferSize));
+    WebHdfsInputStream webfsInputStream =
+        new WebHdfsInputStream(f, bufferSize);
+    if (webfsInputStream.getFileEncryptionInfo() == null) {
+      return new FSDataInputStream(webfsInputStream);
+    } else {
+      return new FSDataInputStream(
+          webfsInputStream.createWrappedInputStream());
+    }
   }
 
   @Override
@@ -1462,7 +1489,8 @@ public class WebHdfsFileSystem extends FileSystem
         final boolean resolved) throws IOException {
       final URL offsetUrl = offset == 0L? url
           : new URL(url + "&" + new OffsetParam(offset));
-      return new URLRunner(GetOpParam.Op.OPEN, offsetUrl, resolved).run();
+      return new URLRunner(GetOpParam.Op.OPEN, offsetUrl, resolved,
+          true).run();
     }
   }
 
@@ -1928,6 +1956,15 @@ public class WebHdfsFileSystem extends FileSystem
     void setReadRunner(ReadRunner rr) {
       this.readRunner = rr;
     }
+
+    FileEncryptionInfo getFileEncryptionInfo() {
+      return readRunner.getFileEncryptionInfo();
+    }
+
+    InputStream createWrappedInputStream() throws IOException {
+      return HdfsKMSUtil.createWrappedInputStream(
+          this, getKeyProvider(), getFileEncryptionInfo(), getConf());
+    }
   }
 
   enum RunnerState {
@@ -1964,7 +2001,7 @@ public class WebHdfsFileSystem extends FileSystem
     private byte[] readBuffer;
     private int readOffset;
     private int readLength;
-    private RunnerState runnerState = RunnerState.DISCONNECTED;
+    private RunnerState runnerState = RunnerState.SEEK;
     private URL originalUrl = null;
     private URL resolvedUrl = null;
 
@@ -1972,6 +2009,7 @@ public class WebHdfsFileSystem extends FileSystem
     private final int bufferSize;
     private long pos = 0;
     private long fileLength = 0;
+    private FileEncryptionInfo feInfo = null;
 
     /* The following methods are WebHdfsInputStream helpers. */
 
@@ -1979,6 +2017,36 @@ public class WebHdfsFileSystem extends FileSystem
       super(GetOpParam.Op.OPEN, p, new BufferSizeParam(bs));
       this.path = p;
       this.bufferSize = bs;
+      getRedirectedUrl();
+    }
+
+    private void getRedirectedUrl() throws IOException {
+      URLRunner urlRunner = new URLRunner(GetOpParam.Op.OPEN, null, false,
+          false) {
+        @Override
+        protected URL getUrl() throws IOException {
+          return toUrl(op, path, new BufferSizeParam(bufferSize));
+        }
+      };
+      HttpURLConnection conn = urlRunner.run();
+      String feInfoStr = conn.getHeaderField(FEFINFO_HEADER);
+      if (feInfoStr != null) {
+        Decoder decoder = Base64.getDecoder();
+        byte[] decodedBytes = decoder.decode(
+            feInfoStr.getBytes(StandardCharsets.UTF_8));
+        feInfo = PBHelperClient
+            .convert(FileEncryptionInfoProto.parseFrom(decodedBytes));
+      }
+      String location = conn.getHeaderField("Location");
+      if (location != null) {
+        // This saves the location for datanode where redirect was issued.
+        // Need to remove offset because seek can be called after open.
+        resolvedUrl = removeOffsetParam(new URL(location));
+      } else {
+        // This is cached for proxies like httpfsfilesystem.
+        cachedConnection = conn;
+      }
+      originalUrl = super.getUrl();
     }
 
     int read(byte[] b, int off, int len) throws IOException {
@@ -2011,7 +2079,8 @@ public class WebHdfsFileSystem extends FileSystem
       if (runnerState == RunnerState.SEEK) {
         try {
           final URL rurl = new URL(resolvedUrl + "&" + new OffsetParam(pos));
-          cachedConnection = new URLRunner(GetOpParam.Op.OPEN, rurl, true).run();
+          cachedConnection = new URLRunner(GetOpParam.Op.OPEN, rurl, true,
+              false).run();
         } catch (IOException ioe) {
           closeInputStream(RunnerState.DISCONNECTED);
         }
@@ -2195,5 +2264,9 @@ public class WebHdfsFileSystem extends FileSystem
     long getPos() {
       return pos;
     }
+
+    protected FileEncryptionInfo getFileEncryptionInfo() {
+      return feInfo;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fde95d46/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsContentLength.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsContentLength.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsContentLength.java
index 19f18b0..6ee8858 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsContentLength.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsContentLength.java
@@ -102,12 +102,14 @@ public class TestWebHdfsContentLength {
   public void testGetOpWithRedirect() {
     Future<String> future1 = contentLengthFuture(redirectResponse);
     Future<String> future2 = contentLengthFuture(errResponse);
+    Future<String> future3 = contentLengthFuture(errResponse);
     try {
       fs.open(p).read();
       Assert.fail();
     } catch (IOException ioe) {} // expected
     Assert.assertEquals(null, getContentLength(future1));
     Assert.assertEquals(null, getContentLength(future2));
+    Assert.assertEquals(null, getContentLength(future3));
   }
   
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fde95d46/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
index e2ba510..d1f16a3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
@@ -28,6 +28,8 @@ import java.net.URISyntaxException;
 import java.net.UnknownHostException;
 import java.security.Principal;
 import java.security.PrivilegedExceptionAction;
+import java.util.Base64;
+import java.util.Base64.Encoder;
 import java.util.EnumSet;
 import java.util.HashSet;
 import java.util.List;
@@ -50,11 +52,14 @@ import javax.ws.rs.core.Context;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
 import javax.ws.rs.core.StreamingOutput;
+import javax.ws.rs.core.Response.ResponseBuilder;
+import javax.ws.rs.core.Response.Status;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsServerDefaults;
@@ -73,6 +78,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
@@ -117,9 +123,9 @@ public class NamenodeWebHdfsMethods {
   private Principal userPrincipal;
   private String remoteAddr;
 
-  private static volatile String serverDefaultsResponse = null;
   private @Context ServletContext context;
   private @Context HttpServletResponse response;
+  private boolean supportEZ;
 
   public NamenodeWebHdfsMethods(@Context HttpServletRequest request) {
     // the request object is a proxy to thread-locals so we have to extract
@@ -130,6 +136,8 @@ public class NamenodeWebHdfsMethods {
     // get the remote address, if coming in via a trusted proxy server then
     // the address with be that of the proxied client
     remoteAddr = JspHelper.getRemoteAddr(request);
+    supportEZ =
+        Boolean.valueOf(request.getHeader(WebHdfsFileSystem.EZ_HEADER));
   }
 
   private void init(final UserGroupInformation ugi,
@@ -228,7 +236,7 @@ public class NamenodeWebHdfsMethods {
   static DatanodeInfo chooseDatanode(final NameNode namenode,
       final String path, final HttpOpParam.Op op, final long openOffset,
       final long blocksize, final String excludeDatanodes,
-      final String remoteAddr) throws IOException {
+      final String remoteAddr, final HdfsFileStatus status) throws IOException {
     FSNamesystem fsn = namenode.getNamesystem();
     if (fsn == null) {
       throw new IOException("Namesystem has not been intialized yet.");
@@ -265,7 +273,6 @@ public class NamenodeWebHdfsMethods {
         || op == PostOpParam.Op.APPEND) {
       //choose a datanode containing a replica 
       final NamenodeProtocols np = getRPCServer(namenode);
-      final HdfsFileStatus status = np.getFileInfo(path);
       if (status == null) {
         throw new FileNotFoundException("File " + path + " not found.");
       }
@@ -285,7 +292,7 @@ public class NamenodeWebHdfsMethods {
           return bestNode(locations.get(0).getLocations(), excludes);
         }
       }
-    } 
+    }
 
     return (DatanodeDescriptor)bm.getDatanodeManager().getNetworkTopology(
         ).chooseRandom(NodeBase.ROOT, excludes);
@@ -322,15 +329,22 @@ public class NamenodeWebHdfsMethods {
     return t;
   }
 
-  private URI redirectURI(final NameNode namenode,
+  private URI redirectURI(ResponseBuilder rb, final NameNode namenode,
       final UserGroupInformation ugi, final DelegationParam delegation,
       final UserParam username, final DoAsParam doAsUser,
       final String path, final HttpOpParam.Op op, final long openOffset,
       final long blocksize, final String excludeDatanodes,
       final Param<?, ?>... parameters) throws URISyntaxException, IOException {
     final DatanodeInfo dn;
+    final NamenodeProtocols np = getRPCServer(namenode);
+    HdfsFileStatus status = null;
+    if (op == GetOpParam.Op.OPEN
+        || op == GetOpParam.Op.GETFILECHECKSUM
+        || op == PostOpParam.Op.APPEND) {
+      status = np.getFileInfo(path);
+    }
     dn = chooseDatanode(namenode, path, op, openOffset, blocksize,
-        excludeDatanodes, remoteAddr);
+        excludeDatanodes, remoteAddr, status);
     if (dn == null) {
       throw new IOException("Failed to find datanode, suggest to check cluster"
           + " health. excludeDatanodes=" + excludeDatanodes);
@@ -349,15 +363,27 @@ public class NamenodeWebHdfsMethods {
           namenode, ugi, null);
       delegationQuery = "&" + new DelegationParam(t.encodeToUrlString());
     }
-    final String query = op.toQueryString() + delegationQuery
-        + "&" + new NamenodeAddressParam(namenode)
-        + Param.toSortedString("&", parameters);
-    final String uripath = WebHdfsFileSystem.PATH_PREFIX + path;
+
+    StringBuilder queryBuilder = new StringBuilder();
+    queryBuilder.append(op.toQueryString());
+    queryBuilder.append(delegationQuery);
+    queryBuilder.append("&").append(new NamenodeAddressParam(namenode));
+    queryBuilder.append(Param.toSortedString("&", parameters));
+
+    boolean prependReservedRawPath  = false;
+    if (op == GetOpParam.Op.OPEN && supportEZ
+        && status.getFileEncryptionInfo() != null) {
+      prependReservedRawPath = true;
+      rb.header(WebHdfsFileSystem.FEFINFO_HEADER,
+          encodeFeInfo(status.getFileEncryptionInfo()));
+    }
+    final String uripath = WebHdfsFileSystem.PATH_PREFIX +
+        (prependReservedRawPath ? "/.reserved/raw" + path : path);
 
     int port = "http".equals(scheme) ? dn.getInfoPort() : dn
         .getInfoSecurePort();
     final URI uri = new URI(scheme, null, dn.getHostName(), port, uripath,
-        query, null);
+        queryBuilder.toString(), null);
 
     if (LOG.isTraceEnabled()) {
       LOG.trace("redirectURI=" + uri);
@@ -581,7 +607,7 @@ public class NamenodeWebHdfsMethods {
     switch(op.getValue()) {
     case CREATE:
     {
-      final URI uri = redirectURI(namenode, ugi, delegation, username,
+      final URI uri = redirectURI(null, namenode, ugi, delegation, username,
           doAsUser, fullpath, op.getValue(), -1L, blockSize.getValue(conf),
           exclDatanodes.getValue(), permission, unmaskedPermission,
           overwrite, bufferSize, replication, blockSize, createParent,
@@ -830,7 +856,7 @@ public class NamenodeWebHdfsMethods {
     case APPEND:
     {
       final NameNode namenode = (NameNode)context.getAttribute("name.node");
-      final URI uri = redirectURI(namenode, ugi, delegation, username,
+      final URI uri = redirectURI(null, namenode, ugi, delegation, username,
           doAsUser, fullpath, op.getValue(), -1L, -1L,
           excludeDatanodes.getValue(), bufferSize);
       if(!noredirectParam.getValue()) {
@@ -967,6 +993,13 @@ public class NamenodeWebHdfsMethods {
     });
   }
 
+  private static String encodeFeInfo(FileEncryptionInfo feInfo) {
+    Encoder encoder = Base64.getEncoder();
+    String encodedValue = encoder
+        .encodeToString(PBHelperClient.convert(feInfo).toByteArray());
+    return encodedValue;
+  }
+
   private Response get(
       final UserGroupInformation ugi,
       final DelegationParam delegation,
@@ -995,15 +1028,17 @@ public class NamenodeWebHdfsMethods {
     case OPEN:
     {
       final NameNode namenode = (NameNode)context.getAttribute("name.node");
-      final URI uri = redirectURI(namenode, ugi, delegation, username,
+      ResponseBuilder rb = Response.noContent();
+      final URI uri = redirectURI(rb, namenode, ugi, delegation, username,
           doAsUser, fullpath, op.getValue(), offset.getValue(), -1L,
           excludeDatanodes.getValue(), offset, length, bufferSize);
       if(!noredirectParam.getValue()) {
-        return Response.temporaryRedirect(uri)
-          .type(MediaType.APPLICATION_OCTET_STREAM).build();
+        return rb.status(Status.TEMPORARY_REDIRECT).location(uri)
+            .type(MediaType.APPLICATION_OCTET_STREAM).build();
       } else {
         final String js = JsonUtil.toJsonString("Location", uri);
-        return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
+        return rb.status(Status.OK).entity(js).type(MediaType.APPLICATION_JSON)
+            .build();
       }
     }
     case GET_BLOCK_LOCATIONS:
@@ -1039,8 +1074,8 @@ public class NamenodeWebHdfsMethods {
     case GETFILECHECKSUM:
     {
       final NameNode namenode = (NameNode)context.getAttribute("name.node");
-      final URI uri = redirectURI(namenode, ugi, delegation, username, doAsUser,
-          fullpath, op.getValue(), -1L, -1L, null);
+      final URI uri = redirectURI(null, namenode, ugi, delegation, username,
+          doAsUser, fullpath, op.getValue(), -1L, -1L, null);
       if(!noredirectParam.getValue()) {
         return Response.temporaryRedirect(uri)
           .type(MediaType.APPLICATION_OCTET_STREAM).build();
@@ -1140,9 +1175,12 @@ public class NamenodeWebHdfsMethods {
     case GETSERVERDEFAULTS: {
       // Since none of the server defaults values are hot reloaded, we can
       // cache the output of serverDefaults.
+      String serverDefaultsResponse =
+          (String) context.getAttribute("serverDefaults");
       if (serverDefaultsResponse == null) {
         FsServerDefaults serverDefaults = cp.getServerDefaults();
         serverDefaultsResponse = JsonUtil.toJsonString(serverDefaults);
+        context.setAttribute("serverDefaults", serverDefaultsResponse);
       }
       return Response.ok(serverDefaultsResponse)
           .type(MediaType.APPLICATION_JSON).build();
@@ -1152,15 +1190,6 @@ public class NamenodeWebHdfsMethods {
     }
   }
 
-  /*
-   * This is used only and only for testing.
-   * Please don't use it otherwise.
-   */
-  @VisibleForTesting
-  public static void resetServerDefaultsResponse() {
-    serverDefaultsResponse = null;
-  }
-
   private static String getTrashRoot(String fullPath,
       Configuration conf) throws IOException {
     FileSystem fs = FileSystem.get(conf != null ? conf : new Configuration());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fde95d46/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
index ea867a8..cf13057 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
@@ -20,10 +20,14 @@ package org.apache.hadoop.hdfs;
 import java.io.ByteArrayOutputStream;
 import java.io.File;
 import java.io.IOException;
+import java.io.InputStream;
 import java.io.PrintStream;
 import java.io.RandomAccessFile;
 import java.io.StringReader;
+import java.net.HttpURLConnection;
+import java.net.InetSocketAddress;
 import java.net.URI;
+import java.net.URL;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -41,12 +45,14 @@ import com.google.common.collect.Lists;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.CipherSuite;
+import org.apache.hadoop.crypto.CryptoInputStream;
 import org.apache.hadoop.crypto.CryptoProtocolVersion;
 import org.apache.hadoop.crypto.key.JavaKeyStoreProvider;
 import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.crypto.key.KeyProviderFactory;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSTestWrapper;
 import org.apache.hadoop.fs.FileContext;
@@ -80,6 +86,7 @@ import org.apache.hadoop.hdfs.web.WebHdfsConstants;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.hdfs.web.WebHdfsTestUtil;
 import org.apache.hadoop.io.EnumSetWritable;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.Credentials;
@@ -1985,4 +1992,185 @@ public class TestEncryptionZones {
     Assert.assertEquals(tokens[1], testToken);
     Assert.assertEquals(1, creds.numberOfTokens());
   }
+
+  /**
+   * Creates a file with stable {@link DistributedFileSystem}.
+   * Tests the following 2 scenarios.
+   * 1. The decrypted data using {@link WebHdfsFileSystem} should be same as
+   * input data.
+   * 2. Gets the underlying raw encrypted stream and verifies that the
+   * encrypted data is different than input data.
+   * @throws Exception
+   */
+  @Test
+  public void testWebhdfsRead() throws Exception {
+    Path zonePath = new Path("/TestEncryptionZone");
+    fsWrapper.mkdir(zonePath, FsPermission.getDirDefault(), false);
+    dfsAdmin.createEncryptionZone(zonePath, TEST_KEY, NO_TRASH);
+    final Path encryptedFilePath =
+        new Path("/TestEncryptionZone/encryptedFile.txt");
+    final Path rawPath = 
+        new Path("/.reserved/raw/TestEncryptionZone/encryptedFile.txt");
+    final String content = "hello world";
+
+    // Create a file using DistributedFileSystem.
+    DFSTestUtil.writeFile(fs, encryptedFilePath, content);
+    final FileSystem webhdfs = WebHdfsTestUtil.getWebHdfsFileSystem(conf,
+        WebHdfsConstants.WEBHDFS_SCHEME);
+    // Verify whether decrypted input stream data is same as content.
+    InputStream decryptedIputStream  = webhdfs.open(encryptedFilePath);
+    verifyStreamsSame(content, decryptedIputStream);
+
+    // Get the underlying stream from CryptoInputStream which should be
+    // raw encrypted bytes.
+    InputStream cryptoStream =
+        webhdfs.open(encryptedFilePath).getWrappedStream();
+    Assert.assertTrue("cryptoStream should be an instance of "
+        + "CryptoInputStream", (cryptoStream instanceof CryptoInputStream));
+    InputStream encryptedStream =
+        ((CryptoInputStream)cryptoStream).getWrappedStream();
+    // Verify that the data read from the raw input stream is different
+    // from the original content. Also check it is identical to the raw
+    // encrypted data from dfs.
+    verifyRaw(content, encryptedStream, fs.open(rawPath));
+  }
+
+  private void verifyStreamsSame(String content, InputStream is)
+      throws IOException {
+    byte[] streamBytes;
+    try (ByteArrayOutputStream os = new ByteArrayOutputStream()) {
+      IOUtils.copyBytes(is, os, 1024, true);
+      streamBytes = os.toByteArray();
+    }
+    Assert.assertArrayEquals(content.getBytes(), streamBytes);
+  }
+
+  private void verifyRaw(String content, InputStream is, InputStream rawIs)
+      throws IOException {
+    byte[] streamBytes, rawBytes;
+    try (ByteArrayOutputStream os = new ByteArrayOutputStream()) {
+      IOUtils.copyBytes(is, os, 1024, true);
+      streamBytes = os.toByteArray();
+    }
+    Assert.assertFalse(Arrays.equals(content.getBytes(), streamBytes));
+
+    // webhdfs raw bytes should match the raw bytes from dfs.
+    try (ByteArrayOutputStream os = new ByteArrayOutputStream()) {
+      IOUtils.copyBytes(rawIs, os, 1024, true);
+      rawBytes = os.toByteArray();
+    }
+    Assert.assertArrayEquals(rawBytes, streamBytes);
+  }
+
+  /* Tests that if client is old and namenode is new then the
+   * data will be decrypted by datanode.
+   * @throws Exception
+   */
+  @Test
+  public void testWebhdfsReadOldBehavior() throws Exception {
+    Path zonePath = new Path("/TestEncryptionZone");
+    fsWrapper.mkdir(zonePath, FsPermission.getDirDefault(), false);
+    dfsAdmin.createEncryptionZone(zonePath, TEST_KEY, NO_TRASH);
+    final Path encryptedFilePath = new Path("/TestEncryptionZone/foo");
+    final String content = "hello world";
+    // Create a file using DistributedFileSystem.
+    DFSTestUtil.writeFile(fs, encryptedFilePath, content);
+
+    InetSocketAddress addr = cluster.getNameNode().getHttpAddress();
+    URL url = new URL("http", addr.getHostString(), addr.getPort(),
+        WebHdfsFileSystem.PATH_PREFIX + encryptedFilePath.toString()
+        + "?op=OPEN");
+    // Return a connection with client not supporting EZ.
+    HttpURLConnection namenodeConnection = returnConnection(url, "GET", false);
+    String location = namenodeConnection.getHeaderField("Location");
+    URL datanodeURL = new URL(location);
+    String path = datanodeURL.getPath();
+    Assert.assertEquals(
+        WebHdfsFileSystem.PATH_PREFIX + encryptedFilePath.toString(), path);
+    HttpURLConnection datanodeConnection = returnConnection(datanodeURL,
+        "GET", false);
+    InputStream in = datanodeConnection.getInputStream();
+    // Comparing with the original contents
+    // and making sure they are decrypted.
+    verifyStreamsSame(content, in);
+  }
+
+  /* Tests namenode returns path starting with /.reserved/raw if client
+   * supports EZ and not if otherwise
+   * @throws Exception
+   */
+  @Test
+  public void testWebhfsEZRedirectLocation()
+      throws Exception {
+    Path zonePath = new Path("/TestEncryptionZone");
+    fsWrapper.mkdir(zonePath, FsPermission.getDirDefault(), false);
+    dfsAdmin.createEncryptionZone(zonePath, TEST_KEY, NO_TRASH);
+    final Path encryptedFilePath =
+        new Path("/TestEncryptionZone/foo");
+    final String content = "hello world";
+    // Create a file using DistributedFileSystem.
+    DFSTestUtil.writeFile(fs, encryptedFilePath, content);
+
+    InetSocketAddress addr = cluster.getNameNode().getHttpAddress();
+    URL url = new URL("http", addr.getHostString(), addr.getPort(),
+        WebHdfsFileSystem.PATH_PREFIX + encryptedFilePath.toString()
+        + "?op=OPEN");
+    // Return a connection with client not supporting EZ.
+    HttpURLConnection namenodeConnection =
+        returnConnection(url, "GET", false);
+    Assert.assertNotNull(namenodeConnection.getHeaderField("Location"));
+    URL datanodeUrl = new URL(namenodeConnection.getHeaderField("Location"));
+    Assert.assertNotNull(datanodeUrl);
+    String path = datanodeUrl.getPath();
+    Assert.assertEquals(
+        WebHdfsFileSystem.PATH_PREFIX + encryptedFilePath.toString(), path);
+
+    url = new URL("http", addr.getHostString(), addr.getPort(),
+        WebHdfsFileSystem.PATH_PREFIX + encryptedFilePath.toString()
+        + "?op=OPEN");
+    // Return a connection with client supporting EZ.
+    namenodeConnection = returnConnection(url, "GET", true);
+    Assert.assertNotNull(namenodeConnection.getHeaderField("Location"));
+    datanodeUrl = new URL(namenodeConnection.getHeaderField("Location"));
+    Assert.assertNotNull(datanodeUrl);
+    path = datanodeUrl.getPath();
+    Assert.assertEquals(WebHdfsFileSystem.PATH_PREFIX
+        + "/.reserved/raw" + encryptedFilePath.toString(), path);
+  }
+
+  private static HttpURLConnection returnConnection(URL url,
+      String httpRequestType, boolean supportEZ) throws Exception {
+    HttpURLConnection conn = null;
+    conn = (HttpURLConnection) url.openConnection();
+    conn.setRequestMethod(httpRequestType);
+    conn.setDoOutput(true);
+    conn.setInstanceFollowRedirects(false);
+    if (supportEZ) {
+      conn.setRequestProperty(WebHdfsFileSystem.EZ_HEADER, "true");
+    }
+    return conn;
+  }
+
+  /*
+   * Test seek behavior of the webhdfs input stream which reads data from
+   * encryption zone.
+   */
+  @Test
+  public void testPread() throws Exception {
+    Path zonePath = new Path("/TestEncryptionZone");
+    fsWrapper.mkdir(zonePath, FsPermission.getDirDefault(), false);
+    dfsAdmin.createEncryptionZone(zonePath, TEST_KEY, NO_TRASH);
+    final Path encryptedFilePath =
+        new Path("/TestEncryptionZone/foo");
+    // Create a file using DistributedFileSystem.
+    WebHdfsFileSystem webfs = WebHdfsTestUtil.getWebHdfsFileSystem(conf,
+        WebHdfsConstants.WEBHDFS_SCHEME);
+    DFSTestUtil.createFile(webfs, encryptedFilePath, 1024, (short)1, 0xFEED);
+    byte[] data = DFSTestUtil.readFileAsBytes(fs, encryptedFilePath);
+    FSDataInputStream in = webfs.open(encryptedFilePath);
+    for (int i = 0; i < 1024; i++) {
+      in.seek(i);
+      Assert.assertEquals((data[i] & 0XFF), in.read());
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fde95d46/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/web/resources/TestWebHdfsDataLocality.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/web/resources/TestWebHdfsDataLocality.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/web/resources/TestWebHdfsDataLocality.java
index 604bf79..759719d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/web/resources/TestWebHdfsDataLocality.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/web/resources/TestWebHdfsDataLocality.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
@@ -101,7 +102,7 @@ public class TestWebHdfsDataLocality {
           //The chosen datanode must be the same as the client address
           final DatanodeInfo chosen = NamenodeWebHdfsMethods.chooseDatanode(
               namenode, f, PutOpParam.Op.CREATE, -1L, blocksize, null,
-              LOCALHOST);
+              LOCALHOST, null);
           Assert.assertEquals(ipAddr, chosen.getIpAddr());
         }
       }
@@ -125,23 +126,26 @@ public class TestWebHdfsDataLocality {
       //the chosen datanode must be the same as the replica location.
 
       { //test GETFILECHECKSUM
+        final HdfsFileStatus status = dfs.getClient().getFileInfo(f);
         final DatanodeInfo chosen = NamenodeWebHdfsMethods.chooseDatanode(
             namenode, f, GetOpParam.Op.GETFILECHECKSUM, -1L, blocksize, null,
-            LOCALHOST);
+            LOCALHOST, status);
         Assert.assertEquals(expected, chosen);
       }
   
       { //test OPEN
+        final HdfsFileStatus status = dfs.getClient().getFileInfo(f);
         final DatanodeInfo chosen = NamenodeWebHdfsMethods.chooseDatanode(
             namenode, f, GetOpParam.Op.OPEN, 0, blocksize, null,
-            LOCALHOST);
+            LOCALHOST, status);
         Assert.assertEquals(expected, chosen);
       }
 
       { //test APPEND
+        final HdfsFileStatus status = dfs.getClient().getFileInfo(f);
         final DatanodeInfo chosen = NamenodeWebHdfsMethods.chooseDatanode(
             namenode, f, PostOpParam.Op.APPEND, -1L, blocksize, null,
-            LOCALHOST);
+            LOCALHOST, status);
         Assert.assertEquals(expected, chosen);
       }
     } finally {
@@ -195,9 +199,10 @@ public class TestWebHdfsDataLocality {
       for (int i = 0; i < 2; i++) {
         sb.append(locations[i].getXferAddr());
         { // test GETFILECHECKSUM
+          final HdfsFileStatus status = dfs.getClient().getFileInfo(f);
           final DatanodeInfo chosen = NamenodeWebHdfsMethods.chooseDatanode(
               namenode, f, GetOpParam.Op.GETFILECHECKSUM, -1L, blocksize,
-              sb.toString(), LOCALHOST);
+              sb.toString(), LOCALHOST, status);
           for (int j = 0; j <= i; j++) {
             Assert.assertNotEquals(locations[j].getHostName(),
                 chosen.getHostName());
@@ -205,9 +210,10 @@ public class TestWebHdfsDataLocality {
         }
 
         { // test OPEN
+          final HdfsFileStatus status = dfs.getClient().getFileInfo(f);
           final DatanodeInfo chosen = NamenodeWebHdfsMethods.chooseDatanode(
               namenode, f, GetOpParam.Op.OPEN, 0, blocksize, sb.toString(),
-              LOCALHOST);
+              LOCALHOST, status);
           for (int j = 0; j <= i; j++) {
             Assert.assertNotEquals(locations[j].getHostName(),
                 chosen.getHostName());
@@ -215,9 +221,10 @@ public class TestWebHdfsDataLocality {
         }
   
         { // test APPEND
+          final HdfsFileStatus status = dfs.getClient().getFileInfo(f);
           final DatanodeInfo chosen = NamenodeWebHdfsMethods
               .chooseDatanode(namenode, f, PostOpParam.Op.APPEND, -1L,
-                  blocksize, sb.toString(), LOCALHOST);
+                  blocksize, sb.toString(), LOCALHOST, status);
           for (int j = 0; j <= i; j++) {
             Assert.assertNotEquals(locations[j].getHostName(),
                 chosen.getHostName());
@@ -238,6 +245,6 @@ public class TestWebHdfsDataLocality {
     exception.expect(IOException.class);
     exception.expectMessage("Namesystem has not been intialized yet.");
     NamenodeWebHdfsMethods.chooseDatanode(nn, "/path", PutOpParam.Op.CREATE, 0,
-        DFSConfigKeys.DFS_BLOCK_SIZE_DEFAULT, null, LOCALHOST);
+        DFSConfigKeys.DFS_BLOCK_SIZE_DEFAULT, null, LOCALHOST, null);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fde95d46/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
index 500ec0a..9a8c9fc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
@@ -1435,7 +1435,6 @@ public class TestWebHDFS {
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
       final WebHdfsFileSystem webfs = WebHdfsTestUtil.getWebHdfsFileSystem(
           conf, WebHdfsConstants.WEBHDFS_SCHEME);
-      NamenodeWebHdfsMethods.resetServerDefaultsResponse();
       FSNamesystem fsnSpy =
           NameNodeAdapter.spyOnNamesystem(cluster.getNameNode());
       Mockito.when(fsnSpy.getServerDefaults()).

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fde95d46/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTokens.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTokens.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTokens.java
index 1862f76..153bd47 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTokens.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTokens.java
@@ -385,7 +385,7 @@ public class TestWebHdfsTokens {
     InputStream is = fs.open(p);
     is.read();
     is.close();
-    verify(fs, times(2)).getDelegationToken(); // first bad, then good
+    verify(fs, times(3)).getDelegationToken(); // first bad, then good
     verify(fs, times(1)).replaceExpiredDelegationToken();
     verify(fs, times(1)).getDelegationToken(null);
     verify(fs, times(1)).setDelegationToken(any());
@@ -402,7 +402,7 @@ public class TestWebHdfsTokens {
     is = fs.open(p);
     is.read();
     is.close();
-    verify(fs, times(2)).getDelegationToken(); // first bad, then good
+    verify(fs, times(3)).getDelegationToken(); // first bad, then good
     verify(fs, times(1)).replaceExpiredDelegationToken();
     verify(fs, times(1)).getDelegationToken(null);
     verify(fs, times(1)).setDelegationToken(any());


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[15/37] hadoop git commit: YARN-7760. [UI2] Clicking 'Master Node' or link next to 'AM Node Web UI' under application's appAttempt page goes to OLD RM UI. (Vasudevan Skm / Sunil G via wangda)

Posted by ae...@apache.org.
YARN-7760. [UI2] Clicking 'Master Node' or link next to 'AM Node Web UI' under application's appAttempt page goes to OLD RM UI. (Vasudevan Skm / Sunil G via wangda)

Change-Id: Ia7e2b955d3551ca43c6852fe253fd01983b56565


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/75618c0d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/75618c0d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/75618c0d

Branch: refs/heads/HDFS-7240
Commit: 75618c0dec9199387fb55c8b4d63ac0003ddc1fe
Parents: e0cfb0a
Author: Wangda Tan <wa...@apache.org>
Authored: Fri Jan 26 15:44:55 2018 +0800
Committer: Wangda Tan <wa...@apache.org>
Committed: Fri Jan 26 15:44:55 2018 +0800

----------------------------------------------------------------------
 .../resourcemanager/webapp/dao/AppInfo.java     | 11 +++
 .../webapp/TestRMWebServicesApps.java           |  2 +-
 .../src/main/webapp/app/models/yarn-app.js      | 96 ++++++++++++--------
 .../src/main/webapp/app/serializers/yarn-app.js |  1 +
 .../src/main/webapp/app/templates/yarn-app.hbs  |  2 +-
 5 files changed, 70 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/75618c0d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java
index 0711b45..631c908 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java
@@ -92,6 +92,7 @@ public class AppInfo {
   protected String amContainerLogs;
   protected String amHostHttpAddress;
   private String amRPCAddress;
+  private String masterNodeId;
   private long allocatedMB;
   private long allocatedVCores;
   private long reservedMB;
@@ -162,6 +163,7 @@ public class AppInfo {
       this.name = app.getName().toString();
       this.queue = app.getQueue().toString();
       this.priority = 0;
+      this.masterNodeId = "";
 
       if (app.getApplicationPriority() != null) {
         this.priority = app.getApplicationPriority().getPriority();
@@ -192,6 +194,7 @@ public class AppInfo {
                 schemePrefix + masterContainer.getNodeHttpAddress(),
                 masterContainer.getId().toString(), app.getUser());
             this.amHostHttpAddress = masterContainer.getNodeHttpAddress();
+            this.masterNodeId = masterContainer.getNodeId().toString();
           }
 
           this.amRPCAddress = getAmRPCAddressFromRMAppAttempt(attempt);
@@ -615,4 +618,12 @@ public class AppInfo {
   public void setName(String name) {
     this.name = name;
   }
+
+  public String getMasterNodeId() {
+    return masterNodeId;
+  }
+
+  public void setMasterNodeId(String masterNodeId) {
+    this.masterNodeId = masterNodeId;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75618c0d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java
index f0704ac..a381ed4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java
@@ -1603,7 +1603,7 @@ public class TestRMWebServicesApps extends JerseyTestBase {
   public void verifyAppInfo(JSONObject info, RMApp app, boolean hasResourceReqs)
       throws JSONException, Exception {
 
-    int expectedNumberOfElements = 38 + (hasResourceReqs ? 2 : 0);
+    int expectedNumberOfElements = 39 + (hasResourceReqs ? 2 : 0);
     String appNodeLabelExpression = null;
     String amNodeLabelExpression = null;
     if (app.getApplicationSubmissionContext()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75618c0d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-app.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-app.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-app.js
index b9897be..7e3050f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-app.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-app.js
@@ -20,42 +20,43 @@ import Converter from 'yarn-ui/utils/converter';
 import DS from 'ember-data';
 
 export default DS.Model.extend({
-  appName: DS.attr('string'),
-  user: DS.attr('string'),
-  queue: DS.attr('string'),
-  state: DS.attr('string'),
-  startTime: DS.attr('string'),
-  elapsedTime: DS.attr('string'),
-  finalStatus: DS.attr('string'),
-  finishedTime: DS.attr('finishedTime'),
-  progress: DS.attr('number'),
-  diagnostics: DS.attr('string'),
-  amContainerLogs: DS.attr('string'),
-  amHostHttpAddress: DS.attr('string'),
-  logAggregationStatus: DS.attr('string'),
-  unmanagedApplication: DS.attr('boolean'),
-  amNodeLabelExpression: DS.attr('string'),
-  applicationTags: DS.attr('string'),
-  applicationType: DS.attr('string'),
-  priority: DS.attr('string'),
-  allocatedMB: DS.attr('number'),
-  allocatedVCores: DS.attr('number'),
-  runningContainers: DS.attr('number'),
-  memorySeconds: DS.attr('number'),
-  vcoreSeconds: DS.attr('number'),
-  preemptedResourceMB: DS.attr('number'),
-  preemptedResourceVCores: DS.attr('number'),
-  numNonAMContainerPreempted: DS.attr('number'),
-  numAMContainerPreempted: DS.attr('number'),
-  clusterUsagePercentage: DS.attr('number'),
-  queueUsagePercentage: DS.attr('number'),
-  currentAppAttemptId: DS.attr('string'),
-  remainingTimeoutInSeconds: DS.attr('number'),
-  applicationExpiryTime: DS.attr('string'),
-  resourceRequests: DS.attr('array'),
+  appName: DS.attr("string"),
+  user: DS.attr("string"),
+  queue: DS.attr("string"),
+  state: DS.attr("string"),
+  startTime: DS.attr("string"),
+  elapsedTime: DS.attr("string"),
+  finalStatus: DS.attr("string"),
+  finishedTime: DS.attr("finishedTime"),
+  progress: DS.attr("number"),
+  diagnostics: DS.attr("string"),
+  amContainerLogs: DS.attr("string"),
+  amHostHttpAddress: DS.attr("string"),
+  masterNodeId: DS.attr("string"),
+  logAggregationStatus: DS.attr("string"),
+  unmanagedApplication: DS.attr("boolean"),
+  amNodeLabelExpression: DS.attr("string"),
+  applicationTags: DS.attr("string"),
+  applicationType: DS.attr("string"),
+  priority: DS.attr("string"),
+  allocatedMB: DS.attr("number"),
+  allocatedVCores: DS.attr("number"),
+  runningContainers: DS.attr("number"),
+  memorySeconds: DS.attr("number"),
+  vcoreSeconds: DS.attr("number"),
+  preemptedResourceMB: DS.attr("number"),
+  preemptedResourceVCores: DS.attr("number"),
+  numNonAMContainerPreempted: DS.attr("number"),
+  numAMContainerPreempted: DS.attr("number"),
+  clusterUsagePercentage: DS.attr("number"),
+  queueUsagePercentage: DS.attr("number"),
+  currentAppAttemptId: DS.attr("string"),
+  remainingTimeoutInSeconds: DS.attr("number"),
+  applicationExpiryTime: DS.attr("string"),
+  resourceRequests: DS.attr("array"),
 
   isFailed: function() {
-    return this.get('finalStatus') === "FAILED";
+    return this.get("finalStatus") === "FAILED";
   }.property("finalStatus"),
 
   validatedFinishedTs: function() {
@@ -70,27 +71,42 @@ export default DS.Model.extend({
   }.property("hasFinishedTime"),
 
   formattedElapsedTime: function() {
-    return Converter.msToElapsedTimeUnit(this.get('elapsedTime'));
-  }.property('elapsedTime'),
+    return Converter.msToElapsedTimeUnit(this.get("elapsedTime"));
+  }.property("elapsedTime"),
 
   allocatedResource: function() {
-    return Converter.resourceToString(this.get("allocatedMB"), this.get("allocatedVCores"));
+    return Converter.resourceToString(
+      this.get("allocatedMB"),
+      this.get("allocatedVCores")
+    );
   }.property("allocatedMB", "allocatedVCores"),
 
   preemptedResource: function() {
-    return Converter.resourceToString(this.get("preemptedResourceMB"), this.get("preemptedResourceVCores"));
+    return Converter.resourceToString(
+      this.get("preemptedResourceMB"),
+      this.get("preemptedResourceVCores")
+    );
   }.property("preemptedResourceMB", "preemptedResourceVCores"),
 
   aggregatedResourceUsage: function() {
-    return Converter.resourceToString(this.get("memorySeconds"), this.get("vcoreSeconds")) + " (× Secs)";
+    return (
+      Converter.resourceToString(
+        this.get("memorySeconds"),
+        this.get("vcoreSeconds")
+      ) + " (× Secs)"
+    );
   }.property("memorySeconds", "vcoreSeconds"),
 
+  masterNodeURL: function() {
+    return `#/yarn-node/${this.get("masterNodeId")}/${this.get("amHostHttpAddress")}/info/`;
+  }.property("masterNodeId", "amHostHttpAddress"),
+
   progressStyle: function() {
     return "width: " + this.get("progress") + "%";
   }.property("progress"),
 
   runningContainersNumber: function() {
-    if(this.get("runningContainers") < 0) {
+    if (this.get("runningContainers") < 0) {
       return 0;
     }
     return this.get("runningContainers");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75618c0d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-app.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-app.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-app.js
index 42d31ca..bd9e984 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-app.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-app.js
@@ -51,6 +51,7 @@ export default DS.JSONAPISerializer.extend({
           diagnostics: (payload.diagnostics && payload.diagnostics !== 'null')? payload.diagnostics : '',
           amContainerLogs: payload.amContainerLogs,
           amHostHttpAddress: payload.amHostHttpAddress,
+          masterNodeId: payload.masterNodeId,
           logAggregationStatus: payload.logAggregationStatus,
           unmanagedApplication: payload.unmanagedApplication,
           amNodeLabelExpression: payload.amNodeLabelExpression,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75618c0d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-app.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-app.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-app.hbs
index ca93428..992b979 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-app.hbs
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-app.hbs
@@ -65,7 +65,7 @@
         <div class="flex-right">
           <div class="links">
             <a href="{{model.app.amContainerLogs}}" target="_blank">Master Container Log</a> &nbsp;|&nbsp;
-            <a href="{{amHostHttpAddressFormatted}}" target="_blank">Master Node</a>  &nbsp;|&nbsp;
+            <a href="{{model.app.masterNodeURL}}">Master Node</a> &nbsp;|&nbsp;
             {{#if isRunningService}}
               <div class="btn-group">
                 <button type="button" class="btn btn-unstyled dropdown-toggle" title="Settings" data-toggle="dropdown" aria-haspopup="true" aria-expanded="false">


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[04/37] hadoop git commit: YARN-7628. [Documentation] Documenting the ability to disable elasticity at Capacity Scheduler's sub queues. (Zian Chen via wangda)

Posted by ae...@apache.org.
YARN-7628. [Documentation] Documenting the ability to disable elasticity at Capacity Scheduler's sub queues. (Zian Chen via wangda)

Change-Id: I5c37df1c5bee9e3a78379775ef307ef94506ebc4


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/051ad9d6
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/051ad9d6
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/051ad9d6

Branch: refs/heads/HDFS-7240
Commit: 051ad9d63f41ae98be728a7cdbdad9b072316318
Parents: 0c559b2
Author: Wangda Tan <wa...@apache.org>
Authored: Thu Jan 25 07:24:57 2018 +0800
Committer: Wangda Tan <wa...@apache.org>
Committed: Thu Jan 25 07:24:57 2018 +0800

----------------------------------------------------------------------
 .../hadoop-yarn-site/src/site/markdown/CapacityScheduler.md        | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/051ad9d6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md
index 2598997..87cfd39 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md
@@ -123,7 +123,7 @@ Configuration
 | Property | Description |
 |:---- |:---- |
 | `yarn.scheduler.capacity.<queue-path>.capacity` | Queue *capacity* in percentage (%) as a float (e.g. 12.5) OR as absolute resource queue minimum capacity. The sum of capacities for all queues, at each level, must be equal to 100. However if absolute resource is configured, sum of absolute resources of child queues could be less than it's parent absolute resource capacity. Applications in the queue may consume more resources than the queue's capacity if there are free resources, providing elasticity. |
-| `yarn.scheduler.capacity.<queue-path>.maximum-capacity` | Maximum queue capacity in percentage (%) as a float OR as absolute resource queue maximum capacity. This limits the *elasticity* for applications in the queue. Defaults to -1 which disables it. |
+| `yarn.scheduler.capacity.<queue-path>.maximum-capacity` | Maximum queue capacity in percentage (%) as a float OR as absolute resource queue maximum capacity. This limits the *elasticity* for applications in the queue. 1) Value is between 0 and 100. 2) Admin needs to make sure absolute maximum capacity >= absolute capacity for each queue. Also, setting this value to -1 sets maximum capacity to 100%. |
 | `yarn.scheduler.capacity.<queue-path>.minimum-user-limit-percent` | Each queue enforces a limit on the percentage of resources allocated to a user at any given time, if there is demand for resources. The user limit can vary between a minimum and maximum value. The former (the minimum value) is set to this property value and the latter (the maximum value) depends on the number of users who have submitted applications. For e.g., suppose the value of this property is 25. If two users have submitted applications to a queue, no single user can use more than 50% of the queue resources. If a third user submits an application, no single user can use more than 33% of the queue resources. With 4 or more users, no user can use more than 25% of the queues resources. A value of 100 implies no user limits are imposed. The default is 100. Value is specified as a integer. |
 | `yarn.scheduler.capacity.<queue-path>.user-limit-factor` | The multiple of the queue capacity which can be configured to allow a single user to acquire more resources. By default this is set to 1 which ensures that a single user can never take more than the queue's configured capacity irrespective of how idle the cluster is. Value is specified as a float. |
 | `yarn.scheduler.capacity.<queue-path>.maximum-allocation-mb` | The per queue maximum limit of memory to allocate to each container request at the Resource Manager. This setting overrides the cluster configuration `yarn.scheduler.maximum-allocation-mb`. This value must be smaller than or equal to the cluster maximum. |


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[10/37] hadoop git commit: YARN-7798. Refactor SLS Reservation Creation. Contributed by Young Chen.

Posted by ae...@apache.org.
YARN-7798. Refactor SLS Reservation Creation. Contributed by Young Chen.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/16be42d3
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/16be42d3
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/16be42d3

Branch: refs/heads/HDFS-7240
Commit: 16be42d3097c13b17d704e5b6dc8d66bd5ff6d9a
Parents: 0c139d5
Author: Yufei Gu <yu...@apache.org>
Authored: Thu Jan 25 13:06:01 2018 -0800
Committer: Yufei Gu <yu...@apache.org>
Committed: Thu Jan 25 13:06:01 2018 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/yarn/sls/SLSRunner.java   | 75 +++++++++-----------
 .../hadoop/yarn/sls/appmaster/AMSimulator.java  | 41 ++++++-----
 .../yarn/sls/appmaster/MRAMSimulator.java       | 74 ++++++++++++++-----
 .../apache/hadoop/yarn/sls/TestSLSRunner.java   |  2 +-
 .../yarn/sls/appmaster/TestAMSimulator.java     |  7 +-
 5 files changed, 119 insertions(+), 80 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/16be42d3/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java
index ad4310f..456602f 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java
@@ -57,7 +57,6 @@ import org.apache.hadoop.tools.rumen.TaskAttemptInfo;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
-import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.ReservationId;
@@ -444,7 +443,7 @@ public class SLSRunner extends Configured implements Tool {
 
     for (int i = 0; i < jobCount; i++) {
       runNewAM(amType, user, queue, oldAppId, jobStartTime, jobFinishTime,
-          getTaskContainers(jsonJob), null, getAMContainerResource(jsonJob));
+          getTaskContainers(jsonJob), getAMContainerResource(jsonJob));
     }
   }
 
@@ -607,7 +606,7 @@ public class SLSRunner extends Configured implements Tool {
 
     // Only supports the default job type currently
     runNewAM(SLSUtils.DEFAULT_JOB_TYPE, user, jobQueue, oldJobId,
-        jobStartTimeMS, jobFinishTimeMS, containerList, null,
+        jobStartTimeMS, jobFinishTimeMS, containerList,
         getAMContainerResource(null));
   }
 
@@ -628,10 +627,6 @@ public class SLSRunner extends Configured implements Tool {
     localConf.set("fs.defaultFS", "file:///");
     long baselineTimeMS = 0;
 
-    // reservations use wall clock time, so need to have a reference for that
-    UTCClock clock = new UTCClock();
-    long now = clock.getTime();
-
     try {
 
       // if we use the nodeFile this could have been not initialized yet.
@@ -670,13 +665,12 @@ public class SLSRunner extends Configured implements Tool {
         ArrayList<NodeId> keyAsArray = new ArrayList<NodeId>(nmMap.keySet());
         Random rand = new Random(stjp.getSeed());
 
-        Resource maxMapRes = Resource.newInstance(0, 0);
-        long maxMapDur = 0;
         // map tasks
         for (int i = 0; i < job.getNumberMaps(); i++) {
           TaskAttemptInfo tai = job.getTaskAttemptInfo(TaskType.MAP, i, 0);
-          RMNode node = nmMap
-              .get(keyAsArray.get(rand.nextInt(keyAsArray.size()))).getNode();
+          RMNode node =
+              nmMap.get(keyAsArray.get(rand.nextInt(keyAsArray.size())))
+                  .getNode();
           String hostname = "/" + node.getRackName() + "/" + node.getHostName();
           long containerLifeTime = tai.getRuntime();
           Resource containerResource =
@@ -684,55 +678,39 @@ public class SLSRunner extends Configured implements Tool {
                   (int) tai.getTaskInfo().getTaskVCores());
           containerList.add(new ContainerSimulator(containerResource,
               containerLifeTime, hostname, DEFAULT_MAPPER_PRIORITY, "map"));
-          maxMapRes = Resources.componentwiseMax(maxMapRes, containerResource);
-          maxMapDur =
-              containerLifeTime > maxMapDur ? containerLifeTime : maxMapDur;
-
         }
 
-        Resource maxRedRes = Resource.newInstance(0, 0);
-        long maxRedDur = 0;
         // reduce tasks
         for (int i = 0; i < job.getNumberReduces(); i++) {
           TaskAttemptInfo tai = job.getTaskAttemptInfo(TaskType.REDUCE, i, 0);
-          RMNode node = nmMap
-              .get(keyAsArray.get(rand.nextInt(keyAsArray.size()))).getNode();
+          RMNode node =
+              nmMap.get(keyAsArray.get(rand.nextInt(keyAsArray.size())))
+                  .getNode();
           String hostname = "/" + node.getRackName() + "/" + node.getHostName();
           long containerLifeTime = tai.getRuntime();
           Resource containerResource =
               Resource.newInstance((int) tai.getTaskInfo().getTaskMemory(),
                   (int) tai.getTaskInfo().getTaskVCores());
-          containerList.add(new ContainerSimulator(containerResource,
-              containerLifeTime, hostname, DEFAULT_REDUCER_PRIORITY, "reduce"));
-          maxRedRes = Resources.componentwiseMax(maxRedRes, containerResource);
-          maxRedDur =
-              containerLifeTime > maxRedDur ? containerLifeTime : maxRedDur;
-
+          containerList.add(
+              new ContainerSimulator(containerResource, containerLifeTime,
+                  hostname, DEFAULT_REDUCER_PRIORITY, "reduce"));
         }
 
-        // generating reservations for the jobs that require them
+        ReservationId reservationId = null;
 
-        ReservationSubmissionRequest rr = null;
         if (job.hasDeadline()) {
-          ReservationId reservationId =
+          reservationId =
               ReservationId.newInstance(this.rm.getStartTime(), AM_ID);
-
-          rr = ReservationClientUtil.createMRReservation(reservationId,
-              "reservation_" + AM_ID, maxMapRes, job.getNumberMaps(), maxMapDur,
-              maxRedRes, job.getNumberReduces(), maxRedDur,
-              now + jobStartTimeMS, now + job.getDeadline(),
-              job.getQueueName());
-
         }
 
         runNewAM(SLSUtils.DEFAULT_JOB_TYPE, user, jobQueue, oldJobId,
-            jobStartTimeMS, jobFinishTimeMS, containerList, rr,
-            getAMContainerResource(null));
+            jobStartTimeMS, jobFinishTimeMS, containerList, reservationId,
+            job.getDeadline(), getAMContainerResource(null));
+
       }
     } finally {
       stjp.close();
     }
-
   }
 
   private Resource getAMContainerResource(Map jsonJob) {
@@ -772,7 +750,17 @@ public class SLSRunner extends Configured implements Tool {
   private void runNewAM(String jobType, String user,
       String jobQueue, String oldJobId, long jobStartTimeMS,
       long jobFinishTimeMS, List<ContainerSimulator> containerList,
-      ReservationSubmissionRequest rr, Resource amContainerResource) {
+      Resource amContainerResource) {
+    runNewAM(jobType, user, jobQueue, oldJobId, jobStartTimeMS,
+        jobFinishTimeMS, containerList, null,  -1,
+        amContainerResource);
+  }
+
+  private void runNewAM(String jobType, String user,
+      String jobQueue, String oldJobId, long jobStartTimeMS,
+      long jobFinishTimeMS, List<ContainerSimulator> containerList,
+      ReservationId reservationId, long deadline,
+      Resource amContainerResource) {
 
     AMSimulator amSim = (AMSimulator) ReflectionUtils.newInstance(
         amClassMap.get(jobType), new Configuration());
@@ -787,10 +775,15 @@ public class SLSRunner extends Configured implements Tool {
         oldJobId = Integer.toString(AM_ID);
       }
       AM_ID++;
-
       amSim.init(heartbeatInterval, containerList, rm, this, jobStartTimeMS,
-          jobFinishTimeMS, user, jobQueue, isTracked, oldJobId, rr,
+          jobFinishTimeMS, user, jobQueue, isTracked, oldJobId,
           runner.getStartTimeMS(), amContainerResource);
+      if(reservationId != null) {
+        // if we have a ReservationId, delegate reservation creation to
+        // AMSim (reservation shape is impl specific)
+        UTCClock clock = new UTCClock();
+        amSim.initReservation(reservationId, deadline, clock.getTime());
+      }
       runner.schedule(amSim);
       maxRuntime = Math.max(maxRuntime, jobFinishTimeMS);
       numTasks += containerList.size();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/16be42d3/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/AMSimulator.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/AMSimulator.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/AMSimulator.java
index 72698ea..5727b5f 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/AMSimulator.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/AMSimulator.java
@@ -85,7 +85,7 @@ public abstract class AMSimulator extends TaskRunner.Task {
   protected final BlockingQueue<AllocateResponse> responseQueue;
   private int responseId = 0;
   // user name
-  protected String user;  
+  private String user;
   // queue name
   protected String queue;
   // am type
@@ -105,7 +105,7 @@ public abstract class AMSimulator extends TaskRunner.Task {
   // waiting for AM container
   volatile boolean isAMContainerRunning = false;
   volatile Container amContainer;
-  
+
   private static final Logger LOG = LoggerFactory.getLogger(AMSimulator.class);
 
   private Resource amContainerResource;
@@ -120,9 +120,8 @@ public abstract class AMSimulator extends TaskRunner.Task {
   public void init(int heartbeatInterval,
       List<ContainerSimulator> containerList, ResourceManager resourceManager,
       SLSRunner slsRunnner, long startTime, long finishTime, String simUser,
-      String simQueue, boolean tracked, String oldApp,
-      ReservationSubmissionRequest rr, long baseTimeMS,
-      Resource amContainerResource) {
+      String simQueue, boolean tracked, String oldApp, long baseTimeMS,
+      Resource amResource) {
     super.init(startTime, startTime + 1000000L * heartbeatInterval,
         heartbeatInterval);
     this.user = simUser;
@@ -134,8 +133,7 @@ public abstract class AMSimulator extends TaskRunner.Task {
     this.baselineTimeMS = baseTimeMS;
     this.traceStartTimeMS = startTime;
     this.traceFinishTimeMS = finishTime;
-    this.reservationRequest = rr;
-    this.amContainerResource = amContainerResource;
+    this.amContainerResource = amResource;
   }
 
   /**
@@ -171,6 +169,10 @@ public abstract class AMSimulator extends TaskRunner.Task {
     isAMContainerRunning = true;
   }
 
+  protected void setReservationRequest(ReservationSubmissionRequest rr){
+    this.reservationRequest = rr;
+  }
+
   private ReservationId submitReservationWhenSpecified()
       throws IOException, InterruptedException {
     if (reservationRequest != null) {
@@ -256,7 +258,7 @@ public abstract class AMSimulator extends TaskRunner.Task {
               simulateStartTimeMS, simulateFinishTimeMS);
     }
   }
-  
+
   protected ResourceRequest createResourceRequest(
           Resource resource, String host, int priority, int numContainers) {
     ResourceRequest request = recordFactory
@@ -269,7 +271,7 @@ public abstract class AMSimulator extends TaskRunner.Task {
     request.setPriority(prio);
     return request;
   }
-  
+
   protected AllocateRequest createAllocateRequest(List<ResourceRequest> ask,
       List<ContainerId> toRelease) {
     AllocateRequest allocateRequest =
@@ -279,36 +281,39 @@ public abstract class AMSimulator extends TaskRunner.Task {
     allocateRequest.setReleaseList(toRelease);
     return allocateRequest;
   }
-  
+
   protected AllocateRequest createAllocateRequest(List<ResourceRequest> ask) {
     return createAllocateRequest(ask, new ArrayList<ContainerId>());
   }
 
   protected abstract void processResponseQueue() throws Exception;
-  
+
   protected abstract void sendContainerRequest() throws Exception;
-  
+
+  public abstract void initReservation(
+      ReservationId reservationId, long deadline, long now);
+
   protected abstract void checkStop();
-  
+
   private void submitApp(ReservationId reservationId)
           throws YarnException, InterruptedException, IOException {
     // ask for new application
     GetNewApplicationRequest newAppRequest =
         Records.newRecord(GetNewApplicationRequest.class);
-    GetNewApplicationResponse newAppResponse = 
+    GetNewApplicationResponse newAppResponse =
         rm.getClientRMService().getNewApplication(newAppRequest);
     appId = newAppResponse.getApplicationId();
-    
+
     // submit the application
     final SubmitApplicationRequest subAppRequest =
         Records.newRecord(SubmitApplicationRequest.class);
-    ApplicationSubmissionContext appSubContext = 
+    ApplicationSubmissionContext appSubContext =
         Records.newRecord(ApplicationSubmissionContext.class);
     appSubContext.setApplicationId(appId);
     appSubContext.setMaxAppAttempts(1);
     appSubContext.setQueue(queue);
     appSubContext.setPriority(Priority.newInstance(0));
-    ContainerLaunchContext conLauContext = 
+    ContainerLaunchContext conLauContext =
         Records.newRecord(ContainerLaunchContext.class);
     conLauContext.setApplicationACLs(new HashMap<>());
     conLauContext.setCommands(new ArrayList<>());
@@ -379,7 +384,7 @@ public abstract class AMSimulator extends TaskRunner.Task {
       }
     }
   }
-  
+
   protected List<ResourceRequest> packageRequests(
           List<ContainerSimulator> csList, int priority) {
     // create requests

http://git-wip-us.apache.org/repos/asf/hadoop/blob/16be42d3/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/MRAMSimulator.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/MRAMSimulator.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/MRAMSimulator.java
index 21bf054..18a155c 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/MRAMSimulator.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/MRAMSimulator.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.sls.appmaster;
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
@@ -34,6 +35,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
 import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -42,8 +44,10 @@ import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.sls.ReservationClientUtil;
 import org.apache.hadoop.yarn.sls.scheduler.ContainerSimulator;
 import org.apache.hadoop.yarn.sls.SLSRunner;
+import org.apache.hadoop.yarn.util.resource.Resources;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -51,51 +55,51 @@ import org.slf4j.LoggerFactory;
 @Unstable
 public class MRAMSimulator extends AMSimulator {
   /*
-  Vocabulary Used: 
+  Vocabulary Used:
   pending -> requests which are NOT yet sent to RM
   scheduled -> requests which are sent to RM but not yet assigned
   assigned -> requests which are assigned to a container
   completed -> request corresponding to which container has completed
-  
+
   Maps are scheduled as soon as their requests are received. Reduces are
   scheduled when all maps have finished (not support slow-start currently).
   */
-  
+
   private static final int PRIORITY_REDUCE = 10;
   private static final int PRIORITY_MAP = 20;
 
   // pending maps
   private LinkedList<ContainerSimulator> pendingMaps =
           new LinkedList<>();
-  
+
   // pending failed maps
   private LinkedList<ContainerSimulator> pendingFailedMaps =
           new LinkedList<ContainerSimulator>();
-  
+
   // scheduled maps
   private LinkedList<ContainerSimulator> scheduledMaps =
           new LinkedList<ContainerSimulator>();
-  
+
   // assigned maps
   private Map<ContainerId, ContainerSimulator> assignedMaps =
           new HashMap<ContainerId, ContainerSimulator>();
-  
+
   // reduces which are not yet scheduled
   private LinkedList<ContainerSimulator> pendingReduces =
           new LinkedList<ContainerSimulator>();
-  
+
   // pending failed reduces
   private LinkedList<ContainerSimulator> pendingFailedReduces =
           new LinkedList<ContainerSimulator>();
- 
+
   // scheduled reduces
   private LinkedList<ContainerSimulator> scheduledReduces =
           new LinkedList<ContainerSimulator>();
-  
+
   // assigned reduces
   private Map<ContainerId, ContainerSimulator> assignedReduces =
           new HashMap<ContainerId, ContainerSimulator>();
-  
+
   // all maps & reduces
   private LinkedList<ContainerSimulator> allMaps =
           new LinkedList<ContainerSimulator>();
@@ -117,14 +121,14 @@ public class MRAMSimulator extends AMSimulator {
   @SuppressWarnings("checkstyle:parameternumber")
   public void init(int heartbeatInterval,
       List<ContainerSimulator> containerList, ResourceManager rm, SLSRunner se,
-      long traceStartTime, long traceFinishTime, String user, String queue, 
-      boolean isTracked, String oldAppId, ReservationSubmissionRequest rr,
-      long baselineStartTimeMS, Resource amContainerResource) {
+      long traceStartTime, long traceFinishTime, String user, String queue,
+      boolean isTracked, String oldAppId, long baselineStartTimeMS,
+      Resource amContainerResource) {
     super.init(heartbeatInterval, containerList, rm, se,
         traceStartTime, traceFinishTime, user, queue, isTracked, oldAppId,
-        rr, baselineStartTimeMS, amContainerResource);
+        baselineStartTimeMS, amContainerResource);
     amtype = "mapreduce";
-    
+
     // get map/reduce tasks
     for (ContainerSimulator cs : containerList) {
       if (cs.getType().equals("map")) {
@@ -202,7 +206,7 @@ public class MRAMSimulator extends AMSimulator {
           }
         }
       }
-      
+
       // check finished
       if (isAMContainerRunning &&
               (mapFinished >= mapTotal) &&
@@ -234,7 +238,7 @@ public class MRAMSimulator extends AMSimulator {
       }
     }
   }
-  
+
   /**
    * restart running because of the am container killed
    */
@@ -322,7 +326,7 @@ public class MRAMSimulator extends AMSimulator {
     if (ask == null) {
       ask = new ArrayList<>();
     }
-    
+
     final AllocateRequest request = createAllocateRequest(ask);
     if (totalContainers == 0) {
       request.setProgress(1.0f);
@@ -349,6 +353,38 @@ public class MRAMSimulator extends AMSimulator {
   }
 
   @Override
+  public void initReservation(ReservationId reservationId, long deadline,
+      long now) {
+
+    Resource mapRes = getMaxResource(allMaps);
+    long mapDur = getMaxDuration(allMaps);
+    Resource redRes = getMaxResource(allReduces);
+    long redDur = getMaxDuration(allReduces);
+
+    ReservationSubmissionRequest rr = ReservationClientUtil.
+        createMRReservation(reservationId,
+            "reservation_" + reservationId.getId(), mapRes, allMaps.size(),
+            mapDur, redRes, allReduces.size(), redDur, now + traceStartTimeMS,
+            now + deadline, queue);
+
+    setReservationRequest(rr);
+  }
+
+  // Helper to compute the component-wise maximum resource used by any container
+  private Resource getMaxResource(Collection<ContainerSimulator> containers) {
+    return containers.parallelStream()
+        .map(ContainerSimulator::getResource)
+        .reduce(Resource.newInstance(0, 0), Resources::componentwiseMax);
+  }
+
+  // Helper to compute the maximum resource used by any map container
+  private long getMaxDuration(Collection<ContainerSimulator> containers) {
+    return containers.parallelStream()
+        .mapToLong(ContainerSimulator::getLifeTime)
+        .reduce(0L, Long::max);
+  }
+
+  @Override
   protected void checkStop() {
     if (isFinished) {
       super.setEndTime(System.currentTimeMillis());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/16be42d3/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSLSRunner.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSLSRunner.java b/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSLSRunner.java
index 567f0d9..abb3b5e 100644
--- a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSLSRunner.java
+++ b/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSLSRunner.java
@@ -78,7 +78,7 @@ public class TestSLSRunner extends BaseSLSRunnerTest {
     exitInvariantFile = "src/test/resources/exit-invariants.txt";
   }
 
-  @Test(timeout = 60000)
+  @Test(timeout = 90000)
   @SuppressWarnings("all")
   public void testSimulatorRunning() throws Exception {
     Configuration conf = new Configuration(false);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/16be42d3/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/appmaster/TestAMSimulator.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/appmaster/TestAMSimulator.java b/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/appmaster/TestAMSimulator.java
index 02dc26e..a67845b 100644
--- a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/appmaster/TestAMSimulator.java
+++ b/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/appmaster/TestAMSimulator.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.yarn.sls.appmaster;
 
 import com.codahale.metrics.MetricRegistry;
 import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
@@ -90,6 +91,10 @@ public class TestAMSimulator {
     }
 
     @Override
+    public void initReservation(ReservationId id, long deadline, long now){
+    }
+
+    @Override
     protected void checkStop() {
     }
   }
@@ -134,7 +139,7 @@ public class TestAMSimulator {
     String queue = "default";
     List<ContainerSimulator> containers = new ArrayList<>();
     app.init(1000, containers, rm, null, 0, 1000000L, "user1", queue, true,
-        appId, null, 0, SLSConfiguration.getAMContainerResource(conf));
+        appId, 0, SLSConfiguration.getAMContainerResource(conf));
     app.firstStep();
 
     verifySchedulerMetrics(appId);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[36/37] hadoop git commit: YARN-7811. Fixed a bug in user defined docker network settings. (Contributed by Billie Rinaldi)

Posted by ae...@apache.org.
YARN-7811.  Fixed a bug in user defined docker network settings.  (Contributed by Billie Rinaldi)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/f9dd5b61
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/f9dd5b61
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/f9dd5b61

Branch: refs/heads/HDFS-7240
Commit: f9dd5b61f4ed0288cc01cb1a676df8c9cd69cdd9
Parents: 901d15a
Author: Eric Yang <ey...@apache.org>
Authored: Tue Jan 30 12:42:38 2018 -0500
Committer: Eric Yang <ey...@apache.org>
Committed: Tue Jan 30 12:42:38 2018 -0500

----------------------------------------------------------------------
 .../containerlaunch/AbstractLauncher.java       | 22 ++++++++++----------
 .../service/provider/docker/DockerKeys.java     |  7 -------
 .../provider/docker/DockerProviderService.java  |  2 +-
 3 files changed, 12 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f9dd5b61/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/AbstractLauncher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/AbstractLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/AbstractLauncher.java
index e1e88cd..f497985 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/AbstractLauncher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/AbstractLauncher.java
@@ -39,8 +39,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 
-import static org.apache.hadoop.yarn.service.provider.docker.DockerKeys.DEFAULT_DOCKER_NETWORK;
-
 /**
  * Launcher of applications: base class
  */
@@ -60,9 +58,9 @@ public class AbstractLauncher {
   private final Map<String, ByteBuffer> serviceData = new HashMap<>();
   protected boolean yarnDockerMode = false;
   protected String dockerImage;
-  protected String dockerNetwork = DEFAULT_DOCKER_NETWORK;
+  protected String dockerNetwork;
   protected String dockerHostname;
-  protected String runPrivilegedContainer;
+  protected boolean runPrivilegedContainer = false;
   private ServiceContext context;
 
   public AbstractLauncher(ServiceContext context) {
@@ -145,10 +143,16 @@ public class AbstractLauncher {
       Map<String, String> env = containerLaunchContext.getEnvironment();
       env.put("YARN_CONTAINER_RUNTIME_TYPE", "docker");
       env.put("YARN_CONTAINER_RUNTIME_DOCKER_IMAGE", dockerImage);
-      env.put("YARN_CONTAINER_RUNTIME_DOCKER_CONTAINER_NETWORK", dockerNetwork);
+      if (ServiceUtils.isSet(dockerNetwork)) {
+        env.put("YARN_CONTAINER_RUNTIME_DOCKER_CONTAINER_NETWORK",
+            dockerNetwork);
+      }
       env.put("YARN_CONTAINER_RUNTIME_DOCKER_CONTAINER_HOSTNAME",
           dockerHostname);
-      env.put("YARN_CONTAINER_RUNTIME_DOCKER_RUN_PRIVILEGED_CONTAINER", runPrivilegedContainer);
+      if (runPrivilegedContainer) {
+        env.put("YARN_CONTAINER_RUNTIME_DOCKER_RUN_PRIVILEGED_CONTAINER",
+            "true");
+      }
       StringBuilder sb = new StringBuilder();
       for (Entry<String,String> mount : mountPaths.entrySet()) {
         if (sb.length() > 0) {
@@ -238,11 +242,7 @@ public class AbstractLauncher {
   }
 
   public void setRunPrivilegedContainer(boolean runPrivilegedContainer) {
-    if (runPrivilegedContainer) {
-      this.runPrivilegedContainer = Boolean.toString(true);
-    } else {
-      this.runPrivilegedContainer = Boolean.toString(false);
-    }
+    this.runPrivilegedContainer = runPrivilegedContainer;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f9dd5b61/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/docker/DockerKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/docker/DockerKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/docker/DockerKeys.java
index f30c002..992a40c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/docker/DockerKeys.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/docker/DockerKeys.java
@@ -18,13 +18,6 @@
 package org.apache.hadoop.yarn.service.provider.docker;
 
 public interface DockerKeys {
-  String PROVIDER_DOCKER = "docker";
   String DOCKER_PREFIX = "docker.";
-  String DOCKER_IMAGE = DOCKER_PREFIX + "image";
   String DOCKER_NETWORK = DOCKER_PREFIX + "network";
-  String DOCKER_USE_PRIVILEGED = DOCKER_PREFIX + "usePrivileged";
-  String DOCKER_START_COMMAND = DOCKER_PREFIX + "startCommand";
-
-  String DEFAULT_DOCKER_NETWORK = "bridge";
-  Boolean DEFAULT_DOCKER_USE_PRIVILEGED = false;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f9dd5b61/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/docker/DockerProviderService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/docker/DockerProviderService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/docker/DockerProviderService.java
index 0741947..6ac8de1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/docker/DockerProviderService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/docker/DockerProviderService.java
@@ -37,7 +37,7 @@ public class DockerProviderService extends AbstractProviderService
     launcher.setYarnDockerMode(true);
     launcher.setDockerImage(compInstance.getCompSpec().getArtifact().getId());
     launcher.setDockerNetwork(compInstance.getCompSpec().getConfiguration()
-        .getProperty(DOCKER_NETWORK, DEFAULT_DOCKER_NETWORK));
+        .getProperty(DOCKER_NETWORK));
     String domain = compInstance.getComponent().getScheduler().getConfig()
         .get(RegistryConstants.KEY_DNS_DOMAIN);
     String hostname;


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[29/37] hadoop git commit: MAPREDUCE-7036. ASF License warning in hadoop-mapreduce-client

Posted by ae...@apache.org.
MAPREDUCE-7036. ASF License warning in hadoop-mapreduce-client

Signed-off-by: Akira Ajisaka <aa...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/56feaa40
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/56feaa40
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/56feaa40

Branch: refs/heads/HDFS-7240
Commit: 56feaa40bb94fcaa96ae668eebfabec4611928c0
Parents: e9c72d0
Author: Takanobu Asanuma <ta...@yahoo-corp.jp>
Authored: Tue Jan 30 00:36:33 2018 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Tue Jan 30 00:39:39 2018 +0900

----------------------------------------------------------------------
 .../apache/hadoop/mapred/pipes/Application.java  |  5 +++--
 .../hadoop/mapred/pipes/TestPipeApplication.java | 19 +++++++++++--------
 2 files changed, 14 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/56feaa40/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/pipes/Application.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/pipes/Application.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/pipes/Application.java
index 83d2509..5c8aab9 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/pipes/Application.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/pipes/Application.java
@@ -45,6 +45,7 @@ import org.apache.hadoop.mapred.RecordReader;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.TaskAttemptID;
 import org.apache.hadoop.mapred.TaskLog;
+import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.filecache.DistributedCache;
 import org.apache.hadoop.mapreduce.security.SecureShuffleUtils;
@@ -103,8 +104,8 @@ class Application<K1 extends WritableComparable, V1 extends Writable,
     // This password is used as shared secret key between this application and
     // child pipes process
     byte[]  password = jobToken.getPassword();
-    String localPasswordFile = new File(".") + Path.SEPARATOR
-        + "jobTokenPassword";
+    String localPasswordFile = new File(conf.get(MRConfig.LOCAL_DIR))
+        + Path.SEPARATOR + "jobTokenPassword";
     writePasswordToLocalFile(localPasswordFile, password, conf);
     env.put("hadoop.pipes.shared.secret.location", localPasswordFile);
  

http://git-wip-us.apache.org/repos/asf/hadoop/blob/56feaa40/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/TestPipeApplication.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/TestPipeApplication.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/TestPipeApplication.java
index 88d8f95..13597e0 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/TestPipeApplication.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/TestPipeApplication.java
@@ -47,6 +47,7 @@ import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapred.IFile.Writer;
+import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.security.TokenCache;
 import org.apache.hadoop.mapred.Counters;
@@ -83,10 +84,10 @@ public class TestPipeApplication {
   public void testRunner() throws Exception {
 
     // clean old password files
-    File[] psw = cleanTokenPasswordFile();
+    JobConf conf = new JobConf();
+    File[] psw = cleanTokenPasswordFile(conf);
     try {
       RecordReader<FloatWritable, NullWritable> rReader = new ReaderPipesMapRunner();
-      JobConf conf = new JobConf();
       conf.set(Submitter.IS_JAVA_RR, "true");
       // for stdour and stderror
 
@@ -162,7 +163,7 @@ public class TestPipeApplication {
 
     TestTaskReporter reporter = new TestTaskReporter();
 
-    File[] psw = cleanTokenPasswordFile();
+    File[] psw = cleanTokenPasswordFile(conf);
     try {
 
       conf.set(MRJobConfig.TASK_ATTEMPT_ID, taskName);
@@ -247,7 +248,7 @@ public class TestPipeApplication {
 
     JobConf conf = new JobConf();
 
-    File[] psw = cleanTokenPasswordFile();
+    File[] psw = cleanTokenPasswordFile(conf);
 
     System.setProperty("test.build.data",
             "target/tmp/build/TEST_SUBMITTER_MAPPER/data");
@@ -388,8 +389,8 @@ public class TestPipeApplication {
   @Test
   public void testPipesReduser() throws Exception {
 
-    File[] psw = cleanTokenPasswordFile();
     JobConf conf = new JobConf();
+    File[] psw = cleanTokenPasswordFile(conf);
     try {
       Token<AMRMTokenIdentifier> token = new Token<AMRMTokenIdentifier>(
               "user".getBytes(), "password".getBytes(), new Text("kind"), new Text(
@@ -506,14 +507,16 @@ public class TestPipeApplication {
 
   }
 
-  private File[] cleanTokenPasswordFile() throws Exception {
+  private File[] cleanTokenPasswordFile(JobConf conf) throws Exception {
     File[] result = new File[2];
-    result[0] = new File("./jobTokenPassword");
+    result[0] = new File(conf.get(MRConfig.LOCAL_DIR) + Path.SEPARATOR
+        + "jobTokenPassword");
     if (result[0].exists()) {
       FileUtil.chmod(result[0].getAbsolutePath(), "700");
       assertTrue(result[0].delete());
     }
-    result[1] = new File("./.jobTokenPassword.crc");
+    result[1] = new File(conf.get(MRConfig.LOCAL_DIR) + Path.SEPARATOR
+        + ".jobTokenPassword.crc");
     if (result[1].exists()) {
       FileUtil.chmod(result[1].getAbsolutePath(), "700");
       result[1].delete();


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[26/37] hadoop git commit: HDFS-12974. Exception message is not printed when creating an encryption zone fails with AuthorizationException. Contributed by fang zhenyi.

Posted by ae...@apache.org.
HDFS-12974. Exception message is not printed when creating an encryption zone fails with AuthorizationException. Contributed by fang zhenyi.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/b63dcd58
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b63dcd58
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b63dcd58

Branch: refs/heads/HDFS-7240
Commit: b63dcd583f0b98e785831004f41bd7c7de8b3c18
Parents: 6bc2f7f
Author: Xiao Chen <xi...@apache.org>
Authored: Sun Jan 28 22:15:58 2018 -0800
Committer: Xiao Chen <xi...@apache.org>
Committed: Sun Jan 28 22:19:49 2018 -0800

----------------------------------------------------------------------
 .../authorize/AuthorizationException.java       |  6 ++--
 .../namenode/EncryptionFaultInjector.java       |  3 ++
 .../server/namenode/FSDirEncryptionZoneOp.java  |  1 +
 .../apache/hadoop/hdfs/TestEncryptionZones.java | 31 +++++++++++++++++++-
 4 files changed, 38 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b63dcd58/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/AuthorizationException.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/AuthorizationException.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/AuthorizationException.java
index 03f4d99..79c7d18 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/AuthorizationException.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/AuthorizationException.java
@@ -64,17 +64,19 @@ public class AuthorizationException extends AccessControlException {
 
   @Override
   public void printStackTrace() {
-    // Do not provide the stack-trace
+    printStackTrace(System.err);
   }
 
   @Override
   public void printStackTrace(PrintStream s) {
     // Do not provide the stack-trace
+    s.println(this);
   }
 
   @Override
   public void printStackTrace(PrintWriter s) {
     // Do not provide the stack-trace
+    s.println(this);
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b63dcd58/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionFaultInjector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionFaultInjector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionFaultInjector.java
index e4a035e..938eacd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionFaultInjector.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionFaultInjector.java
@@ -51,4 +51,7 @@ public class EncryptionFaultInjector {
 
   @VisibleForTesting
   public void reencryptUpdaterProcessCheckpoint() throws IOException {}
+
+  @VisibleForTesting
+  public void ensureKeyIsInitialized() throws IOException {}
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b63dcd58/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java
index 7dcb8ab..943e60d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java
@@ -121,6 +121,7 @@ final class FSDirEncryptionZoneOp {
       throw new IOException("Must specify a key name when creating an "
           + "encryption zone");
     }
+    EncryptionFaultInjector.getInstance().ensureKeyIsInitialized();
     KeyProvider.Metadata metadata = provider.getMetadata(keyName);
     if (metadata == null) {
       /*

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b63dcd58/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
index 4497e23..ea867a8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
@@ -80,9 +80,11 @@ import org.apache.hadoop.hdfs.web.WebHdfsConstants;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.hdfs.web.WebHdfsTestUtil;
 import org.apache.hadoop.io.EnumSetWritable;
+import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.ToolRunner;
@@ -149,6 +151,9 @@ public class TestEncryptionZones {
   private File testRootDir;
   protected final String TEST_KEY = "test_key";
   private static final String NS_METRICS = "FSNamesystem";
+  private static final String  AUTHORIZATION_EXCEPTION_MESSAGE =
+      "User [root] is not authorized to perform [READ] on key " +
+          "with ACL name [key2]!!";
 
   protected FileSystemTestWrapper fsWrapper;
   protected FileContextTestWrapper fcWrapper;
@@ -447,7 +452,6 @@ public class TestEncryptionZones {
     dfsAdmin.createEncryptionZone(zone2, myKeyName, NO_TRASH);
     assertNumZones(++numZones);
     assertZonePresent(myKeyName, zone2.toString());
-
     /* Test failure of create encryption zones as a non super user. */
     final UserGroupInformation user = UserGroupInformation.
         createUserForTesting("user", new String[] { "mygroup" });
@@ -1057,6 +1061,31 @@ public class TestEncryptionZones {
     }
   }
 
+  private class AuthorizationExceptionInjector extends EncryptionFaultInjector {
+    @Override
+    public void ensureKeyIsInitialized() throws IOException {
+      throw new AuthorizationException(AUTHORIZATION_EXCEPTION_MESSAGE);
+    }
+  }
+
+  @Test
+  public void testExceptionInformationReturn() {
+    /* Test exception information can be returned when
+    creating transparent encryption zone.*/
+    final Path zone1 = new Path("/zone1");
+    EncryptionFaultInjector.instance = new AuthorizationExceptionInjector();
+    try {
+      dfsAdmin.createEncryptionZone(zone1, TEST_KEY, NO_TRASH);
+      fail("exception information can be returned when creating " +
+          "transparent encryption zone");
+    } catch (IOException e) {
+      assertTrue(e instanceof RemoteException);
+      assertTrue(((RemoteException) e).unwrapRemoteException()
+          instanceof AuthorizationException);
+      assertExceptionContains(AUTHORIZATION_EXCEPTION_MESSAGE, e);
+    }
+  }
+
   private class MyInjector extends EncryptionFaultInjector {
     volatile int generateCount;
     CountDownLatch ready;


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[37/37] hadoop git commit: Merge branch 'trunk' into HDFS-7240

Posted by ae...@apache.org.
Merge branch 'trunk' into HDFS-7240

 Conflicts:
	hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/7de498c3
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/7de498c3
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/7de498c3

Branch: refs/heads/HDFS-7240
Commit: 7de498c3ab2bcd460388cde51c166d7c118518f9
Parents: 7955ee4 f9dd5b6
Author: Anu Engineer <ae...@apache.org>
Authored: Tue Jan 30 11:28:14 2018 -0800
Committer: Anu Engineer <ae...@apache.org>
Committed: Tue Jan 30 11:28:14 2018 -0800

----------------------------------------------------------------------
 .../hadoop-common/src/main/conf/hadoop-env.cmd  |   9 +
 .../hadoop-common/src/main/conf/hadoop-env.sh   |   9 +
 .../main/java/org/apache/hadoop/io/MapFile.java |  35 +-
 .../authorize/AuthorizationException.java       |   6 +-
 .../org/apache/hadoop/util/CpuTimeTracker.java  |   2 +-
 .../java/org/apache/hadoop/io/TestMapFile.java  |  59 ++-
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  52 +--
 .../org/apache/hadoop/hdfs/HdfsKMSUtil.java     |  41 +++
 .../hadoop/hdfs/protocol/ClientProtocol.java    |   3 +
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      | 101 +++++-
 .../hdfs/web/TestWebHdfsContentLength.java      |   2 +
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |  21 ++
 .../hdfs/server/federation/router/Router.java   |  85 +++++
 .../router/RouterHeartbeatService.java          | 155 ++++++++
 .../federation/router/RouterRpcServer.java      |  43 ++-
 .../router/RouterSafeModeException.java         |  53 +++
 .../router/RouterSafemodeService.java           | 150 ++++++++
 .../federation/router/RouterServiceState.java   |   2 +-
 .../store/StateStoreCacheUpdateService.java     |   7 +-
 .../federation/store/StateStoreService.java     |   9 +
 .../namenode/EncryptionFaultInjector.java       |   3 +
 .../server/namenode/FSDirEncryptionZoneOp.java  |   1 +
 .../web/resources/NamenodeWebHdfsMethods.java   |  85 +++--
 .../src/main/resources/hdfs-default.xml         |  56 ++-
 .../markdown/HDFSHighAvailabilityWithQJM.md     |   2 +-
 .../src/site/markdown/HDFSRouterFederation.md   |   4 +
 .../hadoop/hdfs/TestDistributedFileSystem.java  |  17 +
 .../apache/hadoop/hdfs/TestEncryptionZones.java | 219 +++++++++++-
 .../hdfs/TestErasureCodingMultipleRacks.java    |   8 +-
 .../server/federation/RouterConfigBuilder.java  |  13 +
 .../federation/router/TestRouterSafemode.java   | 192 ++++++++++
 .../store/TestStateStoreRouterState.java        | 194 ++++++++++
 .../web/resources/TestWebHdfsDataLocality.java  |  23 +-
 .../org/apache/hadoop/hdfs/web/TestWebHDFS.java |   1 -
 .../hadoop/hdfs/web/TestWebHdfsTokens.java      |   4 +-
 .../hadoop/mapred/LocalContainerLauncher.java   |   2 +-
 .../hadoop/mapred/TaskAttemptListenerImpl.java  |  15 +-
 .../org/apache/hadoop/mapred/YarnChild.java     |   4 +-
 .../hadoop/mapreduce/v2/app/MRAppMaster.java    |   2 +-
 .../v2/app/job/event/TaskAttemptFailEvent.java  |  53 +++
 .../app/job/event/TaskTAttemptFailedEvent.java  |  39 ++
 .../v2/app/job/impl/TaskAttemptImpl.java        |  40 ++-
 .../mapreduce/v2/app/job/impl/TaskImpl.java     |   6 +-
 .../mapred/TestTaskAttemptListenerImpl.java     |   6 +-
 .../hadoop/mapreduce/v2/app/TestFail.java       |   7 +-
 .../hadoop/mapreduce/v2/app/TestRecovery.java   |   7 +-
 .../mapreduce/v2/app/job/impl/TestJobImpl.java  |   5 +-
 .../v2/app/job/impl/TestTaskAttempt.java        |   9 +-
 .../mapreduce/v2/app/job/impl/TestTaskImpl.java |  42 +--
 .../apache/hadoop/mapred/LocalJobRunner.java    |   4 +-
 .../java/org/apache/hadoop/mapred/MapTask.java  |   3 +-
 .../java/org/apache/hadoop/mapred/Task.java     | 101 +++++-
 .../hadoop/mapred/TaskUmbilicalProtocol.java    |  12 +-
 .../apache/hadoop/mapred/pipes/Application.java |   5 +-
 .../apache/hadoop/mapreduce/MRJobConfig.java    |  14 +
 .../src/main/resources/mapred-default.xml       |  22 ++
 .../hadoop/mapred/TestTaskProgressReporter.java |  90 ++++-
 .../mapreduce/v2/hs/CachedHistoryStorage.java   |   8 +-
 .../mapreduce/v2/hs/HistoryFileManager.java     |  30 +-
 .../hadoop/mapreduce/v2/hs/TestJobHistory.java  |  26 ++
 .../mapreduce/v2/hs/TestJobHistoryParsing.java  |   9 +-
 .../apache/hadoop/mapred/TestMapProgress.java   |   4 +-
 .../apache/hadoop/mapred/TestTaskCommit.java    |   2 +-
 .../mapred/pipes/TestPipeApplication.java       |  19 +-
 hadoop-tools/hadoop-azure-datalake/pom.xml      |   3 +-
 .../org/apache/hadoop/yarn/sls/SLSRunner.java   |  75 ++--
 .../hadoop/yarn/sls/appmaster/AMSimulator.java  |  41 ++-
 .../yarn/sls/appmaster/MRAMSimulator.java       |  74 +++-
 .../yarn/sls/nodemanager/NMSimulator.java       |   4 +-
 .../hadoop/yarn/sls/nodemanager/NodeInfo.java   |  11 +-
 .../yarn/sls/scheduler/RMNodeWrapper.java       |  12 +-
 .../apache/hadoop/yarn/sls/TestSLSRunner.java   |   2 +-
 .../yarn/sls/appmaster/TestAMSimulator.java     |   7 +-
 .../hadoop/yarn/conf/YarnConfiguration.java     |  10 +-
 .../yarn/conf/TestYarnConfigurationFields.java  |   2 -
 .../hadoop/yarn/service/ServiceScheduler.java   |  11 +-
 .../containerlaunch/AbstractLauncher.java       |  22 +-
 .../service/provider/docker/DockerKeys.java     |   7 -
 .../provider/docker/DockerProviderService.java  |   2 +-
 .../yarn/service/utils/ServiceApiUtil.java      |  15 +-
 .../yarn/util/ProcfsBasedProcessTree.java       |   3 +
 .../util/ResourceCalculatorProcessTree.java     |   9 +
 .../src/main/resources/yarn-default.xml         |  34 +-
 .../client/binding/RegistryPathUtils.java       |  20 ++
 .../registry/client/binding/RegistryUtils.java  |   5 +-
 .../server/dns/BaseServiceRecordProcessor.java  |  22 +-
 .../client/binding/TestRegistryPathUtils.java   |  10 +-
 .../yarn/server/nodemanager/NodeManager.java    |  11 +-
 .../launcher/ContainerLaunch.java               |   2 +-
 .../linux/resources/CGroupsHandler.java         |   9 +-
 .../linux/resources/CGroupsHandlerImpl.java     |   3 +-
 .../CGroupsMemoryResourceHandlerImpl.java       |  52 +--
 .../resources/CGroupsResourceCalculator.java    | 357 +++++++++++++++++++
 .../resources/CombinedResourceCalculator.java   | 108 ++++++
 .../linux/resources/ResourceHandlerModule.java  |  43 ++-
 .../runtime/DockerLinuxContainerRuntime.java    |  44 ++-
 .../monitor/ContainersMonitorImpl.java          |  33 +-
 .../timelineservice/NMTimelinePublisher.java    |   1 +
 .../TestCGroupsMemoryResourceHandlerImpl.java   |  45 +++
 .../TestCGroupsResourceCalculator.java          | 274 ++++++++++++++
 .../TestCompareResourceCalculators.java         | 227 ++++++++++++
 .../resources/TestResourceHandlerModule.java    |  47 ++-
 .../runtime/TestDockerContainerRuntime.java     | 272 +++++++-------
 .../server/resourcemanager/ResourceManager.java |   8 +-
 .../resourcemanager/ResourceTrackerService.java |  67 +++-
 .../server/resourcemanager/rmnode/RMNode.java   |  13 +-
 .../resourcemanager/rmnode/RMNodeImpl.java      |  47 +--
 .../rmnode/RMNodeStatusEvent.java               |  13 +-
 .../scheduler/AbstractYarnScheduler.java        |  51 +--
 .../resourcemanager/scheduler/QueueMetrics.java |  18 +
 .../scheduler/SchedulerNode.java                |  16 +
 .../scheduler/capacity/CapacityScheduler.java   |  68 +++-
 .../webapp/JAXBContextResolver.java             |   2 +-
 .../resourcemanager/webapp/dao/AppInfo.java     |  11 +
 .../webapp/dao/ClusterMetricsInfo.java          |  24 ++
 .../resourcemanager/webapp/dao/NodeInfo.java    |  20 ++
 .../webapp/dao/ResourceInfo.java                |  18 +-
 .../webapp/dao/ResourceInformationsInfo.java    |  48 +++
 .../yarn/server/resourcemanager/MockNM.java     |  18 +-
 .../yarn/server/resourcemanager/MockNodes.java  |   9 +-
 .../TestRMHAForAsyncScheduler.java              |  38 +-
 .../resourcemanager/TestRMNodeTransitions.java  |  14 +-
 .../TestResourceTrackerService.java             |  35 +-
 .../TestRMAppLogAggregationStatus.java          |  10 +-
 .../TestCapacitySchedulerAsyncScheduling.java   | 159 ++++++++-
 ...TestCapacitySchedulerSurgicalPreemption.java |   4 +
 .../webapp/TestRMWebServicesApps.java           |   2 +-
 .../storage/HBaseTimelineWriterImpl.java        |   5 +
 .../src/site/markdown/CapacityScheduler.md      |   2 +-
 .../markdown/yarn-service/Configurations.md     |   2 +-
 .../hadoop-yarn/hadoop-yarn-ui/pom.xml          |   4 +-
 .../src/main/webapp/app/adapters/yarn-conf.js   |  79 ++++
 .../main/webapp/app/adapters/yarn-metrics.js    |  76 ++++
 .../src/main/webapp/app/adapters/yarn-rm-log.js |  76 ++++
 .../main/webapp/app/components/timeline-view.js |   5 +-
 .../main/webapp/app/controllers/yarn-tools.js   |  29 ++
 .../app/controllers/yarn-tools/yarn-conf.js     |  48 +++
 .../app/controllers/yarn-tools/yarn-rm-log.js   |  24 ++
 .../src/main/webapp/app/helpers/json-pretty.js  |  25 ++
 .../main/webapp/app/models/cluster-metric.js    |   4 +-
 .../src/main/webapp/app/models/yarn-app.js      |  96 ++---
 .../src/main/webapp/app/models/yarn-conf.js     |  25 ++
 .../src/main/webapp/app/models/yarn-metrics.js  |  23 ++
 .../src/main/webapp/app/models/yarn-rm-log.js   |  23 ++
 .../src/main/webapp/app/models/yarn-rm-node.js  |   4 +-
 .../src/main/webapp/app/router.js               |   6 +
 .../src/main/webapp/app/routes/yarn-tools.js    |  22 ++
 .../webapp/app/routes/yarn-tools/yarn-conf.js   |  22 ++
 .../app/routes/yarn-tools/yarn-metrics.js       |  43 +++
 .../webapp/app/routes/yarn-tools/yarn-rm-log.js |  36 ++
 .../src/main/webapp/app/serializers/yarn-app.js |   1 +
 .../main/webapp/app/serializers/yarn-conf.js    |  43 +++
 .../main/webapp/app/serializers/yarn-metrics.js |  33 ++
 .../main/webapp/app/serializers/yarn-rm-log.js  |  45 +++
 .../main/webapp/app/serializers/yarn-rm-node.js |   4 +-
 .../app/serializers/yarn-timeline-container.js  |  16 +-
 .../src/main/webapp/app/styles/app.scss         |   6 +
 .../main/webapp/app/templates/application.hbs   |   5 +
 .../src/main/webapp/app/templates/yarn-app.hbs  |   2 +-
 .../main/webapp/app/templates/yarn-tools.hbs    | 108 ++++++
 .../app/templates/yarn-tools/yarn-conf.hbs      |  28 ++
 .../app/templates/yarn-tools/yarn-metrics.hbs   |  33 ++
 .../app/templates/yarn-tools/yarn-rm-log.hbs    |  42 +++
 .../src/main/webapp/app/utils/date-utils.js     |   2 +-
 .../hadoop-yarn-ui/src/main/webapp/bower.json   |   3 +-
 .../src/main/webapp/ember-cli-build.js          |   1 +
 .../tests/unit/adapters/yarn-conf-test.js       |  29 ++
 .../tests/unit/adapters/yarn-metrics-test.js    |  30 ++
 .../tests/unit/adapters/yarn-rm-log-test.js     |  30 ++
 .../tests/unit/controllers/yarn-conf-test.js    |  29 ++
 .../tests/unit/controllers/yarn-rm-log-test.js  |  30 ++
 .../tests/unit/controllers/yarn-tools-test.js   |  30 ++
 .../tests/unit/helpers/json-pretty-test.js      |  28 ++
 .../webapp/tests/unit/models/yarn-conf-test.js  |  29 ++
 .../tests/unit/models/yarn-metrics-test.js      |  30 ++
 .../tests/unit/models/yarn-rm-log-test.js       |  30 ++
 .../webapp/tests/unit/routes/yarn-conf-test.js  |  28 ++
 .../tests/unit/routes/yarn-metrics-test.js      |  28 ++
 .../tests/unit/routes/yarn-rm-log-test.js       |  28 ++
 .../webapp/tests/unit/routes/yarn-tools-test.js |  28 ++
 .../tests/unit/serializers/yarn-conf-test.js    |  32 ++
 .../tests/unit/serializers/yarn-metrics-test.js |  33 ++
 .../tests/unit/serializers/yarn-rm-log-test.js  |  33 ++
 183 files changed, 5644 insertions(+), 806 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7de498c3/hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh
----------------------------------------------------------------------
diff --cc hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh
index 7dcf5e6,24aacdf..257b2fd
--- a/hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh
+++ b/hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh
@@@ -396,40 -396,13 +396,49 @@@ esa
  # export HDFS_MOVER_OPTS=""
  
  ###
+ # Router-based HDFS Federation specific parameters
+ # Specify the JVM options to be used when starting the RBF Routers.
+ # These options will be appended to the options specified as HADOOP_OPTS
+ # and therefore may override any similar flags set in HADOOP_OPTS
+ #
+ # export HDFS_DFSROUTER_OPTS=""
++
++
++###
 +# HDFS CBlock Server specific parameters
 +###
 +# Specify the JVM options to be used when starting the HDFS CBlock Server.
 +# These options will be appended to the options specified as HADOOP_OPTS
 +# and therefore may override any similar flags set in HADOOP_OPTS
 +#
 +# export HDFS_CBLOCKSERVER_OPTS=""
 +
 +###
 +# HDFS JSCSI specific parameters
 +###
 +# Specify the JVM options to be used when starting the HDFS JSCSI.
 +# These options will be appended to the options specified as HADOOP_OPTS
 +# and therefore may override any similar flags set in HADOOP_OPTS
 +#
 +# export HDFS_JSCSI_OPTS=""
 +
 +###
 +# HDFS Key Space Manager specific parameters
 +###
 +# Specify the JVM options to be used when starting the HDFS Key Space Manager.
 +# These options will be appended to the options specified as HADOOP_OPTS
 +# and therefore may override any similar flags set in HADOOP_OPTS
 +#
 +# export HDFS_KSM_OPTS=""
 +
 +###
 +# HDFS StorageContainerManager specific parameters
  ###
 +# Specify the JVM options to be used when starting the HDFS Storage Container Manager.
 +# These options will be appended to the options specified as HADOOP_OPTS
 +# and therefore may override any similar flags set in HADOOP_OPTS
 +#
 +# export HDFS_STORAGECONTAINERMANAGER_OPTS=""
  
  ###
  # Advanced Users Only!


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[12/37] hadoop git commit: MAPREDUCE-7041. MR should not try to clean up at first job attempt. (Gergo Repas via Haibo Chen)

Posted by ae...@apache.org.
MAPREDUCE-7041. MR should not try to clean up at first job attempt. (Gergo Repas via Haibo Chen)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/cc108522
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/cc108522
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/cc108522

Branch: refs/heads/HDFS-7240
Commit: cc10852252c2d69294eabc68bd032cc630a53b18
Parents: ff8378e
Author: Haibo Chen <ha...@apache.org>
Authored: Thu Jan 25 16:11:01 2018 -0800
Committer: Haibo Chen <ha...@apache.org>
Committed: Thu Jan 25 16:11:30 2018 -0800

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc108522/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
index e6a45cf..cb65b89 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
@@ -1400,7 +1400,7 @@ public class MRAppMaster extends CompositeService {
 
   private void cleanUpPreviousJobOutput() {
     // recovered application masters should not remove data from previous job
-    if (!recovered()) {
+    if (!isFirstAttempt() && !recovered()) {
       JobContext jobContext = getJobContextFromConf(getConfig());
       try {
         LOG.info("Starting to clean up previous job's temporary files");


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[07/37] hadoop git commit: HDFS-13042. RBF: Heartbeat Router State. Contributed by Inigo Goiri.

Posted by ae...@apache.org.
HDFS-13042. RBF: Heartbeat Router State. Contributed by Inigo Goiri.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/7721fff7
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/7721fff7
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/7721fff7

Branch: refs/heads/HDFS-7240
Commit: 7721fff74494eb7fbbbba7f8bb4b4692d880d301
Parents: eb2dd08
Author: Yiqun Lin <yq...@apache.org>
Authored: Thu Jan 25 15:51:26 2018 +0800
Committer: Yiqun Lin <yq...@apache.org>
Committed: Thu Jan 25 15:51:26 2018 +0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   8 +
 .../hdfs/server/federation/router/Router.java   |  71 +++++++
 .../router/RouterHeartbeatService.java          | 155 +++++++++++++++
 .../federation/router/RouterServiceState.java   |   2 +-
 .../federation/store/StateStoreService.java     |   9 +
 .../src/main/resources/hdfs-default.xml         |  20 ++
 .../store/TestStateStoreRouterState.java        | 194 +++++++++++++++++++
 7 files changed, 458 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7721fff7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index f53badc..84215f3f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -1222,6 +1222,10 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String DFS_ROUTER_MONITOR_LOCAL_NAMENODE =
       FEDERATION_ROUTER_PREFIX + "monitor.localnamenode.enable";
   public static final boolean DFS_ROUTER_MONITOR_LOCAL_NAMENODE_DEFAULT = true;
+  public static final String DFS_ROUTER_HEARTBEAT_STATE_INTERVAL_MS =
+      FEDERATION_ROUTER_PREFIX + "heartbeat-state.interval";
+  public static final long DFS_ROUTER_HEARTBEAT_STATE_INTERVAL_MS_DEFAULT =
+      TimeUnit.SECONDS.toMillis(5);
 
   // HDFS Router NN client
   public static final String DFS_ROUTER_NAMENODE_CONNECTION_POOL_SIZE =
@@ -1282,6 +1286,10 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
       FEDERATION_STORE_PREFIX + "membership.expiration";
   public static final long FEDERATION_STORE_MEMBERSHIP_EXPIRATION_MS_DEFAULT =
       TimeUnit.MINUTES.toMillis(5);
+  public static final String FEDERATION_STORE_ROUTER_EXPIRATION_MS =
+      FEDERATION_STORE_PREFIX + "router.expiration";
+  public static final long FEDERATION_STORE_ROUTER_EXPIRATION_MS_DEFAULT =
+      TimeUnit.MINUTES.toMillis(5);
 
   // HDFS Router-based federation mount table entries
   /** Maximum number of cache entries to have. */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7721fff7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
index ea8a1c0..1e72c93 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
@@ -37,11 +37,13 @@ import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.server.federation.metrics.FederationMetrics;
 import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
 import org.apache.hadoop.hdfs.server.federation.resolver.FileSubclusterResolver;
+import org.apache.hadoop.hdfs.server.federation.store.RouterStore;
 import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.metrics2.source.JvmMetrics;
 import org.apache.hadoop.service.CompositeService;
 import org.apache.hadoop.util.JvmPauseMonitor;
+import org.apache.hadoop.util.Time;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -112,6 +114,18 @@ public class Router extends CompositeService {
   /** Quota cache manager. */
   private RouterQuotaManager quotaManager;
 
+  /** Manages the current state of the router. */
+  private RouterStore routerStateManager;
+  /** Heartbeat our run status to the router state manager. */
+  private RouterHeartbeatService routerHeartbeatService;
+
+  /** The start time of the namesystem. */
+  private final long startTime = Time.now();
+
+  /** State of the Router. */
+  private RouterServiceState state = RouterServiceState.UNINITIALIZED;
+
+
   /////////////////////////////////////////////////////////
   // Constructor
   /////////////////////////////////////////////////////////
@@ -127,6 +141,7 @@ public class Router extends CompositeService {
   @Override
   protected void serviceInit(Configuration configuration) throws Exception {
     this.conf = configuration;
+    updateRouterState(RouterServiceState.INITIALIZING);
 
     if (conf.getBoolean(
         DFSConfigKeys.DFS_ROUTER_STORE_ENABLE,
@@ -188,6 +203,10 @@ public class Router extends CompositeService {
       if (this.namenodeHearbeatServices.isEmpty()) {
         LOG.error("Heartbeat is enabled but there are no namenodes to monitor");
       }
+
+      // Periodically update the router state
+      this.routerHeartbeatService = new RouterHeartbeatService(this);
+      addService(this.routerHeartbeatService);
     }
 
     // Router metrics system
@@ -219,6 +238,8 @@ public class Router extends CompositeService {
   @Override
   protected void serviceStart() throws Exception {
 
+    updateRouterState(RouterServiceState.RUNNING);
+
     if (this.pauseMonitor != null) {
       this.pauseMonitor.start();
       JvmMetrics jvmMetrics = this.metrics.getJvmMetrics();
@@ -233,6 +254,9 @@ public class Router extends CompositeService {
   @Override
   protected void serviceStop() throws Exception {
 
+    // Update state
+    updateRouterState(RouterServiceState.SHUTDOWN);
+
     // JVM pause monitor
     if (this.pauseMonitor != null) {
       this.pauseMonitor.stop();
@@ -454,6 +478,31 @@ public class Router extends CompositeService {
   }
 
   /////////////////////////////////////////////////////////
+  // Router State Management
+  /////////////////////////////////////////////////////////
+
+  /**
+   * Update the router state and heartbeat to the state store.
+   *
+   * @param state The new router state.
+   */
+  public void updateRouterState(RouterServiceState newState) {
+    this.state = newState;
+    if (this.routerHeartbeatService != null) {
+      this.routerHeartbeatService.updateStateAsync();
+    }
+  }
+
+  /**
+   * Get the status of the router.
+   *
+   * @return Status of the router.
+   */
+  public RouterServiceState getRouterState() {
+    return this.state;
+  }
+
+  /////////////////////////////////////////////////////////
   // Submodule getters
   /////////////////////////////////////////////////////////
 
@@ -508,11 +557,33 @@ public class Router extends CompositeService {
     return this.namenodeResolver;
   }
 
+  /**
+   * Get the state store interface for the router heartbeats.
+   *
+   * @return FederationRouterStateStore state store API handle.
+   */
+  public RouterStore getRouterStateManager() {
+    if (this.routerStateManager == null && this.stateStore != null) {
+      this.routerStateManager = this.stateStore.getRegisteredRecordStore(
+          RouterStore.class);
+    }
+    return this.routerStateManager;
+  }
+
   /////////////////////////////////////////////////////////
   // Router info
   /////////////////////////////////////////////////////////
 
   /**
+   * Get the start date of the Router.
+   *
+   * @return Start date of the router.
+   */
+  public long getStartTime() {
+    return this.startTime;
+  }
+
+  /**
    * Unique ID for the router, typically the hostname:port string for the
    * router's RPC server. This ID may be null on router startup before the RPC
    * server has bound to a port.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7721fff7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterHeartbeatService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterHeartbeatService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterHeartbeatService.java
new file mode 100644
index 0000000..86a6210
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterHeartbeatService.java
@@ -0,0 +1,155 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.federation.router;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.federation.store.CachedRecordStore;
+import org.apache.hadoop.hdfs.server.federation.store.MembershipStore;
+import org.apache.hadoop.hdfs.server.federation.store.MountTableStore;
+import org.apache.hadoop.hdfs.server.federation.store.RecordStore;
+import org.apache.hadoop.hdfs.server.federation.store.RouterStore;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.RouterHeartbeatRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.RouterHeartbeatResponse;
+import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
+import org.apache.hadoop.hdfs.server.federation.store.records.RouterState;
+import org.apache.hadoop.hdfs.server.federation.store.records.StateStoreVersion;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Service to periodically update the Router current state in the State Store.
+ */
+public class RouterHeartbeatService extends PeriodicService {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RouterHeartbeatService.class);
+
+  /** Router we are hearbeating. */
+  private final Router router;
+
+  /**
+   * Create a new Router heartbeat service.
+   *
+   * @param router Router to heartbeat.
+   */
+  public RouterHeartbeatService(Router router) {
+    super(RouterHeartbeatService.class.getSimpleName());
+    this.router = router;
+  }
+
+  /**
+   * Trigger the update of the Router state asynchronously.
+   */
+  protected void updateStateAsync() {
+    Thread thread = new Thread(new Runnable() {
+      @Override
+      public void run() {
+        updateStateStore();
+      }
+    }, "Router Heartbeat Async");
+    thread.setDaemon(true);
+    thread.start();
+  }
+
+  /**
+   * Update the state of the Router in the State Store.
+   */
+  private synchronized void updateStateStore() {
+    String routerId = router.getRouterId();
+    if (routerId == null) {
+      LOG.error("Cannot heartbeat for router: unknown router id");
+      return;
+    }
+    RouterStore routerStore = router.getRouterStateManager();
+    if (routerStore != null) {
+      try {
+        RouterState record = RouterState.newInstance(
+            routerId, router.getStartTime(), router.getRouterState());
+        StateStoreVersion stateStoreVersion = StateStoreVersion.newInstance(
+            getStateStoreVersion(MembershipStore.class),
+            getStateStoreVersion(MountTableStore.class));
+        record.setStateStoreVersion(stateStoreVersion);
+        RouterHeartbeatRequest request =
+            RouterHeartbeatRequest.newInstance(record);
+        RouterHeartbeatResponse response = routerStore.routerHeartbeat(request);
+        if (!response.getStatus()) {
+          LOG.warn("Cannot heartbeat router {}", routerId);
+        } else {
+          LOG.debug("Router heartbeat for router {}", routerId);
+        }
+      } catch (IOException e) {
+        LOG.error("Cannot heartbeat router {}: {}", routerId, e.getMessage());
+      }
+    } else {
+      LOG.warn("Cannot heartbeat router {}: State Store unavailable", routerId);
+    }
+  }
+
+  /**
+   * Get the version of the data in the State Store.
+   *
+   * @param clazz Class in the State Store.
+   * @return Version of the data.
+   */
+  private <R extends BaseRecord, S extends RecordStore<R>>
+      long getStateStoreVersion(final Class<S> clazz) {
+    long version = -1;
+    try {
+      StateStoreService stateStore = router.getStateStore();
+      S recordStore = stateStore.getRegisteredRecordStore(clazz);
+      if (recordStore != null) {
+        if (recordStore instanceof CachedRecordStore) {
+          CachedRecordStore<R> cachedRecordStore =
+              (CachedRecordStore<R>) recordStore;
+          List<R> records = cachedRecordStore.getCachedRecords();
+          for (BaseRecord record : records) {
+            if (record.getDateModified() > version) {
+              version = record.getDateModified();
+            }
+          }
+        }
+      }
+    } catch (Exception e) {
+      LOG.error("Cannot get version for {}: {}", clazz, e.getMessage());
+    }
+    return version;
+  }
+
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+
+    long interval = conf.getTimeDuration(
+        DFSConfigKeys.DFS_ROUTER_HEARTBEAT_STATE_INTERVAL_MS,
+        DFSConfigKeys.DFS_ROUTER_HEARTBEAT_STATE_INTERVAL_MS_DEFAULT,
+        TimeUnit.MILLISECONDS);
+    this.setIntervalMs(interval);
+
+    super.serviceInit(conf);
+  }
+
+  @Override
+  public void periodicInvoke() {
+    updateStateStore();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7721fff7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterServiceState.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterServiceState.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterServiceState.java
index 25a6466..3accbe9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterServiceState.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterServiceState.java
@@ -21,7 +21,7 @@ package org.apache.hadoop.hdfs.server.federation.router;
  * States of the Router.
  */
 public enum RouterServiceState {
-  NONE,
+  UNINITIALIZED,
   INITIALIZING,
   SAFEMODE,
   RUNNING,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7721fff7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java
index 0289ba6..aa730ae 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java
@@ -24,6 +24,7 @@ import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.TimeUnit;
 
 import javax.management.NotCompliantMBeanException;
 import javax.management.ObjectName;
@@ -38,8 +39,10 @@ import org.apache.hadoop.hdfs.server.federation.metrics.StateStoreMetrics;
 import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver;
 import org.apache.hadoop.hdfs.server.federation.store.impl.MembershipStoreImpl;
 import org.apache.hadoop.hdfs.server.federation.store.impl.MountTableStoreImpl;
+import org.apache.hadoop.hdfs.server.federation.store.impl.RouterStoreImpl;
 import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
 import org.apache.hadoop.hdfs.server.federation.store.records.MembershipState;
+import org.apache.hadoop.hdfs.server.federation.store.records.RouterState;
 import org.apache.hadoop.metrics2.MetricsException;
 import org.apache.hadoop.metrics2.util.MBeans;
 import org.apache.hadoop.service.CompositeService;
@@ -148,6 +151,7 @@ public class StateStoreService extends CompositeService {
     // Add supported record stores
     addRecordStore(MembershipStoreImpl.class);
     addRecordStore(MountTableStoreImpl.class);
+    addRecordStore(RouterStoreImpl.class);
 
     // Check the connection to the State Store periodically
     this.monitorService = new StateStoreConnectionMonitorService(this);
@@ -158,6 +162,11 @@ public class StateStoreService extends CompositeService {
         DFSConfigKeys.FEDERATION_STORE_MEMBERSHIP_EXPIRATION_MS,
         DFSConfigKeys.FEDERATION_STORE_MEMBERSHIP_EXPIRATION_MS_DEFAULT));
 
+    RouterState.setExpirationMs(conf.getTimeDuration(
+        DFSConfigKeys.FEDERATION_STORE_ROUTER_EXPIRATION_MS,
+        DFSConfigKeys.FEDERATION_STORE_ROUTER_EXPIRATION_MS_DEFAULT,
+        TimeUnit.MILLISECONDS));
+
     // Cache update service
     this.cacheUpdater = new StateStoreCacheUpdateService(this);
     addService(this.cacheUpdater);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7721fff7/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index 7a23eb4..d24310e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -5111,6 +5111,26 @@
   </property>
 
   <property>
+    <name>dfs.federation.router.heartbeat-state.interval</name>
+    <value>5s</value>
+    <description>
+      How often the Router should heartbeat its state into the State Store in
+      milliseconds. This setting supports multiple time unit suffixes as
+      described in dfs.federation.router.quota-cache.update.interval.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.federation.router.store.router.expiration</name>
+    <value>5m</value>
+    <description>
+      Expiration time in milliseconds for a router state record. This setting
+      supports multiple time unit suffixes as described in
+      dfs.federation.router.quota-cache.update.interval.
+    </description>
+  </property>
+
+  <property>
     <name>dfs.federation.router.monitor.namenode</name>
     <value></value>
     <description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7721fff7/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/TestStateStoreRouterState.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/TestStateStoreRouterState.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/TestStateStoreRouterState.java
new file mode 100644
index 0000000..ae15ef6
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/TestStateStoreRouterState.java
@@ -0,0 +1,194 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.federation.store;
+
+import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.verifyException;
+import static org.apache.hadoop.hdfs.server.federation.store.FederationStateStoreTestUtils.clearRecords;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.federation.router.FederationUtil;
+import org.apache.hadoop.hdfs.server.federation.router.RouterServiceState;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetRouterRegistrationRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetRouterRegistrationsRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.RouterHeartbeatRequest;
+import org.apache.hadoop.hdfs.server.federation.store.records.RouterState;
+import org.apache.hadoop.util.Time;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Test the basic {@link StateStoreService} {@link RouterStore} functionality.
+ */
+public class TestStateStoreRouterState extends TestStateStoreBase {
+
+  private static RouterStore routerStore;
+
+  @BeforeClass
+  public static void create() {
+    // Reduce expirations to 5 seconds
+    getConf().setTimeDuration(
+        DFSConfigKeys.FEDERATION_STORE_ROUTER_EXPIRATION_MS,
+        5, TimeUnit.SECONDS);
+  }
+
+  @Before
+  public void setup() throws IOException, InterruptedException {
+
+    if (routerStore == null) {
+      routerStore =
+          getStateStore().getRegisteredRecordStore(RouterStore.class);
+    }
+
+    // Clear router status registrations
+    assertTrue(clearRecords(getStateStore(), RouterState.class));
+  }
+
+  @Test
+  public void testStateStoreDisconnected() throws Exception {
+
+    // Close the data store driver
+    getStateStore().closeDriver();
+    assertEquals(false, getStateStore().isDriverReady());
+
+    // Test all APIs that access the data store to ensure they throw the correct
+    // exception.
+    GetRouterRegistrationRequest getSingleRequest =
+        GetRouterRegistrationRequest.newInstance();
+    verifyException(routerStore, "getRouterRegistration",
+        StateStoreUnavailableException.class,
+        new Class[] {GetRouterRegistrationRequest.class},
+        new Object[] {getSingleRequest});
+
+    GetRouterRegistrationsRequest getRequest =
+        GetRouterRegistrationsRequest.newInstance();
+    routerStore.loadCache(true);
+    verifyException(routerStore, "getRouterRegistrations",
+        StateStoreUnavailableException.class,
+        new Class[] {GetRouterRegistrationsRequest.class},
+        new Object[] {getRequest});
+
+    RouterHeartbeatRequest hbRequest = RouterHeartbeatRequest.newInstance(
+        RouterState.newInstance("test", 0, RouterServiceState.UNINITIALIZED));
+    verifyException(routerStore, "routerHeartbeat",
+        StateStoreUnavailableException.class,
+        new Class[] {RouterHeartbeatRequest.class},
+        new Object[] {hbRequest});
+  }
+
+  //
+  // Router
+  //
+  @Test
+  public void testUpdateRouterStatus()
+      throws IllegalStateException, IOException {
+
+    long dateStarted = Time.now();
+    String address = "testaddress";
+
+    // Set
+    RouterHeartbeatRequest request = RouterHeartbeatRequest.newInstance(
+        RouterState.newInstance(
+            address, dateStarted, RouterServiceState.RUNNING));
+    assertTrue(routerStore.routerHeartbeat(request).getStatus());
+
+    // Verify
+    GetRouterRegistrationRequest getRequest =
+        GetRouterRegistrationRequest.newInstance(address);
+    RouterState record =
+        routerStore.getRouterRegistration(getRequest).getRouter();
+    assertNotNull(record);
+    assertEquals(RouterServiceState.RUNNING, record.getStatus());
+    assertEquals(address, record.getAddress());
+    assertEquals(FederationUtil.getCompileInfo(), record.getCompileInfo());
+    // Build version may vary a bit
+    assertTrue(record.getBuildVersion().length() > 0);
+  }
+
+  @Test
+  public void testRouterStateExpired()
+      throws IOException, InterruptedException {
+
+    long dateStarted = Time.now();
+    String address = "testaddress";
+
+    RouterHeartbeatRequest request = RouterHeartbeatRequest.newInstance(
+        RouterState.newInstance(
+            address, dateStarted, RouterServiceState.RUNNING));
+    // Set
+    assertTrue(routerStore.routerHeartbeat(request).getStatus());
+
+    // Verify
+    GetRouterRegistrationRequest getRequest =
+        GetRouterRegistrationRequest.newInstance(address);
+    RouterState record =
+        routerStore.getRouterRegistration(getRequest).getRouter();
+    assertNotNull(record);
+
+    // Wait past expiration (set to 5 sec in config)
+    Thread.sleep(6000);
+
+    // Verify expired
+    RouterState r = routerStore.getRouterRegistration(getRequest).getRouter();
+    assertEquals(RouterServiceState.EXPIRED, r.getStatus());
+
+    // Heartbeat again and this shouldn't be EXPIRED anymore
+    assertTrue(routerStore.routerHeartbeat(request).getStatus());
+    r = routerStore.getRouterRegistration(getRequest).getRouter();
+    assertEquals(RouterServiceState.RUNNING, r.getStatus());
+  }
+
+  @Test
+  public void testGetAllRouterStates()
+      throws StateStoreUnavailableException, IOException {
+
+    // Set 2 entries
+    RouterHeartbeatRequest heartbeatRequest1 =
+        RouterHeartbeatRequest.newInstance(
+            RouterState.newInstance(
+                "testaddress1", Time.now(), RouterServiceState.RUNNING));
+    assertTrue(routerStore.routerHeartbeat(heartbeatRequest1).getStatus());
+
+    RouterHeartbeatRequest heartbeatRequest2 =
+        RouterHeartbeatRequest.newInstance(
+            RouterState.newInstance(
+                "testaddress2", Time.now(), RouterServiceState.RUNNING));
+    assertTrue(routerStore.routerHeartbeat(heartbeatRequest2).getStatus());
+
+    // Verify
+    routerStore.loadCache(true);
+    GetRouterRegistrationsRequest request =
+        GetRouterRegistrationsRequest.newInstance();
+    List<RouterState> entries =
+        routerStore.getRouterRegistrations(request).getRouters();
+    assertEquals(2, entries.size());
+    Collections.sort(entries);
+    assertEquals("testaddress1", entries.get(0).getAddress());
+    assertEquals("testaddress2", entries.get(1).getAddress());
+    assertEquals(RouterServiceState.RUNNING, entries.get(0).getStatus());
+    assertEquals(RouterServiceState.RUNNING, entries.get(1).getStatus());
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[03/37] hadoop git commit: YARN-7777. Fix user name format in YARN Registry DNS name. Contributed by Jian He

Posted by ae...@apache.org.
YARN-7777. Fix user name format in YARN Registry DNS name. Contributed by Jian He


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/0c559b27
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/0c559b27
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/0c559b27

Branch: refs/heads/HDFS-7240
Commit: 0c559b27820d12ebe5c315c6e2d1685c6de6bd33
Parents: fa8cf4d
Author: Billie Rinaldi <bi...@apache.org>
Authored: Wed Jan 24 13:50:47 2018 -0800
Committer: Billie Rinaldi <bi...@apache.org>
Committed: Wed Jan 24 13:50:47 2018 -0800

----------------------------------------------------------------------
 .../hadoop/yarn/service/ServiceScheduler.java   | 11 +++-------
 .../yarn/service/utils/ServiceApiUtil.java      | 15 +++++++++----
 .../client/binding/RegistryPathUtils.java       | 20 ++++++++++++++++++
 .../registry/client/binding/RegistryUtils.java  |  5 ++++-
 .../server/dns/BaseServiceRecordProcessor.java  | 22 ++------------------
 .../client/binding/TestRegistryPathUtils.java   | 10 ++++++++-
 .../markdown/yarn-service/Configurations.md     |  2 +-
 7 files changed, 50 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0c559b27/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java
index 6cf4e14..dfe9808 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java
@@ -392,14 +392,9 @@ public class ServiceScheduler extends CompositeService {
     // ZK
     globalTokens.put(ServiceApiConstants.CLUSTER_ZK_QUORUM, getConfig()
         .getTrimmed(KEY_REGISTRY_ZK_QUORUM, DEFAULT_REGISTRY_ZK_QUORUM));
-    String user = null;
-    try {
-      user = UserGroupInformation.getCurrentUser().getShortUserName();
-    } catch (IOException e) {
-      LOG.error("Failed to get user.", e);
-    }
-    globalTokens
-        .put(SERVICE_ZK_PATH, ServiceRegistryUtils.mkServiceHomePath(user, app.getName()));
+    String user = RegistryUtils.currentUser();
+    globalTokens.put(SERVICE_ZK_PATH,
+        ServiceRegistryUtils.mkServiceHomePath(user, app.getName()));
 
     globalTokens.put(ServiceApiConstants.USER, user);
     String dnsDomain = getConfig().getTrimmed(KEY_DNS_DOMAIN);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0c559b27/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceApiUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceApiUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceApiUtil.java
index 7f85c95..a5716bd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceApiUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceApiUtil.java
@@ -61,6 +61,9 @@ public class ServiceApiUtil {
   private static final PatternValidator namePattern
       = new PatternValidator("[a-z][a-z0-9-]*");
 
+  private static final PatternValidator userNamePattern
+      = new PatternValidator("[a-z][a-z0-9-.]*");
+
   @VisibleForTesting
   public static void setJsonSerDeser(JsonSerDeser jsd) {
     jsonSerDeser = jsd;
@@ -72,11 +75,15 @@ public class ServiceApiUtil {
       IOException {
     boolean dnsEnabled = conf.getBoolean(RegistryConstants.KEY_DNS_ENABLED,
         RegistryConstants.DEFAULT_DNS_ENABLED);
-    if (dnsEnabled && RegistryUtils.currentUser().length() > RegistryConstants
-        .MAX_FQDN_LABEL_LENGTH) {
-      throw new IllegalArgumentException(RestApiErrorMessages
-          .ERROR_USER_NAME_INVALID);
+    if (dnsEnabled) {
+      if (RegistryUtils.currentUser().length()
+          > RegistryConstants.MAX_FQDN_LABEL_LENGTH) {
+        throw new IllegalArgumentException(
+            RestApiErrorMessages.ERROR_USER_NAME_INVALID);
+      }
+      userNamePattern.validate(RegistryUtils.currentUser());
     }
+
     if (StringUtils.isEmpty(service.getName())) {
       throw new IllegalArgumentException(
           RestApiErrorMessages.ERROR_APPLICATION_NAME_INVALID);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0c559b27/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryPathUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryPathUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryPathUtils.java
index 5fa45f9..b8e9ba1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryPathUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryPathUtils.java
@@ -24,11 +24,13 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.PathNotFoundException;
 import org.apache.hadoop.registry.client.exceptions.InvalidPathnameException;
 import org.apache.hadoop.registry.client.impl.zk.RegistryInternalConstants;
+import org.apache.hadoop.registry.server.dns.BaseServiceRecordProcessor;
 import org.apache.zookeeper.common.PathUtils;
 
 import java.net.IDN;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 /**
@@ -45,6 +47,9 @@ public class RegistryPathUtils {
   private static final Pattern PATH_ENTRY_VALIDATION_PATTERN =
       Pattern.compile(RegistryInternalConstants.VALID_PATH_ENTRY_PATTERN);
 
+  private static final Pattern USER_NAME =
+      Pattern.compile("/users/([a-z][a-z0-9-.]*)");
+
   /**
    * Validate ZK path with the path itself included in
    * the exception text
@@ -215,4 +220,19 @@ public class RegistryPathUtils {
   public static String encodeYarnID(String yarnId) {
     return yarnId.replace("container", "ctr").replace("_", "-");
   }
+
+  /**
+   * Return the username found in the ZK path.
+   *
+   * @param recPath the ZK recPath.
+   * @return the user name.
+   */
+  public static String getUsername(String recPath) {
+    String user = "anonymous";
+    Matcher matcher = USER_NAME.matcher(recPath);
+    if (matcher.find()) {
+      user = matcher.group(1);
+    }
+    return user;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0c559b27/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryUtils.java
index 858b6b1..4ef7b8d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryUtils.java
@@ -296,7 +296,10 @@ public class RegistryUtils {
    */
   public static String currentUser() {
     String shortUserName = currentUsernameUnencoded();
-    return encodeForRegistry(shortUserName);
+    String encodedName =  encodeForRegistry(shortUserName);
+    // DNS name doesn't allow "_", replace it with "-"
+    encodedName = RegistryUtils.convertUsername(encodedName);
+    return encodedName.replace("_", "-");
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0c559b27/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/dns/BaseServiceRecordProcessor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/dns/BaseServiceRecordProcessor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/dns/BaseServiceRecordProcessor.java
index fd5c74f..51ae99a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/dns/BaseServiceRecordProcessor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/dns/BaseServiceRecordProcessor.java
@@ -36,8 +36,6 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
 
 /**
  * Provides common service record processing logic.
@@ -51,7 +49,6 @@ public abstract class BaseServiceRecordProcessor
   private String path;
   private String domain;
 
-  private static final Pattern USER_NAME = Pattern.compile("/users/(\\w*)/?");
   private static final String YARN_SERVICE_API_PREFIX =
       "classpath:org.apache.hadoop.yarn.service.";
   private static final String HTTP_API_TYPE = "http://";
@@ -76,21 +73,6 @@ public abstract class BaseServiceRecordProcessor
   }
 
   /**
-   * Return the username found in the ZK path.
-   *
-   * @param recPath the ZK recPath.
-   * @return the user name.
-   */
-  protected String getUsername(String recPath) {
-    String user = "anonymous";
-    Matcher matcher = USER_NAME.matcher(recPath);
-    if (matcher.find()) {
-      user = matcher.group(1);
-    }
-    return user;
-  }
-
-  /**
    * Return the IPv6 mapped address for the provided IPv4 address. Utilized
    * to create corresponding AAAA records.
    *
@@ -300,7 +282,7 @@ public abstract class BaseServiceRecordProcessor
       String service = RegistryPathUtils.lastPathEntry(
           RegistryPathUtils.parentOf(RegistryPathUtils.parentOf(getPath())));
       String description = getRecord().description.toLowerCase();
-      String user = getUsername(getPath());
+      String user = RegistryPathUtils.getUsername(getPath());
       return Name.fromString(MessageFormat.format("{0}.{1}.{2}.{3}",
           description,
           service,
@@ -352,7 +334,7 @@ public abstract class BaseServiceRecordProcessor
      * @throws TextParseException
      */
     protected Name getServiceName() throws TextParseException {
-      String user = getUsername(getPath());
+      String user = RegistryPathUtils.getUsername(getPath());
       String service =
           String.format("%s.%s.%s",
               RegistryPathUtils.lastPathEntry(getPath()),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0c559b27/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/client/binding/TestRegistryPathUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/client/binding/TestRegistryPathUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/client/binding/TestRegistryPathUtils.java
index 9a24f1c..4346c9a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/client/binding/TestRegistryPathUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/client/binding/TestRegistryPathUtils.java
@@ -80,7 +80,15 @@ public class TestRegistryPathUtils extends Assert {
     assertCreatedPathEquals("/alice", "/alice", "/");
   }
 
-
+  @Test
+  public void testGetUserFromPath() throws Exception {
+    assertEquals("bob", RegistryPathUtils
+        .getUsername("/registry/users/bob/services/yarn-service/test1/"));
+    assertEquals("bob-dev", RegistryPathUtils
+        .getUsername("/registry/users/bob-dev/services/yarn-service/test1"));
+    assertEquals("bob.dev", RegistryPathUtils
+        .getUsername("/registry/users/bob.dev/services/yarn-service/test1"));
+  }
 
 
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0c559b27/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/Configurations.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/Configurations.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/Configurations.md
index a6fd998..c2e6d26 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/Configurations.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/Configurations.md
@@ -155,7 +155,7 @@ where `regionserver-0` is the actual component instance name assigned by the sys
 | Name | Description |
 | ------------ | ------------- |
 | SERVICE_NAME | name of the service defined by the user
-| USER | user who submits the service |
+| USER | user who submits the service. Note that user name which has "\_" will be converted to use "-", to conform with DNS hostname RFC format which doesn't allow "\_", and all characters will be lowercased E.g. "Bob_dev" will be converted to "bob-dev"  |
 | DOMAIN | the domain name for the cluster |
 | COMPONENT_NAME | the name for a given component |
 | COMPONENT_INSTANCE_NAME | the name for a given component instance (i.e. container) |


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[28/37] hadoop git commit: YARN-7790. Improve Capacity Scheduler Async Scheduling to better handle node failures. Contributed by Wangda Tan.

Posted by ae...@apache.org.
YARN-7790. Improve Capacity Scheduler Async Scheduling to better handle node failures. Contributed by Wangda Tan.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/e9c72d04
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e9c72d04
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e9c72d04

Branch: refs/heads/HDFS-7240
Commit: e9c72d04beddfe0252d2e81123a9fe66bdf04078
Parents: 3400d0c
Author: Sunil G <su...@apache.org>
Authored: Mon Jan 29 20:43:08 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Mon Jan 29 20:44:38 2018 +0530

----------------------------------------------------------------------
 .../scheduler/AbstractYarnScheduler.java        |  51 +++---
 .../scheduler/SchedulerNode.java                |  16 ++
 .../scheduler/capacity/CapacityScheduler.java   |  49 +++++-
 .../TestRMHAForAsyncScheduler.java              |  38 ++++-
 .../TestCapacitySchedulerAsyncScheduling.java   | 159 ++++++++++++++++++-
 5 files changed, 276 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e9c72d04/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
index c94c379..4b76327 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
@@ -980,11 +980,11 @@ public abstract class AbstractYarnScheduler
   /**
    * Get lists of new containers from NodeManager and process them.
    * @param nm The RMNode corresponding to the NodeManager
+   * @param schedulerNode schedulerNode
    * @return list of completed containers
    */
-  protected List<ContainerStatus> updateNewContainerInfo(RMNode nm) {
-    SchedulerNode node = getNode(nm.getNodeID());
-
+  private List<ContainerStatus> updateNewContainerInfo(RMNode nm,
+      SchedulerNode schedulerNode) {
     List<UpdatedContainerInfo> containerInfoList = nm.pullContainerUpdates();
     List<ContainerStatus> newlyLaunchedContainers =
         new ArrayList<>();
@@ -999,14 +999,15 @@ public abstract class AbstractYarnScheduler
 
     // Processing the newly launched containers
     for (ContainerStatus launchedContainer : newlyLaunchedContainers) {
-      containerLaunchedOnNode(launchedContainer.getContainerId(), node);
+      containerLaunchedOnNode(launchedContainer.getContainerId(),
+          schedulerNode);
     }
 
     // Processing the newly increased containers
     List<Container> newlyIncreasedContainers =
         nm.pullNewlyIncreasedContainers();
     for (Container container : newlyIncreasedContainers) {
-      containerIncreasedOnNode(container.getId(), node, container);
+      containerIncreasedOnNode(container.getId(), schedulerNode, container);
     }
 
     return completedContainers;
@@ -1017,12 +1018,12 @@ public abstract class AbstractYarnScheduler
    * @param completedContainers Extracted list of completed containers
    * @param releasedResources Reference resource object for completed containers
    * @param nodeId NodeId corresponding to the NodeManager
+   * @param schedulerNode schedulerNode
    * @return The total number of released containers
    */
-  protected int updateCompletedContainers(List<ContainerStatus>
-      completedContainers, Resource releasedResources, NodeId nodeId) {
+  private int updateCompletedContainers(List<ContainerStatus> completedContainers,
+      Resource releasedResources, NodeId nodeId, SchedulerNode schedulerNode) {
     int releasedContainers = 0;
-    SchedulerNode node = getNode(nodeId);
     List<ContainerId> untrackedContainerIdList = new ArrayList<ContainerId>();
     for (ContainerStatus completedContainer : completedContainers) {
       ContainerId containerId = completedContainer.getContainerId();
@@ -1030,8 +1031,8 @@ public abstract class AbstractYarnScheduler
       RMContainer container = getRMContainer(containerId);
       completedContainer(container,
           completedContainer, RMContainerEventType.FINISHED);
-      if (node != null) {
-        node.releaseContainer(containerId, true);
+      if (schedulerNode != null) {
+        schedulerNode.releaseContainer(containerId, true);
       }
 
       if (container != null) {
@@ -1076,14 +1077,14 @@ public abstract class AbstractYarnScheduler
   /**
    * Update container and utilization information on the NodeManager.
    * @param nm The NodeManager to update
+   * @param schedulerNode schedulerNode
    */
-  protected void updateNodeResourceUtilization(RMNode nm) {
-    SchedulerNode node = getNode(nm.getNodeID());
+  protected void updateNodeResourceUtilization(RMNode nm,
+      SchedulerNode schedulerNode) {
     // Updating node resource utilization
-    node.setAggregatedContainersUtilization(
+    schedulerNode.setAggregatedContainersUtilization(
         nm.getAggregatedContainersUtilization());
-    node.setNodeUtilization(nm.getNodeUtilization());
-
+    schedulerNode.setNodeUtilization(nm.getNodeUtilization());
   }
 
   /**
@@ -1097,12 +1098,17 @@ public abstract class AbstractYarnScheduler
     }
 
     // Process new container information
-    List<ContainerStatus> completedContainers = updateNewContainerInfo(nm);
+    SchedulerNode schedulerNode = getNode(nm.getNodeID());
+    List<ContainerStatus> completedContainers = updateNewContainerInfo(nm,
+        schedulerNode);
+
+    // Notify Scheduler Node updated.
+    schedulerNode.notifyNodeUpdate();
 
     // Process completed containers
     Resource releasedResources = Resource.newInstance(0, 0);
     int releasedContainers = updateCompletedContainers(completedContainers,
-        releasedResources, nm.getNodeID());
+        releasedResources, nm.getNodeID(), schedulerNode);
 
     // If the node is decommissioning, send an update to have the total
     // resource equal to the used resource, so no available resource to
@@ -1115,18 +1121,17 @@ public abstract class AbstractYarnScheduler
           .getEventHandler()
           .handle(
               new RMNodeResourceUpdateEvent(nm.getNodeID(), ResourceOption
-                  .newInstance(getSchedulerNode(nm.getNodeID())
-                      .getAllocatedResource(), 0)));
+                  .newInstance(schedulerNode.getAllocatedResource(), 0)));
     }
 
     updateSchedulerHealthInformation(releasedResources, releasedContainers);
-    updateNodeResourceUtilization(nm);
+    updateNodeResourceUtilization(nm, schedulerNode);
 
     // Now node data structures are up-to-date and ready for scheduling.
     if(LOG.isDebugEnabled()) {
-      SchedulerNode node = getNode(nm.getNodeID());
-      LOG.debug("Node being looked for scheduling " + nm +
-          " availableResource: " + node.getUnallocatedResource());
+      LOG.debug(
+          "Node being looked for scheduling " + nm + " availableResource: "
+              + schedulerNode.getUnallocatedResource());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e9c72d04/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
index 05dbf1e..89f748d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
@@ -30,6 +30,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.util.Time;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ExecutionType;
@@ -76,6 +77,9 @@ public abstract class SchedulerNode {
 
   private volatile Set<String> labels = null;
 
+  // Last updated time
+  private volatile long lastHeartbeatMonotonicTime;
+
   public SchedulerNode(RMNode node, boolean usePortForNodeName,
       Set<String> labels) {
     this.rmNode = node;
@@ -87,6 +91,7 @@ public abstract class SchedulerNode {
       nodeName = rmNode.getHostName();
     }
     this.labels = ImmutableSet.copyOf(labels);
+    this.lastHeartbeatMonotonicTime = Time.monotonicNow();
   }
 
   public SchedulerNode(RMNode node, boolean usePortForNodeName) {
@@ -453,6 +458,17 @@ public abstract class SchedulerNode {
     return this.nodeUtilization;
   }
 
+  public long getLastHeartbeatMonotonicTime() {
+    return lastHeartbeatMonotonicTime;
+  }
+
+  /**
+   * This will be called for each node heartbeat.
+   */
+  public void notifyNodeUpdate() {
+    this.lastHeartbeatMonotonicTime = Time.monotonicNow();
+  }
+
 
   private static class ContainerInfo {
     private final RMContainer container;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e9c72d04/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
index 99f4456..03ca507 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
@@ -142,7 +142,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SimpleC
 import org.apache.hadoop.yarn.server.resourcemanager.security.AppPriorityACLsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.utils.Lock;
-import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceUtils;
@@ -181,8 +180,6 @@ public class CapacityScheduler extends
 
   private CSConfigurationProvider csConfProvider;
 
-  protected Clock monotonicClock;
-
   @Override
   public void setConf(Configuration conf) {
       yarnConf = conf;
@@ -243,6 +240,8 @@ public class CapacityScheduler extends
   private RMNodeLabelsManager labelManager;
   private AppPriorityACLsManager appPriorityACLManager;
 
+  private static boolean printedVerboseLoggingForAsyncScheduling = false;
+
   /**
    * EXPERT
    */
@@ -471,6 +470,22 @@ public class CapacityScheduler extends
 
   private final static Random random = new Random(System.currentTimeMillis());
 
+  private static boolean shouldSkipNodeSchedule(FiCaSchedulerNode node,
+      CapacityScheduler cs, boolean printVerboseLog) {
+    // Skip node which missed 2 heartbeats since the node might be dead and
+    // we should not continue allocate containers on that.
+    long timeElapsedFromLastHeartbeat =
+        Time.monotonicNow() - node.getLastHeartbeatMonotonicTime();
+    if (timeElapsedFromLastHeartbeat > cs.nmHeartbeatInterval * 2) {
+      if (printVerboseLog && LOG.isDebugEnabled()) {
+        LOG.debug("Skip scheduling on node because it haven't heartbeated for "
+            + timeElapsedFromLastHeartbeat / 1000.0f + " secs");
+      }
+      return true;
+    }
+    return false;
+  }
+
   /**
    * Schedule on all nodes by starting at a random point.
    * @param cs
@@ -481,16 +496,42 @@ public class CapacityScheduler extends
     Collection<FiCaSchedulerNode> nodes = cs.nodeTracker.getAllNodes();
     int start = random.nextInt(nodes.size());
 
+    // To avoid too verbose DEBUG logging, only print debug log once for
+    // every 10 secs.
+    boolean printSkipedNodeLogging = false;
+    if (Time.monotonicNow() / 1000 % 10 == 0) {
+      printSkipedNodeLogging = (!printedVerboseLoggingForAsyncScheduling);
+    } else {
+      printedVerboseLoggingForAsyncScheduling = false;
+    }
+
+    // Allocate containers of node [start, end)
     for (FiCaSchedulerNode node : nodes) {
       if (current++ >= start) {
+        if (shouldSkipNodeSchedule(node, cs, printSkipedNodeLogging)) {
+          continue;
+        }
         cs.allocateContainersToNode(node.getNodeID(), false);
       }
     }
-    // Now, just get everyone to be safe
+
+    current = 0;
+
+    // Allocate containers of node [0, start)
     for (FiCaSchedulerNode node : nodes) {
+      if (current++ > start) {
+        break;
+      }
+      if (shouldSkipNodeSchedule(node, cs, printSkipedNodeLogging)) {
+        continue;
+      }
       cs.allocateContainersToNode(node.getNodeID(), false);
     }
 
+    if (printSkipedNodeLogging) {
+      printedVerboseLoggingForAsyncScheduling = true;
+    }
+
     Thread.sleep(cs.getAsyncScheduleInterval());
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e9c72d04/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHAForAsyncScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHAForAsyncScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHAForAsyncScheduler.java
index 46d5cda..36f1762 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHAForAsyncScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHAForAsyncScheduler.java
@@ -28,13 +28,19 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptS
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestCapacitySchedulerAsyncScheduling;
 import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
+import java.util.Arrays;
+import java.util.List;
+
 public class TestRMHAForAsyncScheduler extends RMHATestBase {
+  private TestCapacitySchedulerAsyncScheduling.NMHeartbeatThread
+      nmHeartbeatThread = null;
 
   @Before
   @Override
@@ -57,26 +63,49 @@ public class TestRMHAForAsyncScheduler extends RMHATestBase {
         CapacitySchedulerConfiguration.SCHEDULE_ASYNCHRONOUSLY_ENABLE, true);
   }
 
+  private void keepNMHeartbeat(List<MockNM> mockNMs, int interval) {
+    if (nmHeartbeatThread != null) {
+      nmHeartbeatThread.setShouldStop();
+      nmHeartbeatThread = null;
+    }
+    nmHeartbeatThread =
+        new TestCapacitySchedulerAsyncScheduling.NMHeartbeatThread(mockNMs,
+            interval);
+    nmHeartbeatThread.start();
+  }
+
+  private void pauseNMHeartbeat() {
+    if (nmHeartbeatThread != null) {
+      nmHeartbeatThread.setShouldStop();
+      nmHeartbeatThread = null;
+    }
+  }
+
   @Test(timeout = 60000)
   public void testAsyncScheduleThreadStateAfterRMHATransit() throws Exception {
     // start two RMs, and transit rm1 to active, rm2 to standby
     startRMs();
     // register NM
-    rm1.registerNode("h1:1234", 8192, 8);
+    MockNM nm = rm1.registerNode("192.1.1.1:1234", 8192, 8);
     // submit app1 and check
     RMApp app1 = submitAppAndCheckLaunched(rm1);
+    keepNMHeartbeat(Arrays.asList(nm), 1000);
 
     // failover RM1 to RM2
     explicitFailover();
     checkAsyncSchedulerThreads(Thread.currentThread());
+    pauseNMHeartbeat();
 
     // register NM, kill app1
-    rm2.registerNode("h1:1234", 8192, 8);
+    nm = rm2.registerNode("192.1.1.1:1234", 8192, 8);
+    keepNMHeartbeat(Arrays.asList(nm), 1000);
+
     rm2.waitForState(app1.getCurrentAppAttempt().getAppAttemptId(),
         RMAppAttemptState.LAUNCHED);
     rm2.killApp(app1.getApplicationId());
     // submit app3 and check
     RMApp app2 = submitAppAndCheckLaunched(rm2);
+    pauseNMHeartbeat();
 
     // failover RM2 to RM1
     HAServiceProtocol.StateChangeRequestInfo requestInfo =
@@ -92,12 +121,15 @@ public class TestRMHAForAsyncScheduler extends RMHATestBase {
     checkAsyncSchedulerThreads(Thread.currentThread());
 
     // register NM, kill app2
-    rm1.registerNode("h1:1234", 8192, 8);
+    nm = rm1.registerNode("192.1.1.1:1234", 8192, 8);
+    keepNMHeartbeat(Arrays.asList(nm), 1000);
+
     rm1.waitForState(app2.getCurrentAppAttempt().getAppAttemptId(),
         RMAppAttemptState.LAUNCHED);
     rm1.killApp(app2.getApplicationId());
     // submit app3 and check
     submitAppAndCheckLaunched(rm1);
+    pauseNMHeartbeat();
 
     rm1.stop();
     rm2.stop();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e9c72d04/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerAsyncScheduling.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerAsyncScheduling.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerAsyncScheduling.java
index 77596e2..548b909 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerAsyncScheduling.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerAsyncScheduling.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEvent;
@@ -72,6 +73,8 @@ public class TestCapacitySchedulerAsyncScheduling {
 
   RMNodeLabelsManager mgr;
 
+  private NMHeartbeatThread nmHeartbeatThread = null;
+
   @Before
   public void setUp() throws Exception {
     conf = new YarnConfiguration();
@@ -122,9 +125,11 @@ public class TestCapacitySchedulerAsyncScheduling {
     List<MockNM> nms = new ArrayList<>();
     // Add 10 nodes to the cluster, in the cluster we have 200 GB resource
     for (int i = 0; i < 10; i++) {
-      nms.add(rm.registerNode("h-" + i + ":1234", 20 * GB));
+      nms.add(rm.registerNode("127.0.0." + i + ":1234", 20 * GB));
     }
 
+    keepNMHeartbeat(nms, 1000);
+
     List<MockAM> ams = new ArrayList<>();
     // Add 3 applications to the cluster, one app in one queue
     // the i-th app ask (20 * i) containers. So in total we will have
@@ -185,8 +190,8 @@ public class TestCapacitySchedulerAsyncScheduling {
     // init RM & NMs & Nodes
     final MockRM rm = new MockRM(disableAsyncConf);
     rm.start();
-    final MockNM nm1 = rm.registerNode("h1:1234", 9 * GB);
-    final MockNM nm2 = rm.registerNode("h2:2234", 9 * GB);
+    final MockNM nm1 = rm.registerNode("192.168.0.1:1234", 9 * GB);
+    final MockNM nm2 = rm.registerNode("192.168.0.2:2234", 9 * GB);
     List<MockNM> nmLst = new ArrayList<>();
     nmLst.add(nm1);
     nmLst.add(nm2);
@@ -277,8 +282,8 @@ public class TestCapacitySchedulerAsyncScheduling {
     // init RM & NMs & Nodes
     final MockRM rm = new MockRM(disableAsyncConf);
     rm.start();
-    final MockNM nm1 = rm.registerNode("h1:1234", 9 * GB);
-    final MockNM nm2 = rm.registerNode("h2:2234", 9 * GB);
+    final MockNM nm1 = rm.registerNode("127.0.0.1:1234", 9 * GB);
+    final MockNM nm2 = rm.registerNode("127.0.0.2:2234", 9 * GB);
 
     // init scheduler nodes
     int waitTime = 1000;
@@ -416,8 +421,8 @@ public class TestCapacitySchedulerAsyncScheduling {
     // init RM & NMs & Nodes
     final MockRM rm = new MockRM(disableAsyncConf);
     rm.start();
-    final MockNM nm1 = rm.registerNode("h1:1234", 9 * GB);
-    final MockNM nm2 = rm.registerNode("h2:1234", 9 * GB);
+    final MockNM nm1 = rm.registerNode("127.0.0.1:1234", 9 * GB);
+    final MockNM nm2 = rm.registerNode("127.0.0.2:1234", 9 * GB);
     List<MockNM> nmLst = new ArrayList<>();
     nmLst.add(nm1);
     nmLst.add(nm2);
@@ -476,6 +481,146 @@ public class TestCapacitySchedulerAsyncScheduling {
     rm.stop();
   }
 
+  /**
+   * Make sure scheduler skips NMs which haven't heartbeat for a while.
+   * @throws Exception
+   */
+  @Test
+  public void testAsyncSchedulerSkipNoHeartbeatNMs() throws Exception {
+    int heartbeatInterval = 100;
+    conf.setInt(
+        CapacitySchedulerConfiguration.SCHEDULE_ASYNCHRONOUSLY_MAXIMUM_THREAD,
+        1);
+    conf.setInt(CapacitySchedulerConfiguration.SCHEDULE_ASYNCHRONOUSLY_PREFIX
+        + ".scheduling-interval-ms", 100);
+    // Heartbeat interval is 100 ms.
+    conf.setInt(YarnConfiguration.RM_NM_HEARTBEAT_INTERVAL_MS, heartbeatInterval);
+
+    final RMNodeLabelsManager mgr = new NullRMNodeLabelsManager();
+    mgr.init(conf);
+
+    // inject node label manager
+    MockRM rm = new MockRM(TestUtils.getConfigurationWithMultipleQueues(conf)) {
+      @Override
+      public RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+
+    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+
+    rm.getRMContext().setNodeLabelManager(mgr);
+    rm.start();
+
+    List<MockNM> nms = new ArrayList<>();
+    // Add 10 nodes to the cluster, in the cluster we have 200 GB resource
+    for (int i = 0; i < 10; i++) {
+      nms.add(rm.registerNode("127.0.0." + i + ":1234", 20 * GB));
+    }
+
+    List<MockAM> ams = new ArrayList<>();
+
+    keepNMHeartbeat(nms, heartbeatInterval);
+
+    for (int i = 0; i < 3; i++) {
+      RMApp rmApp = rm.submitApp(1024, "app", "user", null, false,
+          Character.toString((char) (i % 34 + 97)), 1, null, null, false);
+      MockAM am = MockRM.launchAMWhenAsyncSchedulingEnabled(rmApp, rm);
+      am.registerAppAttempt();
+      ams.add(am);
+    }
+
+    pauseNMHeartbeat();
+
+    Thread.sleep(heartbeatInterval * 3);
+
+    // Applications request containers.
+    for (int i = 0; i < 3; i++) {
+      ams.get(i).allocate("*", 1024, 20 * (i + 1), new ArrayList<>());
+    }
+
+    for (int i = 0; i < 5; i++) {
+      // Do heartbeat for NM 0-4
+      nms.get(i).nodeHeartbeat(true);
+    }
+
+    // Wait for 2000 ms.
+    Thread.sleep(2000);
+
+    // Make sure that NM5-9 don't have non-AM containers.
+    for (int i = 0; i < 9; i++) {
+      if (i < 5) {
+        Assert.assertTrue(checkNumNonAMContainersOnNode(cs, nms.get(i)) > 0);
+      } else {
+        Assert.assertTrue(checkNumNonAMContainersOnNode(cs, nms.get(i)) == 0);
+      }
+    }
+
+    rm.close();
+  }
+
+  public static class NMHeartbeatThread extends Thread {
+    private List<MockNM> mockNMS;
+    private int interval;
+    private volatile boolean shouldStop = false;
+
+    public NMHeartbeatThread(List<MockNM> mockNMs, int interval) {
+      this.mockNMS = mockNMs;
+      this.interval = interval;
+    }
+
+    public void run() {
+      while (true) {
+        if (shouldStop) {
+          break;
+        }
+        for (MockNM nm : mockNMS) {
+          try {
+            nm.nodeHeartbeat(true);
+          } catch (Exception e) {
+            e.printStackTrace();
+          }
+        }
+        try {
+          Thread.sleep(interval);
+        } catch (InterruptedException e) {
+          e.printStackTrace();
+        }
+      }
+    }
+
+    public void setShouldStop() {
+      shouldStop = true;
+    }
+  }
+
+  private void keepNMHeartbeat(List<MockNM> mockNMs, int interval) {
+    if (nmHeartbeatThread != null) {
+      nmHeartbeatThread.setShouldStop();
+      nmHeartbeatThread = null;
+    }
+    nmHeartbeatThread = new NMHeartbeatThread(mockNMs, interval);
+    nmHeartbeatThread.start();
+  }
+
+  private void pauseNMHeartbeat() {
+    if (nmHeartbeatThread != null) {
+      nmHeartbeatThread.setShouldStop();
+      nmHeartbeatThread = null;
+    }
+  }
+
+  private int checkNumNonAMContainersOnNode(CapacityScheduler cs, MockNM nm) {
+    SchedulerNode node = cs.getNode(nm.getNodeId());
+    int nonAMContainer = 0;
+    for (RMContainer c : node.getCopiedListOfRunningContainers()) {
+      if (!c.isAMContainer()) {
+         nonAMContainer++;
+      }
+    }
+    return nonAMContainer;
+  }
+
   private void allocateAndLaunchContainers(MockAM am, MockNM nm, MockRM rm,
       int nContainer, Resource resource, int priority, int startContainerId)
       throws Exception {


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[19/37] hadoop git commit: YARN-2185. Use pipes when localizing archives. Contributed by Miklos Szegedi

Posted by ae...@apache.org.
YARN-2185. Use pipes when localizing archives. Contributed by Miklos Szegedi


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/1b0f265d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/1b0f265d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/1b0f265d

Branch: refs/heads/HDFS-7240
Commit: 1b0f265db1a5bfccf1d870912237ea9618bd9c34
Parents: f2fa736
Author: Jason Lowe <jl...@apache.org>
Authored: Fri Jan 26 13:25:20 2018 -0600
Committer: Jason Lowe <jl...@apache.org>
Committed: Fri Jan 26 13:25:20 2018 -0600

----------------------------------------------------------------------
 .../java/org/apache/hadoop/fs/FileUtil.java     | 251 ++++++++++++++++++-
 .../java/org/apache/hadoop/util/RunJar.java     |  65 +++++
 .../org/apache/hadoop/yarn/util/FSDownload.java | 215 ++++++++++------
 .../apache/hadoop/yarn/util/TestFSDownload.java |  30 ++-
 4 files changed, 462 insertions(+), 99 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b0f265d/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
index 4d971aa..bf9b146 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
@@ -20,27 +20,35 @@ package org.apache.hadoop.fs;
 
 import java.io.BufferedInputStream;
 import java.io.BufferedOutputStream;
+import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
+import java.io.InputStreamReader;
 import java.io.OutputStream;
 import java.net.InetAddress;
 import java.net.URI;
 import java.net.UnknownHostException;
+import java.nio.charset.Charset;
 import java.nio.file.AccessDeniedException;
 import java.util.ArrayList;
 import java.util.Enumeration;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
 import java.util.jar.Attributes;
 import java.util.jar.JarOutputStream;
 import java.util.jar.Manifest;
 import java.util.zip.GZIPInputStream;
 import java.util.zip.ZipEntry;
 import java.util.zip.ZipFile;
+import java.util.zip.ZipInputStream;
 
 import org.apache.commons.collections.map.CaseInsensitiveMap;
 import org.apache.commons.compress.archivers.tar.TarArchiveEntry;
@@ -75,6 +83,11 @@ public class FileUtil {
   public static final int SYMLINK_NO_PRIVILEGE = 2;
 
   /**
+   * Buffer size for copy the content of compressed file to new file.
+   */
+  private static final int BUFFER_SIZE = 8_192;
+
+  /**
    * convert an array of FileStatus to an array of Path
    *
    * @param stats
@@ -526,6 +539,22 @@ public class FileUtil {
   }
 
   /**
+   * Convert a os-native filename to a path that works for the shell
+   * and avoids script injection attacks.
+   * @param file The filename to convert
+   * @return The unix pathname
+   * @throws IOException on windows, there can be problems with the subprocess
+   */
+  public static String makeSecureShellPath(File file) throws IOException {
+    if (Shell.WINDOWS) {
+      // Currently it is never called, but it might be helpful in the future.
+      throw new UnsupportedOperationException("Not implemented for Windows");
+    } else {
+      return makeShellPath(file, false).replace("'", "'\\''");
+    }
+  }
+
+  /**
    * Convert a os-native filename to a path that works for the shell.
    * @param file The filename to convert
    * @param makeCanonicalPath
@@ -576,11 +605,48 @@ public class FileUtil {
   }
 
   /**
-   * Given a File input it will unzip the file in a the unzip directory
+   * Given a stream input it will unzip the it in the unzip directory.
+   * passed as the second parameter
+   * @param inputStream The zip file as input
+   * @param toDir The unzip directory where to unzip the zip file.
+   * @throws IOException an exception occurred
+   */
+  public static void unZip(InputStream inputStream, File toDir)
+      throws IOException {
+    try (ZipInputStream zip = new ZipInputStream(inputStream)) {
+      int numOfFailedLastModifiedSet = 0;
+      for(ZipEntry entry = zip.getNextEntry();
+          entry != null;
+          entry = zip.getNextEntry()) {
+        if (!entry.isDirectory()) {
+          File file = new File(toDir, entry.getName());
+          File parent = file.getParentFile();
+          if (!parent.mkdirs() &&
+              !parent.isDirectory()) {
+            throw new IOException("Mkdirs failed to create " +
+                parent.getAbsolutePath());
+          }
+          try (OutputStream out = new FileOutputStream(file)) {
+            IOUtils.copyBytes(zip, out, BUFFER_SIZE);
+          }
+          if (!file.setLastModified(entry.getTime())) {
+            numOfFailedLastModifiedSet++;
+          }
+        }
+      }
+      if (numOfFailedLastModifiedSet > 0) {
+        LOG.warn("Could not set last modfied time for {} file(s)",
+            numOfFailedLastModifiedSet);
+      }
+    }
+  }
+
+  /**
+   * Given a File input it will unzip it in the unzip directory.
    * passed as the second parameter
    * @param inFile The zip file as input
    * @param unzipDir The unzip directory where to unzip the zip file.
-   * @throws IOException
+   * @throws IOException An I/O exception has occurred
    */
   public static void unZip(File inFile, File unzipDir) throws IOException {
     Enumeration<? extends ZipEntry> entries;
@@ -621,6 +687,138 @@ public class FileUtil {
   }
 
   /**
+   * Run a command and send the contents of an input stream to it.
+   * @param inputStream Input stream to forward to the shell command
+   * @param command shell command to run
+   * @throws IOException read or write failed
+   * @throws InterruptedException command interrupted
+   * @throws ExecutionException task submit failed
+   */
+  private static void runCommandOnStream(
+      InputStream inputStream, String command)
+      throws IOException, InterruptedException, ExecutionException {
+    ExecutorService executor = null;
+    ProcessBuilder builder = new ProcessBuilder();
+    builder.command(
+        Shell.WINDOWS ? "cmd" : "bash",
+        Shell.WINDOWS ? "/c" : "-c",
+        command);
+    Process process = builder.start();
+    int exitCode;
+    try {
+      // Consume stdout and stderr, to avoid blocking the command
+      executor = Executors.newFixedThreadPool(2);
+      Future output = executor.submit(() -> {
+        try {
+          // Read until the output stream receives an EOF and closed.
+          if (LOG.isDebugEnabled()) {
+            // Log directly to avoid out of memory errors
+            try (BufferedReader reader =
+                     new BufferedReader(
+                         new InputStreamReader(process.getInputStream(),
+                             Charset.forName("UTF-8")))) {
+              String line;
+              while((line = reader.readLine()) != null) {
+                LOG.debug(line);
+              }
+            }
+          } else {
+            org.apache.commons.io.IOUtils.copy(
+                process.getInputStream(),
+                new IOUtils.NullOutputStream());
+          }
+        } catch (IOException e) {
+          LOG.debug(e.getMessage());
+        }
+      });
+      Future error = executor.submit(() -> {
+        try {
+          // Read until the error stream receives an EOF and closed.
+          if (LOG.isDebugEnabled()) {
+            // Log directly to avoid out of memory errors
+            try (BufferedReader reader =
+                     new BufferedReader(
+                         new InputStreamReader(process.getErrorStream(),
+                             Charset.forName("UTF-8")))) {
+              String line;
+              while((line = reader.readLine()) != null) {
+                LOG.debug(line);
+              }
+            }
+          } else {
+            org.apache.commons.io.IOUtils.copy(
+                process.getErrorStream(),
+                new IOUtils.NullOutputStream());
+          }
+        } catch (IOException e) {
+          LOG.debug(e.getMessage());
+        }
+      });
+
+      // Pass the input stream to the command to process
+      try {
+        org.apache.commons.io.IOUtils.copy(
+            inputStream, process.getOutputStream());
+      } finally {
+        process.getOutputStream().close();
+      }
+
+      // Wait for both stdout and stderr futures to finish
+      error.get();
+      output.get();
+    } finally {
+      // Clean up the threads
+      if (executor != null) {
+        executor.shutdown();
+      }
+      // Wait to avoid leaking the child process
+      exitCode = process.waitFor();
+    }
+
+    if (exitCode != 0) {
+      throw new IOException(
+          String.format(
+              "Error executing command. %s " +
+                  "Process exited with exit code %d.",
+              command, exitCode));
+    }
+  }
+
+  /**
+   * Given a Tar File as input it will untar the file in a the untar directory
+   * passed as the second parameter
+   *
+   * This utility will untar ".tar" files and ".tar.gz","tgz" files.
+   *
+   * @param inputStream The tar file as input.
+   * @param untarDir The untar directory where to untar the tar file.
+   * @param gzipped The input stream is gzipped
+   *                TODO Use magic number and PusbackInputStream to identify
+   * @throws IOException an exception occurred
+   * @throws InterruptedException command interrupted
+   * @throws ExecutionException task submit failed
+   */
+  public static void unTar(InputStream inputStream, File untarDir,
+                           boolean gzipped)
+      throws IOException, InterruptedException, ExecutionException {
+    if (!untarDir.mkdirs()) {
+      if (!untarDir.isDirectory()) {
+        throw new IOException("Mkdirs failed to create " + untarDir);
+      }
+    }
+
+    if(Shell.WINDOWS) {
+      // Tar is not native to Windows. Use simple Java based implementation for
+      // tests and simple tar archives
+      unTarUsingJava(inputStream, untarDir, gzipped);
+    } else {
+      // spawn tar utility to untar archive for full fledged unix behavior such
+      // as resolving symlinks in tar archives
+      unTarUsingTar(inputStream, untarDir, gzipped);
+    }
+  }
+
+  /**
    * Given a Tar File as input it will untar the file in a the untar directory
    * passed as the second parameter
    *
@@ -650,23 +848,41 @@ public class FileUtil {
     }
   }
 
+  private static void unTarUsingTar(InputStream inputStream, File untarDir,
+                                    boolean gzipped)
+      throws IOException, InterruptedException, ExecutionException {
+    StringBuilder untarCommand = new StringBuilder();
+    if (gzipped) {
+      untarCommand.append("gzip -dc | (");
+    }
+    untarCommand.append("cd '");
+    untarCommand.append(FileUtil.makeSecureShellPath(untarDir));
+    untarCommand.append("' && ");
+    untarCommand.append("tar -x ");
+
+    if (gzipped) {
+      untarCommand.append(")");
+    }
+    runCommandOnStream(inputStream, untarCommand.toString());
+  }
+
   private static void unTarUsingTar(File inFile, File untarDir,
       boolean gzipped) throws IOException {
     StringBuffer untarCommand = new StringBuffer();
     if (gzipped) {
       untarCommand.append(" gzip -dc '");
-      untarCommand.append(FileUtil.makeShellPath(inFile));
+      untarCommand.append(FileUtil.makeSecureShellPath(inFile));
       untarCommand.append("' | (");
     }
     untarCommand.append("cd '");
-    untarCommand.append(FileUtil.makeShellPath(untarDir));
-    untarCommand.append("' ; ");
+    untarCommand.append(FileUtil.makeSecureShellPath(untarDir));
+    untarCommand.append("' && ");
     untarCommand.append("tar -xf ");
 
     if (gzipped) {
       untarCommand.append(" -)");
     } else {
-      untarCommand.append(FileUtil.makeShellPath(inFile));
+      untarCommand.append(FileUtil.makeSecureShellPath(inFile));
     }
     String[] shellCmd = { "bash", "-c", untarCommand.toString() };
     ShellCommandExecutor shexec = new ShellCommandExecutor(shellCmd);
@@ -701,6 +917,29 @@ public class FileUtil {
     }
   }
 
+  private static void unTarUsingJava(InputStream inputStream, File untarDir,
+                                     boolean gzipped) throws IOException {
+    TarArchiveInputStream tis = null;
+    try {
+      if (gzipped) {
+        inputStream = new BufferedInputStream(new GZIPInputStream(
+            inputStream));
+      } else {
+        inputStream =
+            new BufferedInputStream(inputStream);
+      }
+
+      tis = new TarArchiveInputStream(inputStream);
+
+      for (TarArchiveEntry entry = tis.getNextTarEntry(); entry != null;) {
+        unpackEntries(tis, entry, untarDir);
+        entry = tis.getNextTarEntry();
+      }
+    } finally {
+      IOUtils.cleanupWithLogger(LOG, tis, inputStream);
+    }
+  }
+
   private static void unpackEntries(TarArchiveInputStream tis,
       TarArchiveEntry entry, File outputDir) throws IOException {
     if (entry.isDirectory()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b0f265d/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java
index 19b51ad..89b7d76 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java
@@ -34,9 +34,11 @@ import java.util.Enumeration;
 import java.util.List;
 import java.util.jar.JarEntry;
 import java.util.jar.JarFile;
+import java.util.jar.JarInputStream;
 import java.util.jar.Manifest;
 import java.util.regex.Pattern;
 
+import org.apache.commons.io.input.TeeInputStream;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.FileUtil;
@@ -98,6 +100,69 @@ public class RunJar {
    * Unpack matching files from a jar. Entries inside the jar that do
    * not match the given pattern will be skipped.
    *
+   * @param inputStream the jar stream to unpack
+   * @param toDir the destination directory into which to unpack the jar
+   * @param unpackRegex the pattern to match jar entries against
+   *
+   * @throws IOException if an I/O error has occurred or toDir
+   * cannot be created and does not already exist
+   */
+  public static void unJar(InputStream inputStream, File toDir,
+                           Pattern unpackRegex)
+      throws IOException {
+    try (JarInputStream jar = new JarInputStream(inputStream)) {
+      int numOfFailedLastModifiedSet = 0;
+      for (JarEntry entry = jar.getNextJarEntry();
+           entry != null;
+           entry = jar.getNextJarEntry()) {
+        if (!entry.isDirectory() &&
+            unpackRegex.matcher(entry.getName()).matches()) {
+          File file = new File(toDir, entry.getName());
+          ensureDirectory(file.getParentFile());
+          try (OutputStream out = new FileOutputStream(file)) {
+            IOUtils.copyBytes(jar, out, BUFFER_SIZE);
+          }
+          if (!file.setLastModified(entry.getTime())) {
+            numOfFailedLastModifiedSet++;
+          }
+        }
+      }
+      if (numOfFailedLastModifiedSet > 0) {
+        LOG.warn("Could not set last modfied time for {} file(s)",
+            numOfFailedLastModifiedSet);
+      }
+    }
+  }
+
+  /**
+   * Unpack matching files from a jar. Entries inside the jar that do
+   * not match the given pattern will be skipped. Keep also a copy
+   * of the entire jar in the same directory for backward compatibility.
+   * TODO remove this feature in a new release and do only unJar
+   *
+   * @param inputStream the jar stream to unpack
+   * @param toDir the destination directory into which to unpack the jar
+   * @param unpackRegex the pattern to match jar entries against
+   *
+   * @throws IOException if an I/O error has occurred or toDir
+   * cannot be created and does not already exist
+   */
+  @Deprecated
+  public static void unJarAndSave(InputStream inputStream, File toDir,
+                           String name, Pattern unpackRegex)
+      throws IOException{
+    File file = new File(toDir, name);
+    ensureDirectory(toDir);
+    try (OutputStream jar = new FileOutputStream(file);
+         TeeInputStream teeInputStream = new TeeInputStream(inputStream, jar)) {
+      unJar(teeInputStream, toDir, unpackRegex);
+    }
+  }
+
+  /**
+   * Unpack matching files from a jar. Entries inside the jar that do
+   * not match the given pattern will be skipped.
+   *
    * @param jarFile the .jar file to unpack
    * @param toDir the destination directory into which to unpack the jar
    * @param unpackRegex the pattern to match jar entries against

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b0f265d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/FSDownload.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/FSDownload.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/FSDownload.java
index 6e59574..1a60948 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/FSDownload.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/FSDownload.java
@@ -21,6 +21,8 @@ package org.apache.hadoop.yarn.util;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.net.URISyntaxException;
 import java.security.PrivilegedExceptionAction;
 import java.util.concurrent.Callable;
@@ -29,6 +31,7 @@ import java.util.concurrent.Future;
 import java.util.regex.Pattern;
 
 import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.IOUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
@@ -54,6 +57,7 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
 import com.google.common.util.concurrent.Futures;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 
 /**
  * Download a single URL to the local disk.
@@ -247,9 +251,21 @@ public class FSDownload implements Callable<Path> {
     }
   }
 
-  private Path copy(Path sCopy, Path dstdir) throws IOException {
+  /**
+   * Localize files.
+   * @param destination destination directory
+   * @throws IOException cannot read or write file
+   * @throws YarnException subcommand returned an error
+   */
+  private void verifyAndCopy(Path destination)
+      throws IOException, YarnException {
+    final Path sCopy;
+    try {
+      sCopy = resource.getResource().toPath();
+    } catch (URISyntaxException e) {
+      throw new IOException("Invalid resource", e);
+    }
     FileSystem sourceFs = sCopy.getFileSystem(conf);
-    Path dCopy = new Path(dstdir, "tmp_"+sCopy.getName());
     FileStatus sStat = sourceFs.getFileStatus(sCopy);
     if (sStat.getModificationTime() != resource.getTimestamp()) {
       throw new IOException("Resource " + sCopy +
@@ -264,82 +280,108 @@ public class FSDownload implements Callable<Path> {
       }
     }
 
-    FileUtil.copy(sourceFs, sStat, FileSystem.getLocal(conf), dCopy, false,
-        true, conf);
-    return dCopy;
+    downloadAndUnpack(sCopy, destination);
   }
 
-  private long unpack(File localrsrc, File dst) throws IOException {
-    switch (resource.getType()) {
-    case ARCHIVE: {
-      String lowerDst = StringUtils.toLowerCase(dst.getName());
-      if (lowerDst.endsWith(".jar")) {
-        RunJar.unJar(localrsrc, dst);
-      } else if (lowerDst.endsWith(".zip")) {
-        FileUtil.unZip(localrsrc, dst);
-      } else if (lowerDst.endsWith(".tar.gz") ||
-                 lowerDst.endsWith(".tgz") ||
-                 lowerDst.endsWith(".tar")) {
-        FileUtil.unTar(localrsrc, dst);
+  /**
+   * Copy source path to destination with localization rules.
+   * @param source source path to copy. Typically HDFS
+   * @param destination destination path. Typically local filesystem
+   * @exception YarnException Any error has occurred
+   */
+  private void downloadAndUnpack(Path source, Path destination)
+      throws YarnException {
+    try {
+      FileSystem sourceFileSystem = source.getFileSystem(conf);
+      FileSystem destinationFileSystem = destination.getFileSystem(conf);
+      if (sourceFileSystem.getFileStatus(source).isDirectory()) {
+        FileUtil.copy(
+            sourceFileSystem, source,
+            destinationFileSystem, destination, false,
+            true, conf);
       } else {
-        LOG.warn("Cannot unpack " + localrsrc);
-        if (!localrsrc.renameTo(dst)) {
-            throw new IOException("Unable to rename file: [" + localrsrc
-              + "] to [" + dst + "]");
-        }
+        unpack(source, destination, sourceFileSystem, destinationFileSystem);
       }
+    } catch (Exception e) {
+      throw new YarnException("Download and unpack failed", e);
     }
-    break;
-    case PATTERN: {
+  }
+
+  /**
+   * Do the localization action on the input stream.
+   * We use the deprecated method RunJar.unJarAndSave for compatibility reasons.
+   * We should use the more efficient RunJar.unJar in the future.
+   * @param source Source path
+   * @param destination Destination pth
+   * @param sourceFileSystem Source filesystem
+   * @param destinationFileSystem Destination filesystem
+   * @throws IOException Could not read or write stream
+   * @throws InterruptedException Operation interrupted by caller
+   * @throws ExecutionException Could not create thread pool execution
+   */
+  @SuppressWarnings("deprecation")
+  private void unpack(Path source, Path destination,
+                      FileSystem sourceFileSystem,
+                      FileSystem destinationFileSystem)
+      throws IOException, InterruptedException, ExecutionException {
+    try (InputStream inputStream = sourceFileSystem.open(source)) {
+      File dst = new File(destination.toUri());
       String lowerDst = StringUtils.toLowerCase(dst.getName());
-      if (lowerDst.endsWith(".jar")) {
-        String p = resource.getPattern();
-        RunJar.unJar(localrsrc, dst,
-            p == null ? RunJar.MATCH_ANY : Pattern.compile(p));
-        File newDst = new File(dst, dst.getName());
-        if (!dst.exists() && !dst.mkdir()) {
-          throw new IOException("Unable to create directory: [" + dst + "]");
+      switch (resource.getType()) {
+      case ARCHIVE:
+        if (lowerDst.endsWith(".jar")) {
+          RunJar.unJar(inputStream, dst, RunJar.MATCH_ANY);
+        } else if (lowerDst.endsWith(".zip")) {
+          FileUtil.unZip(inputStream, dst);
+        } else if (lowerDst.endsWith(".tar.gz") ||
+            lowerDst.endsWith(".tgz") ||
+            lowerDst.endsWith(".tar")) {
+          FileUtil.unTar(inputStream, dst, lowerDst.endsWith("gz"));
+        } else {
+          LOG.warn("Cannot unpack " + source);
+          try (OutputStream outputStream =
+                   destinationFileSystem.create(destination, true)) {
+            IOUtils.copy(inputStream, outputStream);
+          }
         }
-        if (!localrsrc.renameTo(newDst)) {
-          throw new IOException("Unable to rename file: [" + localrsrc
-              + "] to [" + newDst + "]");
+        break;
+      case PATTERN:
+        if (lowerDst.endsWith(".jar")) {
+          String p = resource.getPattern();
+          if (!dst.exists() && !dst.mkdir()) {
+            throw new IOException("Unable to create directory: [" + dst + "]");
+          }
+          RunJar.unJarAndSave(inputStream, dst, source.getName(),
+              p == null ? RunJar.MATCH_ANY : Pattern.compile(p));
+        } else if (lowerDst.endsWith(".zip")) {
+          LOG.warn("Treating [" + source + "] as an archive even though it " +
+              "was specified as PATTERN");
+          FileUtil.unZip(inputStream, dst);
+        } else if (lowerDst.endsWith(".tar.gz") ||
+            lowerDst.endsWith(".tgz") ||
+            lowerDst.endsWith(".tar")) {
+          LOG.warn("Treating [" + source + "] as an archive even though it " +
+              "was specified as PATTERN");
+          FileUtil.unTar(inputStream, dst, lowerDst.endsWith("gz"));
+        } else {
+          LOG.warn("Cannot unpack " + source);
+          try (OutputStream outputStream =
+                   destinationFileSystem.create(destination, true)) {
+            IOUtils.copy(inputStream, outputStream);
+          }
         }
-      } else if (lowerDst.endsWith(".zip")) {
-        LOG.warn("Treating [" + localrsrc + "] as an archive even though it " +
-        		"was specified as PATTERN");
-        FileUtil.unZip(localrsrc, dst);
-      } else if (lowerDst.endsWith(".tar.gz") ||
-                 lowerDst.endsWith(".tgz") ||
-                 lowerDst.endsWith(".tar")) {
-        LOG.warn("Treating [" + localrsrc + "] as an archive even though it " +
-        "was specified as PATTERN");
-        FileUtil.unTar(localrsrc, dst);
-      } else {
-        LOG.warn("Cannot unpack " + localrsrc);
-        if (!localrsrc.renameTo(dst)) {
-          throw new IOException("Unable to rename file: [" + localrsrc
-              + "] to [" + dst + "]");
+        break;
+      case FILE:
+      default:
+        try (OutputStream outputStream =
+                 destinationFileSystem.create(destination, true)) {
+          IOUtils.copy(inputStream, outputStream);
         }
+        break;
       }
+      // TODO Should calculate here before returning
+      //return FileUtil.getDU(destDir);
     }
-    break;
-    case FILE:
-    default:
-      if (!localrsrc.renameTo(dst)) {
-        throw new IOException("Unable to rename file: [" + localrsrc
-          + "] to [" + dst + "]");
-      }
-      break;
-    }
-    if(localrsrc.isFile()){
-      try {
-        files.delete(new Path(localrsrc.toString()), false);
-      } catch (IOException ignore) {
-      }
-    }
-    return 0;
-    // TODO Should calculate here before returning
-    //return FileUtil.getDU(destDir);
   }
 
   @Override
@@ -352,27 +394,34 @@ public class FSDownload implements Callable<Path> {
     }
 
     if (LOG.isDebugEnabled()) {
-      LOG.debug("Starting to download " + sCopy);
+      LOG.debug(String.format("Starting to download %s %s %s",
+          sCopy,
+          resource.getType(),
+          resource.getPattern()));
     }
 
-    createDir(destDirPath, cachePerms);
-    final Path dst_work = new Path(destDirPath + "_tmp");
-    createDir(dst_work, cachePerms);
-    Path dFinal = files.makeQualified(new Path(dst_work, sCopy.getName()));
+    final Path destinationTmp = new Path(destDirPath + "_tmp");
+    createDir(destinationTmp, PRIVATE_DIR_PERMS);
+    Path dFinal =
+        files.makeQualified(new Path(destinationTmp, sCopy.getName()));
     try {
-      Path dTmp = null == userUgi ? files.makeQualified(copy(sCopy, dst_work))
-          : userUgi.doAs(new PrivilegedExceptionAction<Path>() {
-            public Path run() throws Exception {
-              return files.makeQualified(copy(sCopy, dst_work));
-            };
-          });
-      unpack(new File(dTmp.toUri()), new File(dFinal.toUri()));
+      if (userUgi == null) {
+        verifyAndCopy(dFinal);
+      } else {
+        userUgi.doAs(new PrivilegedExceptionAction<Void>() {
+          @Override
+          public Void run() throws Exception {
+            verifyAndCopy(dFinal);
+            return null;
+          }
+        });
+      }
       changePermissions(dFinal.getFileSystem(conf), dFinal);
-      files.rename(dst_work, destDirPath, Rename.OVERWRITE);
+      files.rename(destinationTmp, destDirPath, Rename.OVERWRITE);
 
       if (LOG.isDebugEnabled()) {
-        LOG.debug("File has been downloaded to " +
-            new Path(destDirPath, sCopy.getName()));
+        LOG.debug(String.format("File has been downloaded to %s from %s",
+            new Path(destDirPath, sCopy.getName()), sCopy));
       }
     } catch (Exception e) {
       try {
@@ -382,7 +431,7 @@ public class FSDownload implements Callable<Path> {
       throw e;
     } finally {
       try {
-        files.delete(dst_work, true);
+        files.delete(destinationTmp, true);
       } catch (FileNotFoundException ignore) {
       }
       conf = null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b0f265d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestFSDownload.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestFSDownload.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestFSDownload.java
index 877dd08..fa8c039 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestFSDownload.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestFSDownload.java
@@ -82,6 +82,9 @@ import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
 
+/**
+ * Unit test for the FSDownload class.
+ */
 public class TestFSDownload {
 
   private static final Log LOG = LogFactory.getLog(TestFSDownload.class);
@@ -90,7 +93,8 @@ public class TestFSDownload {
   private enum TEST_FILE_TYPE {
     TAR, JAR, ZIP, TGZ
   };
-  
+  private Configuration conf = new Configuration();
+
   @AfterClass
   public static void deleteTestDir() throws IOException {
     FileContext fs = FileContext.getLocalFSFileContext();
@@ -132,6 +136,18 @@ public class TestFSDownload {
     FileOutputStream stream = new FileOutputStream(jarFile);
     LOG.info("Create jar out stream ");
     JarOutputStream out = new JarOutputStream(stream, new Manifest());
+    ZipEntry entry = new ZipEntry("classes/1.class");
+    out.putNextEntry(entry);
+    out.write(1);
+    out.write(2);
+    out.write(3);
+    out.closeEntry();
+    ZipEntry entry2 = new ZipEntry("classes/2.class");
+    out.putNextEntry(entry2);
+    out.write(1);
+    out.write(2);
+    out.write(3);
+    out.closeEntry();
     LOG.info("Done writing jar stream ");
     out.close();
     LocalResource ret = recordFactory.newRecordInstance(LocalResource.class);
@@ -256,7 +272,6 @@ public class TestFSDownload {
   @Test (timeout=10000)
   public void testDownloadBadPublic() throws IOException, URISyntaxException,
       InterruptedException {
-    Configuration conf = new Configuration();
     conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "077");
     FileContext files = FileContext.getLocalFSFileContext(conf);
     final Path basedir = files.makeQualified(new Path("target",
@@ -307,7 +322,6 @@ public class TestFSDownload {
   @Test (timeout=60000)
   public void testDownloadPublicWithStatCache() throws IOException,
       URISyntaxException, InterruptedException, ExecutionException {
-    final Configuration conf = new Configuration();
     FileContext files = FileContext.getLocalFSFileContext(conf);
     Path basedir = files.makeQualified(new Path("target",
       TestFSDownload.class.getSimpleName()));
@@ -382,7 +396,6 @@ public class TestFSDownload {
   @Test (timeout=10000)
   public void testDownload() throws IOException, URISyntaxException,
       InterruptedException {
-    Configuration conf = new Configuration();
     conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "077");
     FileContext files = FileContext.getLocalFSFileContext(conf);
     final Path basedir = files.makeQualified(new Path("target",
@@ -438,7 +451,7 @@ public class TestFSDownload {
         FileStatus status = files.getFileStatus(localized.getParent());
         FsPermission perm = status.getPermission();
         assertEquals("Cache directory permissions are incorrect",
-            new FsPermission((short)0755), perm);
+            new FsPermission((short)0700), perm);
 
         status = files.getFileStatus(localized);
         perm = status.getPermission();
@@ -455,7 +468,6 @@ public class TestFSDownload {
 
   private void downloadWithFileType(TEST_FILE_TYPE fileType) throws IOException, 
       URISyntaxException, InterruptedException{
-    Configuration conf = new Configuration();
     conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "077");
     FileContext files = FileContext.getLocalFSFileContext(conf);
     final Path basedir = files.makeQualified(new Path("target",
@@ -530,7 +542,7 @@ public class TestFSDownload {
     }
   }
 
-  @Test (timeout=10000) 
+  @Test (timeout=10000)
   public void testDownloadArchive() throws IOException, URISyntaxException,
       InterruptedException {
     downloadWithFileType(TEST_FILE_TYPE.TAR);
@@ -542,7 +554,7 @@ public class TestFSDownload {
     downloadWithFileType(TEST_FILE_TYPE.JAR);
   }
 
-  @Test (timeout=10000) 
+  @Test (timeout=10000)
   public void testDownloadArchiveZip() throws IOException, URISyntaxException,
       InterruptedException {
     downloadWithFileType(TEST_FILE_TYPE.ZIP);
@@ -603,7 +615,6 @@ public class TestFSDownload {
 
   @Test (timeout=10000)
   public void testDirDownload() throws IOException, InterruptedException {
-    Configuration conf = new Configuration();
     FileContext files = FileContext.getLocalFSFileContext(conf);
     final Path basedir = files.makeQualified(new Path("target",
       TestFSDownload.class.getSimpleName()));
@@ -668,7 +679,6 @@ public class TestFSDownload {
 
   @Test (timeout=10000)
   public void testUniqueDestinationPath() throws Exception {
-    Configuration conf = new Configuration();
     FileContext files = FileContext.getLocalFSFileContext(conf);
     final Path basedir = files.makeQualified(new Path("target",
         TestFSDownload.class.getSimpleName()));


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[09/37] hadoop git commit: HDFS-13049. RBF: Inconsistent Router OPTS config in branch-2 and branch-3. Contributed by Wei Yan.

Posted by ae...@apache.org.
HDFS-13049. RBF: Inconsistent Router OPTS config in branch-2 and branch-3. Contributed by Wei Yan.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/0c139d5b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/0c139d5b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/0c139d5b

Branch: refs/heads/HDFS-7240
Commit: 0c139d5bcfbcd62fc69111ee6204926c57d57bf1
Parents: 82cc6f6
Author: Wei Yan <we...@apache.org>
Authored: Thu Jan 25 11:36:28 2018 -0800
Committer: Wei Yan <we...@apache.org>
Committed: Thu Jan 25 11:36:40 2018 -0800

----------------------------------------------------------------------
 .../hadoop-common/src/main/conf/hadoop-env.cmd              | 9 +++++++++
 .../hadoop-common/src/main/conf/hadoop-env.sh               | 9 +++++++++
 2 files changed, 18 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0c139d5b/hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.cmd
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.cmd b/hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.cmd
index 5dbd635..159840a 100644
--- a/hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.cmd
+++ b/hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.cmd
@@ -70,6 +70,15 @@ set HADOOP_SECURE_DN_USER=%HADOOP_SECURE_DN_USER%
 @rem Where log files are stored in the secure data environment.
 set HADOOP_SECURE_DN_LOG_DIR=%HADOOP_LOG_DIR%\%HADOOP_HDFS_USER%
 
+@rem
+@rem Router-based HDFS Federation specific parameters
+@rem Specify the JVM options to be used when starting the RBF Routers.
+@rem These options will be appended to the options specified as HADOOP_OPTS
+@rem and therefore may override any similar flags set in HADOOP_OPTS
+@rem
+@rem set HADOOP_DFSROUTER_OPTS=""
+@rem
+
 @rem The directory where pid files are stored. /tmp by default.
 @rem NOTE: this should be set to a directory that can only be written to by 
 @rem       the user that will run the hadoop daemons.  Otherwise there is the

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0c139d5b/hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh b/hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh
index bef4dab..24aacdf 100644
--- a/hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh
+++ b/hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh
@@ -396,6 +396,15 @@ esac
 # export HDFS_MOVER_OPTS=""
 
 ###
+# Router-based HDFS Federation specific parameters
+# Specify the JVM options to be used when starting the RBF Routers.
+# These options will be appended to the options specified as HADOOP_OPTS
+# and therefore may override any similar flags set in HADOOP_OPTS
+#
+# export HDFS_DFSROUTER_OPTS=""
+###
+
+###
 # Advanced Users Only!
 ###
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[34/37] hadoop git commit: YARN-7723. Avoid using docker volume --format option to run against to older docker releases. Contributed by Wangda Tan

Posted by ae...@apache.org.
YARN-7723. Avoid using docker volume --format option to run against to older docker releases. Contributed by Wangda Tan


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/6463e10c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/6463e10c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/6463e10c

Branch: refs/heads/HDFS-7240
Commit: 6463e10c72344e1720c991307cf2e7b67e112a3a
Parents: f666e7c
Author: Sunil G <su...@apache.org>
Authored: Tue Jan 30 15:58:11 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Tue Jan 30 15:58:11 2018 +0530

----------------------------------------------------------------------
 .../runtime/DockerLinuxContainerRuntime.java    |  9 +------
 .../runtime/TestDockerContainerRuntime.java     | 25 ++++++++++++++++----
 2 files changed, 22 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6463e10c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
index f3ce73d..601c32c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
@@ -437,7 +437,6 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
       throws ContainerExecutionException {
     DockerVolumeCommand dockerVolumeInspectCommand = new DockerVolumeCommand(
         DockerVolumeCommand.VOLUME_LS_SUB_COMMAND);
-    dockerVolumeInspectCommand.setFormat("{{.Name}},{{.Driver}}");
     String output = runDockerVolumeCommand(dockerVolumeInspectCommand,
         container);
 
@@ -450,13 +449,7 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
 
     for (String line : output.split("\n")) {
       line = line.trim();
-      String[] arr = line.split(",");
-      String v = arr[0].trim();
-      String d = null;
-      if (arr.length > 1) {
-        d = arr[1].trim();
-      }
-      if (d != null && volumeName.equals(v) && driverName.equals(d)) {
+      if (line.contains(volumeName) && line.contains(driverName)) {
         // Good we found it.
         LOG.info(
             "Docker volume-name=" + volumeName + " driver-name=" + driverName

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6463e10c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java
index 48a96e1..fe4e238 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java
@@ -1473,9 +1473,9 @@ public class TestDockerContainerRuntime {
     commandFile = new File(StringUtils.join(",", op.getArguments()));
     fileInputStream = new FileInputStream(commandFile);
     fileContent = new String(IOUtils.toByteArray(fileInputStream));
-    Assert.assertEquals("[docker-command-execution]\n"
-        + "  docker-command=volume\n" + "  format={{.Name}},{{.Driver}}\n"
-        + "  sub-command=ls\n", fileContent);
+    Assert.assertEquals(
+        "[docker-command-execution]\n" + "  docker-command=volume\n"
+            + "  sub-command=ls\n", fileContent);
     fileInputStream.close();
   }
 
@@ -1577,16 +1577,33 @@ public class TestDockerContainerRuntime {
     // For following tests, we expect to have volume1,local in output
 
     // Failure cases
+    testDockerCommandPluginWithVolumesOutput(
+        "DRIVER              VOLUME NAME\n", true);
     testDockerCommandPluginWithVolumesOutput("", true);
     testDockerCommandPluginWithVolumesOutput("volume1", true);
+    testDockerCommandPluginWithVolumesOutput(
+        "DRIVER              VOLUME NAME\n" +
+        "nvidia-docker       nvidia_driver_375.66\n", true);
+    testDockerCommandPluginWithVolumesOutput(
+        "DRIVER              VOLUME NAME\n" +
+        "                    volume1\n", true);
     testDockerCommandPluginWithVolumesOutput("local", true);
     testDockerCommandPluginWithVolumesOutput("volume2,local", true);
+    testDockerCommandPluginWithVolumesOutput(
+        "DRIVER              VOLUME NAME\n" +
+        "local               volume2\n", true);
     testDockerCommandPluginWithVolumesOutput("volum1,something", true);
+    testDockerCommandPluginWithVolumesOutput(
+        "DRIVER              VOLUME NAME\n" +
+        "something               volume1\n", true);
     testDockerCommandPluginWithVolumesOutput("volum1,something\nvolum2,local",
         true);
 
     // Success case
-    testDockerCommandPluginWithVolumesOutput("volume1,local\n", false);
+    testDockerCommandPluginWithVolumesOutput(
+        "DRIVER              VOLUME NAME\n" +
+        "nvidia-docker       nvidia_driver_375.66\n" +
+        "local               volume1\n", false);
     testDockerCommandPluginWithVolumesOutput(
         "volume_xyz,nvidia\nvolume1,local\n\n", false);
     testDockerCommandPluginWithVolumesOutput(" volume1,  local \n", false);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[27/37] hadoop git commit: YARN-7698. A misleading variable's name in ApplicationAttemptEventDispatcher

Posted by ae...@apache.org.
YARN-7698. A misleading variable's name in ApplicationAttemptEventDispatcher

Signed-off-by: Akira Ajisaka <aa...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/3400d0c5
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/3400d0c5
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/3400d0c5

Branch: refs/heads/HDFS-7240
Commit: 3400d0c535aeb151c3f283cc41111b15d66990e5
Parents: b63dcd5
Author: Jinjiang Ling <li...@zte.com.cn>
Authored: Tue Jan 30 00:00:57 2018 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Tue Jan 30 00:00:57 2018 +0900

----------------------------------------------------------------------
 .../hadoop/yarn/server/resourcemanager/ResourceManager.java  | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3400d0c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
index 7bbf4aa..32c4b0a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
@@ -947,11 +947,11 @@ public class ResourceManager extends CompositeService implements Recoverable {
 
     @Override
     public void handle(RMAppAttemptEvent event) {
-      ApplicationAttemptId appAttemptID = event.getApplicationAttemptId();
-      ApplicationId appAttemptId = appAttemptID.getApplicationId();
-      RMApp rmApp = this.rmContext.getRMApps().get(appAttemptId);
+      ApplicationAttemptId appAttemptId = event.getApplicationAttemptId();
+      ApplicationId appId = appAttemptId.getApplicationId();
+      RMApp rmApp = this.rmContext.getRMApps().get(appId);
       if (rmApp != null) {
-        RMAppAttempt rmAppAttempt = rmApp.getRMAppAttempt(appAttemptID);
+        RMAppAttempt rmAppAttempt = rmApp.getRMAppAttempt(appAttemptId);
         if (rmAppAttempt != null) {
           try {
             rmAppAttempt.handle(event);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[23/37] hadoop git commit: YARN-7064. Use cgroup to get container resource utilization. (Miklos Szegedi via Haibo Chen)

Posted by ae...@apache.org.
YARN-7064. Use cgroup to get container resource utilization. (Miklos Szegedi via Haibo Chen)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/649ef7ac
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/649ef7ac
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/649ef7ac

Branch: refs/heads/HDFS-7240
Commit: 649ef7ac334e63a7c676f8e7406f59d9466eb6f2
Parents: 6eef3d7
Author: Haibo Chen <ha...@apache.org>
Authored: Fri Jan 26 16:27:31 2018 -0800
Committer: Haibo Chen <ha...@apache.org>
Committed: Fri Jan 26 16:27:31 2018 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/util/CpuTimeTracker.java  |   2 +-
 .../hadoop/yarn/conf/YarnConfiguration.java     |  10 +-
 .../yarn/conf/TestYarnConfigurationFields.java  |   2 -
 .../yarn/util/ProcfsBasedProcessTree.java       |   3 +
 .../util/ResourceCalculatorProcessTree.java     |   9 +
 .../src/main/resources/yarn-default.xml         |  34 +-
 .../linux/resources/CGroupsHandler.java         |   9 +-
 .../linux/resources/CGroupsHandlerImpl.java     |   3 +-
 .../CGroupsMemoryResourceHandlerImpl.java       |  52 +--
 .../resources/CGroupsResourceCalculator.java    | 357 +++++++++++++++++++
 .../resources/CombinedResourceCalculator.java   | 108 ++++++
 .../linux/resources/ResourceHandlerModule.java  |  43 ++-
 .../monitor/ContainersMonitorImpl.java          |  33 +-
 .../TestCGroupsMemoryResourceHandlerImpl.java   |  45 +++
 .../TestCGroupsResourceCalculator.java          | 274 ++++++++++++++
 .../TestCompareResourceCalculators.java         | 227 ++++++++++++
 .../resources/TestResourceHandlerModule.java    |  47 ++-
 17 files changed, 1183 insertions(+), 75 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/649ef7ac/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/CpuTimeTracker.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/CpuTimeTracker.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/CpuTimeTracker.java
index b4ebe86..4355367 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/CpuTimeTracker.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/CpuTimeTracker.java
@@ -99,7 +99,7 @@ public class CpuTimeTracker {
   public void updateElapsedJiffies(BigInteger elapsedJiffies, long newTime) {
     BigInteger newValue = elapsedJiffies.multiply(jiffyLengthInMillis);
     cumulativeCpuTime = newValue.compareTo(cumulativeCpuTime) >= 0 ?
-        newValue : cumulativeCpuTime;
+            newValue : cumulativeCpuTime;
     sampleTime = newTime;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/649ef7ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index f132683..bbbfc52 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -1357,22 +1357,20 @@ public class YarnConfiguration extends Configuration {
   public static final String NM_MEMORY_RESOURCE_PREFIX = NM_PREFIX
       + "resource.memory.";
 
-  @Private
   public static final String NM_MEMORY_RESOURCE_ENABLED =
       NM_MEMORY_RESOURCE_PREFIX + "enabled";
-  @Private
   public static final boolean DEFAULT_NM_MEMORY_RESOURCE_ENABLED = false;
 
-  @Private
+  public static final String NM_MEMORY_RESOURCE_ENFORCED =
+      NM_MEMORY_RESOURCE_PREFIX + "enforced";
+  public static final boolean DEFAULT_NM_MEMORY_RESOURCE_ENFORCED = true;
+
   public static final String NM_MEMORY_RESOURCE_CGROUPS_SWAPPINESS =
       NM_MEMORY_RESOURCE_PREFIX + "cgroups.swappiness";
-  @Private
   public static final int DEFAULT_NM_MEMORY_RESOURCE_CGROUPS_SWAPPINESS = 0;
 
-  @Private
   public static final String NM_MEMORY_RESOURCE_CGROUPS_SOFT_LIMIT_PERCENTAGE =
       NM_MEMORY_RESOURCE_PREFIX + "cgroups.soft-limit-percentage";
-  @Private
   public static final float
       DEFAULT_NM_MEMORY_RESOURCE_CGROUPS_SOFT_LIMIT_PERCENTAGE =
       90.0f;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/649ef7ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
index 3976d2d..9fe4f88 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
@@ -159,8 +159,6 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
     configurationPrefixToSkipCompare
         .add(YarnConfiguration.NM_DISK_RESOURCE_ENABLED);
     configurationPrefixToSkipCompare
-        .add(YarnConfiguration.NM_MEMORY_RESOURCE_PREFIX);
-    configurationPrefixToSkipCompare
         .add(YarnConfiguration.NM_CPU_RESOURCE_ENABLED);
     configurationPrefixToSkipCompare.add(
         YarnConfiguration.NM_NETWORK_TAG_MAPPING_MANAGER);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/649ef7ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java
index 7431fdf..55be001 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java
@@ -468,6 +468,9 @@ public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree {
   @Override
   public float getCpuUsagePercent() {
     BigInteger processTotalJiffies = getTotalProcessJiffies();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Process " + pid + " jiffies:" + processTotalJiffies);
+    }
     cpuTimeTracker.updateElapsedJiffies(processTotalJiffies,
         clock.getTime());
     return cpuTimeTracker.getCpuTrackerUsagePercent();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/649ef7ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorProcessTree.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorProcessTree.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorProcessTree.java
index 7e5cf55..c581b83 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorProcessTree.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorProcessTree.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 
 /**
  * Interface class to obtain process resource usage
@@ -51,6 +52,13 @@ public abstract class ResourceCalculatorProcessTree extends Configured {
   }
 
   /**
+   * Initialize the object.
+   * @throws YarnException Throws an exception on error.
+   */
+  public void initialize() throws YarnException {
+  }
+
+  /**
    * Update the process-tree with latest state.
    *
    * Each call to this function should increment the age of the running
@@ -168,6 +176,7 @@ public abstract class ResourceCalculatorProcessTree extends Configured {
         Constructor <? extends ResourceCalculatorProcessTree> c = clazz.getConstructor(String.class);
         ResourceCalculatorProcessTree rctree = c.newInstance(pid);
         rctree.setConf(conf);
+        rctree.initialize();
         return rctree;
       } catch(Exception e) {
         throw new RuntimeException(e);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/649ef7ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
index 1348d6d..0bb4fca 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
@@ -1309,6 +1309,37 @@
   </property>
 
   <property>
+    <description>Whether YARN CGroups memory tracking is enabled.</description>
+    <name>yarn.nodemanager.resource.memory.enabled</name>
+    <value>false</value>
+  </property>
+
+  <property>
+    <description>Whether YARN CGroups strict memory enforcement is enabled.
+    </description>
+    <name>yarn.nodemanager.resource.memory.enforced</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <description>If memory limit is enforced, this the percentage of soft limit
+      compared to the memory assigned to the container. If there is memory
+      pressure container memory usage will be pushed back to its soft limit
+      by swapping out memory.
+    </description>
+    <name>yarn.nodemanager.resource.memory.cgroups.soft-limit-percentage</name>
+    <value>90.0</value>
+  </property>
+
+  <property>
+    <description>Container swappiness is the likelihood a page will be swapped
+      out compared to be kept in memory. Value is between 0-100.
+    </description>
+    <name>yarn.nodemanager.resource.memory.cgroups.swappiness</name>
+    <value>0</value>
+  </property>
+
+  <property>
     <description>Whether physical memory limits will be enforced for
     containers.</description>
     <name>yarn.nodemanager.pmem-check-enabled</name>
@@ -1622,7 +1653,8 @@
     or be allowed to consume spare resources if they need them. For example, turning the
     flag on will restrict apps to use only their share of CPU, even if the node has spare
     CPU cycles. The default value is false i.e. use available resources. Please note that
-    turning this flag on may reduce job throughput on the cluster.</description>
+    turning this flag on may reduce job throughput on the cluster. This setting does
+    not apply to other subsystems like memory.</description>
     <name>yarn.nodemanager.linux-container-executor.cgroups.strict-resource-usage</name>
     <value>false</value>
   </property>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/649ef7ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandler.java
index 5f4d3e4..e279504 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandler.java
@@ -54,7 +54,7 @@ public interface CGroupsHandler {
       this.name = name;
     }
 
-    String getName() {
+    public String getName() {
       return name;
     }
 
@@ -113,6 +113,13 @@ public interface CGroupsHandler {
       ResourceHandlerException;
 
   /**
+   * Gets the absolute path to the specified cgroup controller.
+   * @param controller - controller type for the cgroup
+   * @return the root of the controller.
+   */
+  String getControllerPath(CGroupController controller);
+
+  /**
    * Gets the relative path for the cgroup, independent of a controller, for a
    * given cgroup id.
    * @param cGroupId - id of the cgroup

http://git-wip-us.apache.org/repos/asf/hadoop/blob/649ef7ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandlerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandlerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandlerImpl.java
index 619a65b..008f3d7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandlerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandlerImpl.java
@@ -125,7 +125,8 @@ class CGroupsHandlerImpl implements CGroupsHandler {
     initializeControllerPaths();
   }
 
-  private String getControllerPath(CGroupController controller) {
+  @Override
+  public String getControllerPath(CGroupController controller) {
     try {
       rwLock.readLock().lock();
       return controllerPaths.get(controller);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/649ef7ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsMemoryResourceHandlerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsMemoryResourceHandlerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsMemoryResourceHandlerImpl.java
index d3e787e..558751f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsMemoryResourceHandlerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsMemoryResourceHandlerImpl.java
@@ -52,6 +52,7 @@ public class CGroupsMemoryResourceHandlerImpl implements MemoryResourceHandler {
   private static final int OPPORTUNISTIC_SOFT_LIMIT = 0;
 
   private CGroupsHandler cGroupsHandler;
+  private boolean enforce = true;
   private int swappiness = 0;
   // multiplier to set the soft limit - value should be between 0 and 1
   private float softLimit = 0.0f;
@@ -79,6 +80,9 @@ public class CGroupsMemoryResourceHandlerImpl implements MemoryResourceHandler {
       throw new ResourceHandlerException(msg);
     }
     this.cGroupsHandler.initializeCGroupController(MEMORY);
+    enforce = conf.getBoolean(
+        YarnConfiguration.NM_MEMORY_RESOURCE_ENFORCED,
+        YarnConfiguration.DEFAULT_NM_MEMORY_RESOURCE_ENFORCED);
     swappiness = conf
         .getInt(YarnConfiguration.NM_MEMORY_RESOURCE_CGROUPS_SWAPPINESS,
             YarnConfiguration.DEFAULT_NM_MEMORY_RESOURCE_CGROUPS_SWAPPINESS);
@@ -124,31 +128,33 @@ public class CGroupsMemoryResourceHandlerImpl implements MemoryResourceHandler {
         (long) (container.getResource().getMemorySize() * this.softLimit);
     long containerHardLimit = container.getResource().getMemorySize();
     cGroupsHandler.createCGroup(MEMORY, cgroupId);
-    try {
-      cGroupsHandler.updateCGroupParam(MEMORY, cgroupId,
-          CGroupsHandler.CGROUP_PARAM_MEMORY_HARD_LIMIT_BYTES,
-          String.valueOf(containerHardLimit) + "M");
-      ContainerTokenIdentifier id = container.getContainerTokenIdentifier();
-      if (id != null && id.getExecutionType() ==
-          ExecutionType.OPPORTUNISTIC) {
+    if (enforce) {
+      try {
         cGroupsHandler.updateCGroupParam(MEMORY, cgroupId,
-            CGroupsHandler.CGROUP_PARAM_MEMORY_SOFT_LIMIT_BYTES,
-            String.valueOf(OPPORTUNISTIC_SOFT_LIMIT) + "M");
-        cGroupsHandler.updateCGroupParam(MEMORY, cgroupId,
-            CGroupsHandler.CGROUP_PARAM_MEMORY_SWAPPINESS,
-            String.valueOf(OPPORTUNISTIC_SWAPPINESS));
-      } else {
-        cGroupsHandler.updateCGroupParam(MEMORY, cgroupId,
-            CGroupsHandler.CGROUP_PARAM_MEMORY_SOFT_LIMIT_BYTES,
-            String.valueOf(containerSoftLimit) + "M");
-        cGroupsHandler.updateCGroupParam(MEMORY, cgroupId,
-            CGroupsHandler.CGROUP_PARAM_MEMORY_SWAPPINESS,
-            String.valueOf(swappiness));
+            CGroupsHandler.CGROUP_PARAM_MEMORY_HARD_LIMIT_BYTES,
+            String.valueOf(containerHardLimit) + "M");
+        ContainerTokenIdentifier id = container.getContainerTokenIdentifier();
+        if (id != null && id.getExecutionType() ==
+            ExecutionType.OPPORTUNISTIC) {
+          cGroupsHandler.updateCGroupParam(MEMORY, cgroupId,
+              CGroupsHandler.CGROUP_PARAM_MEMORY_SOFT_LIMIT_BYTES,
+              String.valueOf(OPPORTUNISTIC_SOFT_LIMIT) + "M");
+          cGroupsHandler.updateCGroupParam(MEMORY, cgroupId,
+              CGroupsHandler.CGROUP_PARAM_MEMORY_SWAPPINESS,
+              String.valueOf(OPPORTUNISTIC_SWAPPINESS));
+        } else {
+          cGroupsHandler.updateCGroupParam(MEMORY, cgroupId,
+              CGroupsHandler.CGROUP_PARAM_MEMORY_SOFT_LIMIT_BYTES,
+              String.valueOf(containerSoftLimit) + "M");
+          cGroupsHandler.updateCGroupParam(MEMORY, cgroupId,
+              CGroupsHandler.CGROUP_PARAM_MEMORY_SWAPPINESS,
+              String.valueOf(swappiness));
+        }
+      } catch (ResourceHandlerException re) {
+        cGroupsHandler.deleteCGroup(MEMORY, cgroupId);
+        LOG.warn("Could not update cgroup for container", re);
+        throw re;
       }
-    } catch (ResourceHandlerException re) {
-      cGroupsHandler.deleteCGroup(MEMORY, cgroupId);
-      LOG.warn("Could not update cgroup for container", re);
-      throw re;
     }
     List<PrivilegedOperation> ret = new ArrayList<>();
     ret.add(new PrivilegedOperation(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/649ef7ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsResourceCalculator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsResourceCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsResourceCalculator.java
new file mode 100644
index 0000000..50ce3ea
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsResourceCalculator.java
@@ -0,0 +1,357 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.util.CpuTimeTracker;
+import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.util.SysInfoLinux;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree;
+import org.apache.hadoop.yarn.util.SystemClock;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.math.BigInteger;
+import java.nio.charset.Charset;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * A cgroups file-system based Resource calculator without the process tree
+ * features.
+ *
+ * CGroups has its limitations. It can only be enabled, if both CPU and memory
+ * cgroups are enabled with yarn.nodemanager.resource.cpu.enabled and
+ * yarn.nodemanager.resource.memory.enabled respectively. This means that
+ * memory limits are enforced by default. You can turn this off and keep
+ * memory reporting only with yarn.nodemanager.resource.memory.enforced.
+ *
+ * Another limitation is virtual memory measurement. CGroups does not have the
+ * ability to measure virtual memory usage. This includes memory reserved but
+ * not used. CGroups measures used memory as sa sum of
+ * physical memory and swap usage. This will be returned in the virtual
+ * memory counters.
+ * If the real virtual memory is required please use the legacy procfs based
+ * resource calculator or CombinedResourceCalculator.
+ */
+public class CGroupsResourceCalculator extends ResourceCalculatorProcessTree {
+  enum Result {
+    Continue,
+    Exit
+  }
+  protected static final Log LOG = LogFactory
+      .getLog(CGroupsResourceCalculator.class);
+  private static final String PROCFS = "/proc";
+  static final String CGROUP = "cgroup";
+  static final String CPU_STAT = "cpuacct.stat";
+  static final String MEM_STAT = "memory.usage_in_bytes";
+  static final String MEMSW_STAT = "memory.memsw.usage_in_bytes";
+  private static final String USER = "user ";
+  private static final String SYSTEM = "system ";
+
+  private static final Pattern CGROUP_FILE_FORMAT = Pattern.compile(
+      "^(\\d+):([^:]+):/(.*)$");
+  private final String procfsDir;
+  private CGroupsHandler cGroupsHandler;
+
+  private String pid;
+  private File cpuStat;
+  private File memStat;
+  private File memswStat;
+
+  private BigInteger processTotalJiffies;
+  private long processPhysicalMemory;
+  private long processVirtualMemory;
+
+  private final long jiffyLengthMs;
+  private final CpuTimeTracker cpuTimeTracker;
+  private Clock clock;
+
+  /**
+   * Create resource calculator for all Yarn containers.
+   */
+  public CGroupsResourceCalculator()
+      throws YarnException {
+    this(null, PROCFS, ResourceHandlerModule.getCGroupsHandler(),
+        SystemClock.getInstance(), SysInfoLinux.JIFFY_LENGTH_IN_MILLIS);
+  }
+
+  /**
+   * Create resource calculator for the container that has the specified pid.
+   * @param pid A pid from the cgroup or null for all containers
+   */
+  public CGroupsResourceCalculator(String pid) {
+    this(pid, PROCFS, ResourceHandlerModule.getCGroupsHandler(),
+        SystemClock.getInstance(), SysInfoLinux.JIFFY_LENGTH_IN_MILLIS);
+  }
+
+  /**
+   * Create resource calculator for testing.
+   * @param pid A pid from the cgroup or null for all containers
+   * @param procfsDir Path to /proc or a mock /proc directory
+   * @param cGroupsHandler Initialized cgroups handler object
+   * @param clock A clock object
+   * @param jiffyLengthMs0 Jiffy length in milliseconds
+   */
+  @VisibleForTesting
+  CGroupsResourceCalculator(String pid, String procfsDir,
+                            CGroupsHandler cGroupsHandler,
+                            Clock clock,
+                            long jiffyLengthMs0) {
+    super(pid);
+    this.procfsDir = procfsDir;
+    this.cGroupsHandler = cGroupsHandler;
+    this.pid = pid != null && pid.equals("0") ? "1" : pid;
+    this.jiffyLengthMs = jiffyLengthMs0;
+    this.cpuTimeTracker =
+        new CpuTimeTracker(this.jiffyLengthMs);
+    this.clock = clock;
+    this.processTotalJiffies = BigInteger.ZERO;
+    this.processPhysicalMemory = UNAVAILABLE;
+    this.processVirtualMemory = UNAVAILABLE;
+  }
+
+  @Override
+  public void initialize() throws YarnException {
+    if (!CGroupsResourceCalculator.isAvailable()) {
+      throw new YarnException("CGroupsResourceCalculator is not available");
+    }
+    setCGroupFilePaths();
+  }
+
+  @Override
+  public float getCpuUsagePercent() {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Process " + pid + " jiffies:" + processTotalJiffies);
+    }
+    return cpuTimeTracker.getCpuTrackerUsagePercent();
+  }
+
+  @Override
+  public long getCumulativeCpuTime() {
+    if (jiffyLengthMs < 0) {
+      return UNAVAILABLE;
+    }
+    return processTotalJiffies.longValue() * jiffyLengthMs;
+  }
+
+  @Override
+  public long getRssMemorySize(int olderThanAge) {
+    if (olderThanAge > 1) {
+      return UNAVAILABLE;
+    }
+    return processPhysicalMemory;
+  }
+
+  @Override
+  public long getVirtualMemorySize(int olderThanAge) {
+    if (olderThanAge > 1) {
+      return UNAVAILABLE;
+    }
+    return processVirtualMemory;
+  }
+
+  @Override
+  public void updateProcessTree() {
+    try {
+      this.processTotalJiffies = readTotalProcessJiffies();
+      cpuTimeTracker.updateElapsedJiffies(processTotalJiffies,
+          clock.getTime());
+    } catch (YarnException e) {
+      LOG.warn("Failed to parse " + pid, e);
+    }
+    processPhysicalMemory = getMemorySize(memStat);
+    if (memswStat.exists()) {
+      processVirtualMemory = getMemorySize(memswStat);
+    } else if(LOG.isDebugEnabled()) {
+      LOG.debug("Swap cgroups monitoring is not compiled into the kernel " +
+          memswStat.getAbsolutePath().toString());
+    }
+  }
+
+  @Override
+  public String getProcessTreeDump() {
+    // We do not have a process tree in cgroups return just the pid for tracking
+    return pid;
+  }
+
+  @Override
+  public boolean checkPidPgrpidForMatch() {
+    // We do not have a process tree in cgroups returning default ok
+    return true;
+  }
+
+  /**
+   * Checks if the CGroupsResourceCalculator is available on this system.
+   * This assumes that Linux container executor is already initialized.
+   *
+   * @return true if CGroupsResourceCalculator is available. False otherwise.
+   */
+  public static boolean isAvailable() {
+    try {
+      if (!Shell.LINUX) {
+        LOG.info("CGroupsResourceCalculator currently is supported only on "
+            + "Linux.");
+        return false;
+      }
+      if (ResourceHandlerModule.getCGroupsHandler() == null ||
+          ResourceHandlerModule.getCpuResourceHandler() == null ||
+          ResourceHandlerModule.getMemoryResourceHandler() == null) {
+        LOG.info("CGroupsResourceCalculator requires enabling CGroups" +
+            "cpu and memory");
+        return false;
+      }
+    } catch (SecurityException se) {
+      LOG.warn("Failed to get Operating System name. " + se);
+      return false;
+    }
+    return true;
+  }
+
+  private long getMemorySize(File cgroupUsageFile) {
+    long[] mem = new long[1];
+    try {
+      processFile(cgroupUsageFile, (String line) -> {
+        mem[0] = Long.parseLong(line);
+        return Result.Exit;
+      });
+      return mem[0];
+    } catch (YarnException e) {
+      LOG.warn("Failed to parse cgroups " + memswStat, e);
+    }
+    return UNAVAILABLE;
+  }
+
+  private BigInteger readTotalProcessJiffies() throws YarnException {
+    final BigInteger[] totalCPUTimeJiffies = new BigInteger[1];
+    totalCPUTimeJiffies[0] = BigInteger.ZERO;
+    processFile(cpuStat, (String line) -> {
+      if (line.startsWith(USER)) {
+        totalCPUTimeJiffies[0] = totalCPUTimeJiffies[0].add(
+            new BigInteger(line.substring(USER.length())));
+      }
+      if (line.startsWith(SYSTEM)) {
+        totalCPUTimeJiffies[0] = totalCPUTimeJiffies[0].add(
+            new BigInteger(line.substring(SYSTEM.length())));
+      }
+      return Result.Continue;
+    });
+    return totalCPUTimeJiffies[0];
+  }
+
+  private String getCGroupRelativePath(
+      CGroupsHandler.CGroupController controller)
+      throws YarnException {
+    if (pid == null) {
+      return cGroupsHandler.getRelativePathForCGroup("");
+    } else {
+      return getCGroupRelativePathForPid(controller);
+    }
+  }
+
+  private String getCGroupRelativePathForPid(
+      CGroupsHandler.CGroupController controller)
+      throws YarnException {
+    File pidCgroupFile = new File(new File(procfsDir, pid), CGROUP);
+    String[] result = new String[1];
+    processFile(pidCgroupFile, (String line)->{
+      Matcher m = CGROUP_FILE_FORMAT.matcher(line);
+      boolean mat = m.find();
+      if (mat) {
+        if (m.group(2).contains(controller.getName())) {
+          // Instead of returning the full path we compose it
+          // based on the last item as the container id
+          // This helps to avoid confusion within a privileged Docker container
+          // where the path is referred in /proc/<pid>/cgroup as
+          // /docker/<dcontainerid>/hadoop-yarn/<containerid>
+          // but it is /hadoop-yarn/<containerid> in the cgroups hierarchy
+          String cgroupPath = m.group(3);
+
+          if (cgroupPath != null) {
+            String cgroup =
+                new File(cgroupPath).toPath().getFileName().toString();
+            result[0] = cGroupsHandler.getRelativePathForCGroup(cgroup);
+          } else {
+            LOG.warn("Invalid cgroup path for " + pidCgroupFile);
+          }
+          return Result.Exit;
+        }
+      } else {
+        LOG.warn(
+            "Unexpected: cgroup file is not in the expected format"
+                + " for process with pid " + pid);
+      }
+      return Result.Continue;
+    });
+    if (result[0] == null) {
+      throw new YarnException(controller.getName() + " CGroup for pid " + pid +
+          " not found " + pidCgroupFile);
+    }
+    return result[0];
+  }
+
+  private void processFile(File file, Function<String, Result> processLine)
+      throws YarnException {
+    // Read "procfsDir/<pid>/stat" file - typically /proc/<pid>/stat
+    try (InputStreamReader fReader = new InputStreamReader(
+        new FileInputStream(file), Charset.forName("UTF-8"))) {
+      try (BufferedReader in = new BufferedReader(fReader)) {
+        try {
+          String str;
+          while ((str = in.readLine()) != null) {
+            Result result = processLine.apply(str);
+            if (result == Result.Exit) {
+              return;
+            }
+          }
+        } catch (IOException io) {
+          throw new YarnException("Error reading the stream " + io, io);
+        }
+      }
+    } catch (IOException f) {
+      throw new YarnException("The process vanished in the interim " + pid, f);
+    }
+  }
+
+  void setCGroupFilePaths() throws YarnException {
+    if (cGroupsHandler == null) {
+      throw new YarnException("CGroups handler is not initialized");
+    }
+    File cpuDir = new File(
+        cGroupsHandler.getControllerPath(
+            CGroupsHandler.CGroupController.CPUACCT),
+        getCGroupRelativePath(CGroupsHandler.CGroupController.CPUACCT));
+    File memDir = new File(
+        cGroupsHandler.getControllerPath(
+            CGroupsHandler.CGroupController.MEMORY),
+        getCGroupRelativePath(CGroupsHandler.CGroupController.MEMORY));
+    cpuStat = new File(cpuDir, CPU_STAT);
+    memStat = new File(memDir, MEM_STAT);
+    memswStat = new File(memDir, MEMSW_STAT);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/649ef7ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CombinedResourceCalculator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CombinedResourceCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CombinedResourceCalculator.java
new file mode 100644
index 0000000..84b3ed0
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CombinedResourceCalculator.java
@@ -0,0 +1,108 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.util.ProcfsBasedProcessTree;
+import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree;
+
+/**
+ * CombinedResourceCalculator is a resource calculator that uses cgroups but
+ * it is backward compatible with procfs in terms of virtual memory usage.
+ */
+public class CombinedResourceCalculator  extends ResourceCalculatorProcessTree {
+  protected static final Log LOG = LogFactory
+      .getLog(CombinedResourceCalculator.class);
+  private ProcfsBasedProcessTree procfs;
+  private CGroupsResourceCalculator cgroup;
+
+  public CombinedResourceCalculator(String pid) {
+    super(pid);
+    procfs = new ProcfsBasedProcessTree(pid);
+    cgroup = new CGroupsResourceCalculator(pid);
+  }
+
+  @Override
+  public void initialize() throws YarnException {
+    procfs.initialize();
+    cgroup.initialize();
+  }
+
+  @Override
+  public void updateProcessTree() {
+    procfs.updateProcessTree();
+    cgroup.updateProcessTree();
+  }
+
+  @Override
+  public String getProcessTreeDump() {
+    return procfs.getProcessTreeDump();
+  }
+
+  @Override
+  public float getCpuUsagePercent() {
+    float cgroupUsage = cgroup.getCpuUsagePercent();
+    if (LOG.isDebugEnabled()) {
+      float procfsUsage = procfs.getCpuUsagePercent();
+      LOG.debug("CPU Comparison:" + procfsUsage + " " + cgroupUsage);
+      LOG.debug("Jiffy Comparison:" +
+          procfs.getCumulativeCpuTime() + " " +
+          cgroup.getCumulativeCpuTime());
+    }
+
+    return cgroupUsage;
+  }
+
+  @Override
+  public boolean checkPidPgrpidForMatch() {
+    return procfs.checkPidPgrpidForMatch();
+  }
+
+  @Override
+  public long getCumulativeCpuTime() {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("CPU Comparison:" +
+          procfs.getCumulativeCpuTime() + " " +
+          cgroup.getCumulativeCpuTime());
+    }
+    return cgroup.getCumulativeCpuTime();
+  }
+
+  @Override
+  public long getRssMemorySize(int olderThanAge) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("MEM Comparison:" +
+          procfs.getRssMemorySize(olderThanAge) + " " +
+          cgroup.getRssMemorySize(olderThanAge));
+    }
+    return cgroup.getRssMemorySize(olderThanAge);
+  }
+
+  @Override
+  public long getVirtualMemorySize(int olderThanAge) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("VMEM Comparison:" +
+          procfs.getVirtualMemorySize(olderThanAge) + " " +
+          cgroup.getVirtualMemorySize(olderThanAge));
+    }
+    return procfs.getVirtualMemorySize(olderThanAge);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/649ef7ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/ResourceHandlerModule.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/ResourceHandlerModule.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/ResourceHandlerModule.java
index 921f920..a02204d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/ResourceHandlerModule.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/ResourceHandlerModule.java
@@ -25,7 +25,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperationExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.ResourcePlugin;
@@ -101,7 +100,27 @@ public class ResourceHandlerModule {
     return cGroupsHandler;
   }
 
-  private static CGroupsCpuResourceHandlerImpl getCGroupsCpuResourceHandler(
+  public static NetworkPacketTaggingHandlerImpl
+      getNetworkResourceHandler() {
+    return networkPacketTaggingHandlerImpl;
+  }
+
+  public static DiskResourceHandler
+      getDiskResourceHandler() {
+    return cGroupsBlkioResourceHandler;
+  }
+
+  public static MemoryResourceHandler
+      getMemoryResourceHandler() {
+    return cGroupsMemoryResourceHandler;
+  }
+
+  public static CpuResourceHandler
+      getCpuResourceHandler() {
+    return cGroupsCpuResourceHandler;
+  }
+
+  private static CGroupsCpuResourceHandlerImpl initCGroupsCpuResourceHandler(
       Configuration conf) throws ResourceHandlerException {
     boolean cgroupsCpuEnabled =
         conf.getBoolean(YarnConfiguration.NM_CPU_RESOURCE_ENABLED,
@@ -150,7 +169,7 @@ public class ResourceHandlerModule {
     }
   }
 
-  public static ResourceHandler getNetworkResourceHandler(Configuration conf)
+  public static ResourceHandler initNetworkResourceHandler(Configuration conf)
         throws ResourceHandlerException {
     boolean useNetworkTagHandler = conf.getBoolean(
         YarnConfiguration.NM_NETWORK_TAG_HANDLER_ENABLED,
@@ -181,12 +200,12 @@ public class ResourceHandlerModule {
   }
 
   public static OutboundBandwidthResourceHandler
-      getOutboundBandwidthResourceHandler(Configuration conf)
+      initOutboundBandwidthResourceHandler(Configuration conf)
       throws ResourceHandlerException {
     return getTrafficControlBandwidthHandler(conf);
   }
 
-  public static DiskResourceHandler getDiskResourceHandler(Configuration conf)
+  public static DiskResourceHandler initDiskResourceHandler(Configuration conf)
       throws ResourceHandlerException {
     if (conf.getBoolean(YarnConfiguration.NM_DISK_RESOURCE_ENABLED,
         YarnConfiguration.DEFAULT_NM_DISK_RESOURCE_ENABLED)) {
@@ -210,7 +229,7 @@ public class ResourceHandlerModule {
     return cGroupsBlkioResourceHandler;
   }
 
-  public static MemoryResourceHandler getMemoryResourceHandler(
+  public static MemoryResourceHandler initMemoryResourceHandler(
       Configuration conf) throws ResourceHandlerException {
     if (conf.getBoolean(YarnConfiguration.NM_MEMORY_RESOURCE_ENABLED,
         YarnConfiguration.DEFAULT_NM_MEMORY_RESOURCE_ENABLED)) {
@@ -246,10 +265,14 @@ public class ResourceHandlerModule {
       throws ResourceHandlerException {
     ArrayList<ResourceHandler> handlerList = new ArrayList<>();
 
-    addHandlerIfNotNull(handlerList, getNetworkResourceHandler(conf));
-    addHandlerIfNotNull(handlerList, getDiskResourceHandler(conf));
-    addHandlerIfNotNull(handlerList, getMemoryResourceHandler(conf));
-    addHandlerIfNotNull(handlerList, getCGroupsCpuResourceHandler(conf));
+    addHandlerIfNotNull(handlerList,
+        initNetworkResourceHandler(conf));
+    addHandlerIfNotNull(handlerList,
+        initDiskResourceHandler(conf));
+    addHandlerIfNotNull(handlerList,
+        initMemoryResourceHandler(conf));
+    addHandlerIfNotNull(handlerList,
+        initCGroupsCpuResourceHandler(conf));
     addHandlersFromConfiguredResourcePlugins(handlerList, conf, nmContext);
     resourceHandlerChain = new ResourceHandlerChain(handlerList);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/649ef7ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
index 23c89c0..33986a0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
@@ -215,15 +215,25 @@ public class ContainersMonitorImpl extends AbstractService implements
         YarnConfiguration.DEFAULT_NM_CONTAINER_MONITOR_ENABLED);
   }
 
+  /**
+   * Get the best process tree calculator.
+   * @param pId container process id
+   * @return process tree calculator
+   */
+  private ResourceCalculatorProcessTree
+      getResourceCalculatorProcessTree(String pId) {
+    return ResourceCalculatorProcessTree.
+        getResourceCalculatorProcessTree(
+            pId, processTreeClass, conf);
+  }
+
   private boolean isResourceCalculatorAvailable() {
     if (resourceCalculatorPlugin == null) {
       LOG.info("ResourceCalculatorPlugin is unavailable on this system. " + this
           .getClass().getName() + " is disabled.");
       return false;
     }
-    if (ResourceCalculatorProcessTree
-        .getResourceCalculatorProcessTree("0", processTreeClass, conf)
-        == null) {
+    if (getResourceCalculatorProcessTree("0") == null) {
       LOG.info("ResourceCalculatorProcessTree is unavailable on this system. "
           + this.getClass().getName() + " is disabled.");
       return false;
@@ -535,9 +545,7 @@ public class ContainersMonitorImpl extends AbstractService implements
             LOG.debug("Tracking ProcessTree " + pId + " for the first time");
           }
           ResourceCalculatorProcessTree pt =
-                  ResourceCalculatorProcessTree.
-                        getResourceCalculatorProcessTree(
-                            pId, processTreeClass, conf);
+              getResourceCalculatorProcessTree(pId);
           ptInfo.setPid(pId);
           ptInfo.setProcessTree(pt);
 
@@ -599,11 +607,14 @@ public class ContainersMonitorImpl extends AbstractService implements
       long pmemLimit = ptInfo.getPmemLimit();
       if (AUDITLOG.isDebugEnabled()) {
         AUDITLOG.debug(String.format(
-                "Memory usage of ProcessTree %s for container-id %s: ",
-                pId, containerId.toString()) +
-                formatUsageString(
-                      currentVmemUsage, vmemLimit,
-                      currentPmemUsage, pmemLimit));
+            "Resource usage of ProcessTree %s for container-id %s:" +
+                " %s CPU:%f CPU/core:%f",
+            pId, containerId.toString(),
+            formatUsageString(
+                currentVmemUsage, vmemLimit,
+                currentPmemUsage, pmemLimit),
+            cpuUsagePercentPerCore,
+            cpuUsageTotalCoresPercentage));
       }
 
       // Add resource utilization for this container

http://git-wip-us.apache.org/repos/asf/hadoop/blob/649ef7ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsMemoryResourceHandlerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsMemoryResourceHandlerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsMemoryResourceHandlerImpl.java
index 8fd5a9d..78ccc61 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsMemoryResourceHandlerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsMemoryResourceHandlerImpl.java
@@ -149,6 +149,51 @@ public class TestCGroupsMemoryResourceHandlerImpl {
   }
 
   @Test
+  public void testPreStartNonEnforced() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setBoolean(YarnConfiguration.NM_PMEM_CHECK_ENABLED, false);
+    conf.setBoolean(YarnConfiguration.NM_VMEM_CHECK_ENABLED, false);
+    conf.setBoolean(YarnConfiguration.NM_MEMORY_RESOURCE_ENFORCED, false);
+    cGroupsMemoryResourceHandler.bootstrap(conf);
+    String id = "container_01_01";
+    String path = "test-path/" + id;
+    ContainerId mockContainerId = mock(ContainerId.class);
+    when(mockContainerId.toString()).thenReturn(id);
+    Container mockContainer = mock(Container.class);
+    when(mockContainer.getContainerId()).thenReturn(mockContainerId);
+    when(mockCGroupsHandler
+        .getPathForCGroupTasks(CGroupsHandler.CGroupController.MEMORY, id))
+        .thenReturn(path);
+    int memory = 1024;
+    when(mockContainer.getResource())
+        .thenReturn(Resource.newInstance(memory, 1));
+    List<PrivilegedOperation> ret =
+        cGroupsMemoryResourceHandler.preStart(mockContainer);
+    verify(mockCGroupsHandler, times(1))
+        .createCGroup(CGroupsHandler.CGroupController.MEMORY, id);
+    verify(mockCGroupsHandler, times(0))
+        .updateCGroupParam(CGroupsHandler.CGroupController.MEMORY, id,
+            CGroupsHandler.CGROUP_PARAM_MEMORY_HARD_LIMIT_BYTES,
+            String.valueOf(memory) + "M");
+    verify(mockCGroupsHandler, times(0))
+        .updateCGroupParam(CGroupsHandler.CGroupController.MEMORY, id,
+            CGroupsHandler.CGROUP_PARAM_MEMORY_SOFT_LIMIT_BYTES,
+            String.valueOf((int) (memory * 0.9)) + "M");
+    verify(mockCGroupsHandler, times(0))
+        .updateCGroupParam(CGroupsHandler.CGroupController.MEMORY, id,
+            CGroupsHandler.CGROUP_PARAM_MEMORY_SWAPPINESS, String.valueOf(0));
+    Assert.assertNotNull(ret);
+    Assert.assertEquals(1, ret.size());
+    PrivilegedOperation op = ret.get(0);
+    Assert.assertEquals(PrivilegedOperation.OperationType.ADD_PID_TO_CGROUP,
+        op.getOperationType());
+    List<String> args = op.getArguments();
+    Assert.assertEquals(1, args.size());
+    Assert.assertEquals(PrivilegedOperation.CGROUP_ARG_PREFIX + path,
+        args.get(0));
+  }
+
+  @Test
   public void testReacquireContainer() throws Exception {
     ContainerId containerIdMock = mock(ContainerId.class);
     Assert.assertNull(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/649ef7ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsResourceCalculator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsResourceCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsResourceCalculator.java
new file mode 100644
index 0000000..a2ad11f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsResourceCalculator.java
@@ -0,0 +1,274 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.util.ControlledClock;
+import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.File;
+
+import static org.mockito.Mockito.*;
+
+/**
+ * Unit test for CGroupsResourceCalculator.
+ */
+public class TestCGroupsResourceCalculator {
+
+  private ControlledClock clock = new ControlledClock();
+  private CGroupsHandler cGroupsHandler = mock(CGroupsHandler.class);
+  private String basePath = "/tmp/" + this.getClass().getName();
+
+  public TestCGroupsResourceCalculator() {
+    when(cGroupsHandler.getRelativePathForCGroup("container_1"))
+        .thenReturn("/yarn/container_1");
+    when(cGroupsHandler.getRelativePathForCGroup("")).thenReturn("/yarn/");
+  }
+
+  @Test(expected = YarnException.class)
+  public void testPidNotFound() throws Exception {
+    CGroupsResourceCalculator calculator =
+        new CGroupsResourceCalculator(
+            "1234", ".", cGroupsHandler, clock, 10);
+    calculator.setCGroupFilePaths();
+    Assert.assertEquals("Expected exception", null, calculator);
+  }
+
+  @Test(expected = YarnException.class)
+  public void testNoMemoryCGgroupMount() throws Exception {
+    File procfs = new File(basePath + "/1234");
+    Assert.assertTrue("Setup error", procfs.mkdirs());
+    try {
+      FileUtils.writeStringToFile(
+          new File(procfs, CGroupsResourceCalculator.CGROUP),
+          "7:devices:/yarn/container_1\n" +
+              "6:cpuacct,cpu:/yarn/container_1\n" +
+              "5:pids:/yarn/container_1\n");
+      CGroupsResourceCalculator calculator =
+          new CGroupsResourceCalculator(
+              "1234", basePath,
+              cGroupsHandler, clock, 10);
+      calculator.setCGroupFilePaths();
+      Assert.assertEquals("Expected exception", null, calculator);
+    } finally {
+      FileUtils.deleteDirectory(new File(basePath));
+    }
+  }
+
+  @Test
+  public void testCGgroupNotFound() throws Exception {
+    File procfs = new File(basePath + "/1234");
+    Assert.assertTrue("Setup error", procfs.mkdirs());
+    try {
+      FileUtils.writeStringToFile(
+          new File(procfs, CGroupsResourceCalculator.CGROUP),
+          "7:devices:/yarn/container_1\n" +
+              "6:cpuacct,cpu:/yarn/container_1\n" +
+              "5:pids:/yarn/container_1\n" +
+              "4:memory:/yarn/container_1\n");
+
+      CGroupsResourceCalculator calculator =
+          new CGroupsResourceCalculator(
+              "1234", basePath,
+              cGroupsHandler, clock, 10);
+      calculator.setCGroupFilePaths();
+      calculator.updateProcessTree();
+      Assert.assertEquals("cgroups should be missing",
+          (long)ResourceCalculatorProcessTree.UNAVAILABLE,
+          calculator.getRssMemorySize(0));
+    } finally {
+      FileUtils.deleteDirectory(new File(basePath));
+    }
+  }
+
+  @Test
+  public void testCPUParsing() throws Exception {
+    File cgcpuacctDir =
+        new File(basePath + "/cgcpuacct");
+    File cgcpuacctContainerDir =
+        new File(cgcpuacctDir, "/yarn/container_1");
+    File procfs = new File(basePath + "/1234");
+    when(cGroupsHandler.getControllerPath(
+        CGroupsHandler.CGroupController.CPUACCT)).
+        thenReturn(cgcpuacctDir.getAbsolutePath());
+    Assert.assertTrue("Setup error", procfs.mkdirs());
+    Assert.assertTrue("Setup error", cgcpuacctContainerDir.mkdirs());
+    try {
+      FileUtils.writeStringToFile(
+          new File(procfs, CGroupsResourceCalculator.CGROUP),
+          "7:devices:/yarn/container_1\n" +
+              "6:cpuacct,cpu:/yarn/container_1\n" +
+              "5:pids:/yarn/container_1\n" +
+              "4:memory:/yarn/container_1\n");
+      FileUtils.writeStringToFile(
+          new File(cgcpuacctContainerDir, CGroupsResourceCalculator.CPU_STAT),
+          "Can you handle this?\n" +
+              "user 5415\n" +
+              "system 3632");
+      CGroupsResourceCalculator calculator =
+          new CGroupsResourceCalculator(
+              "1234", basePath,
+              cGroupsHandler, clock, 10);
+      calculator.setCGroupFilePaths();
+      calculator.updateProcessTree();
+      Assert.assertEquals("Incorrect CPU usage",
+          90470,
+          calculator.getCumulativeCpuTime());
+    } finally {
+      FileUtils.deleteDirectory(new File(basePath));
+    }
+  }
+
+  @Test
+  public void testMemoryParsing() throws Exception {
+    File cgcpuacctDir =
+        new File(basePath + "/cgcpuacct");
+    File cgcpuacctContainerDir =
+        new File(cgcpuacctDir, "/yarn/container_1");
+    File cgmemoryDir =
+        new File(basePath + "/memory");
+    File cgMemoryContainerDir =
+        new File(cgmemoryDir, "/yarn/container_1");
+    File procfs = new File(basePath + "/1234");
+    when(cGroupsHandler.getControllerPath(
+        CGroupsHandler.CGroupController.MEMORY)).
+        thenReturn(cgmemoryDir.getAbsolutePath());
+    Assert.assertTrue("Setup error", procfs.mkdirs());
+    Assert.assertTrue("Setup error", cgcpuacctContainerDir.mkdirs());
+    Assert.assertTrue("Setup error", cgMemoryContainerDir.mkdirs());
+    try {
+      FileUtils.writeStringToFile(
+          new File(procfs, CGroupsResourceCalculator.CGROUP),
+              "6:cpuacct,cpu:/yarn/container_1\n" +
+              "4:memory:/yarn/container_1\n");
+      FileUtils.writeStringToFile(
+          new File(cgMemoryContainerDir, CGroupsResourceCalculator.MEM_STAT),
+          "418496512\n");
+
+      CGroupsResourceCalculator calculator =
+          new CGroupsResourceCalculator(
+              "1234", basePath,
+              cGroupsHandler, clock, 10);
+      calculator.setCGroupFilePaths();
+
+      calculator.updateProcessTree();
+      // Test the case where memsw is not available (Ubuntu)
+      Assert.assertEquals("Incorrect memory usage",
+          418496512,
+          calculator.getRssMemorySize());
+      Assert.assertEquals("Incorrect swap usage",
+          (long)ResourceCalculatorProcessTree.UNAVAILABLE,
+          calculator.getVirtualMemorySize());
+
+      // Test the case where memsw is available
+      FileUtils.writeStringToFile(
+          new File(cgMemoryContainerDir, CGroupsResourceCalculator.MEMSW_STAT),
+          "418496513\n");
+      calculator.updateProcessTree();
+      Assert.assertEquals("Incorrect swap usage",
+          418496513,
+          calculator.getVirtualMemorySize());
+    } finally {
+      FileUtils.deleteDirectory(new File(basePath));
+    }
+  }
+
+  @Test
+  public void testCPUParsingRoot() throws Exception {
+    File cgcpuacctDir =
+        new File(basePath + "/cgcpuacct");
+    File cgcpuacctRootDir =
+        new File(cgcpuacctDir, "/yarn");
+    when(cGroupsHandler.getControllerPath(
+        CGroupsHandler.CGroupController.CPUACCT)).
+        thenReturn(cgcpuacctDir.getAbsolutePath());
+    Assert.assertTrue("Setup error", cgcpuacctRootDir.mkdirs());
+    try {
+      FileUtils.writeStringToFile(
+          new File(cgcpuacctRootDir, CGroupsResourceCalculator.CPU_STAT),
+              "user 5415\n" +
+              "system 3632");
+      CGroupsResourceCalculator calculator =
+          new CGroupsResourceCalculator(
+              null, basePath,
+              cGroupsHandler, clock, 10);
+      calculator.setCGroupFilePaths();
+      calculator.updateProcessTree();
+      Assert.assertEquals("Incorrect CPU usage",
+          90470,
+          calculator.getCumulativeCpuTime());
+    } finally {
+      FileUtils.deleteDirectory(new File(basePath));
+    }
+  }
+
+  @Test
+  public void testMemoryParsingRoot() throws Exception {
+    File cgcpuacctDir =
+        new File(basePath + "/cgcpuacct");
+    File cgcpuacctRootDir =
+        new File(cgcpuacctDir, "/yarn");
+    File cgmemoryDir =
+        new File(basePath + "/memory");
+    File cgMemoryRootDir =
+        new File(cgmemoryDir, "/yarn");
+    File procfs = new File(basePath + "/1234");
+    when(cGroupsHandler.getControllerPath(
+        CGroupsHandler.CGroupController.MEMORY)).
+        thenReturn(cgmemoryDir.getAbsolutePath());
+    Assert.assertTrue("Setup error", procfs.mkdirs());
+    Assert.assertTrue("Setup error", cgcpuacctRootDir.mkdirs());
+    Assert.assertTrue("Setup error", cgMemoryRootDir.mkdirs());
+    try {
+      FileUtils.writeStringToFile(
+          new File(cgMemoryRootDir, CGroupsResourceCalculator.MEM_STAT),
+          "418496512\n");
+
+      CGroupsResourceCalculator calculator =
+          new CGroupsResourceCalculator(
+              null, basePath,
+              cGroupsHandler, clock, 10);
+      calculator.setCGroupFilePaths();
+
+      calculator.updateProcessTree();
+
+      // Test the case where memsw is not available (Ubuntu)
+      Assert.assertEquals("Incorrect memory usage",
+          418496512,
+          calculator.getRssMemorySize());
+      Assert.assertEquals("Incorrect swap usage",
+          (long)ResourceCalculatorProcessTree.UNAVAILABLE,
+          calculator.getVirtualMemorySize());
+
+      // Test the case where memsw is available
+      FileUtils.writeStringToFile(
+          new File(cgMemoryRootDir, CGroupsResourceCalculator.MEMSW_STAT),
+          "418496513\n");
+      calculator.updateProcessTree();
+      Assert.assertEquals("Incorrect swap usage",
+          418496513,
+          calculator.getVirtualMemorySize());
+    } finally {
+      FileUtils.deleteDirectory(new File(basePath));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/649ef7ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCompareResourceCalculators.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCompareResourceCalculators.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCompareResourceCalculators.java
new file mode 100644
index 0000000..8be0590
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCompareResourceCalculators.java
@@ -0,0 +1,227 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources;
+
+
+import org.apache.commons.lang3.SystemUtils;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.nodemanager.Context;
+import org.apache.hadoop.yarn.util.ProcfsBasedProcessTree;
+import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree;
+import org.junit.*;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.util.Random;
+
+import static org.mockito.Mockito.mock;
+
+/**
+ * Functional test for CGroupsResourceCalculator to compare two resource
+ * calculators. It is OS dependent.
+ * Ignored in automated tests due to flakiness by design.
+ */
+public class TestCompareResourceCalculators {
+  private Process target = null;
+  private String cgroup = null;
+  private String cgroupCPU = null;
+  private String cgroupMemory = null;
+  public static final long SHMEM_KB = 100 * 1024;
+
+  @Before
+  public void setup() throws IOException, YarnException {
+    Assume.assumeTrue(SystemUtils.IS_OS_LINUX);
+
+    YarnConfiguration conf = new YarnConfiguration();
+    conf.set(YarnConfiguration.NM_LINUX_CONTAINER_CGROUPS_HIERARCHY,
+        "TestCompareResourceCalculators");
+    conf.setBoolean(YarnConfiguration.NM_LINUX_CONTAINER_CGROUPS_MOUNT, false);
+    conf.setStrings(YarnConfiguration.NM_LINUX_CONTAINER_CGROUPS_MOUNT_PATH,
+        "/sys/fs/cgroup");
+    conf.setBoolean(YarnConfiguration.NM_CPU_RESOURCE_ENABLED, true);
+    ResourceHandlerChain module = null;
+    try {
+      module = ResourceHandlerModule.getConfiguredResourceHandlerChain(conf,
+          mock(Context.class));
+    } catch (ResourceHandlerException e) {
+      throw new YarnException("Cannot access cgroups", e);
+    }
+    Assume.assumeNotNull(module);
+    Assume.assumeNotNull(
+        ResourceHandlerModule.getCGroupsHandler()
+            .getControllerPath(CGroupsHandler.CGroupController.CPU));
+    Assume.assumeNotNull(
+        ResourceHandlerModule.getCGroupsHandler()
+            .getControllerPath(CGroupsHandler.CGroupController.MEMORY));
+
+    Random random = new Random(System.currentTimeMillis());
+    cgroup = Long.toString(random.nextLong());
+    cgroupCPU = ResourceHandlerModule.getCGroupsHandler()
+        .getPathForCGroup(CGroupsHandler.CGroupController.CPU, cgroup);
+    cgroupMemory = ResourceHandlerModule.getCGroupsHandler()
+        .getPathForCGroup(CGroupsHandler.CGroupController.MEMORY, cgroup);
+  }
+
+  @After
+  public void tearDown() throws YarnException {
+    stopTestProcess();
+  }
+
+
+  // Ignored in automated tests due to flakiness by design
+  @Ignore
+  @Test
+  public void testCompareResults()
+      throws YarnException, InterruptedException, IOException {
+
+    startTestProcess();
+
+    ProcfsBasedProcessTree legacyCalculator =
+        new ProcfsBasedProcessTree(Long.toString(getPid()));
+    CGroupsResourceCalculator cgroupsCalculator =
+        new CGroupsResourceCalculator(Long.toString(getPid()));
+    cgroupsCalculator.setCGroupFilePaths();
+
+    for (int i = 0; i < 5; ++i) {
+      Thread.sleep(3000);
+      compareMetrics(legacyCalculator, cgroupsCalculator);
+    }
+
+    stopTestProcess();
+
+    ensureCleanedUp(legacyCalculator, cgroupsCalculator);
+  }
+
+  private void ensureCleanedUp(
+          ResourceCalculatorProcessTree metric1,
+          ResourceCalculatorProcessTree metric2) {
+    metric1.updateProcessTree();
+    metric2.updateProcessTree();
+    long pmem1 = metric1.getRssMemorySize(0);
+    long pmem2 = metric2.getRssMemorySize(0);
+    System.out.println(pmem1 + " " + pmem2);
+    Assert.assertTrue("pmem should be invalid " + pmem1 + " " + pmem2,
+            pmem1 == ResourceCalculatorProcessTree.UNAVAILABLE &&
+                    pmem2 == ResourceCalculatorProcessTree.UNAVAILABLE);
+    long vmem1 = metric1.getRssMemorySize(0);
+    long vmem2 = metric2.getRssMemorySize(0);
+    System.out.println(vmem1 + " " + vmem2);
+    Assert.assertTrue("vmem Error outside range " + vmem1 + " " + vmem2,
+            vmem1 == ResourceCalculatorProcessTree.UNAVAILABLE &&
+                    vmem2 == ResourceCalculatorProcessTree.UNAVAILABLE);
+    float cpu1 = metric1.getCpuUsagePercent();
+    float cpu2 = metric2.getCpuUsagePercent();
+    // TODO ProcfsBasedProcessTree may report negative on process exit
+    Assert.assertTrue("CPU% Error outside range " + cpu1 + " " + cpu2,
+            cpu1 == 0 && cpu2 == 0);
+  }
+
+  private void compareMetrics(
+      ResourceCalculatorProcessTree metric1,
+      ResourceCalculatorProcessTree metric2) {
+    metric1.updateProcessTree();
+    metric2.updateProcessTree();
+    long pmem1 = metric1.getRssMemorySize(0);
+    long pmem2 = metric2.getRssMemorySize(0);
+    // TODO The calculation is different and cgroup
+    // can report a small amount after process stop
+    // This is not an issue since the cgroup is deleted
+    System.out.println(pmem1 + " " + (pmem2 - SHMEM_KB * 1024));
+    Assert.assertTrue("pmem Error outside range " + pmem1 + " " + pmem2,
+        Math.abs(pmem1 - (pmem2 - SHMEM_KB * 1024)) < 5000000);
+    long vmem1 = metric1.getRssMemorySize(0);
+    long vmem2 = metric2.getRssMemorySize(0);
+    System.out.println(vmem1 + " " + (vmem2 - SHMEM_KB * 1024));
+    // TODO The calculation is different and cgroup
+    // can report a small amount after process stop
+    // This is not an issue since the cgroup is deleted
+    Assert.assertTrue("vmem Error outside range " + vmem1 + " " + vmem2,
+        Math.abs(vmem1 - (vmem2 - SHMEM_KB * 1024)) < 5000000);
+    float cpu1 = metric1.getCpuUsagePercent();
+    float cpu2 = metric2.getCpuUsagePercent();
+    if (cpu1 > 0) {
+      // TODO ProcfsBasedProcessTree may report negative on process exit
+      Assert.assertTrue("CPU% Error outside range " + cpu1 + " " + cpu2,
+              Math.abs(cpu2 - cpu1) < 10);
+    }
+  }
+
+  private void startTestProcess() throws IOException {
+    ProcessBuilder builder = new ProcessBuilder();
+    String script =
+        "mkdir -p " + cgroupCPU + ";" +
+        "echo $$ >" + cgroupCPU + "/tasks;" +
+        "mkdir -p " + cgroupMemory + ";" +
+        "echo $$ >" + cgroupMemory + "/tasks;" +
+        "dd if=/dev/zero of=/dev/shm/" +
+            cgroup + " bs=1k count=" + SHMEM_KB + ";" +
+        "dd if=/dev/zero of=/dev/null bs=1k &" +
+        "echo $! >/tmp/\" + cgroup + \".pid;" +
+        //"echo while [ -f /tmp/" + cgroup + ".pid ]; do sleep 1; done;" +
+        "sleep 10000;" +
+        "echo kill $(jobs -p);";
+    builder.command("bash", "-c", script);
+    builder.redirectError(new File("/tmp/a.txt"));
+    builder.redirectOutput(new File("/tmp/b.txt"));
+    target = builder.start();
+  }
+
+  private void stopTestProcess() throws YarnException {
+    if (target != null) {
+      target.destroyForcibly();
+      target = null;
+    }
+    try {
+      ProcessBuilder builder = new ProcessBuilder();
+      String script =
+          "rm -f /dev/shm/" + cgroup + ";" +
+          "cat " + cgroupCPU + "/tasks | xargs kill;" +
+          "rm -f /tmp/" + cgroup + ".pid;" +
+          "sleep 4;" +
+          "rmdir " + cgroupCPU + ";" +
+          "rmdir " + cgroupMemory + ";";
+      builder.command("bash", "-c", script);
+      Process cleanup = builder.start();
+      cleanup.waitFor();
+    } catch (IOException|InterruptedException e) {
+      throw new YarnException("Could not clean up", e);
+    }
+  }
+
+  private long getPid() throws YarnException {
+    Class processClass = target.getClass();
+    if (processClass.getName().equals("java.lang.UNIXProcess")) {
+      try {
+        Field pidField = processClass.getDeclaredField("pid");
+        pidField.setAccessible(true);
+        long pid = pidField.getLong(target);
+        pidField.setAccessible(false);
+        return pid;
+      } catch (NoSuchFieldException|IllegalAccessException e) {
+        throw new YarnException("Reflection error", e);
+      }
+    } else {
+      throw new YarnException("Not Unix " + processClass.getName());
+    }
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/649ef7ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestResourceHandlerModule.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestResourceHandlerModule.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestResourceHandlerModule.java
index 0563694..9456303 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestResourceHandlerModule.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestResourceHandlerModule.java
@@ -36,8 +36,8 @@ import static org.mockito.Mockito.mock;
 public class TestResourceHandlerModule {
   private static final Logger LOG =
        LoggerFactory.getLogger(TestResourceHandlerModule.class);
-  Configuration emptyConf;
-  Configuration networkEnabledConf;
+  private Configuration emptyConf;
+  private Configuration networkEnabledConf;
 
   @Before
   public void setup() throws Exception {
@@ -55,23 +55,28 @@ public class TestResourceHandlerModule {
       //This resourceHandler should be non-null only if network as a resource
       //is explicitly enabled
       OutboundBandwidthResourceHandler resourceHandler = ResourceHandlerModule
-          .getOutboundBandwidthResourceHandler(emptyConf);
+          .initOutboundBandwidthResourceHandler(emptyConf);
       Assert.assertNull(resourceHandler);
 
       //When network as a resource is enabled this should be non-null
       resourceHandler = ResourceHandlerModule
-          .getOutboundBandwidthResourceHandler(networkEnabledConf);
+          .initOutboundBandwidthResourceHandler(networkEnabledConf);
       Assert.assertNotNull(resourceHandler);
 
       //Ensure that outbound bandwidth resource handler is present in the chain
       ResourceHandlerChain resourceHandlerChain = ResourceHandlerModule
-          .getConfiguredResourceHandlerChain(networkEnabledConf, mock(Context.class));
-      List<ResourceHandler> resourceHandlers = resourceHandlerChain
-          .getResourceHandlerList();
-      //Exactly one resource handler in chain
-      Assert.assertEquals(resourceHandlers.size(), 1);
-      //Same instance is expected to be in the chain.
-      Assert.assertTrue(resourceHandlers.get(0) == resourceHandler);
+          .getConfiguredResourceHandlerChain(networkEnabledConf,
+              mock(Context.class));
+      if (resourceHandlerChain != null) {
+        List<ResourceHandler> resourceHandlers = resourceHandlerChain
+            .getResourceHandlerList();
+        //Exactly one resource handler in chain
+        Assert.assertEquals(resourceHandlers.size(), 1);
+        //Same instance is expected to be in the chain.
+        Assert.assertTrue(resourceHandlers.get(0) == resourceHandler);
+      } else {
+        Assert.fail("Null returned");
+      }
     } catch (ResourceHandlerException e) {
       Assert.fail("Unexpected ResourceHandlerException: " + e);
     }
@@ -81,23 +86,27 @@ public class TestResourceHandlerModule {
   public void testDiskResourceHandler() throws Exception {
 
     DiskResourceHandler handler =
-        ResourceHandlerModule.getDiskResourceHandler(emptyConf);
+        ResourceHandlerModule.initDiskResourceHandler(emptyConf);
     Assert.assertNull(handler);
 
     Configuration diskConf = new YarnConfiguration();
     diskConf.setBoolean(YarnConfiguration.NM_DISK_RESOURCE_ENABLED, true);
 
-    handler = ResourceHandlerModule.getDiskResourceHandler(diskConf);
+    handler = ResourceHandlerModule.initDiskResourceHandler(diskConf);
     Assert.assertNotNull(handler);
 
     ResourceHandlerChain resourceHandlerChain =
         ResourceHandlerModule.getConfiguredResourceHandlerChain(diskConf,
             mock(Context.class));
-    List<ResourceHandler> resourceHandlers =
-        resourceHandlerChain.getResourceHandlerList();
-    // Exactly one resource handler in chain
-    Assert.assertEquals(resourceHandlers.size(), 1);
-    // Same instance is expected to be in the chain.
-    Assert.assertTrue(resourceHandlers.get(0) == handler);
+    if (resourceHandlerChain != null) {
+      List<ResourceHandler> resourceHandlers =
+          resourceHandlerChain.getResourceHandlerList();
+      // Exactly one resource handler in chain
+      Assert.assertEquals(resourceHandlers.size(), 1);
+      // Same instance is expected to be in the chain.
+      Assert.assertTrue(resourceHandlers.get(0) == handler);
+    } else {
+      Assert.fail("Null returned");
+    }
   }
 }
\ No newline at end of file


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[33/37] hadoop git commit: YARN-5148. [UI2] Add page to new YARN UI to view server side configurations/logs/JVM-metrics. (Kai Sasaki/Sunil G via wangda)

Posted by ae...@apache.org.
YARN-5148. [UI2] Add page to new YARN UI to view server side configurations/logs/JVM-metrics. (Kai Sasaki/Sunil G via wangda)

Change-Id: I5de88ce9850c0bc337dcb2c7d25ee9ad52016925


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/f666e7c4
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/f666e7c4
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/f666e7c4

Branch: refs/heads/HDFS-7240
Commit: f666e7c43d797c1a9a9ff90adcaa70843f4755d6
Parents: dbb9dde
Author: Wangda Tan <wa...@apache.org>
Authored: Tue Jan 30 15:45:06 2018 +0800
Committer: Wangda Tan <wa...@apache.org>
Committed: Tue Jan 30 15:45:06 2018 +0800

----------------------------------------------------------------------
 .../hadoop-yarn/hadoop-yarn-ui/pom.xml          |   4 +-
 .../src/main/webapp/app/adapters/yarn-conf.js   |  79 ++++++++++++++
 .../main/webapp/app/adapters/yarn-metrics.js    |  76 +++++++++++++
 .../src/main/webapp/app/adapters/yarn-rm-log.js |  76 +++++++++++++
 .../main/webapp/app/controllers/yarn-tools.js   |  29 +++++
 .../app/controllers/yarn-tools/yarn-conf.js     |  48 +++++++++
 .../app/controllers/yarn-tools/yarn-rm-log.js   |  24 +++++
 .../src/main/webapp/app/helpers/json-pretty.js  |  25 +++++
 .../src/main/webapp/app/models/yarn-conf.js     |  25 +++++
 .../src/main/webapp/app/models/yarn-metrics.js  |  23 ++++
 .../src/main/webapp/app/models/yarn-rm-log.js   |  23 ++++
 .../src/main/webapp/app/router.js               |   6 ++
 .../src/main/webapp/app/routes/yarn-tools.js    |  22 ++++
 .../webapp/app/routes/yarn-tools/yarn-conf.js   |  22 ++++
 .../app/routes/yarn-tools/yarn-metrics.js       |  43 ++++++++
 .../webapp/app/routes/yarn-tools/yarn-rm-log.js |  36 +++++++
 .../main/webapp/app/serializers/yarn-conf.js    |  43 ++++++++
 .../main/webapp/app/serializers/yarn-metrics.js |  33 ++++++
 .../main/webapp/app/serializers/yarn-rm-log.js  |  45 ++++++++
 .../src/main/webapp/app/styles/app.scss         |   6 ++
 .../main/webapp/app/templates/application.hbs   |   5 +
 .../main/webapp/app/templates/yarn-tools.hbs    | 108 +++++++++++++++++++
 .../app/templates/yarn-tools/yarn-conf.hbs      |  28 +++++
 .../app/templates/yarn-tools/yarn-metrics.hbs   |  33 ++++++
 .../app/templates/yarn-tools/yarn-rm-log.hbs    |  42 ++++++++
 .../hadoop-yarn-ui/src/main/webapp/bower.json   |   3 +-
 .../src/main/webapp/ember-cli-build.js          |   1 +
 .../tests/unit/adapters/yarn-conf-test.js       |  29 +++++
 .../tests/unit/adapters/yarn-metrics-test.js    |  30 ++++++
 .../tests/unit/adapters/yarn-rm-log-test.js     |  30 ++++++
 .../tests/unit/controllers/yarn-conf-test.js    |  29 +++++
 .../tests/unit/controllers/yarn-rm-log-test.js  |  30 ++++++
 .../tests/unit/controllers/yarn-tools-test.js   |  30 ++++++
 .../tests/unit/helpers/json-pretty-test.js      |  28 +++++
 .../webapp/tests/unit/models/yarn-conf-test.js  |  29 +++++
 .../tests/unit/models/yarn-metrics-test.js      |  30 ++++++
 .../tests/unit/models/yarn-rm-log-test.js       |  30 ++++++
 .../webapp/tests/unit/routes/yarn-conf-test.js  |  28 +++++
 .../tests/unit/routes/yarn-metrics-test.js      |  28 +++++
 .../tests/unit/routes/yarn-rm-log-test.js       |  28 +++++
 .../webapp/tests/unit/routes/yarn-tools-test.js |  28 +++++
 .../tests/unit/serializers/yarn-conf-test.js    |  32 ++++++
 .../tests/unit/serializers/yarn-metrics-test.js |  33 ++++++
 .../tests/unit/serializers/yarn-rm-log-test.js  |  33 ++++++
 44 files changed, 1409 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/pom.xml
index b552f35..1fafe77 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/pom.xml
@@ -30,11 +30,9 @@
 
   <properties>
     <packagingType>pom</packagingType>
-
     <webappDir>${basedir}/target/webapp</webappDir>
     <nodeExecutable>${basedir}/target/webapp/node/node</nodeExecutable>
     <packageManagerScript>node/yarn/dist/bin/yarn.js</packageManagerScript>
-
     <keepUIBuildCache>false</keepUIBuildCache>
   </properties>
 
@@ -178,7 +176,7 @@
             <artifactId>exec-maven-plugin</artifactId>
             <executions>
 
-              <!-- Build -->
+              <!-- Ember Build -->
               <execution>
                 <id>ember build</id>
                 <phase>generate-resources</phase>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-conf.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-conf.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-conf.js
new file mode 100644
index 0000000..6621972
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-conf.js
@@ -0,0 +1,79 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+ import DS from 'ember-data';
+ import Ember from 'ember';
+ import Converter from 'yarn-ui/utils/converter';
+ import AbstractAdapter from './abstract';
+
+ export default AbstractAdapter.extend({
+
+   address: 'rmWebAddress',
+
+   headers: {
+     Accept: 'text/plain'
+   },
+
+   host: Ember.computed("address", function () {
+     let address = this.get("address");
+     return this.get(`hosts.${address}`);
+   }),
+
+   pathForType(type) {
+     return  'conf';
+   },
+
+   urlForFindRecord(id, modelName, snapshot) {
+     var extension = this.get("host").split('/').pop();
+     if (extension != id) {
+       this.host = this.get("host") + id;
+     }
+     var url = this._buildURL();
+     return url;
+   },
+
+   ajax(url, method, hash) {
+     hash = hash || {};
+     hash.crossDomain = true;
+     hash.xhrFields = {withCredentials: true};
+     hash.targetServer = "RM";
+     return this._super(url, method, hash);
+   },
+
+   /**
+    * Override options so that result is not expected to be JSON
+    */
+   ajaxOptions: function (url, type, options) {
+     var hash = options || {};
+     hash.url = url;
+     hash.type = type;
+     // Make sure jQuery does not try to convert response to JSON.
+     hash.dataType = 'text';
+     hash.context = this;
+
+     var headers = Ember.get(this, 'headers');
+     if (headers != undefined) {
+       hash.beforeSend = function (xhr) {
+         Object.keys(headers).forEach(function (key) {
+           return xhr.setRequestHeader(key, headers[key]);
+         });
+       };
+     }
+     return hash;
+   },
+ });

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-metrics.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-metrics.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-metrics.js
new file mode 100644
index 0000000..d26de28
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-metrics.js
@@ -0,0 +1,76 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import DS from 'ember-data';
+import Ember from 'ember';
+import Converter from 'yarn-ui/utils/converter';
+
+export default DS.RESTAdapter.extend({
+  address: 'rmWebAddress',
+
+  headers: {
+    Accept: 'text/plain'
+  },
+
+  host: Ember.computed("address", function() {
+    let address = this.get("address");
+    return this.get(`hosts.${address}`);
+  }),
+
+  pathForType(type) {
+    return  'jmx';
+  },
+
+  urlForFindRecord(id, modelName, snapshot) {
+    var extension = this.get("host").split('/').pop();
+    if (extension != id) {
+      this.host = this.get("host") + id;
+    }
+    var url = this._buildURL();
+    return url;
+  },
+
+  ajax(url, method, hash) {
+    hash = hash || {};
+    hash.crossDomain = true;
+    hash.xhrFields = {withCredentials: true};
+    hash.targetServer = "RM";
+    return this._super(url, method, hash);
+  },
+
+  /**
+   * Override options so that result is not expected to be JSON
+   */
+  ajaxOptions: function (url, type, options) {
+    var hash = options || {};
+    hash.url = url;
+    hash.type = type;
+    // Make sure jQuery does not try to convert response to JSON.
+    hash.dataType = 'text';
+    hash.context = this;
+    var headers = Ember.get(this, 'headers');
+    if (headers != undefined) {
+      hash.beforeSend = function (xhr) {
+        Object.keys(headers).forEach(function (key) {
+          return xhr.setRequestHeader(key, headers[key]);
+        });
+      };
+    }
+    return hash;
+  },
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-rm-log.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-rm-log.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-rm-log.js
new file mode 100644
index 0000000..a912ffe
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-rm-log.js
@@ -0,0 +1,76 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import DS from 'ember-data';
+import Ember from 'ember';
+import Converter from 'yarn-ui/utils/converter';
+
+export default DS.RESTAdapter.extend({
+  address: 'rmWebAddress',
+  headers: {
+    Accept: 'text/plain'
+  },
+
+  host: Ember.computed("address", function() {
+    let address = this.get("address");
+    return this.get(`hosts.${address}`);
+  }),
+
+  pathForType(type) {
+    return  'logs';
+  },
+
+  buildURL (modelName, id, snapshot, requestType, query) {
+    return this._super(modelName, id, snapshot, requestType, query) + '/';
+  },
+
+  urlForFindRecord(id, modelName, snapshot) {
+    this.host = this.get('host');
+    let url = this.host + id;
+    return url;
+  },
+
+  ajax(url, method, hash) {
+    hash = hash || {};
+    hash.crossDomain = true;
+    hash.xhrFields = {withCredentials: true};
+    hash.targetServer = "RM";
+    return this._super(url, method, hash);
+  },
+
+  /**
+   * Override options so that result is not expected to be JSON
+   */
+  ajaxOptions: function (url, type, options) {
+    var hash = options || {};
+    hash.url = url;
+    hash.type = type;
+    // Make sure jQuery does not try to convert response to JSON.
+    hash.dataType = 'text';
+    hash.context = this;
+    var headers = Ember.get(this, 'headers');
+    if (headers != undefined) {
+      hash.beforeSend = function (xhr) {
+        Object.keys(headers).forEach(function (key) {
+          return xhr.setRequestHeader(key, headers[key]);
+        });
+      };
+    }
+    return hash;
+  },
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-tools.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-tools.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-tools.js
new file mode 100644
index 0000000..b36098b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-tools.js
@@ -0,0 +1,29 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import Ember from 'ember';
+
+export default Ember.Controller.extend({
+  breadcrumbs: [{
+    text: "Home",
+    routeName: 'application'
+  }, {
+    text: "Yarn Tools",
+    routeName: 'yarn-tools',
+  }],
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-tools/yarn-conf.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-tools/yarn-conf.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-tools/yarn-conf.js
new file mode 100644
index 0000000..86b4177
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-tools/yarn-conf.js
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import Ember from 'ember';
+
+import TableDef from 'em-table/utils/table-definition';
+import ColumnDef from 'em-table/utils/column-definition';
+
+import YarnConf from '../../models/yarn-conf';
+
+export default Ember.Controller.extend({
+  init: function () {
+    var that = this;
+    this.get('store').query('yarn-conf', {})
+      .then(function(conf) {
+        let coreProps = conf.filter(function(o) {
+          return o.get('source') == 'core-default.xml';
+        });
+        that.set('rowsForCoreColumnsFromModel', coreProps);
+        let mapredProps = conf.filter(function(o) {
+          return o.get('source') == 'mapred-default.xml';
+        });
+        that.set('rowsForMapredColumnsFromModel', mapredProps);
+        let yarnProps = conf.filter(function(o) {
+          return o.get('source') == 'yarn-default.xml';
+        });
+        that.set('rowsForYarnColumnsFromModel', yarnProps);
+      });
+  },
+
+  columnsFromModel: ColumnDef.makeFromModel(YarnConf),
+
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-tools/yarn-rm-log.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-tools/yarn-rm-log.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-tools/yarn-rm-log.js
new file mode 100644
index 0000000..a5e0eb5
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-tools/yarn-rm-log.js
@@ -0,0 +1,24 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import Ember from 'ember';
+
+export default Ember.Controller.extend({
+  queryParams: ['filename'],
+  filename: null
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/helpers/json-pretty.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/helpers/json-pretty.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/helpers/json-pretty.js
new file mode 100644
index 0000000..a820c38
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/helpers/json-pretty.js
@@ -0,0 +1,25 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+import Ember from 'ember';
+
+export function jsonPretty(params/*, hash*/) {
+  let j = params[0];
+  return j;
+}
+
+export default Ember.Helper.helper(jsonPretty);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-conf.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-conf.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-conf.js
new file mode 100644
index 0000000..9846ea1
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-conf.js
@@ -0,0 +1,25 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import DS from 'ember-data';
+
+export default DS.Model.extend({
+  name: DS.attr(),
+  source: DS.attr(),
+  value: DS.attr()
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-metrics.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-metrics.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-metrics.js
new file mode 100644
index 0000000..5ed217d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-metrics.js
@@ -0,0 +1,23 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import DS from 'ember-data';
+
+export default DS.Model.extend({
+  metrics: DS.attr()
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-rm-log.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-rm-log.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-rm-log.js
new file mode 100644
index 0000000..2b6febf
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-rm-log.js
@@ -0,0 +1,23 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import DS from 'ember-data';
+
+export default DS.Model.extend({
+  logfileName: DS.attr()
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/router.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/router.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/router.js
index bd7af21..3322a87 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/router.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/router.js
@@ -73,6 +73,12 @@ Router.map(function() {
   this.route('error');
   this.route('notfound', { path: '*:' });
   this.route('yarn-queues', { path: '/yarn-queues/:queue_name' });
+  this.route('yarn-queue-apps', { path: '/yarn-queue-apps/:queue_name' });
+  this.route('yarn-tools', function() {
+    this.route('yarn-conf');
+    this.route('yarn-metrics');
+    this.route('yarn-rm-log');
+  });
 
   this.route('yarn-flow-activity');
   this.route('yarn-flow', { path: '/yarn-flow/:flow_uid'}, function() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-tools.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-tools.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-tools.js
new file mode 100644
index 0000000..8719170
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-tools.js
@@ -0,0 +1,22 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import Ember from 'ember';
+
+export default Ember.Route.extend({
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-tools/yarn-conf.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-tools/yarn-conf.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-tools/yarn-conf.js
new file mode 100644
index 0000000..8719170
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-tools/yarn-conf.js
@@ -0,0 +1,22 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import Ember from 'ember';
+
+export default Ember.Route.extend({
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-tools/yarn-metrics.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-tools/yarn-metrics.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-tools/yarn-metrics.js
new file mode 100644
index 0000000..cca2a53
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-tools/yarn-metrics.js
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+ import Ember from 'ember';
+ import Constants from 'yarn-ui/constants';
+
+ import AbstractRoute from '../abstract';
+
+ export default AbstractRoute.extend({
+   model() {
+     return Ember.RSVP.hash({jmx: this.store.findAll('yarn-metrics')});
+   },
+
+   afterModel(model) {
+     // Handle errors and redirect if promise is rejected.
+     if (model.errors && model.errors[0]) {
+       if (model.errors[0].status == 404) {
+         this.replaceWith('/notfound');
+       } else {
+         this.replaceWith('/error');
+       }
+     }
+   },
+
+   unloadAll() {
+     this.store.unloadAll('yarn-metrics');
+   }
+ });

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-tools/yarn-rm-log.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-tools/yarn-rm-log.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-tools/yarn-rm-log.js
new file mode 100644
index 0000000..a12c280
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-tools/yarn-rm-log.js
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import Ember from 'ember';
+
+export default Ember.Route.extend({
+  queryParams: {
+    filename: {
+      refreshModel: true
+    }
+  },
+
+  model(param) {
+    if (param.filename == null) {
+      return Ember.RSVP.hash({logs: this.store.findAll('yarn-rm-log')});
+    } else {
+      // TODO: Loading log file is disallowed for cross-origin requests that require preflight
+      window.open(this.get('hosts.rmWebAddress') + param.filename);
+    }
+  }
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-conf.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-conf.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-conf.js
new file mode 100644
index 0000000..dc9c64f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-conf.js
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import Ember from 'ember';
+import DS from 'ember-data';
+import Converter from 'yarn-ui/utils/converter';
+
+export default DS.JSONAPISerializer.extend({
+  normalizeResponse(store, primaryModelClass, payload, id, requestType) {
+    var x2js = new X2JS();
+    let props = x2js.xml_str2json(payload);
+    let properties = props.configuration.property;
+    var convertedPayload = [];
+    for (var i = 0; i < properties.length; i++) {
+      var row = {
+        id: i,
+        type: primaryModelClass.modelName,
+        attributes: {
+          name: properties[i].name,
+          source: properties[i].source,
+          value: properties[i].value
+        }
+      };
+      convertedPayload.push(row);
+    }
+    return { data: convertedPayload };
+  },
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-metrics.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-metrics.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-metrics.js
new file mode 100644
index 0000000..70c9ca0
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-metrics.js
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import DS from 'ember-data';
+
+export default DS.JSONAPISerializer.extend({
+  normalizeResponse(store, primaryModelClass, payload, id, requestType) {
+    var ret =
+      {
+        id: 0,
+        type: primaryModelClass.modelName,
+        attributes: {
+          metrics: payload
+        }
+      };
+    return { data : ret };
+  },
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-rm-log.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-rm-log.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-rm-log.js
new file mode 100644
index 0000000..da834f1
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-rm-log.js
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import DS from 'ember-data';
+
+export default DS.JSONAPISerializer.extend({
+  normalizeResponse(store, primaryModelClass, payload, id, requestType) {
+    const pattern = new RegExp('<A HREF="/logs/.+">', 'g');
+    let fileNames = payload.match(pattern);
+
+    if (fileNames == null) {
+      return {data : []};
+    }
+
+    let logfileNames = [];
+    for (var i = 0; i < fileNames.length; i++) {
+      var fileName = fileNames[i].match(/<A HREF="(\/logs\/.+)">/);
+        if (fileName.length != null) {
+          logfileNames.push({
+            id: i,
+            type: primaryModelClass.modelName,
+            attributes: {
+              logfileName: fileName[1]
+            }
+          });
+        }
+    }
+    return { data : logfileNames };
+  },
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/app.scss
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/app.scss b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/app.scss
index 5d99d8e..a85e0eb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/app.scss
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/app.scss
@@ -722,3 +722,9 @@ div.service-action-mask img {
     margin-top: 0;
   }
 }
+
+.yarn-metrics-json {
+  white-space: pre-wrap;
+  word-wrap: nowrap;
+  overflow: scroll;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/application.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/application.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/application.hbs
index 5bc675d..56fef26 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/application.hbs
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/application.hbs
@@ -66,6 +66,11 @@
               <span class="sr-only">(current)</span>
             {{/link-to}}
           {{/link-to}}
+          {{#link-to 'yarn-tools.yarn-conf' tagName="li"}}
+            {{#link-to 'yarn-tools.yarn-conf' class="navigation-link"}}Tools
+              <span class="sr-only">(current)</span>
+            {{/link-to}}
+          {{/link-to}}
         </ul>
       </div><!-- /.navbar-collapse -->
     </div><!-- /.container-fluid -->

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-tools.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-tools.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-tools.hbs
new file mode 100644
index 0000000..2f618fd
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-tools.hbs
@@ -0,0 +1,108 @@
+{{!--
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+--}}
+
+{{!
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+}}
+
+{{breadcrumb-bar breadcrumbs=breadcrumbs}}
+
+<div class="col-md-12 container-fluid">
+  <div class="row">
+
+    <div class="col-md-2 container-fluid">
+      <div class="panel panel-default">
+        <div class="panel-heading">
+          <h4>Tools</h4>
+        </div>
+        <div class="panel-body">
+          <ul class="nav nav-pills nav-stacked" id="stacked-menu">
+            <ul class="nav nav-pills nav-stacked collapse in">
+              {{#link-to 'yarn-tools.yarn-conf' tagName="li"}}
+                {{#link-to 'yarn-tools.yarn-conf'}}YARN Configuration
+                {{/link-to}}
+              {{/link-to}}
+              {{#link-to 'yarn-tools.yarn-rm-log' tagName="li"}}
+                {{#link-to 'yarn-tools.yarn-rm-log'}}YARN Daemon logs
+                {{/link-to}}
+              {{/link-to}}
+            </ul>
+          </ul>
+        </div>
+      </div>
+    </div>
+
+    <div class="col-md-10 container-fluid">
+      {{#if model.clusterMetrics}}
+        <div class="row">
+          <div class="col-lg-4 container-fluid">
+            <div class="panel panel-default">
+              <div class="panel-heading">
+                Finished Apps
+              </div>
+              <div class="container-fluid" id="finishedapps-donut-chart">
+                {{donut-chart data=model.clusterMetrics.firstObject.getFinishedAppsDataForDonutChart
+                showLabels=true
+                parentId="finishedapps-donut-chart"
+                ratio=0.6
+                maxHeight=350
+                colorTargets="good warn error"
+                }}
+              </div>
+            </div>
+          </div>
+
+          <div class="col-lg-4 container-fluid">
+            <div class="panel panel-default">
+              <div class="panel-heading">
+                Running Apps
+              </div>
+              <div class="container-fluid" id="runningapps-donut-chart">
+                {{donut-chart data=model.clusterMetrics.firstObject.getRunningAppsDataForDonutChart
+                showLabels=true
+                parentId="runningapps-donut-chart"
+                ratio=0.6
+                maxHeight=350
+                colorTargets="warn good"
+                }}
+              </div>
+            </div>
+          </div>
+        </div>
+      {{/if}}
+
+      <div class="row">
+        {{outlet}}
+      </div>
+    </div>
+  </div>
+</div>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-tools/yarn-conf.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-tools/yarn-conf.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-tools/yarn-conf.hbs
new file mode 100644
index 0000000..f4c799d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-tools/yarn-conf.hbs
@@ -0,0 +1,28 @@
+{{!
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+}}
+
+<h1>Core Configuration</h1>
+{{em-table columns=columnsFromModel rows=rowsForCoreColumnsFromModel rowCount=10}}
+
+<h1>YARN Configuration</h1>
+{{em-table columns=columnsFromModel rows=rowsForYarnColumnsFromModel rowCount=10}}
+
+<h1>MapReduce Configuration</h1>
+{{em-table columns=columnsFromModel rows=rowsForMapredColumnsFromModel rowCount=10}}
+
+{{outlet}}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-tools/yarn-metrics.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-tools/yarn-metrics.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-tools/yarn-metrics.hbs
new file mode 100644
index 0000000..b11171a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-tools/yarn-metrics.hbs
@@ -0,0 +1,33 @@
+{{!
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+}}
+
+<div class="col-md-12 container-fluid">
+  <div class="col-md-10">
+    <div class="panel panel-default">
+      <div class="panel-body">
+          <div class="yarn-metrics-json">
+        {{#each model.jmx as |jmx|}}
+            {{json-pretty jmx.metrics}}
+        {{/each}}
+          </div>
+      </div>
+    </div>
+  </div>
+</div>
+
+{{outlet}}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-tools/yarn-rm-log.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-tools/yarn-rm-log.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-tools/yarn-rm-log.hbs
new file mode 100644
index 0000000..3cf536d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-tools/yarn-rm-log.hbs
@@ -0,0 +1,42 @@
+{{!
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+}}
+
+<h1>YARN Daemon Logs</h1>
+<div class="col-md-12 container-fluid">
+  <div class="col-md-10">
+    <div class="panel panel-default">
+      <ul class="list-group">
+      {{#if model.logs}}
+        {{#each model.logs as |log|}}
+        <li class=list-group-item>
+          {{#link-to 'yarn-tools.yarn-rm-log' (query-params filename=log.logfileName)}}
+            {{log.logfileName}}
+          {{/link-to}}
+        </li>
+        {{/each}}
+      {{else}}
+        <div class="panel-body">
+          No logs were found.
+        </div>
+      {{/if}}
+      </ul>
+    </div>
+  </div>
+</div>
+
+{{outlet}}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/bower.json
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/bower.json b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/bower.json
index 11fae3e..b7591a5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/bower.json
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/bower.json
@@ -22,6 +22,7 @@
     "momentjs": "~2.10.6",
     "select2": "4.0.0",
     "snippet-ss": "~1.11.0",
-    "alasql": "^0.4.3"
+    "alasql": "^0.4.3",
+    "abdmob/x2js": "1.2.0"
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/ember-cli-build.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/ember-cli-build.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/ember-cli-build.js
index db09ae3..309c53b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/ember-cli-build.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/ember-cli-build.js
@@ -51,6 +51,7 @@ module.exports = function(defaults) {
   app.import('bower_components/bootstrap/dist/css/bootstrap-theme.css');
   app.import('bower_components/bootstrap/dist/js/bootstrap.min.js');
   app.import('bower_components/alasql/dist/alasql.js');
+  app.import('bower_components/abdmob/x2js/xml2json.js');
 
   // Use `app.import` to add additional libraries to the generated
   // output files.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/adapters/yarn-conf-test.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/adapters/yarn-conf-test.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/adapters/yarn-conf-test.js
new file mode 100644
index 0000000..c892933
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/adapters/yarn-conf-test.js
@@ -0,0 +1,29 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+import { moduleFor, test } from 'ember-qunit';
+
+moduleFor('adapter:yarn-conf', 'Unit | Adapter | yarn conf', {
+  // Specify the other units that are required for this test.
+  // needs: ['serializer:foo']
+});
+
+// Replace this with your real tests.
+test('it exists', function(assert) {
+  let adapter = this.subject();
+  assert.ok(adapter);
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/adapters/yarn-metrics-test.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/adapters/yarn-metrics-test.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/adapters/yarn-metrics-test.js
new file mode 100644
index 0000000..10941d2
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/adapters/yarn-metrics-test.js
@@ -0,0 +1,30 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import { moduleFor, test } from 'ember-qunit';
+
+moduleFor('adapter:yarn-metrics', 'Unit | Adapter | yarn metrics', {
+  // Specify the other units that are required for this test.
+  // needs: ['serializer:foo']
+});
+
+// Replace this with your real tests.
+test('it exists', function(assert) {
+  let adapter = this.subject();
+  assert.ok(adapter);
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/adapters/yarn-rm-log-test.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/adapters/yarn-rm-log-test.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/adapters/yarn-rm-log-test.js
new file mode 100644
index 0000000..b736957
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/adapters/yarn-rm-log-test.js
@@ -0,0 +1,30 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import { moduleFor, test } from 'ember-qunit';
+
+moduleFor('adapter:yarn-rm-log', 'Unit | Adapter | yarn rm log', {
+  // Specify the other units that are required for this test.
+  // needs: ['serializer:foo']
+});
+
+// Replace this with your real tests.
+test('it exists', function(assert) {
+  let adapter = this.subject();
+  assert.ok(adapter);
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/controllers/yarn-conf-test.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/controllers/yarn-conf-test.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/controllers/yarn-conf-test.js
new file mode 100644
index 0000000..adec93b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/controllers/yarn-conf-test.js
@@ -0,0 +1,29 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+import { moduleFor, test } from 'ember-qunit';
+
+moduleFor('controller:yarn-conf', 'Unit | Controller | yarn conf', {
+  // Specify the other units that are required for this test.
+  // needs: ['controller:foo']
+});
+
+// Replace this with your real tests.
+test('it exists', function(assert) {
+  let controller = this.subject();
+  assert.ok(controller);
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/controllers/yarn-rm-log-test.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/controllers/yarn-rm-log-test.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/controllers/yarn-rm-log-test.js
new file mode 100644
index 0000000..6f847bf
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/controllers/yarn-rm-log-test.js
@@ -0,0 +1,30 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import { moduleFor, test } from 'ember-qunit';
+
+moduleFor('controller:yarn-rm-log', 'Unit | Controller | yarn rm log', {
+  // Specify the other units that are required for this test.
+  // needs: ['controller:foo']
+});
+
+// Replace this with your real tests.
+test('it exists', function(assert) {
+  let controller = this.subject();
+  assert.ok(controller);
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/controllers/yarn-tools-test.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/controllers/yarn-tools-test.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/controllers/yarn-tools-test.js
new file mode 100644
index 0000000..a92ee59
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/controllers/yarn-tools-test.js
@@ -0,0 +1,30 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import { moduleFor, test } from 'ember-qunit';
+
+moduleFor('controller:yarn-tools', 'Unit | Controller | yarn tools', {
+  // Specify the other units that are required for this test.
+  // needs: ['controller:foo']
+});
+
+// Replace this with your real tests.
+test('it exists', function(assert) {
+  let controller = this.subject();
+  assert.ok(controller);
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/helpers/json-pretty-test.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/helpers/json-pretty-test.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/helpers/json-pretty-test.js
new file mode 100644
index 0000000..a6bc7c2
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/helpers/json-pretty-test.js
@@ -0,0 +1,28 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import { jsonPretty } from '../../../helpers/json-pretty';
+import { module, test } from 'qunit';
+
+module('Unit | Helper | json pretty');
+
+// Replace this with your real tests.
+test('it works', function(assert) {
+  let result = jsonPretty(42);
+  assert.ok(result);
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/models/yarn-conf-test.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/models/yarn-conf-test.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/models/yarn-conf-test.js
new file mode 100644
index 0000000..68ea891
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/models/yarn-conf-test.js
@@ -0,0 +1,29 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+import { moduleForModel, test } from 'ember-qunit';
+
+moduleForModel('yarn-conf', 'Unit | Model | yarn conf', {
+  // Specify the other units that are required for this test.
+  needs: []
+});
+
+test('it exists', function(assert) {
+  let model = this.subject();
+  // let store = this.store();
+  assert.ok(!!model);
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/models/yarn-metrics-test.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/models/yarn-metrics-test.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/models/yarn-metrics-test.js
new file mode 100644
index 0000000..2068dc8
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/models/yarn-metrics-test.js
@@ -0,0 +1,30 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import { moduleForModel, test } from 'ember-qunit';
+
+moduleForModel('yarn-metrics', 'Unit | Model | yarn metrics', {
+  // Specify the other units that are required for this test.
+  needs: []
+});
+
+test('it exists', function(assert) {
+  let model = this.subject();
+  // let store = this.store();
+  assert.ok(!!model);
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/models/yarn-rm-log-test.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/models/yarn-rm-log-test.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/models/yarn-rm-log-test.js
new file mode 100644
index 0000000..6adf3d9
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/models/yarn-rm-log-test.js
@@ -0,0 +1,30 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import { moduleForModel, test } from 'ember-qunit';
+
+moduleForModel('yarn-rm-log', 'Unit | Model | yarn rm log', {
+  // Specify the other units that are required for this test.
+  needs: []
+});
+
+test('it exists', function(assert) {
+  let model = this.subject();
+  // let store = this.store();
+  assert.ok(!!model);
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/routes/yarn-conf-test.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/routes/yarn-conf-test.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/routes/yarn-conf-test.js
new file mode 100644
index 0000000..306fa93
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/routes/yarn-conf-test.js
@@ -0,0 +1,28 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+import { moduleFor, test } from 'ember-qunit';
+
+moduleFor('route:yarn-conf', 'Unit | Route | yarn conf', {
+  // Specify the other units that are required for this test.
+  // needs: ['controller:foo']
+});
+
+test('it exists', function(assert) {
+  let route = this.subject();
+  assert.ok(route);
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/routes/yarn-metrics-test.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/routes/yarn-metrics-test.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/routes/yarn-metrics-test.js
new file mode 100644
index 0000000..13d109b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/routes/yarn-metrics-test.js
@@ -0,0 +1,28 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+import { moduleFor, test } from 'ember-qunit';
+
+moduleFor('route:yarn-metric', 'Unit | Route | yarn metric', {
+  // Specify the other units that are required for this test.
+  // needs: ['controller:foo']
+});
+
+test('it exists', function(assert) {
+  let route = this.subject();
+  assert.ok(route);
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/routes/yarn-rm-log-test.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/routes/yarn-rm-log-test.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/routes/yarn-rm-log-test.js
new file mode 100644
index 0000000..6d696c8
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/routes/yarn-rm-log-test.js
@@ -0,0 +1,28 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+import { moduleFor, test } from 'ember-qunit';
+
+moduleFor('route:yarn-rm-log', 'Unit | Route | yarn rm log', {
+  // Specify the other units that are required for this test.
+  // needs: ['controller:foo']
+});
+
+test('it exists', function(assert) {
+  let route = this.subject();
+  assert.ok(route);
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/routes/yarn-tools-test.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/routes/yarn-tools-test.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/routes/yarn-tools-test.js
new file mode 100644
index 0000000..2950fd4
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/routes/yarn-tools-test.js
@@ -0,0 +1,28 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+import { moduleFor, test } from 'ember-qunit';
+
+moduleFor('route:yarn-tools', 'Unit | Route | yarn tools', {
+  // Specify the other units that are required for this test.
+  // needs: ['controller:foo']
+});
+
+test('it exists', function(assert) {
+  let route = this.subject();
+  assert.ok(route);
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/serializers/yarn-conf-test.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/serializers/yarn-conf-test.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/serializers/yarn-conf-test.js
new file mode 100644
index 0000000..36576fd
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/serializers/yarn-conf-test.js
@@ -0,0 +1,32 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+import { moduleForModel, test } from 'ember-qunit';
+
+moduleForModel('yarn-conf', 'Unit | Serializer | yarn conf', {
+  // Specify the other units that are required for this test.
+  needs: ['serializer:yarn-conf']
+});
+
+// Replace this with your real tests.
+test('it serializes records', function(assert) {
+  let record = this.subject();
+
+  let serializedRecord = record.serialize();
+
+  assert.ok(serializedRecord);
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/serializers/yarn-metrics-test.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/serializers/yarn-metrics-test.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/serializers/yarn-metrics-test.js
new file mode 100644
index 0000000..7cab348
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/serializers/yarn-metrics-test.js
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import { moduleForModel, test } from 'ember-qunit';
+
+moduleForModel('yarn-metrics', 'Unit | Serializer | yarn metrics', {
+  // Specify the other units that are required for this test.
+  needs: ['serializer:yarn-metrics']
+});
+
+// Replace this with your real tests.
+test('it serializes records', function(assert) {
+  let record = this.subject();
+
+  let serializedRecord = record.serialize();
+
+  assert.ok(serializedRecord);
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f666e7c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/serializers/yarn-rm-log-test.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/serializers/yarn-rm-log-test.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/serializers/yarn-rm-log-test.js
new file mode 100644
index 0000000..fa049b3
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/serializers/yarn-rm-log-test.js
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import { moduleForModel, test } from 'ember-qunit';
+
+moduleForModel('yarn-rm-log', 'Unit | Serializer | yarn rm log', {
+  // Specify the other units that are required for this test.
+  needs: ['serializer:yarn-rm-log']
+});
+
+// Replace this with your real tests.
+test('it serializes records', function(assert) {
+  let record = this.subject();
+
+  let serializedRecord = record.serialize();
+
+  assert.ok(serializedRecord);
+});


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[20/37] hadoop git commit: MAPREDUCE-7022. Fast fail rogue jobs based on task scratch dir size. Contributed by Johan Gustavsson

Posted by ae...@apache.org.
MAPREDUCE-7022. Fast fail rogue jobs based on task scratch dir size. Contributed by Johan Gustavsson


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/a37e7f0a
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a37e7f0a
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a37e7f0a

Branch: refs/heads/HDFS-7240
Commit: a37e7f0ad8b68c7ed16c242bedf62f4cde48d6fd
Parents: 1b0f265
Author: Jason Lowe <jl...@apache.org>
Authored: Fri Jan 26 14:36:45 2018 -0600
Committer: Jason Lowe <jl...@apache.org>
Committed: Fri Jan 26 14:36:45 2018 -0600

----------------------------------------------------------------------
 .../hadoop/mapred/LocalContainerLauncher.java   |  2 +-
 .../hadoop/mapred/TaskAttemptListenerImpl.java  |  7 +-
 .../org/apache/hadoop/mapred/YarnChild.java     |  4 +-
 .../v2/app/job/event/TaskAttemptFailEvent.java  | 53 ++++++++++++
 .../app/job/event/TaskTAttemptFailedEvent.java  | 39 +++++++++
 .../v2/app/job/impl/TaskAttemptImpl.java        | 40 ++++++---
 .../mapreduce/v2/app/job/impl/TaskImpl.java     |  6 +-
 .../hadoop/mapreduce/v2/app/TestFail.java       |  7 +-
 .../hadoop/mapreduce/v2/app/TestRecovery.java   |  7 +-
 .../mapreduce/v2/app/job/impl/TestJobImpl.java  |  5 +-
 .../v2/app/job/impl/TestTaskAttempt.java        |  9 +-
 .../mapreduce/v2/app/job/impl/TestTaskImpl.java | 42 ++++-----
 .../apache/hadoop/mapred/LocalJobRunner.java    |  4 +-
 .../java/org/apache/hadoop/mapred/MapTask.java  |  3 +-
 .../java/org/apache/hadoop/mapred/Task.java     | 87 ++++++++++++++++++-
 .../hadoop/mapred/TaskUmbilicalProtocol.java    | 12 ++-
 .../apache/hadoop/mapreduce/MRJobConfig.java    | 14 +++
 .../src/main/resources/mapred-default.xml       | 22 +++++
 .../hadoop/mapred/TestTaskProgressReporter.java | 90 +++++++++++++++++++-
 .../mapreduce/v2/hs/TestJobHistoryParsing.java  |  9 +-
 .../apache/hadoop/mapred/TestMapProgress.java   |  4 +-
 .../apache/hadoop/mapred/TestTaskCommit.java    |  2 +-
 22 files changed, 397 insertions(+), 71 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a37e7f0a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/LocalContainerLauncher.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/LocalContainerLauncher.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/LocalContainerLauncher.java
index 6f9cc34..fed500a 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/LocalContainerLauncher.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/LocalContainerLauncher.java
@@ -510,7 +510,7 @@ public class LocalContainerLauncher extends AbstractService implements
           String cause =
               (tCause == null) ? throwable.getMessage() : StringUtils
                   .stringifyException(tCause);
-          umbilical.fatalError(classicAttemptID, cause);
+          umbilical.fatalError(classicAttemptID, cause, false);
         }
         throw new RuntimeException();
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a37e7f0a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java
index 556c90c..b155af22 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java
@@ -48,6 +48,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.Task;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptDiagnosticsUpdateEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
+import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptFailEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent.TaskAttemptStatus;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent;
 import org.apache.hadoop.mapreduce.v2.app.rm.RMHeartbeatHandler;
@@ -281,7 +282,7 @@ public class TaskAttemptListenerImpl extends CompositeService
   }
 
   @Override
-  public void fatalError(TaskAttemptID taskAttemptID, String msg)
+  public void fatalError(TaskAttemptID taskAttemptID, String msg, boolean fastFail)
       throws IOException {
     // This happens only in Child and in the Task.
     LOG.error("Task: " + taskAttemptID + " - exited : " + msg);
@@ -294,7 +295,7 @@ public class TaskAttemptListenerImpl extends CompositeService
     preemptionPolicy.handleFailedContainer(attemptID);
 
     context.getEventHandler().handle(
-        new TaskAttemptEvent(attemptID, TaskAttemptEventType.TA_FAILMSG));
+        new TaskAttemptFailEvent(attemptID, fastFail));
   }
 
   @Override
@@ -312,7 +313,7 @@ public class TaskAttemptListenerImpl extends CompositeService
     preemptionPolicy.handleFailedContainer(attemptID);
 
     context.getEventHandler().handle(
-        new TaskAttemptEvent(attemptID, TaskAttemptEventType.TA_FAILMSG));
+        new TaskAttemptFailEvent(attemptID));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a37e7f0a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java
index 7ae7a1e..bd40e54 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java
@@ -206,7 +206,7 @@ class YarnChild {
       if (taskid != null) {
         if (!ShutdownHookManager.get().isShutdownInProgress()) {
           umbilical.fatalError(taskid,
-              StringUtils.stringifyException(exception));
+              StringUtils.stringifyException(exception), false);
         }
       }
     } catch (Throwable throwable) {
@@ -218,7 +218,7 @@ class YarnChild {
           String cause =
               tCause == null ? throwable.getMessage() : StringUtils
                   .stringifyException(tCause);
-          umbilical.fatalError(taskid, cause);
+          umbilical.fatalError(taskid, cause, false);
         }
       }
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a37e7f0a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/event/TaskAttemptFailEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/event/TaskAttemptFailEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/event/TaskAttemptFailEvent.java
new file mode 100644
index 0000000..6ea1d15
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/event/TaskAttemptFailEvent.java
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapreduce.v2.app.job.event;
+
+import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
+
+public class TaskAttemptFailEvent extends TaskAttemptEvent {
+  private boolean fastFail;
+
+  /**
+   * Create a new TaskAttemptFailEvent, with task fastFail disabled.
+   *
+   * @param id the id of the task attempt
+   */
+  public TaskAttemptFailEvent(TaskAttemptId id) {
+    this(id, false);
+  }
+
+  /**
+   * Create a new TaskAttemptFailEvent.
+   *
+   * @param id the id of the task attempt
+   * @param fastFail should the task fastFail or not.
+   */
+  public TaskAttemptFailEvent(TaskAttemptId id, boolean fastFail) {
+    super(id, TaskAttemptEventType.TA_FAILMSG);
+    this.fastFail = fastFail;
+  }
+
+  /**
+   * Check if task should fast fail or retry
+   * @return boolean value where true indicates the task should not retry
+   */
+  public boolean isFastFail() {
+    return fastFail;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a37e7f0a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/event/TaskTAttemptFailedEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/event/TaskTAttemptFailedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/event/TaskTAttemptFailedEvent.java
new file mode 100644
index 0000000..30392ac
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/event/TaskTAttemptFailedEvent.java
@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapreduce.v2.app.job.event;
+
+import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
+
+public class TaskTAttemptFailedEvent extends TaskTAttemptEvent {
+
+  private boolean fastFail;
+
+  public TaskTAttemptFailedEvent(TaskAttemptId id) {
+    this(id, false);
+  }
+
+  public TaskTAttemptFailedEvent(TaskAttemptId id, boolean fastFail) {
+    super(id, TaskEventType.T_ATTEMPT_FAILED);
+    this.fastFail = fastFail;
+  }
+
+  public boolean isFastFail() {
+    return fastFail;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a37e7f0a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
index 431128b..6632f27 100755
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
@@ -94,6 +94,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptContainerLaunched
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptDiagnosticsUpdateEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
+import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptFailEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptKillEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptRecoverEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent;
@@ -101,6 +102,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptTooManyFetchFailureEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEventType;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskTAttemptEvent;
+import org.apache.hadoop.mapreduce.v2.app.job.event.TaskTAttemptFailedEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskTAttemptKilledEvent;
 import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncher;
 import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncherEvent;
@@ -194,6 +196,7 @@ public abstract class TaskAttemptImpl implements
   private Locality locality;
   private Avataar avataar;
   private boolean rescheduleNextAttempt = false;
+  private boolean failFast = false;
 
   private static final CleanupContainerTransition
       CLEANUP_CONTAINER_TRANSITION = new CleanupContainerTransition();
@@ -1412,6 +1415,14 @@ public abstract class TaskAttemptImpl implements
   public void setAvataar(Avataar avataar) {
     this.avataar = avataar;
   }
+
+  public void setTaskFailFast(boolean failFast) {
+    this.failFast = failFast;
+  }
+
+  public boolean isTaskFailFast() {
+    return failFast;
+  }
   
   @SuppressWarnings("unchecked")
   public TaskAttemptStateInternal recover(TaskAttemptInfo taInfo,
@@ -1921,9 +1932,12 @@ public abstract class TaskAttemptImpl implements
 
       switch(finalState) {
         case FAILED:
-          taskAttempt.eventHandler.handle(new TaskTAttemptEvent(
-              taskAttempt.attemptId,
-              TaskEventType.T_ATTEMPT_FAILED));
+          boolean fastFail = false;
+          if (event instanceof TaskAttemptFailEvent) {
+            fastFail = ((TaskAttemptFailEvent) event).isFastFail();
+          }
+          taskAttempt.eventHandler.handle(new TaskTAttemptFailedEvent(
+              taskAttempt.attemptId, fastFail));
           break;
         case KILLED:
           taskAttempt.eventHandler.handle(new TaskTAttemptKilledEvent(
@@ -2041,13 +2055,16 @@ public abstract class TaskAttemptImpl implements
 
   private static class FailedTransition implements
       SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent> {
+
+
     @SuppressWarnings("unchecked")
     @Override
     public void transition(TaskAttemptImpl taskAttempt,
         TaskAttemptEvent event) {
       // set the finish time
       taskAttempt.setFinishTime();
-      notifyTaskAttemptFailed(taskAttempt);
+
+      notifyTaskAttemptFailed(taskAttempt, taskAttempt.isTaskFailFast());
     }
   }
 
@@ -2154,8 +2171,8 @@ public abstract class TaskAttemptImpl implements
         LOG.debug("Not generating HistoryFinish event since start event not " +
             "generated for taskAttempt: " + taskAttempt.getID());
       }
-      taskAttempt.eventHandler.handle(new TaskTAttemptEvent(
-          taskAttempt.attemptId, TaskEventType.T_ATTEMPT_FAILED));
+      taskAttempt.eventHandler.handle(new TaskTAttemptFailedEvent(
+          taskAttempt.attemptId));
     }
   }
   
@@ -2332,6 +2349,8 @@ public abstract class TaskAttemptImpl implements
       if (event instanceof TaskAttemptKillEvent) {
         taskAttempt.setRescheduleNextAttempt(
             ((TaskAttemptKillEvent)event).getRescheduleAttempt());
+      } else if (event instanceof TaskAttemptFailEvent) {
+        taskAttempt.setTaskFailFast(((TaskAttemptFailEvent)event).isFastFail());
       }
     }
   }
@@ -2400,12 +2419,13 @@ public abstract class TaskAttemptImpl implements
       // register it to finishing state
       taskAttempt.appContext.getTaskAttemptFinishingMonitor().register(
           taskAttempt.attemptId);
-      notifyTaskAttemptFailed(taskAttempt);
+      notifyTaskAttemptFailed(taskAttempt, false);
     }
   }
 
   @SuppressWarnings("unchecked")
-  private static void notifyTaskAttemptFailed(TaskAttemptImpl taskAttempt) {
+  private static void notifyTaskAttemptFailed(TaskAttemptImpl taskAttempt,
+      boolean fastFail) {
     if (taskAttempt.getLaunchTime() == 0) {
       sendJHStartEventForAssignedFailTask(taskAttempt);
     }
@@ -2419,8 +2439,8 @@ public abstract class TaskAttemptImpl implements
     taskAttempt.eventHandler.handle(new JobHistoryEvent(
         taskAttempt.attemptId.getTaskId().getJobId(), tauce));
 
-    taskAttempt.eventHandler.handle(new TaskTAttemptEvent(
-        taskAttempt.attemptId, TaskEventType.T_ATTEMPT_FAILED));
+    taskAttempt.eventHandler.handle(new TaskTAttemptFailedEvent(
+        taskAttempt.attemptId, fastFail));
 
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a37e7f0a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java
index 086d4d5..ce3b3cc 100755
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java
@@ -74,6 +74,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEventType;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskRecoverEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskTAttemptEvent;
+import org.apache.hadoop.mapreduce.v2.app.job.event.TaskTAttemptFailedEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskTAttemptKilledEvent;
 import org.apache.hadoop.mapreduce.v2.app.metrics.MRAppMetrics;
 import org.apache.hadoop.mapreduce.v2.app.rm.ContainerFailedEvent;
@@ -1054,7 +1055,7 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
 
     @Override
     public TaskStateInternal transition(TaskImpl task, TaskEvent event) {
-      TaskTAttemptEvent castEvent = (TaskTAttemptEvent) event;
+      TaskTAttemptFailedEvent castEvent = (TaskTAttemptFailedEvent) event;
       TaskAttemptId taskAttemptId = castEvent.getTaskAttemptID();
       task.failedAttempts.add(taskAttemptId); 
       if (taskAttemptId.equals(task.commitAttempt)) {
@@ -1068,7 +1069,8 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
       }
       
       task.finishedAttempts.add(taskAttemptId);
-      if (task.failedAttempts.size() < task.maxAttempts) {
+      if (!castEvent.isFastFail()
+          && task.failedAttempts.size() < task.maxAttempts) {
         task.handleTaskAttemptCompletion(
             taskAttemptId, 
             TaskAttemptCompletionEventStatus.FAILED);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a37e7f0a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFail.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFail.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFail.java
index 4d3f6f4..a2f0aba 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFail.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFail.java
@@ -23,6 +23,7 @@ import java.net.InetSocketAddress;
 import java.util.Iterator;
 import java.util.Map;
 
+import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptFailEvent;
 import org.junit.Assert;
 
 import org.apache.hadoop.conf.Configuration;
@@ -288,8 +289,7 @@ public class TestFail {
       if (attemptID.getTaskId().getId() == 0) {//check if it is first task
         // send the Fail event
         getContext().getEventHandler().handle(
-            new TaskAttemptEvent(attemptID, 
-                TaskAttemptEventType.TA_FAILMSG));
+            new TaskAttemptFailEvent(attemptID));
       } else {
         getContext().getEventHandler().handle(
             new TaskAttemptEvent(attemptID,
@@ -310,8 +310,7 @@ public class TestFail {
         //check if it is first task's first attempt
         // send the Fail event
         getContext().getEventHandler().handle(
-            new TaskAttemptEvent(attemptID, 
-                TaskAttemptEventType.TA_FAILMSG));
+            new TaskAttemptFailEvent(attemptID));
       } else {
         getContext().getEventHandler().handle(
             new TaskAttemptEvent(attemptID,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a37e7f0a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRecovery.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRecovery.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRecovery.java
index 893c4a0..b2807c1 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRecovery.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRecovery.java
@@ -38,6 +38,8 @@ import java.util.List;
 import java.util.Map;
 
 import java.util.concurrent.TimeoutException;
+
+import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptFailEvent;
 import org.junit.Assert;
 
 import org.apache.hadoop.conf.Configuration;
@@ -167,9 +169,8 @@ public class TestRecovery {
     /////////// Play some games with the TaskAttempts of the first task //////
     //send the fail signal to the 1st map task attempt
     app.getContext().getEventHandler().handle(
-        new TaskAttemptEvent(
-            task1Attempt1.getID(),
-            TaskAttemptEventType.TA_FAILMSG));
+        new TaskAttemptFailEvent(
+            task1Attempt1.getID()));
     
     app.waitForState(task1Attempt1, TaskAttemptState.FAILED);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a37e7f0a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java
index 1827ce4..8592b20 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java
@@ -81,7 +81,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEventType;
-import org.apache.hadoop.mapreduce.v2.app.job.event.TaskTAttemptEvent;
+import org.apache.hadoop.mapreduce.v2.app.job.event.TaskTAttemptFailedEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl.InitTransition;
 import org.apache.hadoop.mapreduce.v2.app.metrics.MRAppMetrics;
 import org.apache.hadoop.mapreduce.v2.app.rm.RMHeartbeatHandler;
@@ -437,8 +437,7 @@ public class TestJobImpl {
       TaskImpl task = (TaskImpl) t;
       task.handle(new TaskEvent(task.getID(), TaskEventType.T_SCHEDULE));
       for(TaskAttempt ta: task.getAttempts().values()) {
-        task.handle(new TaskTAttemptEvent(ta.getID(),
-          TaskEventType.T_ATTEMPT_FAILED));
+        task.handle(new TaskTAttemptFailedEvent(ta.getID()));
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a37e7f0a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttempt.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttempt.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttempt.java
index fe5d95d..43571a9 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttempt.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttempt.java
@@ -39,6 +39,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.CopyOnWriteArrayList;
 
+import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptFailEvent;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -499,7 +500,7 @@ public class TestTaskAttempt{
           new TaskAttemptDiagnosticsUpdateEvent(attemptID,
               "Test Diagnostic Event"));
       getContext().getEventHandler().handle(
-          new TaskAttemptEvent(attemptID, TaskAttemptEventType.TA_FAILMSG));
+          new TaskAttemptFailEvent(attemptID));
     }
 
     protected EventHandler<JobHistoryEvent> createJobHistoryHandler(
@@ -1357,8 +1358,7 @@ public class TestTaskAttempt{
     MockEventHandler eventHandler = new MockEventHandler();
     TaskAttemptImpl taImpl = createTaskAttemptImpl(eventHandler);
 
-    taImpl.handle(new TaskAttemptEvent(taImpl.getID(),
-        TaskAttemptEventType.TA_FAILMSG));
+    taImpl.handle(new TaskAttemptFailEvent(taImpl.getID()));
 
     assertEquals("Task attempt is not in FAILED state", taImpl.getState(),
         TaskAttemptState.FAILED);
@@ -1484,8 +1484,7 @@ public class TestTaskAttempt{
     MockEventHandler eventHandler = new MockEventHandler();
     TaskAttemptImpl taImpl = createTaskAttemptImpl(eventHandler);
 
-    taImpl.handle(new TaskAttemptEvent(taImpl.getID(),
-        TaskAttemptEventType.TA_FAILMSG));
+    taImpl.handle(new TaskAttemptFailEvent(taImpl.getID()));
 
     assertEquals("Task attempt is not in RUNNING state", taImpl.getState(),
         TaskAttemptState.FAILED);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a37e7f0a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskImpl.java
index 62d4cc0..1225c43 100755
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskImpl.java
@@ -53,6 +53,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEventType;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskTAttemptEvent;
+import org.apache.hadoop.mapreduce.v2.app.job.event.TaskTAttemptFailedEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskTAttemptKilledEvent;
 import org.apache.hadoop.mapreduce.v2.app.metrics.MRAppMetrics;
 import org.apache.hadoop.security.Credentials;
@@ -345,8 +346,7 @@ public class TestTaskImpl {
   }
 
   private void failRunningTaskAttempt(TaskAttemptId attemptId) {
-    mockTask.handle(new TaskTAttemptEvent(attemptId, 
-        TaskEventType.T_ATTEMPT_FAILED));
+    mockTask.handle(new TaskTAttemptFailedEvent(attemptId));
     assertTaskRunningState();
   }
   
@@ -612,11 +612,16 @@ public class TestTaskImpl {
     
     // The task should now have succeeded
     assertTaskSucceededState();
-    
+
     // Now complete the first task attempt, after the second has succeeded
-    mockTask.handle(new TaskTAttemptEvent(taskAttempts.get(0).getAttemptId(), 
-        firstAttemptFinishEvent));
-    
+    if (firstAttemptFinishEvent.equals(TaskEventType.T_ATTEMPT_FAILED)) {
+      mockTask.handle(new TaskTAttemptFailedEvent(taskAttempts
+          .get(0).getAttemptId()));
+    } else {
+      mockTask.handle(new TaskTAttemptEvent(taskAttempts.get(0).getAttemptId(),
+          firstAttemptFinishEvent));
+    }
+
     // The task should still be in the succeeded state
     assertTaskSucceededState();
     
@@ -668,8 +673,8 @@ public class TestTaskImpl {
     assertEquals(2, taskAttempts.size());
 
     // speculative attempt retroactively fails from fetch failures
-    mockTask.handle(new TaskTAttemptEvent(taskAttempts.get(1).getAttemptId(),
-        TaskEventType.T_ATTEMPT_FAILED));
+    mockTask.handle(new TaskTAttemptFailedEvent(
+        taskAttempts.get(1).getAttemptId()));
 
     assertTaskScheduledState();
     assertEquals(3, taskAttempts.size());
@@ -683,8 +688,8 @@ public class TestTaskImpl {
     assertEquals(2, taskAttempts.size());
 
     // speculative attempt retroactively fails from fetch failures
-    mockTask.handle(new TaskTAttemptEvent(taskAttempts.get(1).getAttemptId(),
-        TaskEventType.T_ATTEMPT_FAILED));
+    mockTask.handle(new TaskTAttemptFailedEvent(
+        taskAttempts.get(1).getAttemptId()));
 
     assertTaskScheduledState();
     assertEquals(3, taskAttempts.size());
@@ -698,8 +703,8 @@ public class TestTaskImpl {
     assertEquals(2, taskAttempts.size());
 
     // speculative attempt retroactively fails from fetch failures
-    mockTask.handle(new TaskTAttemptEvent(taskAttempts.get(1).getAttemptId(),
-        TaskEventType.T_ATTEMPT_FAILED));
+    mockTask.handle(new TaskTAttemptFailedEvent(
+        taskAttempts.get(1).getAttemptId()));
 
     assertTaskScheduledState();
     assertEquals(3, taskAttempts.size());
@@ -734,8 +739,8 @@ public class TestTaskImpl {
     // have the first attempt fail, verify task failed due to no retries
     MockTaskAttemptImpl taskAttempt = taskAttempts.get(0);
     taskAttempt.setState(TaskAttemptState.FAILED);
-    mockTask.handle(new TaskTAttemptEvent(taskAttempt.getAttemptId(),
-        TaskEventType.T_ATTEMPT_FAILED));
+    mockTask.handle(new TaskTAttemptFailedEvent(
+        taskAttempt.getAttemptId()));
     assertEquals(TaskState.FAILED, mockTask.getState());
 
     // verify task can no longer be killed
@@ -757,8 +762,7 @@ public class TestTaskImpl {
         TaskEventType.T_ATTEMPT_COMMIT_PENDING));
     assertEquals(TaskState.FAILED, mockTask.getState());
     taskAttempt.setState(TaskAttemptState.FAILED);
-    mockTask.handle(new TaskTAttemptEvent(taskAttempt.getAttemptId(),
-        TaskEventType.T_ATTEMPT_FAILED));
+    mockTask.handle(new TaskTAttemptFailedEvent(taskAttempt.getAttemptId()));
     assertEquals(TaskState.FAILED, mockTask.getState());
     taskAttempt = taskAttempts.get(2);
     taskAttempt.setState(TaskAttemptState.SUCCEEDED);
@@ -808,8 +812,7 @@ public class TestTaskImpl {
     // max attempts is 4
     MockTaskAttemptImpl taskAttempt = taskAttempts.get(0);
     taskAttempt.setState(TaskAttemptState.FAILED);
-    mockTask.handle(new TaskTAttemptEvent(taskAttempt.getAttemptId(),
-        TaskEventType.T_ATTEMPT_FAILED));
+    mockTask.handle(new TaskTAttemptFailedEvent(taskAttempt.getAttemptId()));
     assertEquals(TaskState.RUNNING, mockTask.getState());
 
     // verify a new attempt(#3) added because the speculative attempt(#2)
@@ -829,8 +832,7 @@ public class TestTaskImpl {
     // hasn't reach the max attempts which is 4
     MockTaskAttemptImpl taskAttempt1 = taskAttempts.get(1);
     taskAttempt1.setState(TaskAttemptState.FAILED);
-    mockTask.handle(new TaskTAttemptEvent(taskAttempt1.getAttemptId(),
-        TaskEventType.T_ATTEMPT_FAILED));
+    mockTask.handle(new TaskTAttemptFailedEvent(taskAttempt1.getAttemptId()));
     assertEquals(TaskState.RUNNING, mockTask.getState());
 
     // verify there's no new attempt added because of the running attempt(#3)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a37e7f0a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapred/LocalJobRunner.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapred/LocalJobRunner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapred/LocalJobRunner.java
index c9dff6a..5e7a250 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapred/LocalJobRunner.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapred/LocalJobRunner.java
@@ -729,9 +729,9 @@ public class LocalJobRunner implements ClientProtocol {
       LOG.error("shuffleError: "+ message + "from task: " + taskId);
     }
     
-    public synchronized void fatalError(TaskAttemptID taskId, String msg) 
+    public synchronized void fatalError(TaskAttemptID taskId, String msg, boolean fastFail)
     throws IOException {
-      LOG.error("Fatal: "+ msg + "from task: " + taskId);
+      LOG.error("Fatal: "+ msg + " from task: " + taskId + " fast fail: " + fastFail);
     }
     
     @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a37e7f0a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java
index 27c8976..ab7cba5 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java
@@ -1568,7 +1568,8 @@ public class MapTask extends Task {
         if (lspillException instanceof Error) {
           final String logMsg = "Task " + getTaskID() + " failed : " +
             StringUtils.stringifyException(lspillException);
-          mapTask.reportFatalError(getTaskID(), lspillException, logMsg);
+          mapTask.reportFatalError(getTaskID(), lspillException, logMsg,
+              false);
         }
         throw new IOException("Spill failed", lspillException);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a37e7f0a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Task.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Task.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Task.java
index 730f4ee..87c9e16 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Task.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Task.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
@@ -354,7 +355,7 @@ abstract public class Task implements Writable, Configurable {
    * Report a fatal error to the parent (task) tracker.
    */
   protected void reportFatalError(TaskAttemptID id, Throwable throwable, 
-                                  String logMsg) {
+                                  String logMsg, boolean fastFail) {
     LOG.error(logMsg);
     
     if (ShutdownHookManager.get().isShutdownInProgress()) {
@@ -366,7 +367,7 @@ abstract public class Task implements Writable, Configurable {
                    ? StringUtils.stringifyException(throwable)
                    : StringUtils.stringifyException(tCause);
     try {
-      umbilical.fatalError(id, cause);
+      umbilical.fatalError(id, cause, fastFail);
     } catch (IOException ioe) {
       LOG.error("Failed to contact the tasktracker", ioe);
       System.exit(-1);
@@ -652,6 +653,8 @@ abstract public class Task implements Writable, Configurable {
     private Thread pingThread = null;
     private boolean done = true;
     private Object lock = new Object();
+    private volatile String diskLimitCheckStatus = null;
+    private Thread diskLimitCheckThread = null;
 
     /**
      * flag that indicates whether progress update needs to be sent to parent.
@@ -749,6 +752,65 @@ abstract public class Task implements Writable, Configurable {
     }
 
     /**
+     * disk limit checker, runs in separate thread when activated.
+     */
+    public class DiskLimitCheck implements Runnable {
+      private LocalFileSystem localFS;
+      private long fsLimit;
+      private long checkInterval;
+      private String[] localDirs;
+      private boolean killOnLimitExceeded;
+
+      public DiskLimitCheck(JobConf conf) throws IOException {
+        this.localFS = FileSystem.getLocal(conf);
+        this.fsLimit = conf.getLong(MRJobConfig.JOB_SINGLE_DISK_LIMIT_BYTES,
+            MRJobConfig.DEFAULT_JOB_SINGLE_DISK_LIMIT_BYTES);
+        this.localDirs = conf.getLocalDirs();
+        this.checkInterval = conf.getLong(
+            MRJobConfig.JOB_SINGLE_DISK_LIMIT_CHECK_INTERVAL_MS,
+            MRJobConfig.DEFAULT_JOB_SINGLE_DISK_LIMIT_CHECK_INTERVAL_MS);
+        this.killOnLimitExceeded = conf.getBoolean(
+            MRJobConfig.JOB_SINGLE_DISK_LIMIT_KILL_LIMIT_EXCEED,
+            MRJobConfig.DEFAULT_JOB_SINGLE_DISK_LIMIT_KILL_LIMIT_EXCEED);
+      }
+
+      @Override
+      public void run() {
+        while (!taskDone.get()) {
+          try {
+            long localWritesSize = 0L;
+            String largestWorkDir = null;
+            for (String local : localDirs) {
+              long size = FileUtil.getDU(localFS.pathToFile(new Path(local)));
+              if (localWritesSize < size) {
+                localWritesSize = size;
+                largestWorkDir = local;
+              }
+            }
+            if (localWritesSize > fsLimit) {
+              String localStatus =
+                  "too much data in local scratch dir="
+                      + largestWorkDir
+                      + ". current size is "
+                      + localWritesSize
+                      + " the limit is " + fsLimit;
+              if (killOnLimitExceeded) {
+                LOG.error(localStatus);
+                diskLimitCheckStatus = localStatus;
+              } else {
+                LOG.warn(localStatus);
+              }
+              break;
+            }
+            Thread.sleep(checkInterval);
+          } catch (Exception e) {
+            LOG.error(e.getMessage(), e);
+          }
+        }
+      }
+    }
+
+    /**
      * check the counters to see whether the task has exceeded any configured
      * limits.
      * @throws TaskLimitException
@@ -773,6 +835,9 @@ abstract public class Task implements Writable, Configurable {
                   " the limit is " + limit);
         }
       }
+      if (diskLimitCheckStatus != null) {
+        throw new TaskLimitException(diskLimitCheckStatus);
+      }
     }
 
     /**
@@ -851,7 +916,7 @@ abstract public class Task implements Writable, Configurable {
                   StringUtils.stringifyException(e);
           LOG.error(errMsg);
           try {
-            umbilical.fatalError(taskId, errMsg);
+            umbilical.fatalError(taskId, errMsg, true);
           } catch (IOException ioe) {
             LOG.error("Failed to update failure diagnosis", ioe);
           }
@@ -884,6 +949,22 @@ abstract public class Task implements Writable, Configurable {
         pingThread.setDaemon(true);
         pingThread.start();
       }
+      startDiskLimitCheckerThreadIfNeeded();
+    }
+    public void startDiskLimitCheckerThreadIfNeeded() {
+      if (diskLimitCheckThread == null && conf.getLong(
+          MRJobConfig.JOB_SINGLE_DISK_LIMIT_BYTES,
+          MRJobConfig.DEFAULT_JOB_SINGLE_DISK_LIMIT_BYTES) >= 0) {
+        try {
+          diskLimitCheckThread = new Thread(new DiskLimitCheck(conf),
+              "disk limit check thread");
+          diskLimitCheckThread.setDaemon(true);
+          diskLimitCheckThread.start();
+        } catch (IOException e) {
+          LOG.error("Issues starting disk monitor thread: "
+              + e.getMessage(), e);
+        }
+      }
     }
     public void stopCommunicationThread() throws InterruptedException {
       if (pingThread != null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a37e7f0a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskUmbilicalProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskUmbilicalProtocol.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskUmbilicalProtocol.java
index c3678d6..041ab39 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskUmbilicalProtocol.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskUmbilicalProtocol.java
@@ -68,9 +68,10 @@ public interface TaskUmbilicalProtocol extends VersionedProtocol {
    * Version 18 Added numRequiredSlots to TaskStatus for MAPREDUCE-516
    * Version 19 Added fatalError for child to communicate fatal errors to TT
    * Version 20 Added methods to manage checkpoints
+   * Version 21 Added fastFail parameter to fatalError
    * */
 
-  public static final long versionID = 20L;
+  public static final long versionID = 21L;
   
   /**
    * Called when a child task process starts, to get its task.
@@ -140,8 +141,13 @@ public interface TaskUmbilicalProtocol extends VersionedProtocol {
   /** Report that the task encounted a local filesystem error.*/
   void fsError(TaskAttemptID taskId, String message) throws IOException;
 
-  /** Report that the task encounted a fatal error.*/
-  void fatalError(TaskAttemptID taskId, String message) throws IOException;
+  /**
+   * Report that the task encounted a fatal error.
+   * @param taskId task's id
+   * @param message fail message
+   * @param fastFail flag to enable fast fail for task
+   */
+  void fatalError(TaskAttemptID taskId, String message, boolean fastFail) throws IOException;
   
   /** Called by a reduce task to get the map output locations for finished maps.
    * Returns an update centered around the map-task-completion-events. 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a37e7f0a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
index 6acf1bc..ca18bfe 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
@@ -52,6 +52,20 @@ public interface MRJobConfig {
 
   public static final String TASK_CLEANUP_NEEDED = "mapreduce.job.committer.task.cleanup.needed";
 
+  public static final String JOB_SINGLE_DISK_LIMIT_BYTES =
+          "mapreduce.job.local-fs.single-disk-limit.bytes";
+  // negative values disable the limit
+  public static final long DEFAULT_JOB_SINGLE_DISK_LIMIT_BYTES = -1;
+
+  public static final String JOB_SINGLE_DISK_LIMIT_KILL_LIMIT_EXCEED =
+      "mapreduce.job.local-fs.single-disk-limit.check.kill-limit-exceed";
+  // setting to false only logs the kill
+  public static final boolean DEFAULT_JOB_SINGLE_DISK_LIMIT_KILL_LIMIT_EXCEED = true;
+
+  public static final String JOB_SINGLE_DISK_LIMIT_CHECK_INTERVAL_MS =
+      "mapreduce.job.local-fs.single-disk-limit.check.interval-ms";
+  public static final long DEFAULT_JOB_SINGLE_DISK_LIMIT_CHECK_INTERVAL_MS = 5000;
+
   public static final String TASK_LOCAL_WRITE_LIMIT_BYTES =
           "mapreduce.task.local-fs.write-limit.bytes";
   // negative values disable the limit

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a37e7f0a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
index 62f3dfa..72f509c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
@@ -63,6 +63,28 @@
 </property>
 
 <property>
+  <name>mapreduce.job.local-fs.single-disk-limit.bytes</name>
+  <value>-1</value>
+  <description>Enable an in task monitor thread to watch for single disk
+    consumption by jobs. By setting this to x nr of bytes, the task will fast
+    fail in case it is reached. This is a per disk configuration.</description>
+</property>
+
+<property>
+  <name>mapreduce.job.local-fs.single-disk-limit.check.interval-ms</name>
+  <value>5000</value>
+  <description>Interval of disk limit check to run in ms.</description>
+</property>
+
+<property>
+  <name>mapreduce.job.local-fs.single-disk-limit.check.kill-limit-exceed</name>
+  <value>true</value>
+  <description>If mapreduce.job.local-fs.single-disk-limit.bytes is triggered
+    should the task be killed or logged. If false the intent to kill the task
+    is only logged in the container logs.</description>
+</property>
+
+<property>
   <name>mapreduce.job.maps</name>
   <value>2</value>
   <description>The default number of map tasks per job.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a37e7f0a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestTaskProgressReporter.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestTaskProgressReporter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestTaskProgressReporter.java
index 18442d6..e5ff64e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestTaskProgressReporter.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestTaskProgressReporter.java
@@ -18,15 +18,19 @@
 
 package org.apache.hadoop.mapred;
 
+import java.io.File;
 import java.io.IOException;
 import java.util.Random;
 
+import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.mapred.SortedRanges.Range;
+import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.checkpoint.TaskCheckpointID;
 import org.apache.hadoop.util.ExitUtil;
@@ -43,6 +47,11 @@ public class TestTaskProgressReporter {
 
   private FakeUmbilical fakeUmbilical = new FakeUmbilical();
 
+  private static final String TEST_DIR =
+      System.getProperty("test.build.data",
+          System.getProperty("java.io.tmpdir")) + "/" +
+      TestTaskProgressReporter.class.getName();
+
   private static class DummyTask extends Task {
     @Override
     public void run(JobConf job, TaskUmbilicalProtocol umbilical)
@@ -53,6 +62,11 @@ public class TestTaskProgressReporter {
     public boolean isMapTask() {
       return true;
     }
+
+    @Override
+    public boolean isCommitRequired() {
+      return false;
+    }
   }
 
   private static class FakeUmbilical implements TaskUmbilicalProtocol {
@@ -118,7 +132,7 @@ public class TestTaskProgressReporter {
     }
 
     @Override
-    public void fatalError(TaskAttemptID taskId, String message)
+    public void fatalError(TaskAttemptID taskId, String message, boolean fastFail)
         throws IOException {
     }
 
@@ -163,6 +177,78 @@ public class TestTaskProgressReporter {
     }
   }
 
+  @Test(timeout=60000)
+  public void testScratchDirSize() throws Exception {
+    String tmpPath = TEST_DIR + "/testBytesWrittenLimit-tmpFile-"
+        + new Random(System.currentTimeMillis()).nextInt();
+    File data = new File(tmpPath + "/out");
+    File testDir = new File(tmpPath);
+    testDir.mkdirs();
+    testDir.deleteOnExit();
+    JobConf conf = new JobConf();
+    conf.setStrings(MRConfig.LOCAL_DIR, "file://" + tmpPath);
+    conf.setLong(MRJobConfig.JOB_SINGLE_DISK_LIMIT_BYTES, 1024L);
+    conf.setBoolean(MRJobConfig.JOB_SINGLE_DISK_LIMIT_KILL_LIMIT_EXCEED,
+        true);
+    getBaseConfAndWriteToFile(-1, data);
+    testScratchDirLimit(false, conf);
+    data.delete();
+    getBaseConfAndWriteToFile(100, data);
+    testScratchDirLimit(false, conf);
+    data.delete();
+    getBaseConfAndWriteToFile(1536, data);
+    testScratchDirLimit(true, conf);
+    conf.setBoolean(MRJobConfig.JOB_SINGLE_DISK_LIMIT_KILL_LIMIT_EXCEED,
+        false);
+    testScratchDirLimit(false, conf);
+    conf.setBoolean(MRJobConfig.JOB_SINGLE_DISK_LIMIT_KILL_LIMIT_EXCEED,
+        true);
+    conf.setLong(MRJobConfig.JOB_SINGLE_DISK_LIMIT_BYTES, -1L);
+    testScratchDirLimit(false, conf);
+    data.delete();
+    FileUtil.fullyDelete(testDir);
+  }
+
+  private void getBaseConfAndWriteToFile(int size, File data)
+      throws IOException {
+    if (size > 0) {
+      byte[] b = new byte[size];
+      for (int i = 0; i < size; i++) {
+        b[i] = 1;
+      }
+      FileUtils.writeByteArrayToFile(data, b);
+    }
+  }
+
+  public void testScratchDirLimit(boolean fastFail, JobConf conf)
+          throws Exception {
+    ExitUtil.disableSystemExit();
+    threadExited = false;
+    Thread.UncaughtExceptionHandler h = new Thread.UncaughtExceptionHandler() {
+      public void uncaughtException(Thread th, Throwable ex) {
+        if (ex instanceof ExitUtil.ExitException) {
+          threadExited = true;
+          th.interrupt();
+        }
+      }
+    };
+    Task task = new DummyTask();
+    task.setConf(conf);
+    DummyTaskReporter reporter = new DummyTaskReporter(task);
+    reporter.startDiskLimitCheckerThreadIfNeeded();
+    Thread t = new Thread(reporter);
+    t.setUncaughtExceptionHandler(h);
+    reporter.setProgressFlag();
+    t.start();
+    while (!reporter.taskLimitIsChecked) {
+      Thread.yield();
+    }
+    task.done(fakeUmbilical, reporter);
+    reporter.resetDoneFlag();
+    t.join(1000L);
+    Assert.assertEquals(fastFail, threadExited);
+  }
+
   @Test (timeout=10000)
   public void testTaskProgress() throws Exception {
     JobConf job = new JobConf();
@@ -214,7 +300,7 @@ public class TestTaskProgressReporter {
     conf.getLong(MRJobConfig.TASK_PROGRESS_REPORT_INTERVAL, 0);
     conf.setLong(MRJobConfig.TASK_LOCAL_WRITE_LIMIT_BYTES, limit);
     LocalFileSystem localFS = FileSystem.getLocal(conf);
-    Path tmpPath = new Path("/tmp/testBytesWrittenLimit-tmpFile-"
+    Path tmpPath = new Path(TEST_DIR + "/testBytesWrittenLimit-tmpFile-"
             + new Random(System.currentTimeMillis()).nextInt());
     FSDataOutputStream out = localFS.create(tmpPath, true);
     out.write(new byte[LOCAL_BYTES_WRITTEN]);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a37e7f0a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistoryParsing.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistoryParsing.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistoryParsing.java
index 83e35fe..7b70f98 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistoryParsing.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistoryParsing.java
@@ -36,6 +36,7 @@ import java.util.Map;
 import java.util.StringTokenizer;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptFailEvent;
 import org.junit.Assert;
 
 import org.apache.hadoop.conf.Configuration;
@@ -712,7 +713,7 @@ public class TestJobHistoryParsing {
     protected void attemptLaunched(TaskAttemptId attemptID) {
       if (attemptID.getTaskId().getId() == 0 && attemptID.getId() == 0) {
         getContext().getEventHandler().handle(
-            new TaskAttemptEvent(attemptID, TaskAttemptEventType.TA_FAILMSG));
+            new TaskAttemptFailEvent(attemptID));
       } else {
         getContext().getEventHandler().handle(
             new TaskAttemptEvent(attemptID, TaskAttemptEventType.TA_DONE));
@@ -732,7 +733,7 @@ public class TestJobHistoryParsing {
     protected void attemptLaunched(TaskAttemptId attemptID) {
       if (attemptID.getTaskId().getId() == 0) {
         getContext().getEventHandler().handle(
-            new TaskAttemptEvent(attemptID, TaskAttemptEventType.TA_FAILMSG));
+            new TaskAttemptFailEvent(attemptID));
       } else {
         getContext().getEventHandler().handle(
             new TaskAttemptEvent(attemptID, TaskAttemptEventType.TA_DONE));
@@ -760,10 +761,10 @@ public class TestJobHistoryParsing {
             new TaskEvent(attemptID.getTaskId(), TaskEventType.T_KILL));
       } else if (taskType == TaskType.MAP && taskId == 1) {
         getContext().getEventHandler().handle(
-            new TaskAttemptEvent(attemptID, TaskAttemptEventType.TA_FAILMSG));
+            new TaskAttemptFailEvent(attemptID));
       } else if (taskType == TaskType.REDUCE && taskId == 0) {
         getContext().getEventHandler().handle(
-            new TaskAttemptEvent(attemptID, TaskAttemptEventType.TA_FAILMSG));
+            new TaskAttemptFailEvent(attemptID));
       } else if (taskType == TaskType.REDUCE && taskId == 1) {
         getContext().getEventHandler().handle(
             new TaskEvent(attemptID.getTaskId(), TaskEventType.T_KILL));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a37e7f0a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMapProgress.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMapProgress.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMapProgress.java
index f364c18..9b6ebda 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMapProgress.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMapProgress.java
@@ -91,8 +91,8 @@ public class TestMapProgress {
       LOG.info("Task " + taskId + " reporting shuffle error: " + message);
     }
 
-    public void fatalError(TaskAttemptID taskId, String msg) throws IOException {
-      LOG.info("Task " + taskId + " reporting fatal error: " + msg);
+    public void fatalError(TaskAttemptID taskId, String msg, boolean fastFail) throws IOException {
+      LOG.info("Task " + taskId + " reporting fatal error: " + msg + " fast fail: " + fastFail);
     }
 
     public JvmTask getTask(JvmContext context) throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a37e7f0a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java
index bed545e..a534cfa 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java
@@ -124,7 +124,7 @@ public class TestTaskCommit extends HadoopTestCase {
     }
 
     @Override
-    public void fatalError(TaskAttemptID taskId, String message)
+    public void fatalError(TaskAttemptID taskId, String message, boolean fastFail)
         throws IOException { }
 
     @Override


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[25/37] hadoop git commit: HDFS-13065. TestErasureCodingMultipleRacks#testSkewedRack3 is failing. Contributed by Gabor Bota.

Posted by ae...@apache.org.
HDFS-13065. TestErasureCodingMultipleRacks#testSkewedRack3 is failing. Contributed by Gabor Bota.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/6bc2f7f4
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/6bc2f7f4
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/6bc2f7f4

Branch: refs/heads/HDFS-7240
Commit: 6bc2f7f4b4b8d4c36e92764d4c975c17f9fdd63b
Parents: 443523f
Author: Xiao Chen <xi...@apache.org>
Authored: Sun Jan 28 22:11:08 2018 -0800
Committer: Xiao Chen <xi...@apache.org>
Committed: Sun Jan 28 22:12:05 2018 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/TestErasureCodingMultipleRacks.java   | 8 +++++++-
 1 file changed, 7 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6bc2f7f4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingMultipleRacks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingMultipleRacks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingMultipleRacks.java
index 0689665d..3e87253 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingMultipleRacks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingMultipleRacks.java
@@ -21,6 +21,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyRackFaultTolerant;
@@ -163,7 +164,8 @@ public class TestErasureCodingMultipleRacks {
     // Create enough extra DNs on the 2 racks to test even placement.
     // Desired placement is parityUnits replicas on the 2 racks, and 1 replica
     // on the rest of the racks (which only have 1 DN)
-    setupCluster(dataUnits + parityUnits * 4, dataUnits - parityUnits + 2,
+    int numRacks = dataUnits - parityUnits + 2;
+    setupCluster(dataUnits + parityUnits * 4, numRacks,
         dataUnits - parityUnits);
 
     final int filesize = ecPolicy.getNumDataUnits() * ecPolicy.getCellSize();
@@ -173,6 +175,10 @@ public class TestErasureCodingMultipleRacks {
       final Path path = new Path("/testfile" + i);
       LOG.info("Writing file " + path);
       DFSTestUtil.writeFile(dfs, path, contents);
+      ExtendedBlock extendedBlock = DFSTestUtil.getFirstBlock(dfs, path);
+      // Wait for replication to finish before testing
+      DFSTestUtil.waitForReplication(cluster, extendedBlock, numRacks,
+          ecPolicy.getNumDataUnits() + ecPolicy.getNumParityUnits(), 0);
       BlockLocation[] blocks =
           dfs.getFileBlockLocations(path, 0, Long.MAX_VALUE);
       assertEquals(ecPolicy.getNumDataUnits() + ecPolicy.getNumParityUnits(),


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[17/37] hadoop git commit: MapFile.fix creates a wrong index file in case of block-compressed data file. Contributed by Grigori Rybkine

Posted by ae...@apache.org.
MapFile.fix creates a wrong index file in case of block-compressed data file. Contributed by Grigori Rybkine


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/56872cff
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/56872cff
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/56872cff

Branch: refs/heads/HDFS-7240
Commit: 56872cff92f543bf77206a1324968559dceb7bc2
Parents: 8b5b045
Author: Chris Douglas <cd...@apache.org>
Authored: Fri Jan 26 09:06:48 2018 -0800
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Jan 26 09:18:30 2018 -0800

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/io/MapFile.java | 35 ++++++++++--
 .../java/org/apache/hadoop/io/TestMapFile.java  | 59 +++++++++++++++++++-
 2 files changed, 88 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/56872cff/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/MapFile.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/MapFile.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/MapFile.java
index d56822f..51db0b3 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/MapFile.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/MapFile.java
@@ -811,15 +811,40 @@ public class MapFile {
                                     (LongWritable.class));
     }
     try {
-      long pos = 0L;
+      /** What's the position (in bytes) we wrote when we got the last index */
+      long lastIndexPos = -1;
+      /**
+       * What was size when we last wrote an index. Set to MIN_VALUE to ensure
+       * that we have an index at position zero - midKey will throw an exception
+       * if this is not the case
+       */
+      long lastIndexKeyCount = Long.MIN_VALUE;
+      long pos = dataReader.getPosition();
       LongWritable position = new LongWritable();
+      long nextBlock = pos;
+      boolean blockCompressed = dataReader.isBlockCompressed();
       while(dataReader.next(key, value)) {
-        cnt++;
-        if (cnt % indexInterval == 0) {
+        if (blockCompressed) {
+          long curPos = dataReader.getPosition();
+          if (curPos > nextBlock) {
+            pos = nextBlock;                       // current block position
+            nextBlock = curPos;
+          }
+        }
+        // Follow the same logic as in
+        // {@link MapFile.Writer#append(WritableComparable, Writable)}
+        if (cnt >= lastIndexKeyCount + indexInterval && pos > lastIndexPos) {
           position.set(pos);
-          if (!dryrun) indexWriter.append(key, position);
+          if (!dryrun) {
+            indexWriter.append(key, position);
+          }
+          lastIndexPos = pos;
+          lastIndexKeyCount = cnt;
+        }
+        if (!blockCompressed) {
+          pos = dataReader.getPosition();         // next record position
         }
-        pos = dataReader.getPosition();
+        cnt++;
       }
     } catch(Throwable t) {
       // truncated data file. swallow it.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/56872cff/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestMapFile.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestMapFile.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestMapFile.java
index ff8df7c..7ec4227 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestMapFile.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestMapFile.java
@@ -485,6 +485,63 @@ public class TestMapFile {
       IOUtils.cleanup(null, writer);
     }
   }
+
+  /**
+   * test {@link MapFile#fix(FileSystem, Path, Class<? extends Writable>,
+   *                         Class<? extends Writable>, boolean, Configuration)}
+   * method in case of BLOCK compression
+   */
+  @Test
+  public void testFixBlockCompress() throws Exception {
+    final String indexLessMapFile = "testFixBlockCompress.mapfile";
+    final int compressBlocksize = 100;
+    final int indexInterval = 4;
+    final int noBlocks = 4;
+    final String value = "value-";
+    final int size = noBlocks * compressBlocksize / (4 + value.length());
+
+    conf.setInt("io.seqfile.compress.blocksize", compressBlocksize);
+    MapFile.Writer.setIndexInterval(conf, indexInterval);
+    FileSystem fs = FileSystem.getLocal(conf);
+    Path dir = new Path(TEST_DIR, indexLessMapFile);
+    MapFile.Writer writer = null;
+    MapFile.Reader reader = null;
+    try {
+      writer =
+          new MapFile.Writer(conf, dir,
+          MapFile.Writer.keyClass(IntWritable.class),
+          MapFile.Writer.valueClass(Text.class),
+          MapFile.Writer.compression(CompressionType.BLOCK));
+      for (int i = 0; i < size; i++) {
+        writer.append(new IntWritable(i), new Text(value + i));
+      }
+      writer.close();
+      Path index = new Path(dir, MapFile.INDEX_FILE_NAME);
+      fs.rename(index, index.suffix(".orig"));
+
+      assertEquals("No of valid MapFile entries wrong", size,
+                   MapFile.fix(fs, dir, IntWritable.class, Text.class,
+                               false, conf));
+      reader = new MapFile.Reader(dir, conf);
+      IntWritable key;
+      Text val = new Text();
+      int notFound = 0;
+      for (int i = 0; i < size; i++) {
+        key = new IntWritable(i);
+        if (null == reader.get(key, val)) {
+          notFound++;
+        }
+      }
+      assertEquals("With MapFile.fix-ed index, could not get entries # ",
+                   0, notFound);
+    } finally {
+      IOUtils.cleanupWithLogger(null, writer, reader);
+      if (fs.exists(dir)) {
+        fs.delete(dir, true);
+      }
+    }
+  }
+
   /**
    * test all available constructor for {@code MapFile.Writer}
    */
@@ -619,7 +676,7 @@ public class TestMapFile {
     } catch (Exception ex) {
       fail("testMainMethodMapFile error !!!");
     } finally {
-      IOUtils.cleanup(null, writer);
+      IOUtils.cleanupWithLogger(null, writer);
     }
   }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[05/37] hadoop git commit: YARN-7810. Update TestDockerContainerRuntime to test with current user credential. (contributed by Shane Kumpf)

Posted by ae...@apache.org.
YARN-7810.  Update TestDockerContainerRuntime to test with current user credential.
            (contributed by Shane Kumpf)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/59828be1
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/59828be1
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/59828be1

Branch: refs/heads/HDFS-7240
Commit: 59828be1978ec942dda38774a1d9f741efa96f71
Parents: 051ad9d
Author: Eric Yang <ey...@apache.org>
Authored: Wed Jan 24 21:24:19 2018 -0500
Committer: Eric Yang <ey...@apache.org>
Committed: Wed Jan 24 21:24:19 2018 -0500

----------------------------------------------------------------------
 .../runtime/TestDockerContainerRuntime.java     | 228 ++++++++++---------
 1 file changed, 126 insertions(+), 102 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/59828be1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java
index da1d1eb..a825828 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java
@@ -117,7 +117,8 @@ public class TestDockerContainerRuntime {
   private HashMap<String, String> env;
   private String image;
   private String uidGidPair;
-  private String runAsUser;
+  private String runAsUser = System.getProperty("user.name");
+  private String[] groups = {};
   private String user;
   private String appId;
   private String containerIdStr = containerId;
@@ -166,8 +167,37 @@ public class TestDockerContainerRuntime {
     when(context.getEnvironment()).thenReturn(env);
     when(container.getUser()).thenReturn(submittingUser);
 
-    uidGidPair = "";
-    runAsUser = "run_as_user";
+    // Get the running user's uid and gid for remap
+    String uid = "";
+    String gid = "";
+    Shell.ShellCommandExecutor shexec1 = new Shell.ShellCommandExecutor(
+        new String[]{"id", "-u", runAsUser});
+    Shell.ShellCommandExecutor shexec2 = new Shell.ShellCommandExecutor(
+        new String[]{"id", "-g", runAsUser});
+    Shell.ShellCommandExecutor shexec3 = new Shell.ShellCommandExecutor(
+        new String[]{"id", "-G", runAsUser});
+    try {
+      shexec1.execute();
+      // get rid of newline at the end
+      uid = shexec1.getOutput().replaceAll("\n$", "");
+    } catch (Exception e) {
+      LOG.info("Could not run id -u command: " + e);
+    }
+    try {
+      shexec2.execute();
+      // get rid of newline at the end
+      gid = shexec2.getOutput().replaceAll("\n$", "");
+    } catch (Exception e) {
+      LOG.info("Could not run id -g command: " + e);
+    }
+    try {
+      shexec3.execute();
+      groups = shexec3.getOutput().replace("\n", " ").split(" ");
+    } catch (Exception e) {
+      LOG.info("Could not run id -G command: " + e);
+    }
+    uidGidPair = uid + ":" + gid;
+
     user = "user";
     appId = "app_id";
     containerIdStr = containerId;
@@ -336,7 +366,7 @@ public class TestDockerContainerRuntime {
     List<String> dockerCommands = Files.readAllLines(Paths.get
             (dockerCommandFile), Charset.forName("UTF-8"));
 
-    int expected = 13;
+    int expected = 14;
     int counter = 0;
     Assert.assertEquals(expected, dockerCommands.size());
     Assert.assertEquals("[docker-command-execution]",
@@ -346,6 +376,8 @@ public class TestDockerContainerRuntime {
     Assert.assertEquals("  cap-drop=ALL", dockerCommands.get(counter++));
     Assert.assertEquals("  detach=true", dockerCommands.get(counter++));
     Assert.assertEquals("  docker-command=run", dockerCommands.get(counter++));
+    Assert.assertEquals("  group-add=" + String.join(",", groups),
+        dockerCommands.get(counter++));
     Assert.assertEquals("  hostname=ctr-id", dockerCommands.get(counter++));
     Assert
         .assertEquals("  image=busybox:latest", dockerCommands.get(counter++));
@@ -361,7 +393,7 @@ public class TestDockerContainerRuntime {
             + "/test_container_log_dir:/test_container_log_dir,"
             + "/test_user_local_dir:/test_user_local_dir",
         dockerCommands.get(counter++));
-    Assert.assertEquals("  user=run_as_user", dockerCommands.get(counter++));
+    Assert.assertEquals("  user=" + uidGidPair, dockerCommands.get(counter++));
     Assert.assertEquals("  workdir=/test_container_work_dir",
         dockerCommands.get(counter++));
   }
@@ -372,13 +404,6 @@ public class TestDockerContainerRuntime {
       IOException {
     conf.setBoolean(YarnConfiguration.NM_DOCKER_ENABLE_USER_REMAPPING,
         true);
-    Shell.ShellCommandExecutor shexec = new Shell.ShellCommandExecutor(
-        new String[]{"whoami"});
-    shexec.execute();
-    // get rid of newline at the end
-    runAsUser = shexec.getOutput().replaceAll("\n$", "");
-    builder.setExecutionAttribute(RUN_AS_USER, runAsUser);
-
     DockerLinuxContainerRuntime runtime = new DockerLinuxContainerRuntime(
         mockExecutor, mockCGroupsHandler);
     runtime.initialize(conf, null);
@@ -388,37 +413,6 @@ public class TestDockerContainerRuntime {
     List<String> args = op.getArguments();
     String dockerCommandFile = args.get(11);
 
-    String uid = "";
-    String gid = "";
-    String[] groups = {};
-    Shell.ShellCommandExecutor shexec1 = new Shell.ShellCommandExecutor(
-        new String[]{"id", "-u", runAsUser});
-    Shell.ShellCommandExecutor shexec2 = new Shell.ShellCommandExecutor(
-        new String[]{"id", "-g", runAsUser});
-    Shell.ShellCommandExecutor shexec3 = new Shell.ShellCommandExecutor(
-        new String[]{"id", "-G", runAsUser});
-    try {
-      shexec1.execute();
-      // get rid of newline at the end
-      uid = shexec1.getOutput().replaceAll("\n$", "");
-    } catch (Exception e) {
-      LOG.info("Could not run id -u command: " + e);
-    }
-    try {
-      shexec2.execute();
-      // get rid of newline at the end
-      gid = shexec2.getOutput().replaceAll("\n$", "");
-    } catch (Exception e) {
-      LOG.info("Could not run id -g command: " + e);
-    }
-    try {
-      shexec3.execute();
-      groups = shexec3.getOutput().replace("\n", " ").split(" ");
-    } catch (Exception e) {
-      LOG.info("Could not run id -G command: " + e);
-    }
-    uidGidPair = uid + ":" + gid;
-
     List<String> dockerCommands = Files.readAllLines(
         Paths.get(dockerCommandFile), Charset.forName("UTF-8"));
 
@@ -540,7 +534,7 @@ public class TestDockerContainerRuntime {
     //This is the expected docker invocation for this case
     List<String> dockerCommands = Files
         .readAllLines(Paths.get(dockerCommandFile), Charset.forName("UTF-8"));
-    int expected = 13;
+    int expected = 14;
     int counter = 0;
     Assert.assertEquals(expected, dockerCommands.size());
     Assert.assertEquals("[docker-command-execution]",
@@ -550,6 +544,8 @@ public class TestDockerContainerRuntime {
     Assert.assertEquals("  cap-drop=ALL", dockerCommands.get(counter++));
     Assert.assertEquals("  detach=true", dockerCommands.get(counter++));
     Assert.assertEquals("  docker-command=run", dockerCommands.get(counter++));
+    Assert.assertEquals("  group-add=" + String.join(",", groups),
+        dockerCommands.get(counter++));
     Assert.assertEquals("  hostname=test.hostname",
         dockerCommands.get(counter++));
     Assert
@@ -567,7 +563,7 @@ public class TestDockerContainerRuntime {
             + "/test_container_log_dir:/test_container_log_dir,"
             + "/test_user_local_dir:/test_user_local_dir",
         dockerCommands.get(counter++));
-    Assert.assertEquals("  user=run_as_user", dockerCommands.get(counter++));
+    Assert.assertEquals("  user=" + uidGidPair, dockerCommands.get(counter++));
     Assert.assertEquals("  workdir=/test_container_work_dir",
         dockerCommands.get(counter++));
   }
@@ -606,7 +602,7 @@ public class TestDockerContainerRuntime {
     List<String> dockerCommands = Files
         .readAllLines(Paths.get(dockerCommandFile), Charset.forName("UTF-8"));
 
-    int expected = 13;
+    int expected = 14;
     int counter = 0;
     Assert.assertEquals(expected, dockerCommands.size());
     Assert.assertEquals("[docker-command-execution]",
@@ -616,6 +612,8 @@ public class TestDockerContainerRuntime {
     Assert.assertEquals("  cap-drop=ALL", dockerCommands.get(counter++));
     Assert.assertEquals("  detach=true", dockerCommands.get(counter++));
     Assert.assertEquals("  docker-command=run", dockerCommands.get(counter++));
+    Assert.assertEquals("  group-add=" + String.join(",", groups),
+        dockerCommands.get(counter++));
     Assert.assertEquals("  hostname=ctr-id", dockerCommands.get(counter++));
     Assert
         .assertEquals("  image=busybox:latest", dockerCommands.get(counter++));
@@ -631,7 +629,7 @@ public class TestDockerContainerRuntime {
             + "/test_container_log_dir:/test_container_log_dir,"
             + "/test_user_local_dir:/test_user_local_dir",
         dockerCommands.get(counter++));
-    Assert.assertEquals("  user=run_as_user", dockerCommands.get(counter++));
+    Assert.assertEquals("  user=" + uidGidPair, dockerCommands.get(counter++));
     Assert.assertEquals("  workdir=/test_container_work_dir",
         dockerCommands.get(counter++));
 
@@ -659,6 +657,8 @@ public class TestDockerContainerRuntime {
     Assert.assertEquals("  cap-drop=ALL", dockerCommands.get(counter++));
     Assert.assertEquals("  detach=true", dockerCommands.get(counter++));
     Assert.assertEquals("  docker-command=run", dockerCommands.get(counter++));
+    Assert.assertEquals("  group-add=" + String.join(",", groups),
+        dockerCommands.get(counter++));
     Assert.assertEquals("  hostname=ctr-id", dockerCommands.get(counter++));
     Assert
         .assertEquals("  image=busybox:latest", dockerCommands.get(counter++));
@@ -675,7 +675,7 @@ public class TestDockerContainerRuntime {
             + "/test_container_log_dir:/test_container_log_dir,"
             + "/test_user_local_dir:/test_user_local_dir",
         dockerCommands.get(counter++));
-    Assert.assertEquals("  user=run_as_user", dockerCommands.get(counter++));
+    Assert.assertEquals("  user=" + uidGidPair, dockerCommands.get(counter++));
     Assert.assertEquals("  workdir=/test_container_work_dir",
         dockerCommands.get(counter++));
 
@@ -712,7 +712,7 @@ public class TestDockerContainerRuntime {
     List<String> dockerCommands = Files.readAllLines(Paths.get
         (dockerCommandFile), Charset.forName("UTF-8"));
 
-    int expected = 13;
+    int expected = 14;
     Assert.assertEquals(expected, dockerCommands.size());
 
     String command = dockerCommands.get(0);
@@ -763,7 +763,7 @@ public class TestDockerContainerRuntime {
     List<String> dockerCommands = Files.readAllLines(
         Paths.get(dockerCommandFile), Charset.forName("UTF-8"));
 
-    int expected = 14;
+    int expected = 15;
     int counter = 0;
     Assert.assertEquals(expected, dockerCommands.size());
     Assert.assertEquals("[docker-command-execution]",
@@ -773,6 +773,8 @@ public class TestDockerContainerRuntime {
     Assert.assertEquals("  cap-drop=ALL", dockerCommands.get(counter++));
     Assert.assertEquals("  detach=true", dockerCommands.get(counter++));
     Assert.assertEquals("  docker-command=run", dockerCommands.get(counter++));
+    Assert.assertEquals("  group-add=" + String.join(",", groups),
+        dockerCommands.get(counter++));
     Assert.assertEquals("  hostname=ctr-id", dockerCommands.get(counter++));
     Assert
         .assertEquals("  image=busybox:latest", dockerCommands.get(counter++));
@@ -789,7 +791,7 @@ public class TestDockerContainerRuntime {
             + "/test_container_log_dir:/test_container_log_dir,"
             + "/test_user_local_dir:/test_user_local_dir",
         dockerCommands.get(counter++));
-    Assert.assertEquals("  user=run_as_user", dockerCommands.get(counter++));
+    Assert.assertEquals("  user=" + uidGidPair, dockerCommands.get(counter++));
     Assert.assertEquals("  workdir=/test_container_work_dir",
         dockerCommands.get(counter++));
   }
@@ -813,7 +815,7 @@ public class TestDockerContainerRuntime {
     List<String> dockerCommands = Files.readAllLines(
         Paths.get(dockerCommandFile), Charset.forName("UTF-8"));
 
-    int expected = 13;
+    int expected = 14;
     Assert.assertEquals(expected, dockerCommands.size());
 
     String command = dockerCommands.get(0);
@@ -922,7 +924,7 @@ public class TestDockerContainerRuntime {
     List<String> dockerCommands = Files.readAllLines(Paths.get
         (dockerCommandFile), Charset.forName("UTF-8"));
 
-    int expected = 14;
+    int expected = 15;
     int counter = 0;
     Assert.assertEquals(expected, dockerCommands.size());
     Assert.assertEquals("[docker-command-execution]",
@@ -932,6 +934,8 @@ public class TestDockerContainerRuntime {
     Assert.assertEquals("  cap-drop=ALL", dockerCommands.get(counter++));
     Assert.assertEquals("  detach=true", dockerCommands.get(counter++));
     Assert.assertEquals("  docker-command=run", dockerCommands.get(counter++));
+    Assert.assertEquals("  group-add=" + String.join(",", groups),
+        dockerCommands.get(counter++));
     Assert.assertEquals("  hostname=ctr-id", dockerCommands.get(counter++));
     Assert
         .assertEquals("  image=busybox:latest", dockerCommands.get(counter++));
@@ -948,7 +952,7 @@ public class TestDockerContainerRuntime {
             + "/test_container_log_dir:/test_container_log_dir,"
             + "/test_user_local_dir:/test_user_local_dir",
         dockerCommands.get(counter++));
-    Assert.assertEquals("  user=run_as_user", dockerCommands.get(counter++));
+    Assert.assertEquals("  user=" + uidGidPair, dockerCommands.get(counter++));
     Assert.assertEquals("  workdir=/test_container_work_dir",
         dockerCommands.get(counter++));
   }
@@ -1039,33 +1043,39 @@ public class TestDockerContainerRuntime {
     List<String> dockerCommands = Files.readAllLines(Paths.get
         (dockerCommandFile), Charset.forName("UTF-8"));
 
-    Assert.assertEquals(14, dockerCommands.size());
-    Assert.assertEquals("[docker-command-execution]", dockerCommands.get(0));
+    int expected = 15;
+    int counter = 0;
+    Assert.assertEquals(expected, dockerCommands.size());
+    Assert.assertEquals("[docker-command-execution]",
+        dockerCommands.get(counter++));
     Assert.assertEquals("  cap-add=SYS_CHROOT,NET_BIND_SERVICE",
-        dockerCommands.get(1));
-    Assert.assertEquals("  cap-drop=ALL", dockerCommands.get(2));
-    Assert.assertEquals("  detach=true", dockerCommands.get(3));
-    Assert.assertEquals("  docker-command=run", dockerCommands.get(4));
-    Assert.assertEquals("  hostname=ctr-id", dockerCommands.get(5));
-    Assert.assertEquals("  image=busybox:latest", dockerCommands.get(6));
+        dockerCommands.get(counter++));
+    Assert.assertEquals("  cap-drop=ALL", dockerCommands.get(counter++));
+    Assert.assertEquals("  detach=true", dockerCommands.get(counter++));
+    Assert.assertEquals("  docker-command=run", dockerCommands.get(counter++));
+    Assert.assertEquals("  group-add=" + String.join(",", groups),
+        dockerCommands.get(counter++));
+    Assert.assertEquals("  hostname=ctr-id", dockerCommands.get(counter++));
+    Assert.assertEquals("  image=busybox:latest",
+        dockerCommands.get(counter++));
     Assert.assertEquals(
         "  launch-command=bash,/test_container_work_dir/launch_container.sh",
-        dockerCommands.get(7));
-    Assert.assertEquals("  name=container_id", dockerCommands.get(8));
-    Assert.assertEquals("  net=host", dockerCommands.get(9));
+        dockerCommands.get(counter++));
+    Assert.assertEquals("  name=container_id", dockerCommands.get(counter++));
+    Assert.assertEquals("  net=host", dockerCommands.get(counter++));
     Assert.assertEquals(
         "  ro-mounts=/test_local_dir/test_resource_file:test_mount",
-        dockerCommands.get(10));
+        dockerCommands.get(counter++));
     Assert.assertEquals(
         "  rw-mounts=/test_container_local_dir:/test_container_local_dir,"
             + "/test_filecache_dir:/test_filecache_dir,"
             + "/test_container_work_dir:/test_container_work_dir,"
             + "/test_container_log_dir:/test_container_log_dir,"
             + "/test_user_local_dir:/test_user_local_dir",
-        dockerCommands.get(11));
-    Assert.assertEquals("  user=run_as_user", dockerCommands.get(12));
+        dockerCommands.get(counter++));
+    Assert.assertEquals("  user=" + uidGidPair, dockerCommands.get(counter++));
     Assert.assertEquals("  workdir=/test_container_work_dir",
-        dockerCommands.get(13));
+        dockerCommands.get(counter++));
   }
 
   @Test
@@ -1109,34 +1119,40 @@ public class TestDockerContainerRuntime {
     List<String> dockerCommands = Files.readAllLines(Paths.get
         (dockerCommandFile), Charset.forName("UTF-8"));
 
-    Assert.assertEquals(14, dockerCommands.size());
-    Assert.assertEquals("[docker-command-execution]", dockerCommands.get(0));
+    int expected = 15;
+    int counter = 0;
+    Assert.assertEquals(expected, dockerCommands.size());
+    Assert.assertEquals("[docker-command-execution]",
+        dockerCommands.get(counter++));
     Assert.assertEquals("  cap-add=SYS_CHROOT,NET_BIND_SERVICE",
-        dockerCommands.get(1));
-    Assert.assertEquals("  cap-drop=ALL", dockerCommands.get(2));
-    Assert.assertEquals("  detach=true", dockerCommands.get(3));
-    Assert.assertEquals("  docker-command=run", dockerCommands.get(4));
-    Assert.assertEquals("  hostname=ctr-id", dockerCommands.get(5));
-    Assert.assertEquals("  image=busybox:latest", dockerCommands.get(6));
+        dockerCommands.get(counter++));
+    Assert.assertEquals("  cap-drop=ALL", dockerCommands.get(counter++));
+    Assert.assertEquals("  detach=true", dockerCommands.get(counter++));
+    Assert.assertEquals("  docker-command=run", dockerCommands.get(counter++));
+    Assert.assertEquals("  group-add=" + String.join(",", groups),
+        dockerCommands.get(counter++));
+    Assert.assertEquals("  hostname=ctr-id", dockerCommands.get(counter++));
+    Assert.assertEquals("  image=busybox:latest",
+        dockerCommands.get(counter++));
     Assert.assertEquals(
         "  launch-command=bash,/test_container_work_dir/launch_container.sh",
-        dockerCommands.get(7));
-    Assert.assertEquals("  name=container_id", dockerCommands.get(8));
-    Assert.assertEquals("  net=host", dockerCommands.get(9));
+        dockerCommands.get(counter++));
+    Assert.assertEquals("  name=container_id", dockerCommands.get(counter++));
+    Assert.assertEquals("  net=host", dockerCommands.get(counter++));
     Assert.assertEquals(
         "  ro-mounts=/test_local_dir/test_resource_file:test_mount1,"
             + "/test_local_dir/test_resource_file:test_mount2",
-        dockerCommands.get(10));
+        dockerCommands.get(counter++));
     Assert.assertEquals(
         "  rw-mounts=/test_container_local_dir:/test_container_local_dir,"
             + "/test_filecache_dir:/test_filecache_dir,"
             + "/test_container_work_dir:/test_container_work_dir,"
             + "/test_container_log_dir:/test_container_log_dir,"
             + "/test_user_local_dir:/test_user_local_dir",
-        dockerCommands.get(11));
-    Assert.assertEquals("  user=run_as_user", dockerCommands.get(12));
+        dockerCommands.get(counter++));
+    Assert.assertEquals("  user=" + uidGidPair, dockerCommands.get(counter++));
     Assert.assertEquals("  workdir=/test_container_work_dir",
-        dockerCommands.get(13));
+        dockerCommands.get(counter++));
 
   }
 
@@ -1160,22 +1176,28 @@ public class TestDockerContainerRuntime {
     List<String> dockerCommands = Files.readAllLines(
         Paths.get(dockerCommandFile), Charset.forName("UTF-8"));
 
-    Assert.assertEquals(14, dockerCommands.size());
-    Assert.assertEquals("[docker-command-execution]", dockerCommands.get(0));
+    int expected = 15;
+    int counter = 0;
+    Assert.assertEquals(expected, dockerCommands.size());
+    Assert.assertEquals("[docker-command-execution]",
+        dockerCommands.get(counter++));
     Assert.assertEquals("  cap-add=SYS_CHROOT,NET_BIND_SERVICE",
-        dockerCommands.get(1));
-    Assert.assertEquals("  cap-drop=ALL", dockerCommands.get(2));
-    Assert.assertEquals("  detach=true", dockerCommands.get(3));
-    Assert.assertEquals("  docker-command=run", dockerCommands.get(4));
-    Assert.assertEquals("  hostname=ctr-id", dockerCommands.get(5));
-    Assert.assertEquals("  image=busybox:latest", dockerCommands.get(6));
+        dockerCommands.get(counter++));
+    Assert.assertEquals("  cap-drop=ALL", dockerCommands.get(counter++));
+    Assert.assertEquals("  detach=true", dockerCommands.get(counter++));
+    Assert.assertEquals("  docker-command=run", dockerCommands.get(counter++));
+    Assert.assertEquals("  group-add=" + String.join(",", groups),
+        dockerCommands.get(counter++));
+    Assert.assertEquals("  hostname=ctr-id", dockerCommands.get(counter++));
+    Assert.assertEquals("  image=busybox:latest",
+        dockerCommands.get(counter++));
     Assert.assertEquals(
         "  launch-command=bash,/test_container_work_dir/launch_container.sh",
-        dockerCommands.get(7));
-    Assert.assertEquals("  name=container_id", dockerCommands.get(8));
-    Assert.assertEquals("  net=host", dockerCommands.get(9));
+        dockerCommands.get(counter++));
+    Assert.assertEquals("  name=container_id", dockerCommands.get(counter++));
+    Assert.assertEquals("  net=host", dockerCommands.get(counter++));
     Assert.assertEquals("  ro-mounts=/tmp/foo:/tmp/foo",
-        dockerCommands.get(10));
+        dockerCommands.get(counter++));
     Assert.assertEquals(
         "  rw-mounts=/test_container_local_dir:/test_container_local_dir,"
             + "/test_filecache_dir:/test_filecache_dir,"
@@ -1183,10 +1205,10 @@ public class TestDockerContainerRuntime {
             + "/test_container_log_dir:/test_container_log_dir,"
             + "/test_user_local_dir:/test_user_local_dir,"
             + "/tmp/bar:/tmp/bar",
-        dockerCommands.get(11));
-    Assert.assertEquals("  user=run_as_user", dockerCommands.get(12));
+        dockerCommands.get(counter++));
+    Assert.assertEquals("  user=" + uidGidPair, dockerCommands.get(counter++));
     Assert.assertEquals("  workdir=/test_container_work_dir",
-        dockerCommands.get(13));
+        dockerCommands.get(counter++));
   }
 
   @Test
@@ -1265,7 +1287,7 @@ public class TestDockerContainerRuntime {
     PrivilegedOperation op = capturePrivilegedOperation();
     Assert.assertEquals(op.getOperationType(),
         PrivilegedOperation.OperationType.SIGNAL_CONTAINER);
-    Assert.assertEquals("run_as_user", op.getArguments().get(0));
+    Assert.assertEquals(runAsUser, op.getArguments().get(0));
     Assert.assertEquals("user", op.getArguments().get(1));
     Assert.assertEquals("2", op.getArguments().get(2));
     Assert.assertEquals("1234", op.getArguments().get(3));
@@ -1639,7 +1661,7 @@ public class TestDockerContainerRuntime {
     List<String> dockerCommands = Files.readAllLines(Paths.get
         (dockerCommandFile), Charset.forName("UTF-8"));
 
-    int expected = 15;
+    int expected = 16;
     int counter = 0;
     Assert.assertEquals(expected, dockerCommands.size());
     Assert.assertEquals("[docker-command-execution]",
@@ -1649,6 +1671,8 @@ public class TestDockerContainerRuntime {
     Assert.assertEquals("  cap-drop=ALL", dockerCommands.get(counter++));
     Assert.assertEquals("  detach=true", dockerCommands.get(counter++));
     Assert.assertEquals("  docker-command=run", dockerCommands.get(counter++));
+    Assert.assertEquals("  group-add=" + String.join(",", groups),
+        dockerCommands.get(counter++));
     Assert.assertEquals("  hostname=ctr-id", dockerCommands.get(counter++));
     Assert
         .assertEquals("  image=busybox:latest", dockerCommands.get(counter++));
@@ -1666,7 +1690,7 @@ public class TestDockerContainerRuntime {
             + "/test_container_log_dir:/test_container_log_dir,"
             + "/test_user_local_dir:/test_user_local_dir",
         dockerCommands.get(counter++));
-    Assert.assertEquals("  user=run_as_user", dockerCommands.get(counter++));
+    Assert.assertEquals("  user=" + uidGidPair, dockerCommands.get(counter++));
 
     // Verify volume-driver is set to expected value.
     Assert.assertEquals("  volume-driver=driver-1",


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[18/37] hadoop git commit: YARN-7797. Docker host network can not obtain IP address for RegistryDNS. Contributed by Eric Yang

Posted by ae...@apache.org.
YARN-7797. Docker host network can not obtain IP address for RegistryDNS. Contributed by Eric Yang


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/f2fa736f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/f2fa736f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/f2fa736f

Branch: refs/heads/HDFS-7240
Commit: f2fa736f0ab139b5251d115fd75b833d1d7d1dcd
Parents: 56872cf
Author: Billie Rinaldi <bi...@apache.org>
Authored: Fri Jan 26 09:32:23 2018 -0800
Committer: Billie Rinaldi <bi...@apache.org>
Committed: Fri Jan 26 09:32:23 2018 -0800

----------------------------------------------------------------------
 .../runtime/DockerLinuxContainerRuntime.java    | 28 ++++++++++++++++++++
 1 file changed, 28 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2fa736f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
index 2868dea..f3ce73d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
@@ -58,6 +58,8 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.runtime.Contai
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.runtime.ContainerRuntimeConstants;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.runtime.ContainerRuntimeContext;
 
+import java.net.InetAddress;
+import java.net.UnknownHostException;
 import java.nio.file.Files;
 import java.nio.file.Paths;
 import java.util.ArrayList;
@@ -1013,6 +1015,32 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
       }
       String ips = output.substring(0, index).trim();
       String host = output.substring(index+1).trim();
+      if (ips.equals("")) {
+        String network;
+        try {
+          network = container.getLaunchContext().getEnvironment()
+              .get("YARN_CONTAINER_RUNTIME_DOCKER_CONTAINER_NETWORK");
+          if (network == null || network.isEmpty()) {
+            network = defaultNetwork;
+          }
+        } catch (NullPointerException e) {
+          network = defaultNetwork;
+        }
+        boolean useHostNetwork = network.equalsIgnoreCase("host");
+        if (useHostNetwork) {
+          // Report back node manager IP in the event where docker
+          // inspect reports no IP address.  This is for bridging a gap for
+          // docker environment to run with host network.
+          InetAddress address;
+          try {
+            address = InetAddress.getLocalHost();
+            ips = address.getHostAddress();
+          } catch (UnknownHostException e) {
+            LOG.error("Can not determine IP for container:"
+                + containerId);
+          }
+        }
+      }
       String[] ipAndHost = new String[2];
       ipAndHost[0] = ips;
       ipAndHost[1] = host;


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org