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 as...@apache.org on 2017/09/11 06:48:26 UTC

[01/50] [abbrv] hadoop git commit: HADOOP-14839. DistCp log output should contain copied and deleted files and directories. Contributed by Yiqun Lin. [Forced Update!]

Repository: hadoop
Updated Branches:
  refs/heads/YARN-5972 10eeb8b50 -> 026f877b9 (forced update)


HADOOP-14839. DistCp log output should contain copied and deleted files and directories. Contributed by Yiqun Lin.


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

Branch: refs/heads/YARN-5972
Commit: 63720ef574df6d79a9bab0628edb915d1e3a7c30
Parents: 50506e9
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Tue Sep 5 21:45:30 2017 -0700
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Tue Sep 5 23:34:55 2017 -0700

----------------------------------------------------------------------
 .../apache/hadoop/tools/DistCpConstants.java    |  1 +
 .../apache/hadoop/tools/DistCpOptionSwitch.java |  7 ++
 .../org/apache/hadoop/tools/DistCpOptions.java  | 22 +++++
 .../org/apache/hadoop/tools/OptionsParser.java  |  4 +-
 .../apache/hadoop/tools/mapred/CopyMapper.java  | 32 +++++--
 .../src/site/markdown/DistCp.md.vm              |  1 +
 .../apache/hadoop/tools/TestDistCpOptions.java  | 21 ++++-
 .../hadoop/tools/mapred/TestCopyMapper.java     | 99 +++++++++++++++++++-
 .../hadoop/tools/DistCp_Counter.properties      |  1 +
 9 files changed, 177 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/63720ef5/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpConstants.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpConstants.java
index 8991e09..0bae5d5 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpConstants.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpConstants.java
@@ -45,6 +45,7 @@ public class DistCpConstants {
   public static final String CONF_LABEL_ATOMIC_COPY = "distcp.atomic.copy";
   public static final String CONF_LABEL_WORK_PATH = "distcp.work.path";
   public static final String CONF_LABEL_LOG_PATH = "distcp.log.path";
+  public static final String CONF_LABEL_VERBOSE_LOG = "distcp.verbose.log";
   public static final String CONF_LABEL_IGNORE_FAILURES = "distcp.ignore.failures";
   public static final String CONF_LABEL_PRESERVE_STATUS = "distcp.preserve.status";
   public static final String CONF_LABEL_PRESERVE_RAWXATTRS =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63720ef5/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptionSwitch.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptionSwitch.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptionSwitch.java
index 016172e0..e1c824e 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptionSwitch.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptionSwitch.java
@@ -119,6 +119,13 @@ public enum DistCpOptionSwitch {
       new Option("log", true, "Folder on DFS where distcp execution logs are saved")),
 
   /**
+   * Log additional info (path, size) in the SKIP/COPY log.
+   */
+  VERBOSE_LOG(DistCpConstants.CONF_LABEL_VERBOSE_LOG,
+      new Option("v", false,
+          "Log additional info (path, size) in the SKIP/COPY log")),
+
+  /**
    * Copy strategy is use. This could be dynamic or uniform size etc.
    * DistCp would use an appropriate input format based on this.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63720ef5/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptions.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptions.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptions.java
index af6cb8b..ece1a94 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptions.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptions.java
@@ -101,6 +101,9 @@ public final class DistCpOptions {
   // content at their s1, if src is not the same as tgt.
   private final boolean useRdiff;
 
+  /** Whether to log additional info (path, size) in the SKIP/COPY log. */
+  private final boolean verboseLog;
+
   // For both -diff and -rdiff, given the example command line switches, two
   // steps are taken:
   //   1. Sync Step. This step does renaming/deletion ops in the snapshot diff,
@@ -204,6 +207,7 @@ public final class DistCpOptions {
     this.blocksPerChunk = builder.blocksPerChunk;
 
     this.copyBufferSize = builder.copyBufferSize;
+    this.verboseLog = builder.verboseLog;
   }
 
   public Path getSourceFileListing() {
@@ -323,6 +327,10 @@ public final class DistCpOptions {
     return copyBufferSize;
   }
 
+  public boolean shouldVerboseLog() {
+    return verboseLog;
+  }
+
   /**
    * Add options to configuration. These will be used in the Mapper/committer
    *
@@ -361,6 +369,8 @@ public final class DistCpOptions {
         String.valueOf(blocksPerChunk));
     DistCpOptionSwitch.addToConf(conf, DistCpOptionSwitch.COPY_BUFFER_SIZE,
         String.valueOf(copyBufferSize));
+    DistCpOptionSwitch.addToConf(conf, DistCpOptionSwitch.VERBOSE_LOG,
+        String.valueOf(verboseLog));
   }
 
   /**
@@ -396,6 +406,7 @@ public final class DistCpOptions {
         ", filtersFile='" + filtersFile + '\'' +
         ", blocksPerChunk=" + blocksPerChunk +
         ", copyBufferSize=" + copyBufferSize +
+        ", verboseLog=" + verboseLog +
         '}';
   }
 
@@ -420,6 +431,7 @@ public final class DistCpOptions {
     private boolean append = false;
     private boolean skipCRC = false;
     private boolean blocking = true;
+    private boolean verboseLog = false;
 
     private boolean useDiff = false;
     private boolean useRdiff = false;
@@ -552,6 +564,11 @@ public final class DistCpOptions {
         throw new IllegalArgumentException(
             "-diff and -rdiff are mutually exclusive");
       }
+
+      if (verboseLog && logPath == null) {
+        throw new IllegalArgumentException(
+            "-v is valid only with -log option");
+      }
     }
 
     @VisibleForTesting
@@ -685,6 +702,11 @@ public final class DistCpOptions {
               : DistCpConstants.COPY_BUFFER_SIZE_DEFAULT;
       return this;
     }
+
+    public Builder withVerboseLog(boolean newVerboseLog) {
+      this.verboseLog = newVerboseLog;
+      return this;
+    }
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63720ef5/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/OptionsParser.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/OptionsParser.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/OptionsParser.java
index 2bfaccf..606ed32 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/OptionsParser.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/OptionsParser.java
@@ -111,7 +111,9 @@ public class OptionsParser {
         .withCRC(
             command.hasOption(DistCpOptionSwitch.SKIP_CRC.getSwitch()))
         .withBlocking(
-            !command.hasOption(DistCpOptionSwitch.BLOCKING.getSwitch()));
+            !command.hasOption(DistCpOptionSwitch.BLOCKING.getSwitch()))
+        .withVerboseLog(
+            command.hasOption(DistCpOptionSwitch.VERBOSE_LOG.getSwitch()));
 
     if (command.hasOption(DistCpOptionSwitch.DIFF.getSwitch())) {
       String[] snapshots = getVals(command,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63720ef5/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyMapper.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyMapper.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyMapper.java
index d6b3ba8..faa4aa2 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyMapper.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyMapper.java
@@ -54,6 +54,7 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
    */
   public static enum Counter {
     COPY,         // Number of files received by the mapper for copy.
+    DIR_COPY,     // Number of directories received by the mapper for copy.
     SKIP,         // Number of files skipped.
     FAIL,         // Number of files that failed to be copied.
     BYTESCOPIED,  // Number of bytes actually copied by the copy-mapper, total.
@@ -82,6 +83,7 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
   private boolean skipCrc = false;
   private boolean overWrite = false;
   private boolean append = false;
+  private boolean verboseLog = false;
   private EnumSet<FileAttribute> preserve = EnumSet.noneOf(FileAttribute.class);
 
   private FileSystem targetFS = null;
@@ -105,6 +107,8 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
     skipCrc = conf.getBoolean(DistCpOptionSwitch.SKIP_CRC.getConfigLabel(), false);
     overWrite = conf.getBoolean(DistCpOptionSwitch.OVERWRITE.getConfigLabel(), false);
     append = conf.getBoolean(DistCpOptionSwitch.APPEND.getConfigLabel(), false);
+    verboseLog = conf.getBoolean(
+        DistCpOptionSwitch.VERBOSE_LOG.getConfigLabel(), false);
     preserve = DistCpUtils.unpackAttributes(conf.get(DistCpOptionSwitch.
         PRESERVE_STATUS.getConfigLabel()));
 
@@ -196,6 +200,13 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
         updateSkipCounters(context, sourceCurrStatus);
         context.write(null, new Text("SKIP: " + sourceCurrStatus.getPath()));
 
+        if (verboseLog) {
+          context.write(null,
+              new Text("FILE_SKIPPED: source=" + sourceFileStatus.getPath()
+              + ", size=" + sourceFileStatus.getLen() + " --> "
+              + "target=" + target + ", size=" + (targetStatus == null ?
+                  0 : targetStatus.getLen())));
+        }
       } else {
         if (sourceCurrStatus.isSplit()) {
           tmpTarget = DistCpUtils.getSplitChunkPath(target, sourceCurrStatus);
@@ -203,8 +214,8 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
         if (LOG.isDebugEnabled()) {
           LOG.debug("copying " + sourceCurrStatus + " " + tmpTarget);
         }
-        copyFileWithRetry(description, sourceCurrStatus, tmpTarget, context,
-            action, fileAttributes);
+        copyFileWithRetry(description, sourceCurrStatus, tmpTarget,
+            targetStatus, context, action, fileAttributes);
       }
       DistCpUtils.preserve(target.getFileSystem(conf), tmpTarget,
           sourceCurrStatus, fileAttributes, preserveRawXattrs);
@@ -235,9 +246,10 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
   }
 
   private void copyFileWithRetry(String description,
-      CopyListingFileStatus sourceFileStatus, Path target, Context context,
-      FileAction action, EnumSet<DistCpOptions.FileAttribute> fileAttributes)
-      throws IOException {
+      CopyListingFileStatus sourceFileStatus, Path target,
+      FileStatus targrtFileStatus, Context context, FileAction action,
+      EnumSet<DistCpOptions.FileAttribute> fileAttributes)
+      throws IOException, InterruptedException {
     long bytesCopied;
     try {
       bytesCopied = (Long) new RetriableFileCopyCommand(skipCrc, description,
@@ -251,6 +263,14 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
     incrementCounter(context, Counter.BYTESCOPIED, bytesCopied);
     incrementCounter(context, Counter.COPY, 1);
     totalBytesCopied += bytesCopied;
+
+    if (verboseLog) {
+      context.write(null,
+          new Text("FILE_COPIED: source=" + sourceFileStatus.getPath() + ","
+          + " size=" + sourceFileStatus.getLen() + " --> "
+          + "target=" + target + ", size=" + (targrtFileStatus == null ?
+              0 : targrtFileStatus.getLen())));
+    }
   }
 
   private void createTargetDirsWithRetry(String description,
@@ -260,7 +280,7 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
     } catch (Exception e) {
       throw new IOException("mkdir failed for " + target, e);
     }
-    incrementCounter(context, Counter.COPY, 1);
+    incrementCounter(context, Counter.DIR_COPY, 1);
   }
 
   private static void updateSkipCounters(Context context,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63720ef5/hadoop-tools/hadoop-distcp/src/site/markdown/DistCp.md.vm
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/site/markdown/DistCp.md.vm b/hadoop-tools/hadoop-distcp/src/site/markdown/DistCp.md.vm
index 925b24e..2cd01e2 100644
--- a/hadoop-tools/hadoop-distcp/src/site/markdown/DistCp.md.vm
+++ b/hadoop-tools/hadoop-distcp/src/site/markdown/DistCp.md.vm
@@ -220,6 +220,7 @@ Flag              | Description                          | Notes
 `-p[rbugpcaxt]` | Preserve r: replication number b: block size u: user g: group p: permission c: checksum-type a: ACL x: XAttr t: timestamp | When `-update` is specified, status updates will **not** be synchronized unless the file sizes also differ (i.e. unless the file is re-created). If -pa is specified, DistCp preserves the permissions also because ACLs are a super-set of permissions. The option -pr is only valid if both source and target directory are not erasure coded. **Note:** If -p option's are not specified, then by default block size is preserved.
 `-i` | Ignore failures | As explained in the Appendix, this option will keep more accurate statistics about the copy than the default case. It also preserves logs from failed copies, which can be valuable for debugging. Finally, a failing map will not cause the job to fail before all splits are attempted.
 `-log <logdir>` | Write logs to \<logdir\> | DistCp keeps logs of each file it attempts to copy as map output. If a map fails, the log output will not be retained if it is re-executed.
+`-v` | Log additional info (path, size) in the SKIP/COPY log | This option can only be used with -log option.
 `-m <num_maps>` | Maximum number of simultaneous copies | Specify the number of maps to copy data. Note that more maps may not necessarily improve throughput.
 `-overwrite` | Overwrite destination | If a map fails and `-i` is not specified, all the files in the split, not only those that failed, will be recopied. As discussed in the Usage documentation, it also changes the semantics for generating destination paths, so users should use this carefully.
 `-update` | Overwrite if source and destination differ in size, blocksize, or checksum | As noted in the preceding, this is not a "sync" operation. The criteria examined are the source and destination file sizes, blocksizes, and checksums; if they differ, the source file replaces the destination file. As discussed in the Usage documentation, it also changes the semantics for generating destination paths, so users should use this carefully.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63720ef5/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpOptions.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpOptions.java b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpOptions.java
index 6b59b97..dd8ec69 100644
--- a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpOptions.java
+++ b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpOptions.java
@@ -287,7 +287,7 @@ public class TestDistCpOptions {
         "mapBandwidth=0.0, copyStrategy='uniformsize', preserveStatus=[], " +
         "atomicWorkPath=null, logPath=null, sourceFileListing=abc, " +
         "sourcePaths=null, targetPath=xyz, filtersFile='null'," +
-        " blocksPerChunk=0, copyBufferSize=8192}";
+        " blocksPerChunk=0, copyBufferSize=8192, verboseLog=false}";
     String optionString = option.toString();
     Assert.assertEquals(val, optionString);
     Assert.assertNotSame(DistCpOptionSwitch.ATOMIC_COMMIT.toString(),
@@ -514,4 +514,23 @@ public class TestDistCpOptions {
     Assert.assertEquals(DistCpConstants.COPY_BUFFER_SIZE_DEFAULT,
         builder.build().getCopyBufferSize());
   }
+
+  @Test
+  public void testVerboseLog() {
+    final DistCpOptions.Builder builder = new DistCpOptions.Builder(
+        Collections.singletonList(new Path("hdfs://localhost:8020/source")),
+        new Path("hdfs://localhost:8020/target/"));
+    Assert.assertFalse(builder.build().shouldVerboseLog());
+
+    try {
+      builder.withVerboseLog(true).build();
+      fail("-v should fail if -log option is not specified");
+    } catch (IllegalArgumentException e) {
+      assertExceptionContains("-v is valid only with -log option", e);
+    }
+
+    final Path logPath = new Path("hdfs://localhost:8020/logs");
+    builder.withLogPath(logPath).withVerboseLog(true);
+    Assert.assertTrue(builder.build().shouldVerboseLog());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63720ef5/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyMapper.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyMapper.java b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyMapper.java
index 866ad6e..fd998c8 100644
--- a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyMapper.java
+++ b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyMapper.java
@@ -255,7 +255,13 @@ public class TestCopyMapper {
     context.getConfiguration().setBoolean(
         DistCpOptionSwitch.APPEND.getConfigLabel(), true);
     copyMapper.setup(context);
+
+    int numFiles = 0;
     for (Path path: pathList) {
+      if (fs.getFileStatus(path).isFile()) {
+        numFiles++;
+      }
+
       copyMapper.map(new Text(DistCpUtils.getRelativePath(new Path(SOURCE_PATH), path)),
               new CopyListingFileStatus(cluster.getFileSystem().getFileStatus(
                   path)), context);
@@ -266,7 +272,7 @@ public class TestCopyMapper {
     Assert.assertEquals(nFiles * DEFAULT_FILE_SIZE * 2, stubContext
         .getReporter().getCounter(CopyMapper.Counter.BYTESCOPIED)
         .getValue());
-    Assert.assertEquals(pathList.size(), stubContext.getReporter().
+    Assert.assertEquals(numFiles, stubContext.getReporter().
         getCounter(CopyMapper.Counter.COPY).getValue());
   }
 
@@ -295,7 +301,15 @@ public class TestCopyMapper {
 
     copyMapper.setup(context);
 
-    for (Path path: pathList) {
+    int numFiles = 0;
+    int numDirs = 0;
+    for (Path path : pathList) {
+      if (fs.getFileStatus(path).isDirectory()) {
+        numDirs++;
+      } else {
+        numFiles++;
+      }
+
       copyMapper.map(
           new Text(DistCpUtils.getRelativePath(new Path(SOURCE_PATH), path)),
           new CopyListingFileStatus(fs.getFileStatus(path)), context);
@@ -303,8 +317,10 @@ public class TestCopyMapper {
 
     // Check that the maps worked.
     verifyCopy(fs, preserveChecksum);
-    Assert.assertEquals(pathList.size(), stubContext.getReporter()
+    Assert.assertEquals(numFiles, stubContext.getReporter()
         .getCounter(CopyMapper.Counter.COPY).getValue());
+    Assert.assertEquals(numDirs, stubContext.getReporter()
+        .getCounter(CopyMapper.Counter.DIR_COPY).getValue());
     if (!preserveChecksum) {
       Assert.assertEquals(nFiles * DEFAULT_FILE_SIZE, stubContext
           .getReporter().getCounter(CopyMapper.Counter.BYTESCOPIED)
@@ -1118,4 +1134,81 @@ public class TestCopyMapper {
       e.printStackTrace();
     }
   }
+
+  @Test
+  public void testVerboseLogging() throws Exception {
+    deleteState();
+    createSourceData();
+
+    FileSystem fs = cluster.getFileSystem();
+    CopyMapper copyMapper = new CopyMapper();
+    StubContext stubContext = new StubContext(getConfiguration(), null, 0);
+    Mapper<Text, CopyListingFileStatus, Text, Text>.Context context
+            = stubContext.getContext();
+    copyMapper.setup(context);
+
+    int numFiles = 0;
+    for (Path path : pathList) {
+      if (fs.getFileStatus(path).isFile()) {
+        numFiles++;
+      }
+
+      copyMapper.map(
+          new Text(DistCpUtils.getRelativePath(new Path(SOURCE_PATH), path)),
+          new CopyListingFileStatus(fs.getFileStatus(path)), context);
+    }
+
+    // Check that the maps worked.
+    Assert.assertEquals(numFiles, stubContext.getReporter()
+        .getCounter(CopyMapper.Counter.COPY).getValue());
+
+    testCopyingExistingFiles(fs, copyMapper, context);
+    // verify the verbose log
+    // we shouldn't print verbose log since this option is disabled
+    for (Text value : stubContext.getWriter().values()) {
+      Assert.assertTrue(!value.toString().startsWith("FILE_COPIED:"));
+      Assert.assertTrue(!value.toString().startsWith("FILE_SKIPPED:"));
+    }
+
+    // test with verbose logging
+    deleteState();
+    createSourceData();
+
+    stubContext = new StubContext(getConfiguration(), null, 0);
+    context = stubContext.getContext();
+    copyMapper.setup(context);
+
+    // enables verbose logging
+    context.getConfiguration().setBoolean(
+        DistCpOptionSwitch.VERBOSE_LOG.getConfigLabel(), true);
+    copyMapper.setup(context);
+
+    for (Path path : pathList) {
+      copyMapper.map(
+          new Text(DistCpUtils.getRelativePath(new Path(SOURCE_PATH), path)),
+          new CopyListingFileStatus(fs.getFileStatus(path)), context);
+    }
+
+    Assert.assertEquals(numFiles, stubContext.getReporter()
+        .getCounter(CopyMapper.Counter.COPY).getValue());
+
+    // verify the verbose log of COPY log
+    int numFileCopied = 0;
+    for (Text value : stubContext.getWriter().values()) {
+      if (value.toString().startsWith("FILE_COPIED:")) {
+        numFileCopied++;
+      }
+    }
+    Assert.assertEquals(numFiles, numFileCopied);
+
+    // verify the verbose log of SKIP log
+    int numFileSkipped = 0;
+    testCopyingExistingFiles(fs, copyMapper, context);
+    for (Text value : stubContext.getWriter().values()) {
+      if (value.toString().startsWith("FILE_SKIPPED:")) {
+        numFileSkipped++;
+      }
+    }
+    Assert.assertEquals(numFiles, numFileSkipped);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63720ef5/hadoop-tools/hadoop-extras/src/main/java/org/apache/hadoop/tools/DistCp_Counter.properties
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-extras/src/main/java/org/apache/hadoop/tools/DistCp_Counter.properties b/hadoop-tools/hadoop-extras/src/main/java/org/apache/hadoop/tools/DistCp_Counter.properties
index 2234619..77f6c02 100644
--- a/hadoop-tools/hadoop-extras/src/main/java/org/apache/hadoop/tools/DistCp_Counter.properties
+++ b/hadoop-tools/hadoop-extras/src/main/java/org/apache/hadoop/tools/DistCp_Counter.properties
@@ -15,6 +15,7 @@
 CounterGroupName=       distcp
 
 COPY.name=              Files copied
+DIR_COPY.name=          Directories copied
 SKIP.name=              Files skipped
 FAIL.name=              Files failed
 BYTESCOPIED.name=       Bytes copied


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


[17/50] [abbrv] hadoop git commit: YARN-6978. Add updateContainer API to NMClient. (Kartheek Muthyala via asuresh)

Posted by as...@apache.org.
YARN-6978. Add updateContainer API to NMClient. (Kartheek Muthyala via asuresh)


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

Branch: refs/heads/YARN-5972
Commit: c41118a7f826dcbb269b4bd3d5877b35fbbee2b1
Parents: 13eda50
Author: Arun Suresh <as...@apache.org>
Authored: Thu Sep 7 10:23:12 2017 -0700
Committer: Arun Suresh <as...@apache.org>
Committed: Thu Sep 7 10:23:12 2017 -0700

----------------------------------------------------------------------
 .../distributedshell/ApplicationMaster.java     | 19 +++-
 .../apache/hadoop/yarn/client/api/NMClient.java | 18 ++++
 .../yarn/client/api/async/NMClientAsync.java    | 36 ++++++++
 .../api/async/impl/NMClientAsyncImpl.java       | 92 +++++++++++++++-----
 .../yarn/client/api/impl/NMClientImpl.java      | 29 ++++++
 .../api/async/impl/TestNMClientAsync.java       | 60 ++++++++++++-
 .../yarn/client/api/impl/TestAMRMClient.java    |  1 +
 .../yarn/client/api/impl/TestNMClient.java      |  5 +-
 8 files changed, 227 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c41118a7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
index a02af70..5ec9409 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
@@ -1027,10 +1027,6 @@ public class ApplicationMaster {
     }
 
     @Override
-    public void onContainerResourceIncreased(
-        ContainerId containerId, Resource resource) {}
-
-    @Override
     public void onStartContainerError(ContainerId containerId, Throwable t) {
       LOG.error("Failed to start Container " + containerId, t);
       containers.remove(containerId);
@@ -1050,10 +1046,25 @@ public class ApplicationMaster {
       containers.remove(containerId);
     }
 
+    @Deprecated
     @Override
     public void onIncreaseContainerResourceError(
         ContainerId containerId, Throwable t) {}
 
+    @Deprecated
+    @Override
+    public void onContainerResourceIncreased(
+        ContainerId containerId, Resource resource) {}
+
+    @Override
+    public void onUpdateContainerResourceError(
+        ContainerId containerId, Throwable t) {
+    }
+
+    @Override
+    public void onContainerResourceUpdated(ContainerId containerId,
+        Resource resource) {
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c41118a7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/NMClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/NMClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/NMClient.java
index c1447ba..17168f7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/NMClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/NMClient.java
@@ -104,10 +104,28 @@ public abstract class NMClient extends AbstractService {
    * @throws YarnException YarnException.
    * @throws IOException IOException.
    */
+  @Deprecated
   public abstract void increaseContainerResource(Container container)
       throws YarnException, IOException;
 
   /**
+   * <p>Update the resources of a container.</p>
+   *
+   * <p>The <code>ApplicationMaster</code> or other applications that use the
+   * client must provide the details of the container, including the Id and
+   * the target resource encapsulated in the updated container token via
+   * {@link Container}.
+   * </p>
+   *
+   * @param container the container with updated token.
+   *
+   * @throws YarnException YarnException.
+   * @throws IOException IOException.
+   */
+  public abstract void updateContainerResource(Container container)
+      throws YarnException, IOException;
+
+  /**
    * <p>Stop an started container.</p>
    *
    * @param containerId the Id of the started container

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c41118a7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/NMClientAsync.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/NMClientAsync.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/NMClientAsync.java
index c94942a..62e2d99 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/NMClientAsync.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/NMClientAsync.java
@@ -177,9 +177,23 @@ public abstract class NMClientAsync extends AbstractService {
   public abstract void startContainerAsync(
       Container container, ContainerLaunchContext containerLaunchContext);
 
+  @Deprecated
   public abstract void increaseContainerResourceAsync(Container container);
 
   /**
+   * <p>Update the resources of a container.</p>
+   *
+   * <p>The <code>ApplicationMaster</code> or other applications that use the
+   * client must provide the details of the container, including the Id and
+   * the target resource encapsulated in the updated container token via
+   * {@link Container}.
+   * </p>
+   *
+   * @param container the container with updated token.
+   */
+  public abstract void updateContainerResourceAsync(Container container);
+
+  /**
    * <p>Re-Initialize the Container.</p>
    *
    * @param containerId the Id of the container to Re-Initialize.
@@ -301,10 +315,21 @@ public abstract class NMClientAsync extends AbstractService {
      * @param containerId the Id of the container
      * @param resource the target resource of the container
      */
+    @Deprecated
     public abstract void onContainerResourceIncreased(
         ContainerId containerId, Resource resource);
 
     /**
+     * The API is called when <code>NodeManager</code> responds to indicate
+     * the container resource has been successfully updated.
+     *
+     * @param containerId the Id of the container
+     * @param resource the target resource of the container
+     */
+    public abstract void onContainerResourceUpdated(
+        ContainerId containerId, Resource resource);
+
+    /**
      * The API is called when an exception is raised in the process of
      * querying the status of a container.
      *
@@ -321,11 +346,22 @@ public abstract class NMClientAsync extends AbstractService {
      * @param containerId the Id of the container
      * @param t the raised exception
      */
+    @Deprecated
     public abstract void onIncreaseContainerResourceError(
         ContainerId containerId, Throwable t);
 
     /**
      * The API is called when an exception is raised in the process of
+     * updating container resource.
+     *
+     * @param containerId the Id of the container
+     * @param t the raised exception
+     */
+    public abstract void onUpdateContainerResourceError(
+        ContainerId containerId, Throwable t);
+
+    /**
+     * The API is called when an exception is raised in the process of
      * stopping a container.
      *
      * @param containerId the Id of the container

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c41118a7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/NMClientAsyncImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/NMClientAsyncImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/NMClientAsyncImpl.java
index 20be71e..4d14180 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/NMClientAsyncImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/NMClientAsyncImpl.java
@@ -259,6 +259,7 @@ public class NMClientAsyncImpl extends NMClientAsync {
     }
   }
 
+  @Deprecated
   public void increaseContainerResourceAsync(Container container) {
     if (!(callbackHandler instanceof AbstractCallbackHandler)) {
       LOG.error("Callback handler does not implement container resource "
@@ -274,7 +275,7 @@ public class NMClientAsyncImpl extends NMClientAsync {
                   " is neither started nor scheduled to start"));
     }
     try {
-      events.put(new IncreaseContainerResourceEvent(container));
+      events.put(new UpdateContainerResourceEvent(container, true));
     } catch (InterruptedException e) {
       LOG.warn("Exception when scheduling the event of increasing resource of "
           + "Container " + container.getId());
@@ -283,6 +284,30 @@ public class NMClientAsyncImpl extends NMClientAsync {
   }
 
   @Override
+  public void updateContainerResourceAsync(Container container) {
+    if (!(callbackHandler instanceof AbstractCallbackHandler)) {
+      LOG.error("Callback handler does not implement container resource "
+          + "increase callback methods");
+      return;
+    }
+    AbstractCallbackHandler handler = (AbstractCallbackHandler) callbackHandler;
+    if (containers.get(container.getId()) == null) {
+      handler.onUpdateContainerResourceError(
+          container.getId(),
+          RPCUtil.getRemoteException(
+              "Container " + container.getId() +
+                  " is neither started nor scheduled to start"));
+    }
+    try {
+      events.put(new UpdateContainerResourceEvent(container, false));
+    } catch (InterruptedException e) {
+      LOG.warn("Exception when scheduling the event of increasing resource of "
+          + "Container " + container.getId());
+      handler.onUpdateContainerResourceError(container.getId(), e);
+    }
+  }
+
+  @Override
   public void reInitializeContainerAsync(ContainerId containerId,
       ContainerLaunchContext containerLaunchContex, boolean autoCommit){
     if (!(callbackHandler instanceof AbstractCallbackHandler)) {
@@ -427,7 +452,7 @@ public class NMClientAsyncImpl extends NMClientAsync {
     START_CONTAINER,
     STOP_CONTAINER,
     QUERY_CONTAINER,
-    INCREASE_CONTAINER_RESOURCE,
+    UPDATE_CONTAINER_RESOURCE,
     REINITIALIZE_CONTAINER,
     RESTART_CONTAINER,
     ROLLBACK_LAST_REINIT,
@@ -503,14 +528,20 @@ public class NMClientAsyncImpl extends NMClientAsync {
     }
   }
 
-  protected static class IncreaseContainerResourceEvent extends ContainerEvent {
+  protected static class UpdateContainerResourceEvent extends ContainerEvent {
     private Container container;
+    private boolean isIncreaseEvent;
 
-    public IncreaseContainerResourceEvent(Container container) {
+    // UpdateContainerResourceEvent constructor takes in a
+    // flag to support callback API's calling through the deprecated
+    // increaseContainerResource
+    public UpdateContainerResourceEvent(Container container,
+        boolean isIncreaseEvent) {
       super(container.getId(), container.getNodeId(),
           container.getContainerToken(),
-              ContainerEventType.INCREASE_CONTAINER_RESOURCE);
+          ContainerEventType.UPDATE_CONTAINER_RESOURCE);
       this.container = container;
+      this.isIncreaseEvent = isIncreaseEvent;
     }
 
     public Container getContainer() {
@@ -536,8 +567,8 @@ public class NMClientAsyncImpl extends NMClientAsync {
 
             // Transitions from RUNNING state
             .addTransition(ContainerState.RUNNING, ContainerState.RUNNING,
-                ContainerEventType.INCREASE_CONTAINER_RESOURCE,
-                new IncreaseContainerResourceTransition())
+                ContainerEventType.UPDATE_CONTAINER_RESOURCE,
+                new UpdateContainerResourceTransition())
 
             // Transitions for Container Upgrade
             .addTransition(ContainerState.RUNNING,
@@ -566,7 +597,7 @@ public class NMClientAsyncImpl extends NMClientAsync {
             .addTransition(ContainerState.DONE, ContainerState.DONE,
                 EnumSet.of(ContainerEventType.START_CONTAINER,
                     ContainerEventType.STOP_CONTAINER,
-                    ContainerEventType.INCREASE_CONTAINER_RESOURCE))
+                    ContainerEventType.UPDATE_CONTAINER_RESOURCE))
 
             // Transition from FAILED state
             .addTransition(ContainerState.FAILED, ContainerState.FAILED,
@@ -576,7 +607,7 @@ public class NMClientAsyncImpl extends NMClientAsync {
                     ContainerEventType.RESTART_CONTAINER,
                     ContainerEventType.COMMIT_LAST_REINT,
                     ContainerEventType.ROLLBACK_LAST_REINIT,
-                    ContainerEventType.INCREASE_CONTAINER_RESOURCE));
+                    ContainerEventType.UPDATE_CONTAINER_RESOURCE));
 
     protected static class StartContainerTransition implements
         MultipleArcTransition<StatefulContainer, ContainerEvent,
@@ -628,46 +659,61 @@ public class NMClientAsyncImpl extends NMClientAsync {
       }
     }
 
-    protected static class IncreaseContainerResourceTransition implements
+    protected static class UpdateContainerResourceTransition implements
         SingleArcTransition<StatefulContainer, ContainerEvent> {
+
+      @SuppressWarnings("deprecation")
       @Override
       public void transition(
           StatefulContainer container, ContainerEvent event) {
+        boolean isIncreaseEvent = false;
         if (!(container.nmClientAsync.getCallbackHandler()
             instanceof AbstractCallbackHandler)) {
           LOG.error("Callback handler does not implement container resource "
-              + "increase callback methods");
+              + "update callback methods");
           return;
         }
         AbstractCallbackHandler handler =
             (AbstractCallbackHandler) container.nmClientAsync
                 .getCallbackHandler();
         try {
-          if (!(event instanceof IncreaseContainerResourceEvent)) {
+          if (!(event instanceof UpdateContainerResourceEvent)) {
             throw new AssertionError("Unexpected event type. Expecting:"
-                + "IncreaseContainerResourceEvent. Got:" + event);
+                + "UpdateContainerResourceEvent. Got:" + event);
           }
-          IncreaseContainerResourceEvent increaseEvent =
-              (IncreaseContainerResourceEvent) event;
-          container.nmClientAsync.getClient().increaseContainerResource(
-              increaseEvent.getContainer());
+          UpdateContainerResourceEvent updateEvent =
+              (UpdateContainerResourceEvent) event;
+          container.nmClientAsync.getClient().updateContainerResource(
+              updateEvent.getContainer());
+          isIncreaseEvent = updateEvent.isIncreaseEvent;
           try {
-            handler.onContainerResourceIncreased(
-                increaseEvent.getContainerId(), increaseEvent.getContainer()
-                    .getResource());
+            //If isIncreaseEvent is set, set the appropriate callbacks
+            //for backward compatibility
+            if (isIncreaseEvent) {
+              handler.onContainerResourceIncreased(updateEvent.getContainerId(),
+                  updateEvent.getContainer().getResource());
+            } else {
+              handler.onContainerResourceUpdated(updateEvent.getContainerId(),
+                  updateEvent.getContainer().getResource());
+            }
           } catch (Throwable thr) {
             // Don't process user created unchecked exception
             LOG.info("Unchecked exception is thrown from "
-                + "onContainerResourceIncreased for Container "
+                + "onContainerResourceUpdated for Container "
                 + event.getContainerId(), thr);
           }
         } catch (Exception e) {
           try {
-            handler.onIncreaseContainerResourceError(event.getContainerId(), e);
+            if (isIncreaseEvent) {
+              handler
+                  .onIncreaseContainerResourceError(event.getContainerId(), e);
+            } else {
+              handler.onUpdateContainerResourceError(event.getContainerId(), e);
+            }
           } catch (Throwable thr) {
             // Don't process user created unchecked exception
             LOG.info("Unchecked exception is thrown from "
-                + "onIncreaseContainerResourceError for Container "
+                + "onUpdateContainerResourceError for Container "
                 + event.getContainerId(), thr);
           }
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c41118a7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/NMClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/NMClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/NMClientImpl.java
index 8171de2..ca44e2e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/NMClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/NMClientImpl.java
@@ -230,6 +230,7 @@ public class NMClientImpl extends NMClient {
     }
   }
 
+  @Deprecated
   @Override
   public void increaseContainerResource(Container container)
       throws YarnException, IOException {
@@ -259,6 +260,34 @@ public class NMClientImpl extends NMClient {
   }
 
   @Override
+  public void updateContainerResource(Container container)
+      throws YarnException, IOException {
+    ContainerManagementProtocolProxyData proxy = null;
+    try {
+      proxy =
+          cmProxy.getProxy(container.getNodeId().toString(), container.getId());
+      List<Token> updateTokens = new ArrayList<>();
+      updateTokens.add(container.getContainerToken());
+
+      ContainerUpdateRequest request =
+          ContainerUpdateRequest.newInstance(updateTokens);
+      ContainerUpdateResponse response =
+          proxy.getContainerManagementProtocol().updateContainer(request);
+
+      if (response.getFailedRequests() != null && response.getFailedRequests()
+          .containsKey(container.getId())) {
+        Throwable t =
+            response.getFailedRequests().get(container.getId()).deSerialize();
+        parseAndThrowException(t);
+      }
+    } finally {
+      if (proxy != null) {
+        cmProxy.mayBeCloseProxy(proxy);
+      }
+    }
+  }
+
+  @Override
   public void stopContainer(ContainerId containerId, NodeId nodeId)
       throws YarnException, IOException {
     StartedContainer startedContainer = startedContainers.get(containerId);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c41118a7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestNMClientAsync.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestNMClientAsync.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestNMClientAsync.java
index dda3eec..6c72704 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestNMClientAsync.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestNMClientAsync.java
@@ -253,7 +253,7 @@ public class TestNMClientAsync {
         int t = containerId.getId() % 5;
         switch (t) {
         case 0:
-          asyncClient.increaseContainerResourceAsync(container);
+          asyncClient.updateContainerResourceAsync(container);
           break;
         case 1:
           asyncClient.reInitializeContainerAsync(containerId,
@@ -295,7 +295,7 @@ public class TestNMClientAsync {
       // containerId
       Container container = Container.newInstance(
           containerId, nodeId, null, null, null, containerToken);
-      asyncClient.increaseContainerResourceAsync(container);
+      asyncClient.updateContainerResourceAsync(container);
 
       // Shouldn't crash the test thread
       throw new RuntimeException("Ignorable Exception");
@@ -322,6 +322,25 @@ public class TestNMClientAsync {
 
     @SuppressWarnings("deprecation")
     @Override
+    public void onContainerResourceUpdated(ContainerId containerId,
+        Resource resource) {
+      if (containerId.getId() >= expectedSuccess) {
+        errorMsgs.add("Container " + containerId +
+            " should throw the exception onContainerResourceUpdated");
+        return;
+      }
+      TestData td = testMap.get(OpsToTest.INCR);
+      td.success.addAndGet(1);
+      td.successArray.set(containerId.getId(), 1);
+      // move on to the following success tests
+      asyncClient.reInitializeContainerAsync(containerId,
+          Records.newRecord(ContainerLaunchContext.class), true);
+      // throw a fake user exception, and shouldn't crash the test
+      throw new RuntimeException("Ignorable Exception");
+    }
+
+    @SuppressWarnings("deprecation")
+    @Override
     public void onContainerReInitialize(ContainerId containerId) {
       if (containerId.getId() >= expectedSuccess) {
         errorMsgs.add("Container " + containerId +
@@ -452,6 +471,27 @@ public class TestNMClientAsync {
 
     @SuppressWarnings("deprecation")
     @Override
+    public void onUpdateContainerResourceError(ContainerId containerId,
+        Throwable t) {
+      if (containerId.getId() < expectedSuccess + expectedFailure) {
+        errorMsgs.add("Container " + containerId +
+            " shouldn't throw the exception onUpdatedContainerResourceError");
+        return;
+      }
+      TestData td = testMap.get(OpsToTest.INCR);
+      td.failure.addAndGet(1);
+      td.failureArray.set(
+          containerId.getId() - expectedSuccess - expectedFailure, 1);
+      // increase container resource error should NOT change the
+      // the container status to FAILED
+      // move on to the following failure tests
+      asyncClient.stopContainerAsync(containerId, nodeId);
+      // Shouldn't crash the test thread
+      throw new RuntimeException("Ignorable Exception");
+    }
+
+    @SuppressWarnings("deprecation")
+    @Override
     public void onContainerReInitializeError(ContainerId containerId,
         Throwable t) {
       if (containerId.getId() < expectedSuccess + expectedFailure) {
@@ -673,7 +713,7 @@ public class TestNMClientAsync {
         when(client.getContainerStatus(any(ContainerId.class),
             any(NodeId.class))).thenReturn(
                 recordFactory.newRecordInstance(ContainerStatus.class));
-        doNothing().when(client).increaseContainerResource(
+        doNothing().when(client).updateContainerResource(
             any(Container.class));
         doNothing().when(client).reInitializeContainer(
             any(ContainerId.class), any(ContainerLaunchContext.class),
@@ -703,7 +743,7 @@ public class TestNMClientAsync {
             any(NodeId.class))).thenReturn(
                 recordFactory.newRecordInstance(ContainerStatus.class));
         doThrow(RPCUtil.getRemoteException("Increase Resource Exception"))
-            .when(client).increaseContainerResource(any(Container.class));
+            .when(client).updateContainerResource(any(Container.class));
         doThrow(RPCUtil.getRemoteException("ReInitialize Exception"))
             .when(client).reInitializeContainer(
             any(ContainerId.class), any(ContainerLaunchContext.class),
@@ -818,11 +858,17 @@ public class TestNMClientAsync {
         ContainerStatus containerStatus) {
     }
 
+    @Deprecated
     @Override
     public void onContainerResourceIncreased(
         ContainerId containerId, Resource resource) {}
 
     @Override
+    public void onContainerResourceUpdated(ContainerId containerId,
+        Resource resource) {
+    }
+
+    @Override
     public void onContainerStopped(ContainerId containerId) {
     }
 
@@ -847,11 +893,17 @@ public class TestNMClientAsync {
         Throwable t) {
     }
 
+    @Deprecated
     @Override
     public void onIncreaseContainerResourceError(
         ContainerId containerId, Throwable t) {}
 
     @Override
+    public void onUpdateContainerResourceError(ContainerId containerId,
+        Throwable t) {
+    }
+
+    @Override
     public void onStopContainerError(ContainerId containerId, Throwable t) {
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c41118a7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
index 09b12f2..fa2e7a5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
@@ -1428,6 +1428,7 @@ public class TestAMRMClient {
     amClient.ask.clear();
   }
 
+  @SuppressWarnings("deprecation")
   private void updateContainerExecType(AllocateResponse allocResponse,
       ExecutionType expectedExecType, NMClientImpl nmClient)
       throws IOException, YarnException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c41118a7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestNMClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestNMClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestNMClient.java
index 9b79e2d..b23a923 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestNMClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestNMClient.java
@@ -301,10 +301,10 @@ public class TestNMClient {
         assertTrue("The thrown exception is not expected",
             e.getMessage().contains("is not handled by this NodeManager"));
       }
-      // increaseContainerResource shouldn't be called before startContainer,
+      // upadateContainerResource shouldn't be called before startContainer,
       // otherwise, NodeManager cannot find the container
       try {
-        nmClient.increaseContainerResource(container);
+        nmClient.updateContainerResource(container);
         fail("Exception is expected");
       } catch (YarnException e) {
         assertTrue("The thrown exception is not expected",
@@ -469,6 +469,7 @@ public class TestNMClient {
     }
   }
 
+  @SuppressWarnings("deprecation")
   private void testIncreaseContainerResource(Container container)
     throws YarnException, IOException {
     try {


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


[18/50] [abbrv] hadoop git commit: HADOOP-14520. WASB: Block compaction for Azure Block Blobs. Contributed by Georgi Chalakov

Posted by as...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/13eda500/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemBlockCompaction.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemBlockCompaction.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemBlockCompaction.java
new file mode 100644
index 0000000..820ce4f
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemBlockCompaction.java
@@ -0,0 +1,266 @@
+/*
+ * 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.fs.azure;
+
+import com.microsoft.azure.storage.blob.BlockEntry;
+import org.apache.commons.lang.RandomStringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.ByteArrayOutputStream;
+import java.io.OutputStream;
+import java.net.URI;
+import java.util.List;
+
+/**
+ * Test class that runs WASB block compaction process for block blobs.
+ */
+
+public class TestNativeAzureFileSystemBlockCompaction extends AbstractWasbTestBase {
+
+  private static final String TEST_FILE = "/user/active/test.dat";
+  private static final Path TEST_PATH = new Path(TEST_FILE);
+
+  private static final String TEST_FILE_NORMAL = "/user/normal/test.dat";
+  private static final Path TEST_PATH_NORMAL = new Path(TEST_FILE_NORMAL);
+
+  private AzureBlobStorageTestAccount testAccount = null;
+
+  @Before
+  public void setUp() throws Exception {
+    super.setUp();
+    testAccount = createTestAccount();
+    fs = testAccount.getFileSystem();
+    Configuration conf = fs.getConf();
+    conf.setBoolean(NativeAzureFileSystem.APPEND_SUPPORT_ENABLE_PROPERTY_NAME, true);
+    conf.set(AzureNativeFileSystemStore.KEY_BLOCK_BLOB_WITH_COMPACTION_DIRECTORIES, "/user/active");
+    URI uri = fs.getUri();
+    fs.initialize(uri, conf);
+  }
+
+  /*
+   * Helper method that creates test data of size provided by the
+   * "size" parameter.
+   */
+  private static byte[] getTestData(int size) {
+    byte[] testData = new byte[size];
+    System.arraycopy(RandomStringUtils.randomAlphabetic(size).getBytes(), 0, testData, 0, size);
+    return testData;
+  }
+
+  @Override
+  protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
+    return AzureBlobStorageTestAccount.create();
+  }
+
+  private BlockBlobAppendStream getBlockBlobAppendStream(FSDataOutputStream appendStream) {
+    SyncableDataOutputStream dataOutputStream = null;
+
+    if (appendStream.getWrappedStream() instanceof NativeAzureFileSystem.NativeAzureFsOutputStream) {
+      NativeAzureFileSystem.NativeAzureFsOutputStream fsOutputStream =
+              (NativeAzureFileSystem.NativeAzureFsOutputStream) appendStream.getWrappedStream();
+
+      dataOutputStream = (SyncableDataOutputStream) fsOutputStream.getOutStream();
+    }
+
+    if (appendStream.getWrappedStream() instanceof SyncableDataOutputStream) {
+      dataOutputStream = (SyncableDataOutputStream) appendStream.getWrappedStream();
+    }
+
+    Assert.assertNotNull("Did not recognize " + dataOutputStream,
+        dataOutputStream);
+
+    return (BlockBlobAppendStream) dataOutputStream.getOutStream();
+  }
+
+  private void verifyBlockList(BlockBlobAppendStream blockBlobStream,
+                               int[] testData) throws Throwable {
+    List<BlockEntry> blockList = blockBlobStream.getBlockList();
+    Assert.assertEquals("Block list length", testData.length, blockList.size());
+
+    int i = 0;
+    for (BlockEntry block: blockList) {
+      Assert.assertTrue(block.getSize() == testData[i++]);
+    }
+  }
+
+  private void appendBlockList(FSDataOutputStream fsStream,
+                              ByteArrayOutputStream memStream,
+                              int[] testData) throws Throwable {
+
+    for (int d: testData) {
+      byte[] data = getTestData(d);
+      memStream.write(data);
+      fsStream.write(data);
+    }
+    fsStream.hflush();
+  }
+
+  @Test
+  public void testCompactionDisabled() throws Throwable {
+
+    try (FSDataOutputStream appendStream = fs.create(TEST_PATH_NORMAL)) {
+
+      // testing new file
+
+      SyncableDataOutputStream dataOutputStream = null;
+
+      OutputStream wrappedStream = appendStream.getWrappedStream();
+      if (wrappedStream instanceof NativeAzureFileSystem.NativeAzureFsOutputStream) {
+        NativeAzureFileSystem.NativeAzureFsOutputStream fsOutputStream =
+                (NativeAzureFileSystem.NativeAzureFsOutputStream) wrappedStream;
+
+        dataOutputStream = (SyncableDataOutputStream) fsOutputStream.getOutStream();
+      } else if (wrappedStream instanceof SyncableDataOutputStream) {
+        dataOutputStream = (SyncableDataOutputStream) wrappedStream;
+      } else {
+        Assert.fail("Unable to determine type of " + wrappedStream
+            + " class of " + wrappedStream.getClass());
+      }
+
+      Assert.assertFalse("Data output stream is a BlockBlobAppendStream: "
+          + dataOutputStream,
+          dataOutputStream.getOutStream() instanceof BlockBlobAppendStream);
+
+    }
+  }
+
+  @Test
+  public void testCompaction() throws Throwable {
+
+    final int n2 = 2;
+    final int n4 = 4;
+    final int n10 = 10;
+    final int n12 = 12;
+    final int n14 = 14;
+    final int n16 = 16;
+
+    final int maxBlockSize = 16;
+    final int compactionBlockCount = 4;
+
+    ByteArrayOutputStream memStream = new ByteArrayOutputStream();
+
+    try (FSDataOutputStream appendStream = fs.create(TEST_PATH)) {
+
+      // test new file
+
+      BlockBlobAppendStream blockBlobStream = getBlockBlobAppendStream(appendStream);
+      blockBlobStream.setMaxBlockSize(maxBlockSize);
+      blockBlobStream.setCompactionBlockCount(compactionBlockCount);
+
+      appendBlockList(appendStream, memStream, new int[]{n2});
+      verifyBlockList(blockBlobStream, new int[]{n2});
+
+      appendStream.hflush();
+      verifyBlockList(blockBlobStream, new int[]{n2});
+
+      appendBlockList(appendStream, memStream, new int[]{n4});
+      verifyBlockList(blockBlobStream, new int[]{n2, n4});
+
+      appendStream.hsync();
+      verifyBlockList(blockBlobStream, new int[]{n2, n4});
+
+      appendBlockList(appendStream, memStream, new int[]{n4});
+      verifyBlockList(blockBlobStream, new int[]{n2, n4, n4});
+
+      appendBlockList(appendStream, memStream, new int[]{n4});
+      verifyBlockList(blockBlobStream, new int[]{n2, n4, n4, n4});
+
+      appendBlockList(appendStream, memStream, new int[]{n4});
+      verifyBlockList(blockBlobStream, new int[]{n14, n4});
+
+      appendBlockList(appendStream, memStream, new int[]{n4});
+      verifyBlockList(blockBlobStream, new int[]{n14, n4, n4});
+
+      appendBlockList(appendStream, memStream, new int[]{n4});
+      verifyBlockList(blockBlobStream, new int[]{n14, n4, n4, n4});
+
+      appendBlockList(appendStream, memStream, new int[]{n2, n4, n4});
+      verifyBlockList(blockBlobStream, new int[]{n14, n12, n10});
+
+      appendBlockList(appendStream, memStream, new int[]{n4});
+      verifyBlockList(blockBlobStream, new int[]{n14, n12, n10, n4});
+
+      appendBlockList(appendStream, memStream,
+              new int[]{n4, n4, n4, n4});
+      verifyBlockList(blockBlobStream,
+              new int[]{n14, n12, n14, n16});
+
+      appendBlockList(appendStream, memStream,
+              new int[]{n4, n4, n4, n4, n4});
+      verifyBlockList(blockBlobStream,
+              new int[]{n14, n12, n14, n16, n16, n4});
+
+      appendBlockList(appendStream, memStream,
+              new int[]{n4});
+      verifyBlockList(blockBlobStream,
+              new int[]{n14, n12, n14, n16, n16, n4, n4});
+
+      appendBlockList(appendStream, memStream,
+              new int[]{n4});
+      verifyBlockList(blockBlobStream,
+              new int[]{n14, n12, n14, n16, n16, n4, n4, n4});
+
+      appendBlockList(appendStream, memStream,
+              new int[]{n4});
+      verifyBlockList(blockBlobStream,
+              new int[]{n14, n12, n14, n16, n16, n4, n4, n4, n4});
+
+      appendBlockList(appendStream, memStream, new int[]{n4});
+
+      appendStream.close();
+
+      ContractTestUtils.verifyFileContents(fs, TEST_PATH, memStream.toByteArray());
+    }
+
+    try (FSDataOutputStream appendStream = fs.append(TEST_PATH)) {
+
+      // test existing file
+
+      BlockBlobAppendStream blockBlobStream = getBlockBlobAppendStream(appendStream);
+      blockBlobStream.setMaxBlockSize(maxBlockSize);
+      blockBlobStream.setCompactionBlockCount(compactionBlockCount);
+
+      appendBlockList(appendStream, memStream, new int[]{n4});
+      verifyBlockList(blockBlobStream,
+              new int[]{n14, n12, n14, n16, n16, n16, n4, n4});
+
+      appendBlockList(appendStream, memStream, new int[]{n4});
+      verifyBlockList(blockBlobStream,
+              new int[]{n14, n12, n14, n16, n16, n16, n4, n4, n4});
+
+      appendBlockList(appendStream, memStream, new int[]{n4});
+      verifyBlockList(blockBlobStream,
+              new int[]{n14, n12, n14, n16, n16, n16, n4, n4, n4, n4});
+
+      appendBlockList(appendStream, memStream, new int[]{n4});
+      verifyBlockList(blockBlobStream,
+              new int[]{n14, n12, n14, n16, n16, n16, n16, n4});
+
+      appendStream.close();
+
+      ContractTestUtils.verifyFileContents(fs, TEST_PATH, memStream.toByteArray());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13eda500/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties b/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties
index 73ee3f9..a5e0c4f 100644
--- a/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties
+++ b/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties
@@ -23,3 +23,4 @@ log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
 log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p [%t]: %c{2} (%F:%M(%L)) - %m%n
 
 log4j.logger.org.apache.hadoop.fs.azure.AzureFileSystemThreadPoolExecutor=DEBUG
+log4j.logger.org.apache.hadoop.fs.azure.BlockBlobAppendStream=DEBUG


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


[27/50] [abbrv] hadoop git commit: HDFS-12404. Rename hdfs config authorization.provider.bypass.users to attributes.provider.bypass.users.

Posted by as...@apache.org.
HDFS-12404. Rename hdfs config authorization.provider.bypass.users to attributes.provider.bypass.users.


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

Branch: refs/heads/YARN-5972
Commit: 3b3be355b35d08a78d9dcd647650812a2d28207b
Parents: 4e50dc9
Author: Manoj Govindassamy <ma...@apache.org>
Authored: Thu Sep 7 17:20:42 2017 -0700
Committer: Manoj Govindassamy <ma...@apache.org>
Committed: Thu Sep 7 17:20:42 2017 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/src/main/resources/hdfs-default.xml              | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b3be355/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 36c74f6..f85996b 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
@@ -4160,11 +4160,11 @@
 </property>
 
 <property>
-  <name>dfs.namenode.authorization.provider.bypass.users</name>
+  <name>dfs.namenode.inode.attributes.provider.bypass.users</name>
   <value></value>
   <description>
     A list of user principals (in secure cluster) or user names (in insecure
-    cluster) for whom the external attribute provider will be bypassed for all
+    cluster) for whom the external attributes provider will be bypassed for all
     operations. This means file attributes stored in HDFS instead of the
     external provider will be used for permission checking and be returned when
     requested.


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


[23/50] [abbrv] hadoop git commit: YARN-6930. Admins should be able to explicitly enable specific LinuxContainerRuntime in the NodeManager. Contributed by Shane Kumpf

Posted by as...@apache.org.
YARN-6930. Admins should be able to explicitly enable specific LinuxContainerRuntime in the NodeManager. 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/b0b535d9
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b0b535d9
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b0b535d9

Branch: refs/heads/YARN-5972
Commit: b0b535d9d5727cd84fd6368c6d1b38363616504e
Parents: f155ab7
Author: Jason Lowe <jl...@apache.org>
Authored: Thu Sep 7 16:12:09 2017 -0500
Committer: Jason Lowe <jl...@apache.org>
Committed: Thu Sep 7 16:17:03 2017 -0500

----------------------------------------------------------------------
 .../hadoop/yarn/conf/YarnConfiguration.java     |  17 +++
 .../src/main/resources/yarn-default.xml         |   8 ++
 .../server/nodemanager/ContainerExecutor.java   |   4 +-
 .../nodemanager/LinuxContainerExecutor.java     |   3 +-
 .../DelegatingLinuxContainerRuntime.java        |  79 ++++++++---
 .../runtime/LinuxContainerRuntimeConstants.java |   9 ++
 .../monitor/ContainersMonitorImpl.java          |   4 +-
 .../runtime/ContainerRuntime.java               |   6 +-
 .../TestDelegatingLinuxContainerRuntime.java    | 137 ++++++++++++++++++
 .../src/site/markdown/DockerContainers.md       | 139 ++++++++++---------
 10 files changed, 320 insertions(+), 86 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0b535d9/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 27ca957..be63233 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
@@ -1470,6 +1470,23 @@ public class YarnConfiguration extends Configuration {
   /** Prefix for runtime configuration constants. */
   public static final String LINUX_CONTAINER_RUNTIME_PREFIX = NM_PREFIX +
       "runtime.linux.";
+
+  /**
+   * Comma separated list of runtimes that are allowed when using
+   * LinuxContainerExecutor. The allowed values are:
+   * <ul>
+   *   <li>default</li>
+   *   <li>docker</li>
+   *   <li>javasandbox</li>
+   * </ul>
+   */
+  public static final String LINUX_CONTAINER_RUNTIME_ALLOWED_RUNTIMES =
+      LINUX_CONTAINER_RUNTIME_PREFIX + "allowed-runtimes";
+
+  /** The default list of allowed runtimes when using LinuxContainerExecutor. */
+  public static final String[] DEFAULT_LINUX_CONTAINER_RUNTIME_ALLOWED_RUNTIMES
+      = {"default"};
+
   public static final String DOCKER_CONTAINER_RUNTIME_PREFIX =
       LINUX_CONTAINER_RUNTIME_PREFIX + "docker.";
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0b535d9/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 0cad167..afde222 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
@@ -1572,6 +1572,14 @@
   </property>
 
   <property>
+    <description>Comma separated list of runtimes that are allowed when using
+    LinuxContainerExecutor. The allowed values are default, docker, and
+    javasandbox.</description>
+    <name>yarn.nodemanager.runtime.linux.allowed-runtimes</name>
+    <value>default</value>
+  </property>
+
+  <property>
     <description>This configuration setting determines the capabilities
       assigned to docker containers when they are launched. While these may not
       be case-sensitive from a docker perspective, it is best to keep these

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0b535d9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerExecutor.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/ContainerExecutor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerExecutor.java
index b6fb4ec..072cca7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerExecutor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerExecutor.java
@@ -51,6 +51,7 @@ import org.apache.hadoop.yarn.exceptions.ConfigurationException;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerDiagnosticsUpdateEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.runtime.ContainerExecutionException;
 import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerPrepareContext;
 import org.apache.hadoop.yarn.server.nodemanager.util.NodeManagerHardwareUtils;
 import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerLivenessContext;
@@ -663,7 +664,8 @@ public abstract class ContainerExecutor implements Configurable {
   }
 
   // LinuxContainerExecutor overrides this method and behaves differently.
-  public String[] getIpAndHost(Container container) {
+  public String[] getIpAndHost(Container container)
+      throws ContainerExecutionException {
     return getLocalIpAndHost(container);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0b535d9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.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/LinuxContainerExecutor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java
index dc68680..2971f83 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java
@@ -625,7 +625,8 @@ public class LinuxContainerExecutor extends ContainerExecutor {
   }
 
   @Override
-  public String[] getIpAndHost(Container container) {
+  public String[] getIpAndHost(Container container)
+      throws ContainerExecutionException {
     return linuxContainerRuntime.getIpAndHost(container);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0b535d9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DelegatingLinuxContainerRuntime.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/DelegatingLinuxContainerRuntime.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/DelegatingLinuxContainerRuntime.java
index 5273334..9fe4927 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/DelegatingLinuxContainerRuntime.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/DelegatingLinuxContainerRuntime.java
@@ -20,9 +20,11 @@
 
 package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime;
 
+import com.google.common.annotations.VisibleForTesting;
 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.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperationExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.runtime.ContainerExecutionException;
@@ -31,6 +33,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.runtime.Contai
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.EnumSet;
 import java.util.Map;
 
 /**
@@ -50,34 +53,62 @@ public class DelegatingLinuxContainerRuntime implements LinuxContainerRuntime {
   private DefaultLinuxContainerRuntime defaultLinuxContainerRuntime;
   private DockerLinuxContainerRuntime dockerLinuxContainerRuntime;
   private JavaSandboxLinuxContainerRuntime javaSandboxLinuxContainerRuntime;
+  private EnumSet<LinuxContainerRuntimeConstants.RuntimeType> allowedRuntimes =
+      EnumSet.noneOf(LinuxContainerRuntimeConstants.RuntimeType.class);
 
   @Override
   public void initialize(Configuration conf)
       throws ContainerExecutionException {
-    PrivilegedOperationExecutor privilegedOperationExecutor =
-        PrivilegedOperationExecutor.getInstance(conf);
-    defaultLinuxContainerRuntime = new DefaultLinuxContainerRuntime(
-        privilegedOperationExecutor);
-    defaultLinuxContainerRuntime.initialize(conf);
-    dockerLinuxContainerRuntime = new DockerLinuxContainerRuntime(
-        privilegedOperationExecutor);
-    dockerLinuxContainerRuntime.initialize(conf);
-    javaSandboxLinuxContainerRuntime = new JavaSandboxLinuxContainerRuntime(
-        privilegedOperationExecutor);
-    javaSandboxLinuxContainerRuntime.initialize(conf);
+    String[] configuredRuntimes = conf.getTrimmedStrings(
+        YarnConfiguration.LINUX_CONTAINER_RUNTIME_ALLOWED_RUNTIMES,
+        YarnConfiguration.DEFAULT_LINUX_CONTAINER_RUNTIME_ALLOWED_RUNTIMES);
+    for (String configuredRuntime : configuredRuntimes) {
+      try {
+        allowedRuntimes.add(
+            LinuxContainerRuntimeConstants.RuntimeType.valueOf(
+                configuredRuntime.toUpperCase()));
+      } catch (IllegalArgumentException e) {
+        throw new ContainerExecutionException("Invalid runtime set in "
+            + YarnConfiguration.LINUX_CONTAINER_RUNTIME_ALLOWED_RUNTIMES + " : "
+            + configuredRuntime);
+      }
+    }
+    if (isRuntimeAllowed(
+        LinuxContainerRuntimeConstants.RuntimeType.JAVASANDBOX)) {
+      javaSandboxLinuxContainerRuntime = new JavaSandboxLinuxContainerRuntime(
+          PrivilegedOperationExecutor.getInstance(conf));
+      javaSandboxLinuxContainerRuntime.initialize(conf);
+    }
+    if (isRuntimeAllowed(
+        LinuxContainerRuntimeConstants.RuntimeType.DOCKER)) {
+      dockerLinuxContainerRuntime = new DockerLinuxContainerRuntime(
+          PrivilegedOperationExecutor.getInstance(conf));
+      dockerLinuxContainerRuntime.initialize(conf);
+    }
+    if (isRuntimeAllowed(
+        LinuxContainerRuntimeConstants.RuntimeType.DEFAULT)) {
+      defaultLinuxContainerRuntime = new DefaultLinuxContainerRuntime(
+          PrivilegedOperationExecutor.getInstance(conf));
+      defaultLinuxContainerRuntime.initialize(conf);
+    }
   }
 
-  private LinuxContainerRuntime pickContainerRuntime(
-      Map<String, String> environment){
+  @VisibleForTesting
+  LinuxContainerRuntime pickContainerRuntime(
+      Map<String, String> environment) throws ContainerExecutionException {
     LinuxContainerRuntime runtime;
     //Sandbox checked first to ensure DockerRuntime doesn't circumvent controls
-    if (javaSandboxLinuxContainerRuntime.isSandboxContainerRequested()){
-        runtime = javaSandboxLinuxContainerRuntime;
-    } else if (DockerLinuxContainerRuntime
-        .isDockerContainerRequested(environment)){
+    if (javaSandboxLinuxContainerRuntime != null &&
+        javaSandboxLinuxContainerRuntime.isSandboxContainerRequested()){
+      runtime = javaSandboxLinuxContainerRuntime;
+    } else if (dockerLinuxContainerRuntime != null &&
+        DockerLinuxContainerRuntime.isDockerContainerRequested(environment)){
       runtime = dockerLinuxContainerRuntime;
-    } else {
+    } else if (defaultLinuxContainerRuntime != null &&
+        !DockerLinuxContainerRuntime.isDockerContainerRequested(environment)) {
       runtime = defaultLinuxContainerRuntime;
+    } else {
+      throw new ContainerExecutionException("Requested runtime not allowed.");
     }
 
     if (LOG.isDebugEnabled()) {
@@ -88,7 +119,8 @@ public class DelegatingLinuxContainerRuntime implements LinuxContainerRuntime {
     return runtime;
   }
 
-  private LinuxContainerRuntime pickContainerRuntime(Container container) {
+  private LinuxContainerRuntime pickContainerRuntime(Container container)
+      throws ContainerExecutionException {
     return pickContainerRuntime(container.getLaunchContext().getEnvironment());
   }
 
@@ -127,8 +159,15 @@ public class DelegatingLinuxContainerRuntime implements LinuxContainerRuntime {
   }
 
   @Override
-  public String[] getIpAndHost(Container container) {
+  public String[] getIpAndHost(Container container)
+      throws ContainerExecutionException {
     LinuxContainerRuntime runtime = pickContainerRuntime(container);
     return runtime.getIpAndHost(container);
   }
+
+  @VisibleForTesting
+  boolean isRuntimeAllowed(
+      LinuxContainerRuntimeConstants.RuntimeType runtimeType) {
+    return allowedRuntimes.contains(runtimeType);
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0b535d9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/LinuxContainerRuntimeConstants.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/LinuxContainerRuntimeConstants.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/LinuxContainerRuntimeConstants.java
index 2e632fa..3a47523 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/LinuxContainerRuntimeConstants.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/LinuxContainerRuntimeConstants.java
@@ -31,6 +31,15 @@ public final class LinuxContainerRuntimeConstants {
   private LinuxContainerRuntimeConstants() {
   }
 
+  /**
+   * Linux container runtime types for {@link DelegatingLinuxContainerRuntime}.
+   */
+  public enum RuntimeType {
+    DEFAULT,
+    DOCKER,
+    JAVASANDBOX;
+  }
+
   public static final Attribute<Map> LOCALIZED_RESOURCES = Attribute
       .attribute(Map.class, "localized_resources");
   public static final Attribute<List> CONTAINER_LAUNCH_PREFIX_COMMANDS =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0b535d9/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 d764e1d..2b99cc7 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
@@ -38,6 +38,7 @@ import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerImpl;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerKillEvent;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.runtime.ContainerExecutionException;
 import org.apache.hadoop.yarn.server.nodemanager.timelineservice.NMTimelinePublisher;
 import org.apache.hadoop.yarn.server.nodemanager.util.NodeManagerHardwareUtils;
 import org.apache.hadoop.yarn.util.ResourceCalculatorPlugin;
@@ -502,7 +503,8 @@ public class ContainersMonitorImpl extends AbstractService implements
      * @param entry process tree entry to fill in
      */
     private void initializeProcessTrees(
-            Entry<ContainerId, ProcessTreeInfo> entry) {
+            Entry<ContainerId, ProcessTreeInfo> entry)
+        throws ContainerExecutionException {
       ContainerId containerId = entry.getKey();
       ProcessTreeInfo ptInfo = entry.getValue();
       String pId = ptInfo.getPID();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0b535d9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/runtime/ContainerRuntime.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/runtime/ContainerRuntime.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/runtime/ContainerRuntime.java
index b15690f..7caa0ed 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/runtime/ContainerRuntime.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/runtime/ContainerRuntime.java
@@ -77,6 +77,10 @@ public interface ContainerRuntime {
 
   /**
    * Return the host and ip of the container
+   *
+   * @param container the {@link Container}
+   * @throws ContainerExecutionException if an error occurs while getting the ip
+   * and hostname
    */
-  String[] getIpAndHost(Container container);
+  String[] getIpAndHost(Container container) throws ContainerExecutionException;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0b535d9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDelegatingLinuxContainerRuntime.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/TestDelegatingLinuxContainerRuntime.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/TestDelegatingLinuxContainerRuntime.java
new file mode 100644
index 0000000..7f4bbc4
--- /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/runtime/TestDelegatingLinuxContainerRuntime.java
@@ -0,0 +1,137 @@
+/*
+ * 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.runtime;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.runtime.ContainerRuntime;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.runtime.ContainerRuntimeConstants;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.*;
+
+/**
+ * Test container runtime delegation.
+ */
+public class TestDelegatingLinuxContainerRuntime {
+
+  private DelegatingLinuxContainerRuntime delegatingLinuxContainerRuntime;
+  private Configuration conf;
+  private Map<String, String> env = new HashMap<>();
+
+  @Before
+  public void setUp() throws Exception {
+    delegatingLinuxContainerRuntime = new DelegatingLinuxContainerRuntime();
+    conf = new Configuration();
+    env.clear();
+  }
+
+  @Test
+  public void testIsRuntimeAllowedDefault() throws Exception {
+    conf.set(YarnConfiguration.LINUX_CONTAINER_RUNTIME_ALLOWED_RUNTIMES,
+        YarnConfiguration.DEFAULT_LINUX_CONTAINER_RUNTIME_ALLOWED_RUNTIMES[0]);
+    System.out.println(conf.get(
+        YarnConfiguration.LINUX_CONTAINER_RUNTIME_ALLOWED_RUNTIMES));
+    delegatingLinuxContainerRuntime.initialize(conf);
+    assertTrue(delegatingLinuxContainerRuntime.isRuntimeAllowed(
+        LinuxContainerRuntimeConstants.RuntimeType.DEFAULT));
+    assertFalse(delegatingLinuxContainerRuntime.isRuntimeAllowed(
+        LinuxContainerRuntimeConstants.RuntimeType.DOCKER));
+    assertFalse(delegatingLinuxContainerRuntime.isRuntimeAllowed(
+        LinuxContainerRuntimeConstants.RuntimeType.JAVASANDBOX));
+  }
+
+  @Test
+  public void testIsRuntimeAllowedDocker() throws Exception {
+    conf.set(YarnConfiguration.LINUX_CONTAINER_RUNTIME_ALLOWED_RUNTIMES,
+        "docker");
+    delegatingLinuxContainerRuntime.initialize(conf);
+    assertTrue(delegatingLinuxContainerRuntime.isRuntimeAllowed(
+        LinuxContainerRuntimeConstants.RuntimeType.DOCKER));
+    assertFalse(delegatingLinuxContainerRuntime.isRuntimeAllowed(
+        LinuxContainerRuntimeConstants.RuntimeType.DEFAULT));
+    assertFalse(delegatingLinuxContainerRuntime.isRuntimeAllowed(
+        LinuxContainerRuntimeConstants.RuntimeType.JAVASANDBOX));
+  }
+
+  @Test
+  public void testIsRuntimeAllowedJavaSandbox() throws Exception {
+    conf.set(YarnConfiguration.LINUX_CONTAINER_RUNTIME_ALLOWED_RUNTIMES,
+        "javasandbox");
+    delegatingLinuxContainerRuntime.initialize(conf);
+    assertTrue(delegatingLinuxContainerRuntime.isRuntimeAllowed(
+        LinuxContainerRuntimeConstants.RuntimeType.JAVASANDBOX));
+    assertFalse(delegatingLinuxContainerRuntime.isRuntimeAllowed(
+        LinuxContainerRuntimeConstants.RuntimeType.DEFAULT));
+    assertFalse(delegatingLinuxContainerRuntime.isRuntimeAllowed(
+        LinuxContainerRuntimeConstants.RuntimeType.DOCKER));
+  }
+
+  @Test
+  public void testIsRuntimeAllowedMultiple() throws Exception {
+    conf.set(YarnConfiguration.LINUX_CONTAINER_RUNTIME_ALLOWED_RUNTIMES,
+        "docker,javasandbox");
+    delegatingLinuxContainerRuntime.initialize(conf);
+    assertTrue(delegatingLinuxContainerRuntime.isRuntimeAllowed(
+        LinuxContainerRuntimeConstants.RuntimeType.DOCKER));
+    assertTrue(delegatingLinuxContainerRuntime.isRuntimeAllowed(
+        LinuxContainerRuntimeConstants.RuntimeType.JAVASANDBOX));
+    assertFalse(delegatingLinuxContainerRuntime.isRuntimeAllowed(
+        LinuxContainerRuntimeConstants.RuntimeType.DEFAULT));
+  }
+
+  @Test
+  public void testIsRuntimeAllowedAll() throws Exception {
+    conf.set(YarnConfiguration.LINUX_CONTAINER_RUNTIME_ALLOWED_RUNTIMES,
+        "default,docker,javasandbox");
+    delegatingLinuxContainerRuntime.initialize(conf);
+    assertTrue(delegatingLinuxContainerRuntime.isRuntimeAllowed(
+        LinuxContainerRuntimeConstants.RuntimeType.DEFAULT));
+    assertTrue(delegatingLinuxContainerRuntime.isRuntimeAllowed(
+        LinuxContainerRuntimeConstants.RuntimeType.DOCKER));
+    assertTrue(delegatingLinuxContainerRuntime.isRuntimeAllowed(
+        LinuxContainerRuntimeConstants.RuntimeType.JAVASANDBOX));
+  }
+
+  @Test
+  public void testJavaSandboxNotAllowedButPermissive() throws Exception {
+    conf.set(YarnConfiguration.LINUX_CONTAINER_RUNTIME_ALLOWED_RUNTIMES,
+        "default,docker");
+    conf.set(YarnConfiguration.YARN_CONTAINER_SANDBOX, "permissive");
+    delegatingLinuxContainerRuntime.initialize(conf);
+    ContainerRuntime runtime =
+        delegatingLinuxContainerRuntime.pickContainerRuntime(env);
+    assertTrue(runtime instanceof DefaultLinuxContainerRuntime);
+  }
+
+  @Test
+  public void testJavaSandboxNotAllowedButPermissiveDockerRequested()
+      throws Exception {
+    env.put(ContainerRuntimeConstants.ENV_CONTAINER_TYPE, "docker");
+    conf.set(YarnConfiguration.LINUX_CONTAINER_RUNTIME_ALLOWED_RUNTIMES,
+        "default,docker");
+    conf.set(YarnConfiguration.YARN_CONTAINER_SANDBOX, "permissive");
+    delegatingLinuxContainerRuntime.initialize(conf);
+    ContainerRuntime runtime =
+        delegatingLinuxContainerRuntime.pickContainerRuntime(env);
+    assertTrue(runtime instanceof DockerLinuxContainerRuntime);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0b535d9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/DockerContainers.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/DockerContainers.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/DockerContainers.md
index 4de0a6a..bf94169 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/DockerContainers.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/DockerContainers.md
@@ -71,68 +71,83 @@ request. For example:
 The following properties should be set in yarn-site.xml:
 
 ```xml
-<property>
-  <name>yarn.nodemanager.container-executor.class</name>
-  <value>org.apache.hadoop.yarn.server.nodemanager.LinuxContainerExecutor</value>
-  <description>
-    This is the container executor setting that ensures that all applications
-    are started with the LinuxContainerExecutor.
-  </description>
-</property>
-
-<property>
-  <name>yarn.nodemanager.linux-container-executor.group</name>
-  <value>hadoop</value>
-  <description>
-    The POSIX group of the NodeManager. It should match the setting in
-    "container-executor.cfg". This configuration is required for validating
-    the secure access of the container-executor binary.
-  </description>
-</property>
-
-<property>
-  <name>yarn.nodemanager.linux-container-executor.nonsecure-mode.limit-users</name>
-  <value>false</value>
-  <description>
-    Whether all applications should be run as the NodeManager process' owner.
-    When false, applications are launched instead as the application owner.
-  </description>
-</property>
-
-<property>
-  <name>yarn.nodemanager.runtime.linux.docker.allowed-container-networks</name>
-  <value>host,none,bridge</value>
-  <description>
-    Optional. A comma-separated set of networks allowed when launching
-    containers. Valid values are determined by Docker networks available from
-    `docker network ls`
-  </description>
-</property>
-
-<property>
-  <description>The network used when launching Docker containers when no
-    network is specified in the request. This network must be one of the
-    (configurable) set of allowed container networks.</description>
-  <name>yarn.nodemanager.runtime.linux.docker.default-container-network</name>
-  <value>host</value>
-</property>
-
-<property>
-  <name>yarn.nodemanager.runtime.linux.docker.privileged-containers.allowed</name>
-  <value>false</value>
-  <description>
-    Optional. Whether applications are allowed to run in privileged containers.
-  </description>
-</property>
-
-<property>
-  <name>yarn.nodemanager.runtime.linux.docker.privileged-containers.acl</name>
-  <value></value>
-  <description>
-    Optional. A comma-separated list of users who are allowed to request
-    privileged contains if privileged containers are allowed.
-  </description>
-</property>
+<configuration>
+  <property>
+    <name>yarn.nodemanager.container-executor.class</name>
+    <value>org.apache.hadoop.yarn.server.nodemanager.LinuxContainerExecutor</value>
+    <description>
+      This is the container executor setting that ensures that all applications
+      are started with the LinuxContainerExecutor.
+    </description>
+  </property>
+
+  <property>
+    <name>yarn.nodemanager.linux-container-executor.group</name>
+    <value>hadoop</value>
+    <description>
+      The POSIX group of the NodeManager. It should match the setting in
+      "container-executor.cfg". This configuration is required for validating
+      the secure access of the container-executor binary.
+    </description>
+  </property>
+
+  <property>
+    <name>yarn.nodemanager.linux-container-executor.nonsecure-mode.limit-users</name>
+    <value>false</value>
+    <description>
+      Whether all applications should be run as the NodeManager process' owner.
+      When false, applications are launched instead as the application owner.
+    </description>
+  </property>
+
+  <property>
+    <name>yarn.nodemanager.runtime.linux.allowed-runtimes</name>
+    <value>default,docker</value>
+    <description>
+      Comma separated list of runtimes that are allowed when using
+      LinuxContainerExecutor. The allowed values are default, docker, and
+      javasandbox.
+    </description>
+  </property>
+
+  <property>
+    <name>yarn.nodemanager.runtime.linux.docker.allowed-container-networks</name>
+    <value>host,none,bridge</value>
+    <description>
+      Optional. A comma-separated set of networks allowed when launching
+      containers. Valid values are determined by Docker networks available from
+      `docker network ls`
+    </description>
+  </property>
+
+  <property>
+    <name>yarn.nodemanager.runtime.linux.docker.default-container-network</name>
+    <value>host</value>
+    <description>
+      The network used when launching Docker containers when no
+      network is specified in the request. This network must be one of the
+      (configurable) set of allowed container networks.
+    </description>
+  </property>
+
+  <property>
+    <name>yarn.nodemanager.runtime.linux.docker.privileged-containers.allowed</name>
+    <value>false</value>
+    <description>
+      Optional. Whether applications are allowed to run in privileged
+      containers.
+    </description>
+  </property>
+
+  <property>
+    <name>yarn.nodemanager.runtime.linux.docker.privileged-containers.acl</name>
+    <value></value>
+    <description>
+      Optional. A comma-separated list of users who are allowed to request
+      privileged contains if privileged containers are allowed.
+    </description>
+  </property>
+</configuration>
 ```
 
 In addition, a container-executer.cfg file must exist and contain settings for


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


[45/50] [abbrv] hadoop git commit: HDFS-10391. Always enable NameNode service RPC port. Contributed by Gergely Novak.

Posted by as...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa4b6fbe/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java
index b3bb3dd..4d04970 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java
@@ -171,15 +171,18 @@ public class TestEditLogTailer {
     for (int i = 0; i < 5; i++) {
       try {
         // Have to specify IPC ports so the NNs can talk to each other.
-        int[] ports = ServerSocketUtil.getPorts(3);
+        int[] ports = ServerSocketUtil.getPorts(6);
         MiniDFSNNTopology topology = new MiniDFSNNTopology()
             .addNameservice(new MiniDFSNNTopology.NSConf("ns1")
                 .addNN(new MiniDFSNNTopology.NNConf("nn1")
-                    .setIpcPort(ports[0]))
+                    .setIpcPort(ports[0])
+                    .setServicePort(ports[1]))
                 .addNN(new MiniDFSNNTopology.NNConf("nn2")
-                    .setIpcPort(ports[1]))
+                    .setIpcPort(ports[2])
+                    .setServicePort(ports[3]))
                 .addNN(new MiniDFSNNTopology.NNConf("nn3")
-                    .setIpcPort(ports[2])));
+                    .setIpcPort(ports[4])
+                    .setServicePort(ports[5])));
 
         cluster = new MiniDFSCluster.Builder(conf)
           .nnTopology(topology)
@@ -219,11 +222,14 @@ public class TestEditLogTailer {
     MiniDFSNNTopology topology = new MiniDFSNNTopology()
         .addNameservice(new MiniDFSNNTopology.NSConf("ns1")
             .addNN(new MiniDFSNNTopology.NNConf("nn1")
-                .setIpcPort(ServerSocketUtil.getPort(0, 100)))
+                .setIpcPort(ServerSocketUtil.getPort(0, 100))
+                .setServicePort(ServerSocketUtil.getPort(0, 100)))
             .addNN(new MiniDFSNNTopology.NNConf("nn2")
-                .setIpcPort(ServerSocketUtil.getPort(0, 100)))
+                .setIpcPort(ServerSocketUtil.getPort(0, 100))
+                .setServicePort(ServerSocketUtil.getPort(0, 100)))
             .addNN(new MiniDFSNNTopology.NNConf("nn3")
-                .setIpcPort(ServerSocketUtil.getPort(0, 100))));
+                .setIpcPort(ServerSocketUtil.getPort(0, 100))
+                .setServicePort(ServerSocketUtil.getPort(0, 100))));
 
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
         .nnTopology(topology)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa4b6fbe/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdmin.java
index f1f5793..f2274f9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdmin.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hdfs.tools;
 
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -179,9 +180,11 @@ public class TestDFSHAAdmin {
     Mockito.doReturn(STANDBY_READY_RESULT).when(mockProtocol)
         .getServiceStatus();
     assertEquals(0, runTool("-getAllServiceState"));
-    assertOutputContains(String.format("%-50s %-10s", (HOST_A + ":" + 12345),
+    assertOutputContains(String.format("%-50s %-10s", (HOST_A + ":" +
+            DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT),
         STANDBY_READY_RESULT.getState()));
-    assertOutputContains(String.format("%-50s %-10s", (HOST_B + ":" + 12345),
+    assertOutputContains(String.format("%-50s %-10s", (HOST_B + ":" +
+            DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT),
         STANDBY_READY_RESULT.getState()));
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa4b6fbe/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdminMiniCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdminMiniCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdminMiniCluster.java
index a21a31d..d49a3cf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdminMiniCluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdminMiniCluster.java
@@ -77,7 +77,7 @@ public class TestDFSHAAdminMiniCluster {
     tool.setErrOut(new PrintStream(errOutBytes));
     cluster.waitActive();
     
-    nn1Port = cluster.getNameNodePort(0);
+    nn1Port = cluster.getNameNodeServicePort(0);
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa4b6fbe/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSZKFailoverController.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSZKFailoverController.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSZKFailoverController.java
index bbb787e..9943260 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSZKFailoverController.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSZKFailoverController.java
@@ -88,9 +88,11 @@ public class TestDFSZKFailoverController extends ClientBaseWithFixes {
     MiniDFSNNTopology topology = new MiniDFSNNTopology()
     .addNameservice(new MiniDFSNNTopology.NSConf("ns1")
         .addNN(new MiniDFSNNTopology.NNConf("nn1")
-            .setIpcPort(ServerSocketUtil.getPort(10021, 100)))
+            .setIpcPort(ServerSocketUtil.getPort(10021, 100))
+            .setServicePort(ServerSocketUtil.getPort(10025, 100)))
         .addNN(new MiniDFSNNTopology.NNConf("nn2")
-            .setIpcPort(ServerSocketUtil.getPort(10022, 100))));
+            .setIpcPort(ServerSocketUtil.getPort(10022, 100))
+            .setServicePort(ServerSocketUtil.getPort(10026, 100))));
     cluster = new MiniDFSCluster.Builder(conf)
         .nnTopology(topology)
         .numDataNodes(0)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa4b6fbe/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestGetConf.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestGetConf.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestGetConf.java
index 942719e..36d57a9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestGetConf.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestGetConf.java
@@ -24,6 +24,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMESERVICES;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
@@ -121,13 +122,13 @@ public class TestGetConf {
       TestType type, HdfsConfiguration conf) throws IOException {
     switch (type) {
     case NAMENODE:
-      return DFSUtil.getNNServiceRpcAddressesForCluster(conf);
+      return DFSUtil.getNNServiceRpcAddresses(conf);
     case BACKUP:
       return DFSUtil.getBackupNodeAddresses(conf);
     case SECONDARY:
       return DFSUtil.getSecondaryNameNodeAddresses(conf);
     case NNRPCADDRESSES:
-      return DFSUtil.getNNServiceRpcAddressesForCluster(conf);
+      return DFSUtil.getNNServiceRpcAddresses(conf);
     }
     return null;
   }
@@ -278,10 +279,12 @@ public class TestGetConf {
   public void testNonFederation() throws Exception {
     HdfsConfiguration conf = new HdfsConfiguration(false);
   
-    // Returned namenode address should match default address
+    // Returned namenode address should match the default service address
     conf.set(FS_DEFAULT_NAME_KEY, "hdfs://localhost:1000");
-    verifyAddresses(conf, TestType.NAMENODE, false, "localhost:1000");
-    verifyAddresses(conf, TestType.NNRPCADDRESSES, true, "localhost:1000");
+    verifyAddresses(conf, TestType.NAMENODE, false, "localhost:" +
+        DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT);
+    verifyAddresses(conf, TestType.NNRPCADDRESSES, true, "localhost:" +
+        DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT);
   
     // Returned address should match backupnode RPC address
     conf.set(DFS_NAMENODE_BACKUP_ADDRESS_KEY,"localhost:1001");
@@ -297,12 +300,14 @@ public class TestGetConf {
     conf.set(DFS_NAMENODE_RPC_ADDRESS_KEY, "localhost:1001");
     verifyAddresses(conf, TestType.NAMENODE, false, "localhost:1000");
     verifyAddresses(conf, TestType.NNRPCADDRESSES, true, "localhost:1000");
-  
-    // Returned address should match RPC address
+
+    // Returned namenode address should match the default service address
     conf = new HdfsConfiguration();
     conf.set(DFS_NAMENODE_RPC_ADDRESS_KEY, "localhost:1001");
-    verifyAddresses(conf, TestType.NAMENODE, false, "localhost:1001");
-    verifyAddresses(conf, TestType.NNRPCADDRESSES, true, "localhost:1001");
+    verifyAddresses(conf, TestType.NAMENODE, false, "localhost:" +
+        DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT);
+    verifyAddresses(conf, TestType.NNRPCADDRESSES, true, "localhost:" +
+        DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT);
   }
 
   /**
@@ -330,23 +335,6 @@ public class TestGetConf {
     verifyAddresses(conf, TestType.BACKUP, false, backupAddresses);
     verifyAddresses(conf, TestType.SECONDARY, false, secondaryAddresses);
     verifyAddresses(conf, TestType.NNRPCADDRESSES, true, nnAddresses);
-  
-    // Test to ensure namenode, backup, secondary namenode addresses and 
-    // namenode rpc addresses are  returned from federation configuration. 
-    // Returned namenode addresses are based on regular RPC address
-    // in the absence of service RPC address.
-    conf = new HdfsConfiguration(false);
-    setupNameServices(conf, nsCount);
-    nnAddresses = setupAddress(conf,
-        DFS_NAMENODE_RPC_ADDRESS_KEY, nsCount, 1000);
-    backupAddresses = setupAddress(conf,
-        DFS_NAMENODE_BACKUP_ADDRESS_KEY, nsCount, 2000);
-    secondaryAddresses = setupAddress(conf,
-        DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY, nsCount, 3000);
-    verifyAddresses(conf, TestType.NAMENODE, false, nnAddresses);
-    verifyAddresses(conf, TestType.BACKUP, false, backupAddresses);
-    verifyAddresses(conf, TestType.SECONDARY, false, secondaryAddresses);
-    verifyAddresses(conf, TestType.NNRPCADDRESSES, true, nnAddresses);
   }
   
   @Test(timeout=10000)


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


[43/50] [abbrv] hadoop git commit: YARN-7072. Add a new log aggregation file format controller. Contributed by Xuan Gong.

Posted by as...@apache.org.
YARN-7072. Add a new log aggregation file format controller. Contributed by Xuan Gong.


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

Branch: refs/heads/YARN-5972
Commit: 3fddabc2fe4fbdb8ef3f9ce7558955c4f0794dcc
Parents: 8edc605
Author: Junping Du <ju...@apache.org>
Authored: Fri Sep 8 15:16:19 2017 -0700
Committer: Junping Du <ju...@apache.org>
Committed: Fri Sep 8 15:16:19 2017 -0700

----------------------------------------------------------------------
 .../file/tfile/BoundedRangeFileInputStream.java |    2 +-
 .../hadoop/io/file/tfile/Compression.java       |    6 +-
 .../file/tfile/SimpleBufferedOutputStream.java  |    2 +-
 .../apache/hadoop/yarn/client/cli/LogsCLI.java  |   25 +-
 .../logaggregation/ContainerLogFileInfo.java    |   93 ++
 .../yarn/logaggregation/ContainerLogMeta.java   |    8 +-
 .../logaggregation/LogAggregationUtils.java     |   27 +
 .../yarn/logaggregation/LogCLIHelpers.java      |   20 +-
 .../yarn/logaggregation/LogToolUtils.java       |   26 +
 .../logaggregation/PerContainerLogFileInfo.java |   93 --
 .../LogAggregationFileController.java           |   45 +-
 .../ifile/IndexedFileAggregatedLogsBlock.java   |  275 +++++
 .../LogAggregationIndexedFileController.java    | 1056 ++++++++++++++++++
 .../filecontroller/ifile/package-info.java      |   21 +
 .../tfile/LogAggregationTFileController.java    |   10 +-
 .../TestLogAggregationIndexFileController.java  |  316 ++++++
 .../webapp/TestAHSWebServices.java              |    8 +-
 .../server/webapp/dao/ContainerLogsInfo.java    |   10 +-
 .../webapp/dao/NMContainerLogsInfo.java         |    8 +-
 .../nodemanager/webapp/TestNMWebServices.java   |    8 +-
 20 files changed, 1886 insertions(+), 173 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fddabc2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/BoundedRangeFileInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/BoundedRangeFileInputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/BoundedRangeFileInputStream.java
index e7f4c83..050c15b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/BoundedRangeFileInputStream.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/BoundedRangeFileInputStream.java
@@ -28,7 +28,7 @@ import org.apache.hadoop.fs.FSDataInputStream;
  * BoundedRangeFileInputStream on top of the same FSDataInputStream and they
  * would not interfere with each other.
  */
-class BoundedRangeFileInputStream extends InputStream {
+public class BoundedRangeFileInputStream extends InputStream {
 
   private FSDataInputStream in;
   private long pos;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fddabc2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/Compression.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/Compression.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/Compression.java
index f82f4df..fa85ed7 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/Compression.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/Compression.java
@@ -43,7 +43,7 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_
 /**
  * Compression related stuff.
  */
-final class Compression {
+public final class Compression {
   static final Logger LOG = LoggerFactory.getLogger(Compression.class);
 
   /**
@@ -75,7 +75,7 @@ final class Compression {
   /**
    * Compression algorithms.
    */
-  enum Algorithm {
+  public enum Algorithm {
     LZO(TFile.COMPRESSION_LZO) {
       private transient boolean checked = false;
       private static final String defaultClazz =
@@ -348,7 +348,7 @@ final class Compression {
     }
   }
 
-  static Algorithm getCompressionAlgorithmByName(String compressName) {
+  public static Algorithm getCompressionAlgorithmByName(String compressName) {
     Algorithm[] algos = Algorithm.class.getEnumConstants();
 
     for (Algorithm a : algos) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fddabc2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/SimpleBufferedOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/SimpleBufferedOutputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/SimpleBufferedOutputStream.java
index a26a02d..0a194a3 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/SimpleBufferedOutputStream.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/SimpleBufferedOutputStream.java
@@ -25,7 +25,7 @@ import java.io.OutputStream;
  * A simplified BufferedOutputStream with borrowed buffer, and allow users to
  * see how much data have been buffered.
  */
-class SimpleBufferedOutputStream extends FilterOutputStream {
+public class SimpleBufferedOutputStream extends FilterOutputStream {
   protected byte buf[]; // the borrowed buffer
   protected int count = 0; // bytes used in buffer.
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fddabc2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
index 5528412..1a3db26 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
@@ -62,9 +62,10 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.client.api.YarnClient;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.logaggregation.ContainerLogFileInfo;
 import org.apache.hadoop.yarn.logaggregation.ContainerLogsRequest;
 import org.apache.hadoop.yarn.logaggregation.LogCLIHelpers;
-import org.apache.hadoop.yarn.logaggregation.PerContainerLogFileInfo;
+import org.apache.hadoop.yarn.logaggregation.LogToolUtils;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 import org.apache.hadoop.yarn.webapp.util.YarnWebServiceUtils;
 import org.codehaus.jettison.json.JSONArray;
@@ -411,10 +412,10 @@ public class LogsCLI extends Configured implements Tool {
     return false;
   }
 
-  private List<Pair<PerContainerLogFileInfo, String>> getContainerLogFiles(
+  private List<Pair<ContainerLogFileInfo, String>> getContainerLogFiles(
       Configuration conf, String containerIdStr, String nodeHttpAddress)
       throws IOException {
-    List<Pair<PerContainerLogFileInfo, String>> logFileInfos
+    List<Pair<ContainerLogFileInfo, String>> logFileInfos
         = new ArrayList<>();
     Client webServiceClient = Client.create();
     try {
@@ -453,12 +454,12 @@ public class LogsCLI extends Configured implements Tool {
             if (ob instanceof JSONArray) {
               JSONArray obArray = (JSONArray)ob;
               for (int j = 0; j < obArray.length(); j++) {
-                logFileInfos.add(new Pair<PerContainerLogFileInfo, String>(
+                logFileInfos.add(new Pair<ContainerLogFileInfo, String>(
                     generatePerContainerLogFileInfoFromJSON(
                         obArray.getJSONObject(j)), aggregateType));
               }
             } else if (ob instanceof JSONObject) {
-              logFileInfos.add(new Pair<PerContainerLogFileInfo, String>(
+              logFileInfos.add(new Pair<ContainerLogFileInfo, String>(
                   generatePerContainerLogFileInfoFromJSON(
                       (JSONObject)ob), aggregateType));
             }
@@ -477,7 +478,7 @@ public class LogsCLI extends Configured implements Tool {
     return logFileInfos;
   }
 
-  private PerContainerLogFileInfo generatePerContainerLogFileInfoFromJSON(
+  private ContainerLogFileInfo generatePerContainerLogFileInfoFromJSON(
       JSONObject meta) throws JSONException {
     String fileName = meta.has("fileName") ?
         meta.getString("fileName") : "N/A";
@@ -485,7 +486,7 @@ public class LogsCLI extends Configured implements Tool {
         meta.getString("fileSize") : "N/A";
     String lastModificationTime = meta.has("lastModifiedTime") ?
         meta.getString("lastModifiedTime") : "N/A";
-    return new PerContainerLogFileInfo(fileName, fileSize,
+    return new ContainerLogFileInfo(fileName, fileSize,
         lastModificationTime);
   }
 
@@ -506,7 +507,7 @@ public class LogsCLI extends Configured implements Tool {
       return -1;
     }
     String nodeId = request.getNodeId();
-    PrintStream out = logCliHelper.createPrintStream(localDir, nodeId,
+    PrintStream out = LogToolUtils.createPrintStream(localDir, nodeId,
         containerIdStr);
     try {
       Set<String> matchedFiles = getMatchedContainerLogFiles(request,
@@ -1235,9 +1236,9 @@ public class LogsCLI extends Configured implements Tool {
     outStream.printf(LogCLIHelpers.PER_LOG_FILE_INFO_PATTERN,
         "LogFile", "LogLength", "LastModificationTime", "LogAggregationType");
     outStream.println(StringUtils.repeat("=", containerString.length() * 2));
-    List<Pair<PerContainerLogFileInfo, String>> infos = getContainerLogFiles(
+    List<Pair<ContainerLogFileInfo, String>> infos = getContainerLogFiles(
         getConf(), containerId, nodeHttpAddress);
-    for (Pair<PerContainerLogFileInfo, String> info : infos) {
+    for (Pair<ContainerLogFileInfo, String> info : infos) {
       outStream.printf(LogCLIHelpers.PER_LOG_FILE_INFO_PATTERN,
           info.getKey().getFileName(), info.getKey().getFileSize(),
           info.getKey().getLastModifiedTime(), info.getValue());
@@ -1249,11 +1250,11 @@ public class LogsCLI extends Configured implements Tool {
       boolean useRegex) throws IOException {
     // fetch all the log files for the container
     // filter the log files based on the given -log_files pattern
-    List<Pair<PerContainerLogFileInfo, String>> allLogFileInfos=
+    List<Pair<ContainerLogFileInfo, String>> allLogFileInfos=
         getContainerLogFiles(getConf(), request.getContainerId(),
             request.getNodeHttpAddress());
     List<String> fileNames = new ArrayList<String>();
-    for (Pair<PerContainerLogFileInfo, String> fileInfo : allLogFileInfos) {
+    for (Pair<ContainerLogFileInfo, String> fileInfo : allLogFileInfos) {
       fileNames.add(fileInfo.getKey().getFileName());
     }
     return getMatchedLogFiles(request, fileNames,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fddabc2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/ContainerLogFileInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/ContainerLogFileInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/ContainerLogFileInfo.java
new file mode 100644
index 0000000..b461ebb
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/ContainerLogFileInfo.java
@@ -0,0 +1,93 @@
+/**
+ * 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.logaggregation;
+
+/**
+ * ContainerLogFileInfo represents the meta data for a container log file,
+ * which includes:
+ * <ul>
+ *   <li>The filename of the container log.</li>
+ *   <li>The size of the container log.</li>
+ *   <li>The last modification time of the container log.</li>
+ * </ul>
+ *
+ */
+public class ContainerLogFileInfo {
+  private String fileName;
+  private String fileSize;
+  private String lastModifiedTime;
+
+  //JAXB needs this
+  public ContainerLogFileInfo() {}
+
+  public ContainerLogFileInfo(String fileName, String fileSize,
+      String lastModifiedTime) {
+    this.setFileName(fileName);
+    this.setFileSize(fileSize);
+    this.setLastModifiedTime(lastModifiedTime);
+  }
+
+  public String getFileName() {
+    return fileName;
+  }
+
+  public void setFileName(String fileName) {
+    this.fileName = fileName;
+  }
+
+  public String getFileSize() {
+    return fileSize;
+  }
+
+  public void setFileSize(String fileSize) {
+    this.fileSize = fileSize;
+  }
+
+  public String getLastModifiedTime() {
+    return lastModifiedTime;
+  }
+
+  public void setLastModifiedTime(String lastModifiedTime) {
+    this.lastModifiedTime = lastModifiedTime;
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + ((fileName == null) ? 0 : fileName.hashCode());
+    result = prime * result + ((fileSize == null) ? 0 : fileSize.hashCode());
+    result = prime * result + ((lastModifiedTime == null) ?
+        0 : lastModifiedTime.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object otherObj) {
+    if (otherObj == this) {
+      return true;
+    }
+    if (!(otherObj instanceof ContainerLogFileInfo)) {
+      return false;
+    }
+    ContainerLogFileInfo other = (ContainerLogFileInfo)otherObj;
+    return other.fileName.equals(fileName) && other.fileSize.equals(fileSize)
+        && other.lastModifiedTime.equals(lastModifiedTime);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fddabc2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/ContainerLogMeta.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/ContainerLogMeta.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/ContainerLogMeta.java
index 26a620e..4c6b0de 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/ContainerLogMeta.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/ContainerLogMeta.java
@@ -26,14 +26,14 @@ import java.util.List;
  * <ul>
  *   <li>The Container Id.</li>
  *   <li>The NodeManager Id.</li>
- *   <li>A list of {@link PerContainerLogFileInfo}.</li>
+ *   <li>A list of {@link ContainerLogFileInfo}.</li>
  * </ul>
  *
  */
 public class ContainerLogMeta {
   private String containerId;
   private String nodeId;
-  private List<PerContainerLogFileInfo> logMeta;
+  private List<ContainerLogFileInfo> logMeta;
 
   public ContainerLogMeta(String containerId, String nodeId) {
     this.containerId = containerId;
@@ -51,11 +51,11 @@ public class ContainerLogMeta {
 
   public void addLogMeta(String fileName, String fileSize,
       String lastModificationTime) {
-    logMeta.add(new PerContainerLogFileInfo(fileName, fileSize,
+    logMeta.add(new ContainerLogFileInfo(fileName, fileSize,
         lastModificationTime));
   }
 
-  public List<PerContainerLogFileInfo> getContainerLogMeta() {
+  public List<ContainerLogFileInfo> getContainerLogMeta() {
     return this.logMeta;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fddabc2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogAggregationUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogAggregationUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogAggregationUtils.java
index 6d04c29..edf2cf3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogAggregationUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogAggregationUtils.java
@@ -30,6 +30,9 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 
 import com.google.common.annotations.VisibleForTesting;
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
 
 @Private
 public class LogAggregationUtils {
@@ -200,6 +203,30 @@ public class LogAggregationUtils {
    * @param conf the configuration
    * @param appId the applicationId
    * @param appOwner the application owner
+   * @param remoteRootLogDir the remote root log directory
+   * @param suffix the log directory suffix
+   * @return the list of available log files
+   * @throws IOException if there is no log file available
+   */
+  public static List<FileStatus> getRemoteNodeFileList(
+      Configuration conf, ApplicationId appId, String appOwner,
+      org.apache.hadoop.fs.Path remoteRootLogDir, String suffix)
+      throws IOException {
+    Path remoteAppLogDir = getRemoteAppLogDir(conf, appId, appOwner,
+        remoteRootLogDir, suffix);
+    List<FileStatus> nodeFiles = new ArrayList<>();
+    Path qualifiedLogDir =
+        FileContext.getFileContext(conf).makeQualified(remoteAppLogDir);
+    nodeFiles.addAll(Arrays.asList(FileContext.getFileContext(
+        qualifiedLogDir.toUri(), conf).util().listStatus(remoteAppLogDir)));
+    return nodeFiles;
+  }
+
+  /**
+   * Get all available log files under remote app log directory.
+   * @param conf the configuration
+   * @param appId the applicationId
+   * @param appOwner the application owner
    * @return the iterator of available log files
    * @throws IOException if there is no log file available
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fddabc2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java
index 0068eae..97b78ec 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java
@@ -22,8 +22,6 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.nio.file.AccessDeniedException;
-import java.nio.file.Files;
-import java.nio.file.Paths;
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
@@ -229,7 +227,7 @@ public class LogCLIHelpers implements Configurable {
       out.printf(PER_LOG_FILE_INFO_PATTERN, "LogFile", "LogLength",
           "LastModificationTime", "LogAggregationType");
       out.println(StringUtils.repeat("=", containerString.length() * 2));
-      for (PerContainerLogFileInfo logMeta : containerLogMeta
+      for (ContainerLogFileInfo logMeta : containerLogMeta
           .getContainerLogMeta()) {
         out.printf(PER_LOG_FILE_INFO_PATTERN, logMeta.getFileName(),
             logMeta.getFileSize(), logMeta.getLastModifiedTime(), "AGGREGATED");
@@ -345,20 +343,6 @@ public class LogCLIHelpers implements Configurable {
         + ". Error message found: " + errorMessage);
   }
 
-  @Private
-  public PrintStream createPrintStream(String localDir, String nodeId,
-      String containerId) throws IOException {
-    PrintStream out = System.out;
-    if(localDir != null && !localDir.isEmpty()) {
-      Path nodePath = new Path(localDir, LogAggregationUtils
-          .getNodeString(nodeId));
-      Files.createDirectories(Paths.get(nodePath.toString()));
-      Path containerLogPath = new Path(nodePath, containerId);
-      out = new PrintStream(containerLogPath.toString(), "UTF-8");
-    }
-    return out;
-  }
-
   public void closePrintStream(PrintStream out) {
     if (out != System.out) {
       IOUtils.closeQuietly(out);
@@ -379,7 +363,7 @@ public class LogCLIHelpers implements Configurable {
       return logTypes;
     }
     for (ContainerLogMeta logMeta: containersLogMeta) {
-      for (PerContainerLogFileInfo fileInfo : logMeta.getContainerLogMeta()) {
+      for (ContainerLogFileInfo fileInfo : logMeta.getContainerLogMeta()) {
         logTypes.add(fileInfo.getFileName());
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fddabc2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogToolUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogToolUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogToolUtils.java
index ddee445..90faa19 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogToolUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogToolUtils.java
@@ -21,11 +21,15 @@ import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.io.PrintStream;
 import java.nio.channels.Channels;
 import java.nio.channels.FileChannel;
 import java.nio.channels.WritableByteChannel;
 import java.nio.charset.Charset;
+import java.nio.file.Files;
+import java.nio.file.Paths;
 import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.fs.Path;
 
 /**
  * This class contains several utility function which could be used in different
@@ -158,4 +162,26 @@ public final class LogToolUtils {
     }
   }
 
+
+  /**
+   * Create the container log file under given (local directory/nodeId) and
+   * return the PrintStream object.
+   * @param localDir the Local Dir
+   * @param nodeId the NodeId
+   * @param containerId the ContainerId
+   * @return the printStream object
+   * @throws IOException if an I/O error occurs
+   */
+  public static PrintStream createPrintStream(String localDir, String nodeId,
+      String containerId) throws IOException {
+    PrintStream out = System.out;
+    if(localDir != null && !localDir.isEmpty()) {
+      Path nodePath = new Path(localDir, LogAggregationUtils
+          .getNodeString(nodeId));
+      Files.createDirectories(Paths.get(nodePath.toString()));
+      Path containerLogPath = new Path(nodePath, containerId);
+      out = new PrintStream(containerLogPath.toString(), "UTF-8");
+    }
+    return out;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fddabc2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/PerContainerLogFileInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/PerContainerLogFileInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/PerContainerLogFileInfo.java
deleted file mode 100644
index 867815f..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/PerContainerLogFileInfo.java
+++ /dev/null
@@ -1,93 +0,0 @@
-/**
- * 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.logaggregation;
-
-/**
- * PerContainerLogFileInfo represents the meta data for a container log file,
- * which includes:
- * <ul>
- *   <li>The filename of the container log.</li>
- *   <li>The size of the container log.</li>
- *   <li>The last modification time of the container log.</li>
- * </ul>
- *
- */
-public class PerContainerLogFileInfo {
-  private String fileName;
-  private String fileSize;
-  private String lastModifiedTime;
-
-  //JAXB needs this
-  public PerContainerLogFileInfo() {}
-
-  public PerContainerLogFileInfo(String fileName, String fileSize,
-      String lastModifiedTime) {
-    this.setFileName(fileName);
-    this.setFileSize(fileSize);
-    this.setLastModifiedTime(lastModifiedTime);
-  }
-
-  public String getFileName() {
-    return fileName;
-  }
-
-  public void setFileName(String fileName) {
-    this.fileName = fileName;
-  }
-
-  public String getFileSize() {
-    return fileSize;
-  }
-
-  public void setFileSize(String fileSize) {
-    this.fileSize = fileSize;
-  }
-
-  public String getLastModifiedTime() {
-    return lastModifiedTime;
-  }
-
-  public void setLastModifiedTime(String lastModifiedTime) {
-    this.lastModifiedTime = lastModifiedTime;
-  }
-
-  @Override
-  public int hashCode() {
-    final int prime = 31;
-    int result = 1;
-    result = prime * result + ((fileName == null) ? 0 : fileName.hashCode());
-    result = prime * result + ((fileSize == null) ? 0 : fileSize.hashCode());
-    result = prime * result + ((lastModifiedTime == null) ?
-        0 : lastModifiedTime.hashCode());
-    return result;
-  }
-
-  @Override
-  public boolean equals(Object otherObj) {
-    if (otherObj == this) {
-      return true;
-    }
-    if (!(otherObj instanceof PerContainerLogFileInfo)) {
-      return false;
-    }
-    PerContainerLogFileInfo other = (PerContainerLogFileInfo)otherObj;
-    return other.fileName.equals(fileName) && other.fileSize.equals(fileSize)
-        && other.lastModifiedTime.equals(lastModifiedTime);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fddabc2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationFileController.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationFileController.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationFileController.java
index 39f3dc3..5df900b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationFileController.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationFileController.java
@@ -25,9 +25,6 @@ import com.google.common.collect.Sets;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.OutputStream;
-import java.io.PrintStream;
-import java.nio.file.Files;
-import java.nio.file.Paths;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -37,6 +34,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
@@ -91,6 +89,12 @@ public abstract class LogAggregationFileController {
   protected static final FsPermission APP_DIR_PERMISSIONS = FsPermission
       .createImmutable((short) 0770);
 
+  /**
+   * Umask for the log file.
+   */
+  protected static final FsPermission APP_LOG_FILE_UMASK = FsPermission
+      .createImmutable((short) (0640 ^ 0777));
+
   // This is temporary solution. The configuration will be deleted once
   // we find a more scalable method to only write a single log file per LRS.
   private static final String NM_LOG_AGGREGATION_NUM_LOG_FILES_SIZE_PER_APP
@@ -98,6 +102,11 @@ public abstract class LogAggregationFileController {
   private static final int
       DEFAULT_NM_LOG_AGGREGATION_NUM_LOG_FILES_SIZE_PER_APP = 30;
 
+  // This is temporary solution. The configuration will be deleted once we have
+  // the FileSystem API to check whether append operation is supported or not.
+  public static final String LOG_AGGREGATION_FS_SUPPORT_APPEND
+      = YarnConfiguration.YARN_PREFIX+ "log-aggregation.fs-support-append";
+
   protected Configuration conf;
   protected Path remoteRootLogDir;
   protected String remoteRootLogDirSuffix;
@@ -178,19 +187,6 @@ public abstract class LogAggregationFileController {
   public abstract void postWrite(LogAggregationFileControllerContext record)
       throws Exception;
 
-  protected PrintStream createPrintStream(String localDir, String nodeId,
-      String containerId) throws IOException {
-    PrintStream out = System.out;
-    if(localDir != null && !localDir.isEmpty()) {
-      Path nodePath = new Path(localDir, LogAggregationUtils
-          .getNodeString(nodeId));
-      Files.createDirectories(Paths.get(nodePath.toString()));
-      Path containerLogPath = new Path(nodePath, containerId);
-      out = new PrintStream(containerLogPath.toString(), "UTF-8");
-    }
-    return out;
-  }
-
   protected void closePrintStream(OutputStream out) {
     if (out != System.out) {
       IOUtils.cleanupWithLogger(LOG, out);
@@ -481,4 +477,21 @@ public abstract class LogAggregationFileController {
       LOG.error("Failed to clean old logs", e);
     }
   }
+
+  /**
+   * Create the aggregated log suffix. The LogAggregationFileController
+   * should call this to get the suffix and append the suffix to the end
+   * of each log. This would keep the aggregated log format consistent.
+   *
+   * @param fileName the File Name
+   * @return the aggregated log suffix String
+   */
+  protected String aggregatedLogSuffix(String fileName) {
+    StringBuilder sb = new StringBuilder();
+    String endOfFile = "End of LogType:" + fileName;
+    sb.append("\n" + endOfFile + "\n");
+    sb.append(StringUtils.repeat("*", endOfFile.length() + 50)
+        + "\n\n");
+    return sb.toString();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fddabc2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/IndexedFileAggregatedLogsBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/IndexedFileAggregatedLogsBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/IndexedFileAggregatedLogsBlock.java
new file mode 100644
index 0000000..c4cbfda
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/IndexedFileAggregatedLogsBlock.java
@@ -0,0 +1,275 @@
+/**
+ * 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.logaggregation.filecontroller.ifile;
+
+import static org.apache.hadoop.yarn.webapp.YarnWebParams.APP_OWNER;
+import static org.apache.hadoop.yarn.webapp.YarnWebParams.CONTAINER_ID;
+import static org.apache.hadoop.yarn.webapp.YarnWebParams.CONTAINER_LOG_TYPE;
+import static org.apache.hadoop.yarn.webapp.YarnWebParams.ENTITY_STRING;
+import static org.apache.hadoop.yarn.webapp.YarnWebParams.NM_NODENAME;
+
+import com.google.inject.Inject;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.io.compress.Decompressor;
+import org.apache.hadoop.io.file.tfile.BoundedRangeFileInputStream;
+import org.apache.hadoop.io.file.tfile.Compression;
+import org.apache.hadoop.io.file.tfile.Compression.Algorithm;
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.logaggregation.LogAggregationUtils;
+import org.apache.hadoop.yarn.logaggregation.filecontroller.LogAggregationHtmlBlock;
+import org.apache.hadoop.yarn.logaggregation.filecontroller.ifile.LogAggregationIndexedFileController.IndexedFileLogMeta;
+import org.apache.hadoop.yarn.logaggregation.filecontroller.ifile.LogAggregationIndexedFileController.IndexedLogsMeta;
+import org.apache.hadoop.yarn.logaggregation.filecontroller.ifile.LogAggregationIndexedFileController.IndexedPerAggregationLogMeta;
+import org.apache.hadoop.yarn.util.Times;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.PRE;
+
+/**
+ * The Aggregated Logs Block implementation for Indexed File.
+ */
+@InterfaceAudience.LimitedPrivate({"YARN", "MapReduce"})
+public class IndexedFileAggregatedLogsBlock extends LogAggregationHtmlBlock {
+
+  private final LogAggregationIndexedFileController fileController;
+  private final Configuration conf;
+
+  @Inject
+  public IndexedFileAggregatedLogsBlock(ViewContext ctx,
+      Configuration conf,
+      LogAggregationIndexedFileController fileController) {
+    super(ctx);
+    this.conf = conf;
+    this.fileController = fileController;
+  }
+
+  @Override
+  protected void render(Block html) {
+    BlockParameters params = verifyAndParseParameters(html);
+    if (params == null) {
+      return;
+    }
+
+    ApplicationId appId = params.getAppId();
+    ContainerId containerId = params.getContainerId();
+    NodeId nodeId = params.getNodeId();
+    String appOwner = params.getAppOwner();
+    String logEntity = params.getLogEntity();
+    long start = params.getStartIndex();
+    long end = params.getEndIndex();
+
+    List<FileStatus> nodeFiles = null;
+    try {
+      nodeFiles = LogAggregationUtils
+          .getRemoteNodeFileList(conf, appId, appOwner,
+              this.fileController.getRemoteRootLogDir(),
+              this.fileController.getRemoteRootLogDirSuffix());
+    } catch(Exception ex) {
+      html.h1("Unable to locate any logs for container "
+          + containerId.toString());
+      LOG.error(ex.getMessage());
+      return;
+    }
+
+    Map<String, FileStatus> checkSumFiles;
+    try {
+      checkSumFiles = fileController.filterFiles(nodeFiles,
+          LogAggregationIndexedFileController.CHECK_SUM_FILE_SUFFIX);
+    } catch (IOException ex) {
+      LOG.error("Error getting logs for " + logEntity, ex);
+      html.h1("Error getting logs for " + logEntity);
+      return;
+    }
+
+    List<FileStatus> fileToRead;
+    try {
+      fileToRead = fileController.getNodeLogFileToRead(nodeFiles,
+          nodeId.toString(), appId);
+    } catch (IOException ex) {
+      LOG.error("Error getting logs for " + logEntity, ex);
+      html.h1("Error getting logs for " + logEntity);
+      return;
+    }
+
+    boolean foundLog = false;
+    String desiredLogType = $(CONTAINER_LOG_TYPE);
+    try {
+      for (FileStatus thisNodeFile : fileToRead) {
+        FileStatus checkSum = fileController.getAllChecksumFiles(
+            checkSumFiles, thisNodeFile.getPath().getName());
+        long endIndex = -1;
+        if (checkSum != null) {
+          endIndex = fileController.loadIndexedLogsCheckSum(
+             checkSum.getPath());
+        }
+        IndexedLogsMeta indexedLogsMeta = null;
+        try {
+          indexedLogsMeta = fileController.loadIndexedLogsMeta(
+              thisNodeFile.getPath(), endIndex);
+        } catch (Exception ex) {
+          // DO NOTHING
+          LOG.warn("Can not load log meta from the log file:"
+              + thisNodeFile.getPath());
+          continue;
+        }
+        if (indexedLogsMeta == null) {
+          continue;
+        }
+        Map<ApplicationAccessType, String> appAcls = indexedLogsMeta.getAcls();
+        String user = indexedLogsMeta.getUser();
+        String remoteUser = request().getRemoteUser();
+        if (!checkAcls(conf, appId, user, appAcls, remoteUser)) {
+          html.h1().__("User [" + remoteUser
+              + "] is not authorized to view the logs for " + logEntity
+              + " in log file [" + thisNodeFile.getPath().getName() + "]")
+              .__();
+          LOG.error("User [" + remoteUser
+              + "] is not authorized to view the logs for " + logEntity);
+          continue;
+        }
+        String compressAlgo = indexedLogsMeta.getCompressName();
+        List<IndexedFileLogMeta> candidates = new ArrayList<>();
+        for (IndexedPerAggregationLogMeta logMeta
+            : indexedLogsMeta.getLogMetas()) {
+          for (Entry<String, List<IndexedFileLogMeta>> meta
+              : logMeta.getLogMetas().entrySet()) {
+            for (IndexedFileLogMeta log : meta.getValue()) {
+              if (!log.getContainerId().equals(containerId.toString())) {
+                continue;
+              }
+              if (desiredLogType != null && !desiredLogType.isEmpty()
+                  && !desiredLogType.equals(log.getFileName())) {
+                continue;
+              }
+              candidates.add(log);
+            }
+          }
+        }
+        if (candidates.isEmpty()) {
+          continue;
+        }
+
+        Algorithm compressName = Compression.getCompressionAlgorithmByName(
+            compressAlgo);
+        Decompressor decompressor = compressName.getDecompressor();
+        FileContext fileContext = FileContext.getFileContext(
+            thisNodeFile.getPath().toUri(), conf);
+        FSDataInputStream fsin = fileContext.open(thisNodeFile.getPath());
+        int bufferSize = 65536;
+        for (IndexedFileLogMeta candidate : candidates) {
+          byte[] cbuf = new byte[bufferSize];
+          InputStream in = null;
+          try {
+            in = compressName.createDecompressionStream(
+                new BoundedRangeFileInputStream(fsin,
+                    candidate.getStartIndex(),
+                    candidate.getFileCompressedSize()),
+                    decompressor,
+                    LogAggregationIndexedFileController.getFSInputBufferSize(
+                        conf));
+            long logLength = candidate.getFileSize();
+            html.pre().__("\n\n").__();
+            html.p().__("Log Type: " + candidate.getFileName()).__();
+            html.p().__("Log Upload Time: " + Times.format(
+                candidate.getLastModificatedTime())).__();
+            html.p().__("Log Length: " + Long.toString(
+                logLength)).__();
+            long startIndex = start < 0
+                ? logLength + start : start;
+            startIndex = startIndex < 0 ? 0 : startIndex;
+            startIndex = startIndex > logLength ? logLength : startIndex;
+            long endLogIndex = end < 0
+                ? logLength + end : end;
+            endLogIndex = endLogIndex < 0 ? 0 : endLogIndex;
+            endLogIndex = endLogIndex > logLength ? logLength : endLogIndex;
+            endLogIndex = endLogIndex < startIndex ?
+                startIndex : endLogIndex;
+            long toRead = endLogIndex - startIndex;
+            if (toRead < logLength) {
+              html.p().__("Showing " + toRead + " bytes of " + logLength
+                  + " total. Click ").a(url("logs", $(NM_NODENAME),
+                      $(CONTAINER_ID), $(ENTITY_STRING), $(APP_OWNER),
+                      candidate.getFileName(), "?start=0"), "here").
+                      __(" for the full log.").__();
+            }
+            long totalSkipped = 0;
+            while (totalSkipped < start) {
+              long ret = in.skip(start - totalSkipped);
+              if (ret == 0) {
+                //Read one byte
+                int nextByte = in.read();
+                // Check if we have reached EOF
+                if (nextByte == -1) {
+                  throw new IOException("Premature EOF from container log");
+                }
+                ret = 1;
+              }
+              totalSkipped += ret;
+            }
+            int len = 0;
+            int currentToRead = toRead > bufferSize ? bufferSize : (int) toRead;
+            PRE<Hamlet> pre = html.pre();
+
+            while (toRead > 0
+                && (len = in.read(cbuf, 0, currentToRead)) > 0) {
+              pre.__(new String(cbuf, 0, len, Charset.forName("UTF-8")));
+              toRead = toRead - len;
+              currentToRead = toRead > bufferSize ? bufferSize : (int) toRead;
+            }
+
+            pre.__();
+            foundLog = true;
+          } catch (Exception ex) {
+            LOG.error("Error getting logs for " + logEntity, ex);
+            continue;
+          } finally {
+            IOUtils.closeQuietly(in);
+          }
+        }
+      }
+      if (!foundLog) {
+        if (desiredLogType.isEmpty()) {
+          html.h1("No logs available for container " + containerId.toString());
+        } else {
+          html.h1("Unable to locate '" + desiredLogType
+              + "' log for container " + containerId.toString());
+        }
+      }
+    } catch (RuntimeException e) {
+      throw e;
+    } catch (Exception ex) {
+      html.h1().__("Error getting logs for " + logEntity).__();
+      LOG.error("Error getting logs for " + logEntity, ex);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fddabc2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/LogAggregationIndexedFileController.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/LogAggregationIndexedFileController.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/LogAggregationIndexedFileController.java
new file mode 100644
index 0000000..6cb2062
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/LogAggregationIndexedFileController.java
@@ -0,0 +1,1056 @@
+/**
+ * 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.logaggregation.filecontroller.ifile;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Sets;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.UUID;
+import org.apache.commons.lang.SerializationUtils;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.HarFs;
+import org.apache.hadoop.fs.Options;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.SecureIOUtils;
+import org.apache.hadoop.io.compress.Compressor;
+import org.apache.hadoop.io.compress.Decompressor;
+import org.apache.hadoop.io.file.tfile.BoundedRangeFileInputStream;
+import org.apache.hadoop.io.file.tfile.Compression;
+import org.apache.hadoop.io.file.tfile.SimpleBufferedOutputStream;
+import org.apache.hadoop.io.file.tfile.Compression.Algorithm;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.logaggregation.ContainerLogAggregationType;
+import org.apache.hadoop.yarn.logaggregation.ContainerLogMeta;
+import org.apache.hadoop.yarn.logaggregation.ContainerLogsRequest;
+import org.apache.hadoop.yarn.logaggregation.LogAggregationUtils;
+import org.apache.hadoop.yarn.logaggregation.LogToolUtils;
+import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogKey;
+import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogValue;
+import org.apache.hadoop.yarn.logaggregation.filecontroller.LogAggregationFileController;
+import org.apache.hadoop.yarn.logaggregation.filecontroller.LogAggregationFileControllerContext;
+import org.apache.hadoop.yarn.util.Times;
+import org.apache.hadoop.yarn.webapp.View.ViewContext;
+import org.apache.hadoop.yarn.webapp.view.HtmlBlock.Block;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The Indexed Log Aggregation File Format implementation.
+ *
+ */
+@Private
+@Unstable
+public class LogAggregationIndexedFileController
+    extends LogAggregationFileController {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      LogAggregationIndexedFileController.class);
+  private static final String FS_OUTPUT_BUF_SIZE_ATTR =
+      "indexedFile.fs.output.buffer.size";
+  private static final String FS_INPUT_BUF_SIZE_ATTR =
+      "indexedFile.fs.input.buffer.size";
+  private static final String FS_NUM_RETRIES_ATTR =
+      "indexedFile.fs.op.num-retries";
+  private static final String FS_RETRY_INTERVAL_MS_ATTR =
+      "indexedFile.fs.retry-interval-ms";
+  private static final int UUID_LENGTH = 36;
+
+  @VisibleForTesting
+  public static final String CHECK_SUM_FILE_SUFFIX = "-checksum";
+
+  private int fsNumRetries = 3;
+  private long fsRetryInterval = 1000L;
+  private static final int VERSION = 1;
+  private IndexedLogsMeta indexedLogsMeta = null;
+  private IndexedPerAggregationLogMeta logsMetaInThisCycle;
+  private long logAggregationTimeInThisCycle;
+  private FSDataOutputStream fsDataOStream;
+  private Algorithm compressAlgo;
+  private CachedIndexedLogsMeta cachedIndexedLogsMeta = null;
+  private boolean logAggregationSuccessfullyInThisCyCle = false;
+  private long currentOffSet = 0;
+  private Path remoteLogCheckSumFile;
+  private FileContext fc;
+  private UserGroupInformation ugi;
+  private String uuid = null;
+
+  public LogAggregationIndexedFileController() {}
+
+  @Override
+  public void initInternal(Configuration conf) {
+    // Currently, we need the underlying File System to support append
+    // operation. Will remove this check after we finish
+    // LogAggregationIndexedFileController for non-append mode.
+    boolean append = conf.getBoolean(LOG_AGGREGATION_FS_SUPPORT_APPEND, true);
+    if (!append) {
+      throw new YarnRuntimeException("The configuration:"
+          + LOG_AGGREGATION_FS_SUPPORT_APPEND + " is set as False. We can only"
+          + " use LogAggregationIndexedFileController when the FileSystem "
+          + "support append operations.");
+    }
+    String remoteDirStr = String.format(
+        YarnConfiguration.LOG_AGGREGATION_REMOTE_APP_LOG_DIR_FMT,
+        this.fileControllerName);
+    String remoteDir = conf.get(remoteDirStr);
+    if (remoteDir == null || remoteDir.isEmpty()) {
+      remoteDir = conf.get(YarnConfiguration.NM_REMOTE_APP_LOG_DIR,
+          YarnConfiguration.DEFAULT_NM_REMOTE_APP_LOG_DIR);
+    }
+    this.remoteRootLogDir = new Path(remoteDir);
+    String suffix = String.format(
+        YarnConfiguration.LOG_AGGREGATION_REMOTE_APP_LOG_DIR_SUFFIX_FMT,
+        this.fileControllerName);
+    this.remoteRootLogDirSuffix = conf.get(suffix);
+    if (this.remoteRootLogDirSuffix == null
+        || this.remoteRootLogDirSuffix.isEmpty()) {
+      this.remoteRootLogDirSuffix = conf.get(
+          YarnConfiguration.NM_REMOTE_APP_LOG_DIR_SUFFIX,
+          YarnConfiguration.DEFAULT_NM_REMOTE_APP_LOG_DIR_SUFFIX)
+          + "-ifile";
+    }
+    String compressName = conf.get(
+        YarnConfiguration.NM_LOG_AGG_COMPRESSION_TYPE,
+        YarnConfiguration.DEFAULT_NM_LOG_AGG_COMPRESSION_TYPE);
+    this.compressAlgo = Compression.getCompressionAlgorithmByName(
+        compressName);
+    this.fsNumRetries = conf.getInt(FS_NUM_RETRIES_ATTR, 3);
+    this.fsRetryInterval = conf.getLong(FS_RETRY_INTERVAL_MS_ATTR, 1000L);
+  }
+
+  @Override
+  public void initializeWriter(
+      final LogAggregationFileControllerContext context)
+      throws IOException {
+    final UserGroupInformation userUgi = context.getUserUgi();
+    final Map<ApplicationAccessType, String> appAcls = context.getAppAcls();
+    final String nodeId = context.getNodeId().toString();
+    final Path remoteLogFile = context.getRemoteNodeLogFileForApp();
+    this.ugi = userUgi;
+    logAggregationSuccessfullyInThisCyCle = false;
+    logsMetaInThisCycle = new IndexedPerAggregationLogMeta();
+    logAggregationTimeInThisCycle = System.currentTimeMillis();
+    logsMetaInThisCycle.setUploadTimeStamp(logAggregationTimeInThisCycle);
+    logsMetaInThisCycle.setRemoteNodeFile(remoteLogFile.getName());
+    try {
+      userUgi.doAs(new PrivilegedExceptionAction<Object>() {
+        @Override
+        public Object run() throws Exception {
+          fc = FileContext.getFileContext(
+              remoteRootLogDir.toUri(), conf);
+          fc.setUMask(APP_LOG_FILE_UMASK);
+          boolean fileExist = fc.util().exists(remoteLogFile);
+          if (fileExist && context.isLogAggregationInRolling()) {
+            fsDataOStream = fc.create(remoteLogFile,
+                EnumSet.of(CreateFlag.APPEND),
+                new Options.CreateOpts[] {});
+            if (uuid == null) {
+              FSDataInputStream fsDataInputStream = null;
+              try {
+                fsDataInputStream = fc.open(remoteLogFile);
+                byte[] b = new byte[UUID_LENGTH];
+                int actual = fsDataInputStream.read(b);
+                if (actual != UUID_LENGTH) {
+                  // Get an error when parse the UUID from existed log file.
+                  // Simply OverWrite the existed log file and re-create the
+                  // UUID.
+                  fsDataOStream = fc.create(remoteLogFile,
+                      EnumSet.of(CreateFlag.OVERWRITE),
+                          new Options.CreateOpts[] {});
+                  uuid = UUID.randomUUID().toString();
+                  fsDataOStream.write(uuid.getBytes(Charset.forName("UTF-8")));
+                  fsDataOStream.flush();
+                } else {
+                  uuid = new String(b, Charset.forName("UTF-8"));
+                }
+              } finally {
+                IOUtils.cleanupWithLogger(LOG, fsDataInputStream);
+              }
+            }
+            // if the remote log file exists, but we do not have any
+            // indexedLogsMeta. We need to re-load indexedLogsMeta from
+            // the existing remote log file. If the re-load fails, we simply
+            // re-create a new indexedLogsMeta object. And will re-load
+            // the indexedLogsMeta from checksum file later.
+            if (indexedLogsMeta == null) {
+              try {
+                indexedLogsMeta = loadIndexedLogsMeta(remoteLogFile);
+              } catch (IOException ex) {
+                // DO NOTHING
+              }
+            }
+          } else {
+            fsDataOStream = fc.create(remoteLogFile,
+                EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE),
+                new Options.CreateOpts[] {});
+            if (uuid == null) {
+              uuid = UUID.randomUUID().toString();
+            }
+            byte[] b = uuid.getBytes(Charset.forName("UTF-8"));
+            fsDataOStream.write(b);
+            fsDataOStream.flush();
+          }
+          if (indexedLogsMeta == null) {
+            indexedLogsMeta = new IndexedLogsMeta();
+            indexedLogsMeta.setVersion(VERSION);
+            indexedLogsMeta.setUser(userUgi.getShortUserName());
+            indexedLogsMeta.setAcls(appAcls);
+            indexedLogsMeta.setNodeId(nodeId);
+            String compressName = conf.get(
+                YarnConfiguration.NM_LOG_AGG_COMPRESSION_TYPE,
+                YarnConfiguration.DEFAULT_NM_LOG_AGG_COMPRESSION_TYPE);
+            indexedLogsMeta.setCompressName(compressName);
+          }
+          final long currentAggregatedLogFileLength = fc
+              .getFileStatus(remoteLogFile).getLen();
+          // only check the check-sum file when we are in append mode
+          if (context.isLogAggregationInRolling()) {
+            // check whether the checksum file exists to figure out
+            // whether the previous log aggregation process is successful
+            // and the aggregated log file is corrupted or not.
+            remoteLogCheckSumFile = new Path(remoteLogFile.getParent(),
+                (remoteLogFile.getName() + CHECK_SUM_FILE_SUFFIX));
+            boolean exist = fc.util().exists(remoteLogCheckSumFile);
+            if (!exist) {
+              FSDataOutputStream checksumFileOutputStream = null;
+              try {
+                checksumFileOutputStream = fc.create(remoteLogCheckSumFile,
+                    EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE),
+                    new Options.CreateOpts[] {});
+                checksumFileOutputStream.writeLong(
+                    currentAggregatedLogFileLength);
+              } finally {
+                IOUtils.cleanupWithLogger(LOG, checksumFileOutputStream);
+              }
+            } else {
+              FSDataInputStream checksumFileInputStream = null;
+              try {
+                checksumFileInputStream = fc.open(remoteLogCheckSumFile);
+                long endIndex = checksumFileInputStream.readLong();
+                IndexedLogsMeta recoveredLogsMeta = loadIndexedLogsMeta(
+                    remoteLogFile, endIndex);
+                if (recoveredLogsMeta == null) {
+                  indexedLogsMeta.getLogMetas().clear();
+                } else {
+                  indexedLogsMeta = recoveredLogsMeta;
+                }
+              } finally {
+                IOUtils.cleanupWithLogger(LOG, checksumFileInputStream);
+              }
+            }
+          }
+          // append a simple character("\n") to move the writer cursor, so
+          // we could get the correct position when we call
+          // fsOutputStream.getStartPos()
+          final byte[] dummyBytes = "\n".getBytes(Charset.forName("UTF-8"));
+          fsDataOStream.write(dummyBytes);
+          fsDataOStream.flush();
+
+          if (fsDataOStream.getPos() >= (currentAggregatedLogFileLength
+              + dummyBytes.length)) {
+            currentOffSet = 0;
+          } else {
+            currentOffSet = currentAggregatedLogFileLength;
+          }
+          return null;
+        }
+      });
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
+  }
+
+  @Override
+  public void closeWriter() {
+    IOUtils.cleanupWithLogger(LOG, this.fsDataOStream);
+  }
+
+  @Override
+  public void write(LogKey logKey, LogValue logValue) throws IOException {
+    String containerId = logKey.toString();
+    Set<File> pendingUploadFiles = logValue
+        .getPendingLogFilesToUploadForThisContainer();
+    List<IndexedFileLogMeta> metas = new ArrayList<>();
+    for (File logFile : pendingUploadFiles) {
+      FileInputStream in = null;
+      try {
+        in = SecureIOUtils.openForRead(logFile, logValue.getUser(), null);
+      } catch (IOException e) {
+        logErrorMessage(logFile, e);
+        IOUtils.cleanupWithLogger(LOG, in);
+        continue;
+      }
+      final long fileLength = logFile.length();
+      IndexedFileOutputStreamState outputStreamState = null;
+      try {
+        outputStreamState = new IndexedFileOutputStreamState(
+            this.compressAlgo, this.fsDataOStream, conf, this.currentOffSet);
+        byte[] buf = new byte[65535];
+        int len = 0;
+        long bytesLeft = fileLength;
+        while ((len = in.read(buf)) != -1) {
+          //If buffer contents within fileLength, write
+          if (len < bytesLeft) {
+            outputStreamState.getOutputStream().write(buf, 0, len);
+            bytesLeft-=len;
+          } else {
+            //else only write contents within fileLength, then exit early
+            outputStreamState.getOutputStream().write(buf, 0,
+                (int)bytesLeft);
+            break;
+          }
+        }
+        long newLength = logFile.length();
+        if(fileLength < newLength) {
+          LOG.warn("Aggregated logs truncated by approximately "+
+              (newLength-fileLength) +" bytes.");
+        }
+        logAggregationSuccessfullyInThisCyCle = true;
+      } catch (IOException e) {
+        String message = logErrorMessage(logFile, e);
+        if (outputStreamState != null &&
+            outputStreamState.getOutputStream() != null) {
+          outputStreamState.getOutputStream().write(
+              message.getBytes(Charset.forName("UTF-8")));
+        }
+      } finally {
+        IOUtils.cleanupWithLogger(LOG, in);
+      }
+
+      IndexedFileLogMeta meta = new IndexedFileLogMeta();
+      meta.setContainerId(containerId.toString());
+      meta.setFileName(logFile.getName());
+      if (outputStreamState != null) {
+        outputStreamState.finish();
+        meta.setFileCompressedSize(outputStreamState.getCompressedSize());
+        meta.setStartIndex(outputStreamState.getStartPos());
+        meta.setFileSize(fileLength);
+      }
+      meta.setLastModificatedTime(logFile.lastModified());
+      metas.add(meta);
+    }
+    logsMetaInThisCycle.addContainerLogMeta(containerId, metas);
+  }
+
+  @Override
+  public void postWrite(LogAggregationFileControllerContext record)
+      throws Exception {
+    // always aggregate the previous logsMeta, and append them together
+    // at the end of the file
+    indexedLogsMeta.addLogMeta(logsMetaInThisCycle);
+    byte[] b = SerializationUtils.serialize(indexedLogsMeta);
+    this.fsDataOStream.write(b);
+    int length = b.length;
+    this.fsDataOStream.writeInt(length);
+    byte[] separator = this.uuid.getBytes(Charset.forName("UTF-8"));
+    this.fsDataOStream.write(separator);
+    if (logAggregationSuccessfullyInThisCyCle) {
+      deleteFileWithRetries(fc, ugi, remoteLogCheckSumFile);
+    }
+  }
+
+  private void deleteFileWithRetries(final FileContext fileContext,
+      final UserGroupInformation userUgi,
+      final Path deletePath) throws Exception {
+    new FSAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        deleteFileWithPrivilege(fileContext, userUgi, deletePath);
+        return null;
+      }
+    }.runWithRetries();
+  }
+
+  private Object deleteFileWithPrivilege(final FileContext fileContext,
+      final UserGroupInformation userUgi, final Path fileToDelete)
+      throws Exception {
+    return userUgi.doAs(new PrivilegedExceptionAction<Object>() {
+      @Override
+      public Object run() throws Exception {
+        if (fileContext.util().exists(fileToDelete)) {
+          fileContext.delete(fileToDelete, false);
+        }
+        return null;
+      }
+    });
+  }
+
+  @Override
+  public boolean readAggregatedLogs(ContainerLogsRequest logRequest,
+      OutputStream os) throws IOException {
+    boolean findLogs = false;
+    boolean createPrintStream = (os == null);
+    ApplicationId appId = logRequest.getAppId();
+    String nodeId = logRequest.getNodeId();
+    String nodeIdStr = (nodeId == null || nodeId.isEmpty()) ? null
+        : LogAggregationUtils.getNodeString(nodeId);
+    List<String> logTypes = new ArrayList<>();
+    if (logRequest.getLogTypes() != null && !logRequest
+        .getLogTypes().isEmpty()) {
+      logTypes.addAll(logRequest.getLogTypes());
+    }
+    String containerIdStr = logRequest.getContainerId();
+    boolean getAllContainers = (containerIdStr == null
+        || containerIdStr.isEmpty());
+    long size = logRequest.getBytes();
+    List<FileStatus> nodeFiles = LogAggregationUtils
+        .getRemoteNodeFileList(conf, appId, logRequest.getAppOwner(),
+        this.remoteRootLogDir, this.remoteRootLogDirSuffix);
+    if (nodeFiles.isEmpty()) {
+      throw new IOException("There is no available log fils for "
+          + "application:" + appId);
+    }
+    Map<String, FileStatus> checkSumFiles = filterFiles(
+        nodeFiles, CHECK_SUM_FILE_SUFFIX);
+    List<FileStatus> fileToRead = getNodeLogFileToRead(
+        nodeFiles, nodeIdStr, appId);
+    byte[] buf = new byte[65535];
+    for (FileStatus thisNodeFile : fileToRead) {
+      String nodeName = thisNodeFile.getPath().getName();
+      FileStatus checkSum = getAllChecksumFiles(checkSumFiles,
+          thisNodeFile.getPath().getName());
+      long endIndex = -1;
+      if (checkSum != null) {
+        endIndex = loadIndexedLogsCheckSum(checkSum.getPath());
+      }
+      IndexedLogsMeta indexedLogsMeta = null;
+      try {
+        indexedLogsMeta = loadIndexedLogsMeta(thisNodeFile.getPath(),
+            endIndex);
+      } catch (Exception ex) {
+        // DO NOTHING
+        LOG.warn("Can not load log meta from the log file:"
+            + thisNodeFile.getPath());
+        continue;
+      }
+      if (indexedLogsMeta == null) {
+        continue;
+      }
+      String compressAlgo = indexedLogsMeta.getCompressName();
+      List<IndexedFileLogMeta> candidates = new ArrayList<>();
+      for (IndexedPerAggregationLogMeta logMeta
+          : indexedLogsMeta.getLogMetas()) {
+        for (Entry<String, List<IndexedFileLogMeta>> meta
+            : logMeta.getLogMetas().entrySet()) {
+          for (IndexedFileLogMeta log : meta.getValue()) {
+            if (!getAllContainers && !log.getContainerId()
+                .equals(containerIdStr)) {
+              continue;
+            }
+            if (logTypes != null && !logTypes.isEmpty() &&
+                !logTypes.contains(log.getFileName())) {
+              continue;
+            }
+            candidates.add(log);
+          }
+        }
+      }
+      if (candidates.isEmpty()) {
+        continue;
+      }
+
+      Algorithm compressName = Compression.getCompressionAlgorithmByName(
+          compressAlgo);
+      Decompressor decompressor = compressName.getDecompressor();
+      FileContext fileContext = FileContext.getFileContext(
+          thisNodeFile.getPath().toUri(), conf);
+      FSDataInputStream fsin = fileContext.open(thisNodeFile.getPath());
+      String currentContainer = "";
+      for (IndexedFileLogMeta candidate : candidates) {
+        if (!candidate.getContainerId().equals(currentContainer)) {
+          if (createPrintStream) {
+            closePrintStream(os);
+            os = LogToolUtils.createPrintStream(
+                logRequest.getOutputLocalDir(),
+                thisNodeFile.getPath().getName(),
+                candidate.getContainerId());
+            currentContainer = candidate.getContainerId();
+          }
+        }
+        InputStream in = null;
+        try {
+          in = compressName.createDecompressionStream(
+              new BoundedRangeFileInputStream(fsin,
+                  candidate.getStartIndex(),
+                  candidate.getFileCompressedSize()),
+              decompressor, getFSInputBufferSize(conf));
+          LogToolUtils.outputContainerLog(candidate.getContainerId(),
+              nodeName, candidate.getFileName(), candidate.getFileSize(), size,
+              Times.format(candidate.getLastModificatedTime()),
+              in, os, buf, ContainerLogAggregationType.AGGREGATED);
+          byte[] b = aggregatedLogSuffix(candidate.getFileName())
+              .getBytes(Charset.forName("UTF-8"));
+          os.write(b, 0, b.length);
+          findLogs = true;
+        } catch (IOException e) {
+          System.err.println(e.getMessage());
+          compressName.returnDecompressor(decompressor);
+          continue;
+        } finally {
+          os.flush();
+          IOUtils.cleanupWithLogger(LOG, in);
+        }
+      }
+    }
+    return findLogs;
+  }
+
+  // TODO: fix me if the remote file system does not support append operation.
+  @Override
+  public List<ContainerLogMeta> readAggregatedLogsMeta(
+      ContainerLogsRequest logRequest) throws IOException {
+    List<IndexedLogsMeta> listOfLogsMeta = new ArrayList<>();
+    List<ContainerLogMeta> containersLogMeta = new ArrayList<>();
+    String containerIdStr = logRequest.getContainerId();
+    String nodeId = logRequest.getNodeId();
+    ApplicationId appId = logRequest.getAppId();
+    String appOwner = logRequest.getAppOwner();
+    boolean getAllContainers = (containerIdStr == null ||
+        containerIdStr.isEmpty());
+    String nodeIdStr = (nodeId == null || nodeId.isEmpty()) ? null
+        : LogAggregationUtils.getNodeString(nodeId);
+    List<FileStatus> nodeFiles = LogAggregationUtils
+        .getRemoteNodeFileList(conf, appId, appOwner, this.remoteRootLogDir,
+        this.remoteRootLogDirSuffix);
+    if (nodeFiles.isEmpty()) {
+      throw new IOException("There is no available log fils for "
+          + "application:" + appId);
+    }
+    Map<String, FileStatus> checkSumFiles = filterFiles(
+        nodeFiles, CHECK_SUM_FILE_SUFFIX);
+    List<FileStatus> fileToRead = getNodeLogFileToRead(
+        nodeFiles, nodeIdStr, appId);
+    for(FileStatus thisNodeFile : fileToRead) {
+      try {
+        FileStatus checkSum = getAllChecksumFiles(checkSumFiles,
+            thisNodeFile.getPath().getName());
+        long endIndex = -1;
+        if (checkSum != null) {
+          endIndex = loadIndexedLogsCheckSum(checkSum.getPath());
+        }
+        IndexedLogsMeta current = loadIndexedLogsMeta(
+            thisNodeFile.getPath(), endIndex);
+        if (current != null) {
+          listOfLogsMeta.add(current);
+        }
+      } catch (IOException ex) {
+        // DO NOTHING
+        LOG.warn("Can not get log meta from the log file:"
+            + thisNodeFile.getPath());
+      }
+    }
+    for (IndexedLogsMeta indexedLogMeta : listOfLogsMeta) {
+      String curNodeId = indexedLogMeta.getNodeId();
+      for (IndexedPerAggregationLogMeta logMeta :
+          indexedLogMeta.getLogMetas()) {
+        if (getAllContainers) {
+          for (Entry<String, List<IndexedFileLogMeta>> log : logMeta
+              .getLogMetas().entrySet()) {
+            ContainerLogMeta meta = new ContainerLogMeta(
+                log.getKey().toString(), curNodeId);
+            for (IndexedFileLogMeta aMeta : log.getValue()) {
+              meta.addLogMeta(aMeta.getFileName(), Long.toString(
+                  aMeta.getFileSize()),
+                  Times.format(aMeta.getLastModificatedTime()));
+            }
+            containersLogMeta.add(meta);
+          }
+        } else if (logMeta.getContainerLogMeta(containerIdStr) != null) {
+          ContainerLogMeta meta = new ContainerLogMeta(containerIdStr,
+              curNodeId);
+          for (IndexedFileLogMeta log :
+              logMeta.getContainerLogMeta(containerIdStr)) {
+            meta.addLogMeta(log.getFileName(), Long.toString(
+                log.getFileSize()),
+                Times.format(log.getLastModificatedTime()));
+          }
+          containersLogMeta.add(meta);
+        }
+      }
+    }
+    Collections.sort(containersLogMeta, new Comparator<ContainerLogMeta>() {
+      @Override
+      public int compare(ContainerLogMeta o1, ContainerLogMeta o2) {
+        return o1.getContainerId().compareTo(o2.getContainerId());
+      }
+    });
+    return containersLogMeta;
+  }
+
+  @Private
+  public Map<String, FileStatus> filterFiles(
+      List<FileStatus> fileList, final String suffix) throws IOException {
+    Map<String, FileStatus> checkSumFiles = new HashMap<>();
+    Set<FileStatus> status = new HashSet<FileStatus>(fileList);
+    Iterable<FileStatus> mask =
+        Iterables.filter(status, new Predicate<FileStatus>() {
+          @Override
+          public boolean apply(FileStatus next) {
+            return next.getPath().getName().endsWith(
+                suffix);
+          }
+        });
+    status = Sets.newHashSet(mask);
+    for (FileStatus file : status) {
+      checkSumFiles.put(file.getPath().getName(), file);
+    }
+    return checkSumFiles;
+  }
+
+  @Private
+  public List<FileStatus> getNodeLogFileToRead(
+      List<FileStatus> nodeFiles, String nodeId, ApplicationId appId)
+      throws IOException {
+    List<FileStatus> listOfFiles = new ArrayList<>();
+    List<FileStatus> files = new ArrayList<>(nodeFiles);
+    for (FileStatus file : files) {
+      String nodeName = file.getPath().getName();
+      if ((nodeId == null || nodeId.isEmpty()
+          || nodeName.contains(LogAggregationUtils
+          .getNodeString(nodeId))) && !nodeName.endsWith(
+              LogAggregationUtils.TMP_FILE_SUFFIX) &&
+          !nodeName.endsWith(CHECK_SUM_FILE_SUFFIX)) {
+        if (nodeName.equals(appId + ".har")) {
+          Path p = new Path("har:///" + file.getPath().toUri().getRawPath());
+          files = Arrays.asList(HarFs.get(p.toUri(), conf).listStatus(p));
+          continue;
+        }
+        listOfFiles.add(file);
+      }
+    }
+    return listOfFiles;
+  }
+
+  @Private
+  public FileStatus getAllChecksumFiles(Map<String, FileStatus> fileMap,
+      String fileName) {
+    for (Entry<String, FileStatus> file : fileMap.entrySet()) {
+      if (file.getKey().startsWith(fileName) && file.getKey()
+          .endsWith(CHECK_SUM_FILE_SUFFIX)) {
+        return file.getValue();
+      }
+    }
+    return null;
+  }
+
+  @Override
+  public void renderAggregatedLogsBlock(Block html, ViewContext context) {
+    IndexedFileAggregatedLogsBlock block = new IndexedFileAggregatedLogsBlock(
+        context, this.conf, this);
+    block.render(html);
+  }
+
+  @Override
+  public String getApplicationOwner(Path aggregatedLogPath)
+      throws IOException {
+    if (this.cachedIndexedLogsMeta == null
+        || !this.cachedIndexedLogsMeta.getRemoteLogPath()
+            .equals(aggregatedLogPath)) {
+      this.cachedIndexedLogsMeta = new CachedIndexedLogsMeta(
+          loadIndexedLogsMeta(aggregatedLogPath), aggregatedLogPath);
+    }
+    return this.cachedIndexedLogsMeta.getCachedIndexedLogsMeta().getUser();
+  }
+
+  @Override
+  public Map<ApplicationAccessType, String> getApplicationAcls(
+      Path aggregatedLogPath) throws IOException {
+    if (this.cachedIndexedLogsMeta == null
+        || !this.cachedIndexedLogsMeta.getRemoteLogPath()
+            .equals(aggregatedLogPath)) {
+      this.cachedIndexedLogsMeta = new CachedIndexedLogsMeta(
+          loadIndexedLogsMeta(aggregatedLogPath), aggregatedLogPath);
+    }
+    return this.cachedIndexedLogsMeta.getCachedIndexedLogsMeta().getAcls();
+  }
+
+  @Override
+  public Path getRemoteAppLogDir(ApplicationId appId, String user)
+      throws IOException {
+    return LogAggregationUtils.getRemoteAppLogDir(conf, appId, user,
+        this.remoteRootLogDir, this.remoteRootLogDirSuffix);
+  }
+
+  @Private
+  public IndexedLogsMeta loadIndexedLogsMeta(Path remoteLogPath, long end)
+      throws IOException {
+    FileContext fileContext =
+        FileContext.getFileContext(remoteLogPath.toUri(), conf);
+    FSDataInputStream fsDataIStream = null;
+    try {
+      fsDataIStream = fileContext.open(remoteLogPath);
+      if (end == 0) {
+        return null;
+      }
+      long fileLength = end < 0 ? fileContext.getFileStatus(
+          remoteLogPath).getLen() : end;
+      fsDataIStream.seek(fileLength - Integer.SIZE/ Byte.SIZE - UUID_LENGTH);
+      int offset = fsDataIStream.readInt();
+      byte[] array = new byte[offset];
+      fsDataIStream.seek(
+          fileLength - offset - Integer.SIZE/ Byte.SIZE - UUID_LENGTH);
+      int actual = fsDataIStream.read(array);
+      if (actual != offset) {
+        throw new IOException("Error on loading log meta from "
+            + remoteLogPath);
+      }
+      return (IndexedLogsMeta)SerializationUtils
+          .deserialize(array);
+    } finally {
+      IOUtils.cleanupWithLogger(LOG, fsDataIStream);
+    }
+  }
+
+  private IndexedLogsMeta loadIndexedLogsMeta(Path remoteLogPath)
+      throws IOException {
+    return loadIndexedLogsMeta(remoteLogPath, -1);
+  }
+
+  @Private
+  public long loadIndexedLogsCheckSum(Path remoteLogCheckSumPath)
+      throws IOException {
+    FileContext fileContext =
+        FileContext.getFileContext(remoteLogCheckSumPath.toUri(), conf);
+    FSDataInputStream fsDataIStream = null;
+    try {
+      fsDataIStream = fileContext.open(remoteLogCheckSumPath);
+      return fsDataIStream.readLong();
+    } finally {
+      IOUtils.cleanupWithLogger(LOG, fsDataIStream);
+    }
+  }
+
+  /**
+   * This IndexedLogsMeta includes all the meta information
+   * for the aggregated log file.
+   */
+  @Private
+  @VisibleForTesting
+  public static class IndexedLogsMeta implements Serializable {
+
+    private static final long serialVersionUID = 5439875373L;
+    private int version;
+    private String user;
+    private String compressName;
+    private Map<ApplicationAccessType, String> acls;
+    private String nodeId;
+    private List<IndexedPerAggregationLogMeta> logMetas = new ArrayList<>();
+
+    public int getVersion() {
+      return this.version;
+    }
+
+    public void setVersion(int version) {
+      this.version = version;
+    }
+
+    public String getUser() {
+      return this.user;
+    }
+
+    public void setUser(String user) {
+      this.user = user;
+    }
+
+    public Map<ApplicationAccessType, String> getAcls() {
+      return this.acls;
+    }
+
+    public void setAcls(Map<ApplicationAccessType, String> acls) {
+      this.acls = acls;
+    }
+
+    public String getCompressName() {
+      return compressName;
+    }
+
+    public void setCompressName(String compressName) {
+      this.compressName = compressName;
+    }
+
+    public String getNodeId() {
+      return nodeId;
+    }
+
+    public void setNodeId(String nodeId) {
+      this.nodeId = nodeId;
+    }
+
+    public void addLogMeta(IndexedPerAggregationLogMeta logMeta) {
+      logMetas.add(logMeta);
+    }
+
+    public List<IndexedPerAggregationLogMeta> getLogMetas() {
+      return logMetas;
+    }
+  }
+
+  /**
+   * This IndexedPerAggregationLogMeta includes the meta information
+   * for all files which would be aggregated in one
+   * Log aggregation cycle.
+   */
+  public static class IndexedPerAggregationLogMeta implements Serializable {
+    private static final long serialVersionUID = 3929298383L;
+    private String remoteNodeLogFileName;
+    private Map<String, List<IndexedFileLogMeta>> logMetas = new HashMap<>();
+    private long uploadTimeStamp;
+
+    public String getRemoteNodeFile() {
+      return remoteNodeLogFileName;
+    }
+    public void setRemoteNodeFile(String remoteNodeLogFileName) {
+      this.remoteNodeLogFileName = remoteNodeLogFileName;
+    }
+
+    public void addContainerLogMeta(String containerId,
+        List<IndexedFileLogMeta> logMeta) {
+      logMetas.put(containerId, logMeta);
+    }
+
+    public List<IndexedFileLogMeta> getContainerLogMeta(String containerId) {
+      return logMetas.get(containerId);
+    }
+
+    public Map<String, List<IndexedFileLogMeta>> getLogMetas() {
+      return logMetas;
+    }
+
+    public long getUploadTimeStamp() {
+      return uploadTimeStamp;
+    }
+
+    public void setUploadTimeStamp(long uploadTimeStamp) {
+      this.uploadTimeStamp = uploadTimeStamp;
+    }
+  }
+
+  /**
+   * This IndexedFileLogMeta includes the meta information
+   * for a single file which would be aggregated in one
+   * Log aggregation cycle.
+   *
+   */
+  @Private
+  @VisibleForTesting
+  public static class IndexedFileLogMeta implements Serializable {
+    private static final long serialVersionUID = 1L;
+    private String containerId;
+    private String fileName;
+    private long fileSize;
+    private long fileCompressedSize;
+    private long lastModificatedTime;
+    private long startIndex;
+
+    public String getFileName() {
+      return fileName;
+    }
+    public void setFileName(String fileName) {
+      this.fileName = fileName;
+    }
+
+    public long getFileSize() {
+      return fileSize;
+    }
+    public void setFileSize(long fileSize) {
+      this.fileSize = fileSize;
+    }
+
+    public long getFileCompressedSize() {
+      return fileCompressedSize;
+    }
+    public void setFileCompressedSize(long fileCompressedSize) {
+      this.fileCompressedSize = fileCompressedSize;
+    }
+
+    public long getLastModificatedTime() {
+      return lastModificatedTime;
+    }
+    public void setLastModificatedTime(long lastModificatedTime) {
+      this.lastModificatedTime = lastModificatedTime;
+    }
+
+    public long getStartIndex() {
+      return startIndex;
+    }
+    public void setStartIndex(long startIndex) {
+      this.startIndex = startIndex;
+    }
+
+    public String getContainerId() {
+      return containerId;
+    }
+    public void setContainerId(String containerId) {
+      this.containerId = containerId;
+    }
+  }
+
+  private static String logErrorMessage(File logFile, Exception e) {
+    String message = "Error aggregating log file. Log file : "
+        + logFile.getAbsolutePath() + ". " + e.getMessage();
+    LOG.error(message, e);
+    return message;
+  }
+
+  private static class IndexedFileOutputStreamState {
+    private final Algorithm compressAlgo;
+    private Compressor compressor;
+    private final FSDataOutputStream fsOut;
+    private long posStart;
+    private final SimpleBufferedOutputStream fsBufferedOutput;
+    private OutputStream out;
+    private long offset;
+
+    IndexedFileOutputStreamState(Algorithm compressionName,
+        FSDataOutputStream fsOut, Configuration conf, long offset)
+        throws IOException {
+      this.compressAlgo = compressionName;
+      this.fsOut = fsOut;
+      this.offset = offset;
+      this.posStart = fsOut.getPos();
+
+      BytesWritable fsOutputBuffer = new BytesWritable();
+      fsOutputBuffer.setCapacity(LogAggregationIndexedFileController
+          .getFSOutputBufferSize(conf));
+
+      this.fsBufferedOutput = new SimpleBufferedOutputStream(this.fsOut,
+          fsOutputBuffer.getBytes());
+
+      this.compressor = compressAlgo.getCompressor();
+
+      try {
+        this.out = compressAlgo.createCompressionStream(
+            fsBufferedOutput, compressor, 0);
+      } catch (IOException e) {
+        compressAlgo.returnCompressor(compressor);
+        throw e;
+      }
+    }
+
+    OutputStream getOutputStream() {
+      return out;
+    }
+
+    long getCurrentPos() throws IOException {
+      return fsOut.getPos() + fsBufferedOutput.size();
+    }
+
+    long getStartPos() {
+      return posStart + offset;
+    }
+
+    long getCompressedSize() throws IOException {
+      long ret = getCurrentPos() - posStart;
+      return ret;
+    }
+
+    void finish() throws IOException {
+      try {
+        if (out != null) {
+          out.flush();
+          out = null;
+        }
+      } finally {
+        compressAlgo.returnCompressor(compressor);
+        compressor = null;
+      }
+    }
+  }
+
+  private static class CachedIndexedLogsMeta {
+    private final Path remoteLogPath;
+    private final IndexedLogsMeta indexedLogsMeta;
+    CachedIndexedLogsMeta(IndexedLogsMeta indexedLogsMeta,
+        Path remoteLogPath) {
+      this.indexedLogsMeta = indexedLogsMeta;
+      this.remoteLogPath = remoteLogPath;
+    }
+
+    public Path getRemoteLogPath() {
+      return this.remoteLogPath;
+    }
+
+    public IndexedLogsMeta getCachedIndexedLogsMeta() {
+      return this.indexedLogsMeta;
+    }
+  }
+
+  @Private
+  public static int getFSOutputBufferSize(Configuration conf) {
+    return conf.getInt(FS_OUTPUT_BUF_SIZE_ATTR, 256 * 1024);
+  }
+
+  @Private
+  public static int getFSInputBufferSize(Configuration conf) {
+    return conf.getInt(FS_INPUT_BUF_SIZE_ATTR, 256 * 1024);
+  }
+
+  private abstract class FSAction<T> {
+    abstract T run() throws Exception;
+
+    T runWithRetries() throws Exception {
+      int retry = 0;
+      while (true) {
+        try {
+          return run();
+        } catch (IOException e) {
+          LOG.info("Exception while executing an FS operation.", e);
+          if (++retry > fsNumRetries) {
+            LOG.info("Maxed out FS retries. Giving up!");
+            throw e;
+          }
+          LOG.info("Retrying operation on FS. Retry no. " + retry);
+          Thread.sleep(fsRetryInterval);
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fddabc2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/package-info.java
new file mode 100644
index 0000000..08ddece
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+@InterfaceAudience.Public
+package org.apache.hadoop.yarn.logaggregation.filecontroller.ifile;
+import org.apache.hadoop.classification.InterfaceAudience;
+


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


[41/50] [abbrv] hadoop git commit: HADOOP-14847. Remove Guava Supplier and change to java Supplier in AMRMClient and AMRMClientAysnc. (Bharat Viswanadham via Haibo Chen)

Posted by as...@apache.org.
HADOOP-14847. Remove Guava Supplier and change to java Supplier in AMRMClient and AMRMClientAysnc. (Bharat Viswanadham 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/8edc6053
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8edc6053
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8edc6053

Branch: refs/heads/YARN-5972
Commit: 8edc60531fec4f4070955b3e82a78ba70ba40ec0
Parents: a323f73
Author: Haibo Chen <ha...@apache.org>
Authored: Fri Sep 8 14:27:05 2017 -0700
Committer: Haibo Chen <ha...@apache.org>
Committed: Fri Sep 8 14:33:11 2017 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/yarn/client/api/AMRMClient.java   | 8 ++++----
 .../apache/hadoop/yarn/client/api/async/AMRMClientAsync.java | 8 ++++----
 .../yarn/client/api/async/impl/TestAMRMClientAsync.java      | 2 +-
 .../apache/hadoop/yarn/client/api/impl/TestAMRMClient.java   | 2 +-
 4 files changed, 10 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8edc6053/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
index b11c094..fc64093 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.client.api;
 
 import java.io.IOException;
 import java.util.Collection;
+import java.util.function.Supplier;
 import java.util.List;
 
 import org.apache.commons.logging.Log;
@@ -45,7 +46,6 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
 import com.google.common.base.Preconditions;
-import com.google.common.base.Supplier;
 import com.google.common.collect.ImmutableList;
 
 @InterfaceAudience.Public
@@ -702,8 +702,8 @@ public abstract class AMRMClient<T extends AMRMClient.ContainerRequest> extends
 
   /**
    * Wait for <code>check</code> to return true for each 1000 ms.
-   * See also {@link #waitFor(com.google.common.base.Supplier, int)}
-   * and {@link #waitFor(com.google.common.base.Supplier, int, int)}
+   * See also {@link #waitFor(java.util.function.Supplier, int)}
+   * and {@link #waitFor(java.util.function.Supplier, int, int)}
    * @param check the condition for which it should wait
    */
   public void waitFor(Supplier<Boolean> check) throws InterruptedException {
@@ -713,7 +713,7 @@ public abstract class AMRMClient<T extends AMRMClient.ContainerRequest> extends
   /**
    * Wait for <code>check</code> to return true for each
    * <code>checkEveryMillis</code> ms.
-   * See also {@link #waitFor(com.google.common.base.Supplier, int, int)}
+   * See also {@link #waitFor(java.util.function.Supplier, int, int)}
    * @param check user defined checker
    * @param checkEveryMillis interval to call <code>check</code>
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8edc6053/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java
index 1ecfe1f..44a36af 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.util.Collection;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -52,7 +53,6 @@ import org.apache.hadoop.yarn.util.resource.Resources;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
-import com.google.common.base.Supplier;
 
 /**
  * <code>AMRMClientAsync</code> handles communication with the ResourceManager
@@ -375,8 +375,8 @@ extends AbstractService {
 
   /**
    * Wait for <code>check</code> to return true for each 1000 ms.
-   * See also {@link #waitFor(com.google.common.base.Supplier, int)}
-   * and {@link #waitFor(com.google.common.base.Supplier, int, int)}
+   * See also {@link #waitFor(java.util.function.Supplier, int)}
+   * and {@link #waitFor(java.util.function.Supplier, int, int)}
    * @param check the condition for which it should wait
    */
   public void waitFor(Supplier<Boolean> check) throws InterruptedException {
@@ -386,7 +386,7 @@ extends AbstractService {
   /**
    * Wait for <code>check</code> to return true for each
    * <code>checkEveryMillis</code> ms.
-   * See also {@link #waitFor(com.google.common.base.Supplier, int, int)}
+   * See also {@link #waitFor(java.util.function.Supplier, int, int)}
    * @param check user defined checker
    * @param checkEveryMillis interval to call <code>check</code>
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8edc6053/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestAMRMClientAsync.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestAMRMClientAsync.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestAMRMClientAsync.java
index 56826c4..8d912a6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestAMRMClientAsync.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestAMRMClientAsync.java
@@ -34,6 +34,7 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -62,7 +63,6 @@ import org.junit.Test;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
-import com.google.common.base.Supplier;
 
 public class TestAMRMClientAsync {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8edc6053/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
index fa2e7a5..b6beb38 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
@@ -33,6 +33,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.function.Supplier;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -88,7 +89,6 @@ import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 import org.eclipse.jetty.util.log.Log;
 
-import com.google.common.base.Supplier;
 
 /**
  * Test application master client class to resource manager.


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


[46/50] [abbrv] hadoop git commit: HDFS-10391. Always enable NameNode service RPC port. Contributed by Gergely Novak.

Posted by as...@apache.org.
HDFS-10391. Always enable NameNode service RPC port. Contributed by Gergely Novak.


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

Branch: refs/heads/YARN-5972
Commit: aa4b6fbe754ab7e3cf8ee106598d550f6e14783e
Parents: 180e814
Author: Arpit Agarwal <ar...@apache.org>
Authored: Sat Sep 9 08:40:53 2017 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Sat Sep 9 08:40:53 2017 -0700

----------------------------------------------------------------------
 .../hdfs/client/HdfsClientConfigKeys.java       |   1 +
 .../java/org/apache/hadoop/hdfs/DFSUtil.java    | 106 +++++++--------
 .../hdfs/server/datanode/BlockPoolManager.java  |   2 +-
 .../hadoop/hdfs/server/namenode/BackupNode.java |   2 +-
 .../hdfs/server/namenode/FSNamesystem.java      |   5 +-
 .../hadoop/hdfs/server/namenode/NameNode.java   |  29 ++--
 .../hdfs/server/namenode/NameNodeRpcServer.java | 129 ++++++++----------
 .../hdfs/server/namenode/SecondaryNameNode.java |   2 +-
 .../hdfs/server/namenode/ha/EditLogTailer.java  |   3 +-
 .../server/namenode/ha/RemoteNameNodeInfo.java  |   2 +-
 .../server/namenode/ha/StandbyCheckpointer.java |   2 +-
 .../org/apache/hadoop/hdfs/tools/GetConf.java   |   4 +-
 .../src/main/resources/hdfs-default.xml         |   3 +-
 .../org/apache/hadoop/hdfs/MiniDFSCluster.java  |  66 +++++++--
 .../apache/hadoop/hdfs/MiniDFSNNTopology.java   |  17 ++-
 .../org/apache/hadoop/hdfs/TestDFSUtil.java     | 133 ++++++++++++++++---
 .../apache/hadoop/hdfs/TestHDFSServerPorts.java |  85 ++++++------
 .../org/apache/hadoop/hdfs/TestSafeMode.java    |   2 +-
 .../hadoop/hdfs/qjournal/MiniQJMHACluster.java  |   9 +-
 .../balancer/TestBalancerWithHANameNodes.java   |   5 +-
 .../datanode/InternalDataNodeTestUtils.java     |   9 +-
 .../hdfs/server/datanode/TestBlockRecovery.java |   6 +-
 .../datanode/TestDataNodeMetricsLogger.java     |  10 +-
 .../TestDataNodeMultipleRegistrations.java      |   8 +-
 .../datanode/TestDataNodeReconfiguration.java   |  11 +-
 .../TestDatanodeProtocolRetryPolicy.java        |   6 +-
 .../server/datanode/TestRefreshNamenodes.java   |  17 ++-
 .../hdfs/server/namenode/TestBackupNode.java    |   2 +
 .../hdfs/server/namenode/TestCheckpoint.java    |   6 +-
 .../server/namenode/TestNameNodeMXBean.java     |  11 +-
 .../namenode/TestNameNodeMetricsLogger.java     |   1 +
 .../TestValidateConfigurationSettings.java      |   2 +
 .../hdfs/server/namenode/ha/HATestUtil.java     |   1 -
 .../server/namenode/ha/TestEditLogTailer.java   |  20 ++-
 .../hadoop/hdfs/tools/TestDFSHAAdmin.java       |   7 +-
 .../hdfs/tools/TestDFSHAAdminMiniCluster.java   |   2 +-
 .../hdfs/tools/TestDFSZKFailoverController.java |   6 +-
 .../apache/hadoop/hdfs/tools/TestGetConf.java   |  40 ++----
 38 files changed, 475 insertions(+), 297 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa4b6fbe/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
index e99b099..d6efb5c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
@@ -74,6 +74,7 @@ public interface HdfsClientConfigKeys {
   String  DFS_NAMENODE_HTTPS_ADDRESS_KEY = "dfs.namenode.https-address";
   String DFS_HA_NAMENODES_KEY_PREFIX = "dfs.ha.namenodes";
   int DFS_NAMENODE_RPC_PORT_DEFAULT = 9820;
+  int DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT = 9840;
   String DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY =
       "dfs.namenode.kerberos.principal";
   String  DFS_CLIENT_WRITE_PACKET_SIZE_KEY = "dfs.client-write-packet-size";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa4b6fbe/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
index 7776dc2..3c71e76 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
@@ -35,6 +35,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMESERVICE_ID;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_KEYPASSWORD_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_KEYSTORE_PASSWORD_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_TRUSTSTORE_PASSWORD_KEY;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT;
 
 import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
@@ -492,61 +493,25 @@ public class DFSUtil {
   }
 
   /**
-   * Returns list of InetSocketAddresses corresponding to namenodes from the
-   * configuration.
-   * 
-   * Returns namenode address specifically configured for datanodes (using
-   * service ports), if found. If not, regular RPC address configured for other
-   * clients is returned.
-   * 
-   * @param conf configuration
-   * @return list of InetSocketAddress
-   * @throws IOException on error
-   */
-  public static Map<String, Map<String, InetSocketAddress>> getNNServiceRpcAddresses(
-      Configuration conf) throws IOException {
-    // Use default address as fall back
-    String defaultAddress;
-    try {
-      defaultAddress = NetUtils.getHostPortString(
-          DFSUtilClient.getNNAddress(conf));
-    } catch (IllegalArgumentException e) {
-      defaultAddress = null;
-    }
-    
-    Map<String, Map<String, InetSocketAddress>> addressList =
-      DFSUtilClient.getAddresses(conf, defaultAddress,
-                                 DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY,
-                                 DFS_NAMENODE_RPC_ADDRESS_KEY);
-    if (addressList.isEmpty()) {
-      throw new IOException("Incorrect configuration: namenode address "
-          + DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY + " or "  
-          + DFS_NAMENODE_RPC_ADDRESS_KEY
-          + " is not configured.");
-    }
-    return addressList;
-  }
-
-  /**
    * Returns list of InetSocketAddresses corresponding to the namenode
    * that manages this cluster. Note this is to be used by datanodes to get
    * the list of namenode addresses to talk to.
    *
-   * Returns namenode address specifically configured for datanodes (using
-   * service ports), if found. If not, regular RPC address configured for other
-   * clients is returned.
+   * Returns namenode address specifically configured for datanodes
    *
    * @param conf configuration
    * @return list of InetSocketAddress
    * @throws IOException on error
    */
   public static Map<String, Map<String, InetSocketAddress>>
-    getNNServiceRpcAddressesForCluster(Configuration conf) throws IOException {
+      getNNServiceRpcAddresses(Configuration conf) throws IOException {
     // Use default address as fall back
     String defaultAddress;
     try {
-      defaultAddress = NetUtils.getHostPortString(
-          DFSUtilClient.getNNAddress(conf));
+      InetSocketAddress rpcAddress = DFSUtilClient.getNNAddress(conf);
+      InetSocketAddress serviceAddress = InetSocketAddress.createUnresolved(
+          rpcAddress.getHostName(), DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT);
+      defaultAddress = NetUtils.getHostPortString(serviceAddress);
     } catch (IllegalArgumentException e) {
       defaultAddress = null;
     }
@@ -569,16 +534,46 @@ public class DFSUtil {
       }
     }
 
+    // If true, then replace the port numbers in the final address list
+    // with the default service RPC port.
+    boolean replacePortNumbers = false;
+
+    // First try to lookup using the service RPC address keys.
     Map<String, Map<String, InetSocketAddress>> addressList =
-            DFSUtilClient.getAddressesForNsIds(conf, parentNameServices,
-                                               defaultAddress,
-                                               DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY,
-                                               DFS_NAMENODE_RPC_ADDRESS_KEY);
+            DFSUtilClient.getAddressesForNsIds(
+                conf, parentNameServices, null,
+                DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY);
+
+    // Next try to lookup using the RPC address key.
+    if (addressList.isEmpty()) {
+      replacePortNumbers = true;
+      addressList = DFSUtilClient.getAddressesForNsIds(
+          conf, parentNameServices, null, DFS_NAMENODE_RPC_ADDRESS_KEY);
+    }
+
+    // Finally, fallback to the default address.
+    // This will not yield the correct address in a federated/HA setup.
+    if (addressList.isEmpty()) {
+      addressList = DFSUtilClient.getAddressesForNsIds(
+          conf, parentNameServices, defaultAddress);
+    }
+
     if (addressList.isEmpty()) {
       throw new IOException("Incorrect configuration: namenode address "
-              + DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY + " or "
-              + DFS_NAMENODE_RPC_ADDRESS_KEY
-              + " is not configured.");
+          + DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY + " or "
+          + DFS_NAMENODE_RPC_ADDRESS_KEY
+          + " is not configured.");
+    }
+
+    if (replacePortNumbers) {
+      // Replace the RPC port(s) with the default service RPC port(s)
+      addressList.forEach((nsId, addresses) -> {
+        addresses.forEach((nnId, address) -> {
+          InetSocketAddress serviceAddress = InetSocketAddress.createUnresolved(
+              address.getHostName(), DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT);
+          addresses.put(nnId, serviceAddress);
+        });
+      });
     }
     return addressList;
   }
@@ -1230,12 +1225,17 @@ public class DFSUtil {
     String serviceAddrKey = DFSUtilClient.concatSuffixes(
         DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, nsId, nnId);
 
-    String addrKey = DFSUtilClient.concatSuffixes(
-        DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY, nsId, nnId);
-
     String serviceRpcAddr = conf.get(serviceAddrKey);
     if (serviceRpcAddr == null) {
-      serviceRpcAddr = conf.get(addrKey);
+      String addrKey = DFSUtilClient.concatSuffixes(
+          DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY, nsId, nnId);
+      String rpcAddress = conf.get(addrKey);
+      if (rpcAddress != null) {
+        InetSocketAddress rpcAddr = NetUtils.createSocketAddr(rpcAddress);
+        InetSocketAddress serviceAddr = InetSocketAddress.createUnresolved(
+            rpcAddr.getHostName(), DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT);
+        serviceRpcAddr = NetUtils.getHostPortString(serviceAddr);
+      }
     }
     return serviceRpcAddr;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa4b6fbe/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolManager.java
index f6a11c2..677559c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolManager.java
@@ -150,7 +150,7 @@ class BlockPoolManager {
             (DFSConfigKeys.DFS_NAMESERVICES));
 
     Map<String, Map<String, InetSocketAddress>> newAddressMap = DFSUtil
-            .getNNServiceRpcAddressesForCluster(conf);
+            .getNNServiceRpcAddresses(conf);
     Map<String, Map<String, InetSocketAddress>> newLifelineAddressMap = DFSUtil
             .getNNLifelineRpcAddressesForCluster(conf);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa4b6fbe/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
index 318d8e0..5c2dcdc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
@@ -318,7 +318,7 @@ public class BackupNode extends NameNode {
 
   private NamespaceInfo handshake(Configuration conf) throws IOException {
     // connect to name node
-    InetSocketAddress nnAddress = NameNode.getServiceAddress(conf, true);
+    InetSocketAddress nnAddress = NameNode.getServiceAddress(conf);
     this.namenode = NameNodeProxies.createNonHAProxy(conf, nnAddress,
         NamenodeProtocol.class, UserGroupInformation.getCurrentUser(),
         true).getProxy();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa4b6fbe/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index d9f3c0e..55695b3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -1157,9 +1157,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       dir.setINodeAttributeProvider(inodeAttributeProvider);
     }
     snapshotManager.registerMXBean();
-    InetSocketAddress serviceAddress = NameNode.getServiceAddress(conf, true);
-    this.nameNodeHostName = (serviceAddress != null) ?
-        serviceAddress.getHostName() : "";
+    InetSocketAddress serviceAddress = NameNode.getServiceAddress(conf);
+    this.nameNodeHostName = serviceAddress.getHostName();
   }
   
   /** 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa4b6fbe/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
index 79bbbc5..d700439 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
@@ -505,18 +505,17 @@ public class NameNode extends ReconfigurableBase implements
   
   /**
    * Fetches the address for services to use when connecting to namenode
-   * based on the value of fallback returns null if the special
-   * address is not specified or returns the default namenode address
-   * to be used by both clients and services.
    * Services here are datanodes, backup node, any non client connection
    */
-  public static InetSocketAddress getServiceAddress(Configuration conf,
-                                                        boolean fallback) {
-    String addr = conf.getTrimmed(DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY);
-    if (addr == null || addr.isEmpty()) {
-      return fallback ? DFSUtilClient.getNNAddress(conf) : null;
+  public static InetSocketAddress getServiceAddress(Configuration conf) {
+    String address = conf.getTrimmed(DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY);
+    if (address == null || address.isEmpty()) {
+      InetSocketAddress rpcAddress = DFSUtilClient.getNNAddress(conf);
+      return NetUtils.createSocketAddr(rpcAddress.getHostName(),
+          HdfsClientConfigKeys.DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT);
     }
-    return DFSUtilClient.getNNAddress(addr);
+    return NetUtils.createSocketAddr(address,
+        HdfsClientConfigKeys.DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT);
   }
 
   //
@@ -554,7 +553,7 @@ public class NameNode extends ReconfigurableBase implements
    * If the service rpc is not configured returns null
    */
   protected InetSocketAddress getServiceRpcServerAddress(Configuration conf) {
-    return NameNode.getServiceAddress(conf, false);
+    return NameNode.getServiceAddress(conf);
   }
 
   protected InetSocketAddress getRpcServerAddress(Configuration conf) {
@@ -615,7 +614,8 @@ public class NameNode extends ReconfigurableBase implements
   }
 
   /**
-   * Modifies the configuration passed to contain the service rpc address setting
+   * Modifies the configuration passed to contain the service rpc address
+   * setting.
    */
   protected void setRpcServiceServerAddress(Configuration conf,
       InetSocketAddress serviceRPCAddress) {
@@ -1071,6 +1071,13 @@ public class NameNode extends ReconfigurableBase implements
   }
 
   /**
+   * @return NameNode service RPC address in "host:port" string form
+   */
+  public String getServiceRpcAddressHostPortString() {
+    return NetUtils.getHostPortString(getServiceRpcAddress());
+  }
+
+  /**
    * @return NameNode HTTP address, used by the Web UI, image transfer,
    *    and HTTP-based file system clients like WebHDFS
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa4b6fbe/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index 1ef3f55..78790bd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -333,66 +333,63 @@ public class NameNodeRpcServer implements NamenodeProtocols {
         .newReflectiveBlockingService(traceAdminXlator);
 
     InetSocketAddress serviceRpcAddr = nn.getServiceRpcServerAddress(conf);
-    if (serviceRpcAddr != null) {
-      String bindHost = nn.getServiceRpcServerBindHost(conf);
-      if (bindHost == null) {
-        bindHost = serviceRpcAddr.getHostName();
-      }
-      LOG.info("Service RPC server is binding to " + bindHost + ":" +
-          serviceRpcAddr.getPort());
-
-      int serviceHandlerCount =
-        conf.getInt(DFS_NAMENODE_SERVICE_HANDLER_COUNT_KEY,
-                    DFS_NAMENODE_SERVICE_HANDLER_COUNT_DEFAULT);
-      serviceRpcServer = new RPC.Builder(conf)
-          .setProtocol(
-              org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB.class)
-          .setInstance(clientNNPbService)
-          .setBindAddress(bindHost)
-          .setPort(serviceRpcAddr.getPort())
-          .setNumHandlers(serviceHandlerCount)
-          .setVerbose(false)
-          .setSecretManager(namesystem.getDelegationTokenSecretManager())
-          .build();
+    String bindHost = nn.getServiceRpcServerBindHost(conf);
+    if (bindHost == null) {
+      bindHost = serviceRpcAddr.getHostName();
+    }
 
-      // Add all the RPC protocols that the namenode implements
-      DFSUtil.addPBProtocol(conf, HAServiceProtocolPB.class, haPbService,
-          serviceRpcServer);
-      DFSUtil.addPBProtocol(conf, ReconfigurationProtocolPB.class,
-          reconfigurationPbService, serviceRpcServer);
-      DFSUtil.addPBProtocol(conf, NamenodeProtocolPB.class, NNPbService,
-          serviceRpcServer);
-      DFSUtil.addPBProtocol(conf, DatanodeProtocolPB.class, dnProtoPbService,
-          serviceRpcServer);
-      DFSUtil.addPBProtocol(conf, RefreshAuthorizationPolicyProtocolPB.class,
-          refreshAuthService, serviceRpcServer);
-      DFSUtil.addPBProtocol(conf, RefreshUserMappingsProtocolPB.class, 
-          refreshUserMappingService, serviceRpcServer);
-      // We support Refreshing call queue here in case the client RPC queue is full
-      DFSUtil.addPBProtocol(conf, RefreshCallQueueProtocolPB.class,
-          refreshCallQueueService, serviceRpcServer);
-      DFSUtil.addPBProtocol(conf, GenericRefreshProtocolPB.class,
-          genericRefreshService, serviceRpcServer);
-      DFSUtil.addPBProtocol(conf, GetUserMappingsProtocolPB.class, 
-          getUserMappingService, serviceRpcServer);
-      DFSUtil.addPBProtocol(conf, TraceAdminProtocolPB.class,
-          traceAdminService, serviceRpcServer);
+    LOG.info("Service RPC server is binding to " + bindHost + ":" +
+        serviceRpcAddr.getPort());
 
-      // Update the address with the correct port
-      InetSocketAddress listenAddr = serviceRpcServer.getListenerAddress();
-      serviceRPCAddress = new InetSocketAddress(
-            serviceRpcAddr.getHostName(), listenAddr.getPort());
-      nn.setRpcServiceServerAddress(conf, serviceRPCAddress);
-    } else {
-      serviceRpcServer = null;
-      serviceRPCAddress = null;
-    }
+    int serviceHandlerCount = conf.getInt(
+        DFS_NAMENODE_SERVICE_HANDLER_COUNT_KEY,
+        DFS_NAMENODE_SERVICE_HANDLER_COUNT_DEFAULT);
+    serviceRpcServer = new RPC.Builder(conf)
+        .setProtocol(
+            org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB.class)
+        .setInstance(clientNNPbService)
+        .setBindAddress(bindHost)
+        .setPort(serviceRpcAddr.getPort())
+        .setNumHandlers(serviceHandlerCount)
+        .setVerbose(false)
+        .setSecretManager(namesystem.getDelegationTokenSecretManager())
+        .build();
+
+    // Add all the RPC protocols that the namenode implements
+    DFSUtil.addPBProtocol(conf, HAServiceProtocolPB.class, haPbService,
+        serviceRpcServer);
+    DFSUtil.addPBProtocol(conf, ReconfigurationProtocolPB.class,
+        reconfigurationPbService, serviceRpcServer);
+    DFSUtil.addPBProtocol(conf, NamenodeProtocolPB.class, NNPbService,
+        serviceRpcServer);
+    DFSUtil.addPBProtocol(conf, DatanodeProtocolPB.class, dnProtoPbService,
+        serviceRpcServer);
+    DFSUtil.addPBProtocol(conf, RefreshAuthorizationPolicyProtocolPB.class,
+        refreshAuthService, serviceRpcServer);
+    DFSUtil.addPBProtocol(conf, RefreshUserMappingsProtocolPB.class,
+        refreshUserMappingService, serviceRpcServer);
+    // We support Refreshing call queue here in case the client RPC queue
+    // is full.
+    DFSUtil.addPBProtocol(conf, RefreshCallQueueProtocolPB.class,
+        refreshCallQueueService, serviceRpcServer);
+    DFSUtil.addPBProtocol(conf, GenericRefreshProtocolPB.class,
+        genericRefreshService, serviceRpcServer);
+    DFSUtil.addPBProtocol(conf, GetUserMappingsProtocolPB.class,
+        getUserMappingService, serviceRpcServer);
+    DFSUtil.addPBProtocol(conf, TraceAdminProtocolPB.class,
+        traceAdminService, serviceRpcServer);
+
+    // Update the address with the correct port.
+    InetSocketAddress listenAddr = serviceRpcServer.getListenerAddress();
+    serviceRPCAddress = new InetSocketAddress(
+          serviceRpcAddr.getHostName(), listenAddr.getPort());
+    nn.setRpcServiceServerAddress(conf, serviceRPCAddress);
 
     InetSocketAddress lifelineRpcAddr = nn.getLifelineRpcServerAddress(conf);
     if (lifelineRpcAddr != null) {
       RPC.setProtocolEngine(conf, HAServiceProtocolPB.class,
           ProtobufRpcEngine.class);
-      String bindHost = nn.getLifelineRpcServerBindHost(conf);
+      bindHost = nn.getLifelineRpcServerBindHost(conf);
       if (bindHost == null) {
         bindHost = lifelineRpcAddr.getHostName();
       }
@@ -422,7 +419,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
           lifelineProtoPbService, lifelineRpcServer);
 
       // Update the address with the correct port
-      InetSocketAddress listenAddr = lifelineRpcServer.getListenerAddress();
+      listenAddr = lifelineRpcServer.getListenerAddress();
       lifelineRPCAddress = new InetSocketAddress(lifelineRpcAddr.getHostName(),
           listenAddr.getPort());
       nn.setRpcLifelineServerAddress(conf, lifelineRPCAddress);
@@ -432,7 +429,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     }
 
     InetSocketAddress rpcAddr = nn.getRpcServerAddress(conf);
-    String bindHost = nn.getRpcServerBindHost(conf);
+    bindHost = nn.getRpcServerBindHost(conf);
     if (bindHost == null) {
       bindHost = rpcAddr.getHostName();
     }
@@ -476,16 +473,14 @@ public class NameNodeRpcServer implements NamenodeProtocols {
           conf.getBoolean(
             CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, false)) {
       clientRpcServer.refreshServiceAcl(conf, new HDFSPolicyProvider());
-      if (serviceRpcServer != null) {
-        serviceRpcServer.refreshServiceAcl(conf, new HDFSPolicyProvider());
-      }
+      serviceRpcServer.refreshServiceAcl(conf, new HDFSPolicyProvider());
       if (lifelineRpcServer != null) {
         lifelineRpcServer.refreshServiceAcl(conf, new HDFSPolicyProvider());
       }
     }
 
     // The rpc-server port can be ephemeral... ensure we have the correct info
-    InetSocketAddress listenAddr = clientRpcServer.getListenerAddress();
+    listenAddr = clientRpcServer.getListenerAddress();
     clientRpcAddress = new InetSocketAddress(
         rpcAddr.getHostName(), listenAddr.getPort());
     nn.setRpcServerAddress(conf, clientRpcAddress);
@@ -523,9 +518,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     clientRpcServer.addSuppressedLoggingExceptions(StandbyException.class);
 
     clientRpcServer.setTracer(nn.tracer);
-    if (serviceRpcServer != null) {
-      serviceRpcServer.setTracer(nn.tracer);
-    }
+    serviceRpcServer.setTracer(nn.tracer);
     if (lifelineRpcServer != null) {
       lifelineRpcServer.setTracer(nn.tracer);
     }
@@ -554,9 +547,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
    */
   void start() {
     clientRpcServer.start();
-    if (serviceRpcServer != null) {
-      serviceRpcServer.start();      
-    }
+    serviceRpcServer.start();
     if (lifelineRpcServer != null) {
       lifelineRpcServer.start();
     }
@@ -567,9 +558,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
    */
   void join() throws InterruptedException {
     clientRpcServer.join();
-    if (serviceRpcServer != null) {
-      serviceRpcServer.join();      
-    }
+    serviceRpcServer.join();
     if (lifelineRpcServer != null) {
       lifelineRpcServer.join();
     }
@@ -582,9 +571,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     if (clientRpcServer != null) {
       clientRpcServer.stop();
     }
-    if (serviceRpcServer != null) {
-      serviceRpcServer.stop();
-    }
+    serviceRpcServer.stop();
     if (lifelineRpcServer != null) {
       lifelineRpcServer.stop();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa4b6fbe/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
index ff83e34..e8dfb72 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
@@ -228,7 +228,7 @@ public class SecondaryNameNode implements Runnable,
 
     // Create connection to the namenode.
     shouldRun = true;
-    nameNodeAddr = NameNode.getServiceAddress(conf, true);
+    nameNodeAddr = NameNode.getServiceAddress(conf);
 
     this.conf = conf;
     this.namenode = NameNodeProxies.createNonHAProxy(conf, nameNodeAddr, 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa4b6fbe/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java
index f57cb4b..fd5a70e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java
@@ -159,7 +159,8 @@ public class EditLogTailer {
 
       for (RemoteNameNodeInfo info : nns) {
         // overwrite the socket address, if we need to
-        InetSocketAddress ipc = NameNode.getServiceAddress(info.getConfiguration(), true);
+        InetSocketAddress ipc = NameNode.getServiceAddress(
+            info.getConfiguration());
         // sanity check the ipc address
         Preconditions.checkArgument(ipc.getPort() > 0,
             "Active NameNode must have an IPC port configured. " + "Got address '%s'", ipc);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa4b6fbe/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/RemoteNameNodeInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/RemoteNameNodeInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/RemoteNameNodeInfo.java
index 9a51190..248be55 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/RemoteNameNodeInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/RemoteNameNodeInfo.java
@@ -54,7 +54,7 @@ public class RemoteNameNodeInfo {
     for (Configuration otherNode : otherNodes) {
       String otherNNId = HAUtil.getNameNodeId(otherNode, nsId);
       // don't do any validation here as in some cases, it can be overwritten later
-      InetSocketAddress otherIpcAddr = NameNode.getServiceAddress(otherNode, true);
+      InetSocketAddress otherIpcAddr = NameNode.getServiceAddress(otherNode);
 
 
       final String scheme = DFSUtil.getHttpClientScheme(conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa4b6fbe/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java
index 789ed9c..3cbcd9c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java
@@ -121,7 +121,7 @@ public class StandbyCheckpointer {
   
   private URL getHttpAddress(Configuration conf) throws IOException {
     final String scheme = DFSUtil.getHttpClientScheme(conf);
-    String defaultHost = NameNode.getServiceAddress(conf, true).getHostName();
+    String defaultHost = NameNode.getServiceAddress(conf).getHostName();
     URI addr = DFSUtil.getInfoServerWithDefaultHost(defaultHost, conf, scheme);
     return addr.toURL();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa4b6fbe/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java
index e6cf16c..e780393 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java
@@ -187,7 +187,7 @@ public class GetConf extends Configured implements Tool {
   static class NameNodesCommandHandler extends CommandHandler {
     @Override
     int doWorkInternal(GetConf tool, String []args) throws IOException {
-      tool.printMap(DFSUtil.getNNServiceRpcAddressesForCluster(tool.getConf()));
+      tool.printMap(DFSUtil.getNNServiceRpcAddresses(tool.getConf()));
       return 0;
     }
   }
@@ -224,7 +224,7 @@ public class GetConf extends Configured implements Tool {
     public int doWorkInternal(GetConf tool, String []args) throws IOException {
       Configuration config = tool.getConf();
       List<ConfiguredNNAddress> cnnlist = DFSUtil.flattenAddressMap(
-          DFSUtil.getNNServiceRpcAddressesForCluster(config));
+          DFSUtil.getNNServiceRpcAddresses(config));
       if (!cnnlist.isEmpty()) {
         for (ConfiguredNNAddress cnn : cnnlist) {
           InetSocketAddress rpc = cnn.getAddress();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa4b6fbe/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 f85996b..4f37b56 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
@@ -61,8 +61,7 @@
     connecting to this address if it is configured. In the case of HA/Federation where multiple namenodes exist,
     the name service id is added to the name e.g. dfs.namenode.servicerpc-address.ns1
     dfs.namenode.rpc-address.EXAMPLENAMESERVICE
-    The value of this property will take the form of nn-host1:rpc-port.
-    If the value of this property is unset the value of dfs.namenode.rpc-address will be used as the default.
+    The value of this property will take the form of nn-host1:rpc-port. The NameNode's default service RPC port is 9840.
   </description>
 </property>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa4b6fbe/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
index 0345cf5..aa3ed30 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
@@ -166,6 +166,7 @@ public class MiniDFSCluster implements AutoCloseable {
    */
   public static class Builder {
     private int nameNodePort = 0;
+    private int nameNodeServicePort = 0;
     private int nameNodeHttpPort = 0;
     private final Configuration conf;
     private int numDataNodes = 1;
@@ -208,6 +209,14 @@ public class MiniDFSCluster implements AutoCloseable {
       this.nameNodePort = val;
       return this;
     }
+
+    /**
+     * Default: 0
+     */
+    public Builder nameNodeServicePort(int val) {
+      this.nameNodeServicePort = val;
+      return this;
+    }
     
     /**
      * Default: 0
@@ -399,8 +408,8 @@ public class MiniDFSCluster implements AutoCloseable {
     }
 
     /**
-     * Default: false
-     * When true the hosts file/include file for the cluster is setup
+     * Default: false.
+     * When true the hosts file/include file for the cluster is setup.
      */
     public Builder setupHostsFile(boolean val) {
       this.setupHostsFile = val;
@@ -410,7 +419,7 @@ public class MiniDFSCluster implements AutoCloseable {
     /**
      * Default: a single namenode.
      * See {@link MiniDFSNNTopology#simpleFederatedTopology(int)} to set up
-     * federated nameservices
+     * federated nameservices.
      */
     public Builder nnTopology(MiniDFSNNTopology topology) {
       this.nnTopology = topology;
@@ -461,7 +470,8 @@ public class MiniDFSCluster implements AutoCloseable {
     if (builder.nnTopology == null) {
       // If no topology is specified, build a single NN. 
       builder.nnTopology = MiniDFSNNTopology.simpleSingleNN(
-          builder.nameNodePort, builder.nameNodeHttpPort);
+          builder.nameNodePort, builder.nameNodeServicePort,
+          builder.nameNodeHttpPort);
     }
     assert builder.storageTypes == null ||
            builder.storageTypes.length == builder.numDataNodes;
@@ -770,7 +780,7 @@ public class MiniDFSCluster implements AutoCloseable {
                        manageNameDfsDirs, true, manageDataDfsDirs, manageDataDfsDirs,
                        operation, null, racks, hosts,
                        null, simulatedCapacities, null, true, false,
-                       MiniDFSNNTopology.simpleSingleNN(nameNodePort, 0),
+                       MiniDFSNNTopology.simpleSingleNN(nameNodePort, 0, 0),
                        true, false, false, null, true, false);
   }
 
@@ -1249,6 +1259,11 @@ public class MiniDFSCluster implements AutoCloseable {
         DFS_NAMENODE_RPC_ADDRESS_KEY, nameserviceId,
         nnConf.getNnId());
     conf.set(key, "127.0.0.1:" + nnConf.getIpcPort());
+
+    key = DFSUtil.addKeySuffixes(
+        DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, nameserviceId,
+        nnConf.getNnId());
+    conf.set(key, "127.0.0.1:" + nnConf.getServicePort());
   }
   
   private static String[] createArgs(StartupOption operation) {
@@ -1282,6 +1297,8 @@ public class MiniDFSCluster implements AutoCloseable {
     // the conf
     hdfsConf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY,
         nameserviceId, nnId), nn.getNameNodeAddressHostPortString());
+    hdfsConf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY,
+        nameserviceId, nnId), nn.getServiceRpcAddressHostPortString());
     if (nn.getHttpAddress() != null) {
       hdfsConf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_HTTP_ADDRESS_KEY,
           nameserviceId, nnId), NetUtils.getHostPortString(nn.getHttpAddress()));
@@ -1337,6 +1354,14 @@ public class MiniDFSCluster implements AutoCloseable {
     return getNN(nnIndex).conf;
   }
 
+  /**
+   * Return the cluster-wide configuration.
+   * @return
+   */
+  public Configuration getClusterConfiguration() {
+    return conf;
+  }
+
   private NameNodeInfo getNN(int nnIndex) {
     int count = 0;
     for (NameNodeInfo nn : namenodes.values()) {
@@ -1930,6 +1955,16 @@ public class MiniDFSCluster implements AutoCloseable {
   }
 
   /**
+   * Gets the service rpc port used by the NameNode, because the caller
+   * supplied port is not necessarily the actual port used.
+   * Assumption: cluster has a single namenode
+   */
+  public int getNameNodeServicePort() {
+    checkSingleNameNode();
+    return getNameNodeServicePort(0);
+  }
+
+  /**
    * @return the service rpc port used by the NameNode at the given index.
    */     
   public int getNameNodeServicePort(int nnIndex) {
@@ -2556,12 +2591,14 @@ public class MiniDFSCluster implements AutoCloseable {
     }
 
     NameNodeInfo info = getNN(nnIndex);
-    InetSocketAddress addr = info.nameNode.getServiceRpcAddress();
-    assert addr.getPort() != 0;
-    DFSClient client = new DFSClient(addr, conf);
+    InetSocketAddress nameNodeAddress = info.nameNode.getNameNodeAddress();
+    assert nameNodeAddress.getPort() != 0;
+    DFSClient client = new DFSClient(nameNodeAddress, conf);
 
     // ensure all datanodes have registered and sent heartbeat to the namenode
-    while (shouldWait(client.datanodeReport(DatanodeReportType.LIVE), addr)) {
+    InetSocketAddress serviceAddress = info.nameNode.getServiceRpcAddress();
+    while (shouldWait(client.datanodeReport(DatanodeReportType.LIVE),
+        serviceAddress)) {
       try {
         LOG.info("Waiting for cluster to become active");
         Thread.sleep(100);
@@ -3056,13 +3093,18 @@ public class MiniDFSCluster implements AutoCloseable {
     }
   }
 
+  public void addNameNode(Configuration conf, int namenodePort)
+      throws IOException{
+    addNameNode(conf, namenodePort, 0);
+  }
+
   /**
    * Add a namenode to a federated cluster and start it. Configuration of
    * datanodes in the cluster is refreshed to register with the new namenode.
    * 
    * @return newly started namenode
    */
-  public void addNameNode(Configuration conf, int namenodePort)
+  public void addNameNode(Configuration conf, int namenodePort, int servicePort)
       throws IOException {
     if(!federation)
       throw new IOException("cannot add namenode to non-federated cluster");
@@ -3076,7 +3118,9 @@ public class MiniDFSCluster implements AutoCloseable {
   
     String nnId = null;
     initNameNodeAddress(conf, nameserviceId,
-        new NNConf(nnId).setIpcPort(namenodePort));
+        new NNConf(nnId)
+            .setIpcPort(namenodePort)
+            .setServicePort(servicePort));
     // figure out the current number of NNs
     NameNodeInfo[] infos = this.getNameNodeInfos(nameserviceId);
     int nnIndex = infos == null ? 0 : infos.length;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa4b6fbe/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSNNTopology.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSNNTopology.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSNNTopology.java
index b9786a3..b1d609a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSNNTopology.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSNNTopology.java
@@ -43,12 +43,13 @@ public class MiniDFSNNTopology {
    * Set up a simple non-federated non-HA NN.
    */
   public static MiniDFSNNTopology simpleSingleNN(
-      int nameNodePort, int nameNodeHttpPort) {
+      int rpcPort, int servicePort, int httpPort) {
     return new MiniDFSNNTopology()
       .addNameservice(new MiniDFSNNTopology.NSConf(null)
         .addNN(new MiniDFSNNTopology.NNConf(null)
-          .setHttpPort(nameNodeHttpPort)
-          .setIpcPort(nameNodePort)));
+          .setIpcPort(rpcPort)
+          .setServicePort(servicePort)
+          .setHttpPort(httpPort)));
   }
   
 
@@ -221,6 +222,7 @@ public class MiniDFSNNTopology {
     private final String nnId;
     private int httpPort;
     private int ipcPort;
+    private int servicePort;
     private String clusterId;
     
     public NNConf(String nnId) {
@@ -234,6 +236,10 @@ public class MiniDFSNNTopology {
     int getIpcPort() {
       return ipcPort;
     }
+
+    int getServicePort() {
+      return servicePort;
+    }
     
     int getHttpPort() {
       return httpPort;
@@ -253,6 +259,11 @@ public class MiniDFSNNTopology {
       return this;
     }
 
+    public NNConf setServicePort(int servicePort) {
+      this.servicePort = servicePort;
+      return this;
+    }
+
     public NNConf setClusterId(String clusterId) {
       this.clusterId = clusterId;
       return this;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa4b6fbe/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
index f811d3d..4ae2a77 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
@@ -33,6 +33,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMESERVICE_ID;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_KEYPASSWORD_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_KEYSTORE_PASSWORD_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_TRUSTSTORE_PASSWORD_KEY;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT;
 import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
 import static org.apache.hadoop.test.PlatformAssumptions.assumeNotWindows;
 import static org.hamcrest.CoreMatchers.not;
@@ -83,9 +84,9 @@ import com.google.common.collect.Sets;
 
 public class TestDFSUtil {
 
-  static final String NS1_NN_ADDR    = "ns1-nn.example.com:9820";
-  static final String NS1_NN1_ADDR   = "ns1-nn1.example.com:9820";
-  static final String NS1_NN2_ADDR   = "ns1-nn2.example.com:9820";
+  private static final String NS1_NN_ADDR    = "ns1-nn.example.com:9820";
+  private static final String NS1_NN1_ADDR   = "ns1-nn1.example.com:9820";
+  private static final String NS1_NN2_ADDR   = "ns1-nn2.example.com:9820";
 
   /**
    * Reset to default UGI settings since some tests change them.
@@ -273,13 +274,13 @@ public class TestDFSUtil {
     assertEquals(1, nn1Map.size());
     InetSocketAddress addr = nn1Map.get(null);
     assertEquals("localhost", addr.getHostName());
-    assertEquals(9000, addr.getPort());
+    assertEquals(DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT, addr.getPort());
     
     Map<String, InetSocketAddress> nn2Map = nnMap.get("nn2");
     assertEquals(1, nn2Map.size());
     addr = nn2Map.get(null);
     assertEquals("localhost", addr.getHostName());
-    assertEquals(9001, addr.getPort());
+    assertEquals(DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT, addr.getPort());
 
     // Test - can look up nameservice ID from service address
     checkNameServiceId(conf, NN1_ADDRESS, "nn1");
@@ -314,7 +315,8 @@ public class TestDFSUtil {
     Map<String, InetSocketAddress> defaultNsMap = addrMap.get(null);
     assertEquals(1, defaultNsMap.size());
     
-    assertEquals(9999, defaultNsMap.get(null).getPort());
+    assertEquals(DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT,
+        defaultNsMap.get(null).getPort());
   }
   
   /**
@@ -491,6 +493,10 @@ public class TestDFSUtil {
     final String NS1_NN2_HOST = "ns1-nn2.example.com:9820";
     final String NS2_NN1_HOST = "ns2-nn1.example.com:9820";
     final String NS2_NN2_HOST = "ns2-nn2.example.com:9820";
+    final String NS1_NN1_SERVICE_HOST = "ns1-nn1.example.com:9840";
+    final String NS1_NN2_SERVICE_HOST = "ns1-nn2.example.com:9840";
+    final String NS2_NN1_SERVICE_HOST = "ns2-nn1.example.com:9840";
+    final String NS2_NN2_SERVICE_HOST = "ns2-nn2.example.com:9840";
     conf.set(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY, "hdfs://ns1");
     
     // Two nameservices, each with two NNs.
@@ -524,12 +530,14 @@ public class TestDFSUtil {
     assertEquals(NS2_NN1_HOST, map.get("ns2").get("ns2-nn1").toString());
     assertEquals(NS2_NN2_HOST, map.get("ns2").get("ns2-nn2").toString());
     
-    assertEquals(NS1_NN1_HOST, 
+    assertEquals(NS1_NN1_SERVICE_HOST,
         DFSUtil.getNamenodeServiceAddr(conf, "ns1", "ns1-nn1"));
-    assertEquals(NS1_NN2_HOST, 
+    assertEquals(NS1_NN2_SERVICE_HOST,
         DFSUtil.getNamenodeServiceAddr(conf, "ns1", "ns1-nn2"));
-    assertEquals(NS2_NN1_HOST, 
+    assertEquals(NS2_NN1_SERVICE_HOST,
         DFSUtil.getNamenodeServiceAddr(conf, "ns2", "ns2-nn1"));
+    assertEquals(NS2_NN2_SERVICE_HOST,
+        DFSUtil.getNamenodeServiceAddr(conf, "ns2", "ns2-nn2"));
 
     // No nameservice was given and we can't determine which service addr
     // to use as two nameservices could share a namenode ID.
@@ -555,9 +563,11 @@ public class TestDFSUtil {
     
     // One nameservice with two NNs
     final String NS1_NN1_HOST = "ns1-nn1.example.com:9820";
-    final String NS1_NN1_HOST_SVC = "ns1-nn2.example.com:9821";
-    final String NS1_NN2_HOST = "ns1-nn1.example.com:9820";
+    final String NS1_NN1_HOST_SVC = "ns1-nn1.example.com:9821";
+    final String NS1_NN1_HOST_DEFAULT_SVC = "ns1-nn1.example.com:9840";
+    final String NS1_NN2_HOST = "ns1-nn2.example.com:9820";
     final String NS1_NN2_HOST_SVC = "ns1-nn2.example.com:9821";
+    final String NS1_NN2_HOST_DEFAULT_SVC = "ns1-nn2.example.com:9840";
    
     conf.set(DFS_NAMESERVICES, "ns1");
     conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY_PREFIX, "ns1"),"nn1,nn2"); 
@@ -567,12 +577,15 @@ public class TestDFSUtil {
     conf.set(DFSUtil.addKeySuffixes(
         DFS_NAMENODE_RPC_ADDRESS_KEY, "ns1", "nn2"), NS1_NN2_HOST);
 
-    // The rpc address is used if no service address is defined
-    assertEquals(NS1_NN1_HOST, DFSUtil.getNamenodeServiceAddr(conf, null, "nn1"));
-    assertEquals(NS1_NN2_HOST, DFSUtil.getNamenodeServiceAddr(conf, null, "nn2"));
+    // The default service rpc address is used if no service address is defined
+    assertEquals(NS1_NN1_HOST_DEFAULT_SVC,
+        DFSUtil.getNamenodeServiceAddr(conf, null, "nn1"));
+    assertEquals(NS1_NN2_HOST_DEFAULT_SVC,
+        DFSUtil.getNamenodeServiceAddr(conf, null, "nn2"));
 
     // A nameservice is specified explicitly
-    assertEquals(NS1_NN1_HOST, DFSUtil.getNamenodeServiceAddr(conf, "ns1", "nn1"));
+    assertEquals(NS1_NN1_HOST_DEFAULT_SVC,
+        DFSUtil.getNamenodeServiceAddr(conf, "ns1", "nn1"));
     assertEquals(null, DFSUtil.getNamenodeServiceAddr(conf, "invalid", "nn1"));
     
     // The service addrs are used when they are defined
@@ -996,6 +1009,92 @@ public class TestDFSUtil {
   }
 
   @Test
+  public void testGetNNServiceRpcAddresses() throws IOException {
+    Configuration conf = new HdfsConfiguration();
+    final String NN_HOST = "nn.example.com";
+    final String NN_ADDRESS = "hdfs://" + NN_HOST + ":9000/";
+    conf.set(FS_DEFAULT_NAME_KEY, NN_ADDRESS);
+
+    // No service RPC, no rpc
+    Map<String, Map<String, InetSocketAddress>> nsMap = DFSUtil
+        .getNNServiceRpcAddresses(conf);
+    assertEquals(1, nsMap.size());
+    InetSocketAddress address = nsMap.get(null).get(null);
+    assertEquals(DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT,
+        address.getPort());
+    assertEquals(NN_HOST, address.getHostName());
+
+    // No service RPC
+    final String RPC_ADDRESS = NN_HOST + ":9191";
+    conf.set(DFS_NAMENODE_RPC_ADDRESS_KEY, RPC_ADDRESS);
+    nsMap = DFSUtil.getNNServiceRpcAddresses(conf);
+    assertEquals(1, nsMap.size());
+    address = nsMap.get(null).get(null);
+    assertEquals(DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT,
+        address.getPort());
+    assertEquals(NN_HOST, address.getHostName());
+
+    // Service RPC present
+    final String SERVICE_RPC_ADDRESS = NN_HOST + ":9292";
+    conf.set(DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, SERVICE_RPC_ADDRESS);
+    nsMap = DFSUtil.getNNServiceRpcAddresses(conf);
+    assertEquals(1, nsMap.size());
+    address = nsMap.get(null).get(null);
+    assertEquals(9292, address.getPort());
+    assertEquals(NN_HOST, address.getHostName());
+  }
+
+  @Test
+  public void testGetNNServiceRpcAddressesForHA() throws IOException {
+    Configuration conf = new HdfsConfiguration();
+
+    final String NS = "mycluster";
+    final String NN1_HOST = "nn1.example.com";
+    final String NN2_HOST = "nn2.example.com";
+    conf.set(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY, "hdfs://mycluster");
+
+    conf.set(DFS_NAMESERVICES, NS);
+    conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY_PREFIX, NS),
+        "nn1,nn2");
+    conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY, NS, "nn1"),
+        NN1_HOST + ":9820");
+    conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY, NS, "nn2"),
+        NN2_HOST + ":9820");
+
+    assertTrue(HAUtil.isHAEnabled(conf, NS));
+
+    // Without Service RPC keys
+    Map<String, Map<String, InetSocketAddress>> nsMap =
+        DFSUtil.getNNServiceRpcAddresses(conf);
+    assertEquals(1, nsMap.size());
+    Map<String, InetSocketAddress> nnMap = nsMap.get(NS);
+    assertEquals(2, nnMap.size());
+    InetSocketAddress nn1Address = nnMap.get("nn1");
+    assertEquals(NN1_HOST, nn1Address.getHostName());
+    assertEquals(DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT, nn1Address.getPort());
+    InetSocketAddress nn2Address = nnMap.get("nn2");
+    assertEquals(NN2_HOST, nn2Address.getHostName());
+    assertEquals(DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT, nn2Address.getPort());
+
+    // With Service RPC keys
+    final int CUSTOM_SERVICE_PORT = 9191;
+    conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY,
+        NS, "nn1"), NN1_HOST + ":" + CUSTOM_SERVICE_PORT);
+    conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY,
+        NS, "nn2"), NN2_HOST + ":" + CUSTOM_SERVICE_PORT);
+    nsMap = DFSUtil.getNNServiceRpcAddresses(conf);
+    assertEquals(1, nsMap.size());
+    nnMap = nsMap.get(NS);
+    assertEquals(2, nnMap.size());
+    nn1Address = nnMap.get("nn1");
+    assertEquals(NN1_HOST, nn1Address.getHostName());
+    assertEquals(CUSTOM_SERVICE_PORT, nn1Address.getPort());
+    nn2Address = nnMap.get("nn2");
+    assertEquals(NN2_HOST, nn2Address.getHostName());
+    assertEquals(CUSTOM_SERVICE_PORT, nn2Address.getPort());
+  }
+
+  @Test
   public void testGetNNServiceRpcAddressesForNsIds() throws IOException {
     Configuration conf = new HdfsConfiguration();
     conf.set(DFS_NAMESERVICES, "nn1,nn2");
@@ -1017,13 +1116,13 @@ public class TestDFSUtil {
     }
 
     Map<String, Map<String, InetSocketAddress>> nnMap = DFSUtil
-            .getNNServiceRpcAddressesForCluster(conf);
+            .getNNServiceRpcAddresses(conf);
     assertEquals(1, nnMap.size());
     assertTrue(nnMap.containsKey("nn1"));
 
     conf.set(DFS_INTERNAL_NAMESERVICES_KEY, "nn3");
     try {
-      DFSUtil.getNNServiceRpcAddressesForCluster(conf);
+      DFSUtil.getNNServiceRpcAddresses(conf);
       fail("Should fail for misconfiguration");
     } catch (IOException ignored) {
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa4b6fbe/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSServerPorts.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSServerPorts.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSServerPorts.java
index 59e8555..1914b78 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSServerPorts.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSServerPorts.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.net.DNS;
 import org.apache.hadoop.test.PathUtils;
+import org.junit.Ignore;
 import org.junit.Test;
 
 import java.io.File;
@@ -277,17 +278,14 @@ public class TestHDFSServerPorts {
       // different http port
       conf2.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, THIS_HOST);
       started = canStartNameNode(conf2);
+      assertFalse("Should've failed on service port", started);
 
-      if (withService) {
-        assertFalse("Should've failed on service port", started);
-
-        // reset conf2 since NameNode modifies it
-        FileSystem.setDefaultUri(conf2, "hdfs://" + THIS_HOST);
-        conf2.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, THIS_HOST);
-        // Set Service address      
-        conf2.set(DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY,  THIS_HOST);
-        started = canStartNameNode(conf2);        
-      }
+      // reset conf2 since NameNode modifies it
+      FileSystem.setDefaultUri(conf2, "hdfs://" + THIS_HOST);
+      conf2.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, THIS_HOST);
+      // Set Service address
+      conf2.set(DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY,  THIS_HOST);
+      started = canStartNameNode(conf2);
       assertTrue(started);
     } finally {
       stopNameNode(nn);
@@ -359,38 +357,39 @@ public class TestHDFSServerPorts {
     }
   }
     
-    /**
-     * Verify BackupNode port usage.
-     */
-    @Test(timeout = 300000)
-    public void testBackupNodePorts() throws Exception {
-      NameNode nn = null;
-      try {
-        nn = startNameNode();
-
-        Configuration backup_config = new HdfsConfiguration(config);
-        backup_config.set(
-            DFSConfigKeys.DFS_NAMENODE_BACKUP_ADDRESS_KEY, THIS_HOST);
-        // bind http server to the same port as name-node
-        backup_config.set(DFSConfigKeys.DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY, 
-            backup_config.get(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY));
-
-        LOG.info("= Starting 1 on: " + backup_config.get(
-            DFSConfigKeys.DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY));
-
-        assertFalse("Backup started on same port as Namenode", 
-                           canStartBackupNode(backup_config)); // should fail
-
-        // bind http server to a different port
-        backup_config.set(
-            DFSConfigKeys.DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY, THIS_HOST);
-        LOG.info("= Starting 2 on: " + backup_config.get(
-            DFSConfigKeys.DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY));
-
-        boolean started = canStartBackupNode(backup_config);
-        assertTrue("Backup Namenode should've started", started); // should start now
-      } finally {
-        stopNameNode(nn);
-      }
+  /**
+   * Verify BackupNode port usage.
+   */
+  @Ignore
+  @Test(timeout = 300000)
+  public void testBackupNodePorts() throws Exception {
+    NameNode nn = null;
+    try {
+      nn = startNameNode();
+
+      Configuration backup_config = new HdfsConfiguration(config);
+      backup_config.set(
+          DFSConfigKeys.DFS_NAMENODE_BACKUP_ADDRESS_KEY, THIS_HOST);
+      // bind http server to the same port as name-node
+      backup_config.set(DFSConfigKeys.DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY,
+          backup_config.get(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY));
+
+      LOG.info("= Starting 1 on: " + backup_config.get(
+          DFSConfigKeys.DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY));
+
+      assertFalse("Backup started on same port as Namenode",
+                         canStartBackupNode(backup_config)); // should fail
+
+      // bind http server to a different port
+      backup_config.set(
+          DFSConfigKeys.DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY, THIS_HOST);
+      LOG.info("= Starting 2 on: " + backup_config.get(
+          DFSConfigKeys.DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY));
+
+      boolean started = canStartBackupNode(backup_config);
+      assertTrue("Backup Namenode should've started", started); // should start now
+    } finally {
+      stopNameNode(nn);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa4b6fbe/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
index f25d28f..df6dc03 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
@@ -324,7 +324,7 @@ public class TestSafeMode {
     } catch (RemoteException re) {
       assertEquals(SafeModeException.class.getName(), re.getClassName());
       GenericTestUtils.assertExceptionContains(
-          NameNode.getServiceAddress(conf, true).getHostName(), re);
+          NameNode.getServiceAddress(conf).getHostName(), re);
     } catch (IOException ioe) {
       fail("Encountered exception" + " " + StringUtils.stringifyException(ioe));
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa4b6fbe/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniQJMHACluster.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniQJMHACluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniQJMHACluster.java
index c163894..501ba77 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniQJMHACluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniQJMHACluster.java
@@ -77,7 +77,9 @@ public class MiniQJMHACluster {
   public static MiniDFSNNTopology createDefaultTopology(int nns, int startingPort) {
     MiniDFSNNTopology.NSConf nameservice = new MiniDFSNNTopology.NSConf(NAMESERVICE);
     for (int i = 0; i < nns; i++) {
-      nameservice.addNN(new MiniDFSNNTopology.NNConf("nn" + i).setIpcPort(startingPort++)
+      nameservice.addNN(new MiniDFSNNTopology.NNConf("nn" + i)
+          .setIpcPort(startingPort++)
+          .setServicePort(startingPort++)
           .setHttpPort(startingPort++));
     }
 
@@ -148,8 +150,9 @@ public class MiniQJMHACluster {
     int port = basePort;
     for (int i = 0; i < numNNs; i++) {
       nns.add("127.0.0.1:" + port);
-      // increment by 2 each time to account for the http port in the config setting
-      port += 2;
+      // increment by 3 each time to account for the http and the service port
+      // in the config setting
+      port += 3;
     }
 
     // use standard failover configurations

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa4b6fbe/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithHANameNodes.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithHANameNodes.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithHANameNodes.java
index 1444193..516f159 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithHANameNodes.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithHANameNodes.java
@@ -89,8 +89,9 @@ public class TestBalancerWithHANameNodes {
           / numOfDatanodes, (short) numOfDatanodes, 1);
 
       // start up an empty node with the same capacity and on the same rack
-      cluster.startDataNodes(conf, 1, true, null, new String[] { newNodeRack },
-          new long[] { newNodeCapacity });
+      cluster.startDataNodes(cluster.getClusterConfiguration(),
+          1, true, null, new String[] {newNodeRack},
+          new long[] {newNodeCapacity});
       totalCapacity += newNodeCapacity;
       TestBalancer.waitForHeartBeat(totalUsedSpace, totalCapacity, client,
           cluster);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa4b6fbe/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/InternalDataNodeTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/InternalDataNodeTestUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/InternalDataNodeTestUtils.java
index 876a854..c199c9c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/InternalDataNodeTestUtils.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/InternalDataNodeTestUtils.java
@@ -105,8 +105,11 @@ public class InternalDataNodeTestUtils {
    *
    * @throws IOException
    */
-  public static DataNode startDNWithMockNN(Configuration conf,
-      final InetSocketAddress nnSocketAddr, final String dnDataDir)
+  public static DataNode startDNWithMockNN(
+      Configuration conf,
+      final InetSocketAddress nnSocketAddr,
+      final InetSocketAddress nnServiceAddr,
+      final String dnDataDir)
       throws IOException {
 
     FileSystem.setDefaultUri(conf, "hdfs://" + nnSocketAddr.getHostName() + ":"
@@ -149,7 +152,7 @@ public class InternalDataNodeTestUtils {
       @Override
       DatanodeProtocolClientSideTranslatorPB connectToNN(
           InetSocketAddress nnAddr) throws IOException {
-        Assert.assertEquals(nnSocketAddr, nnAddr);
+        Assert.assertEquals(nnServiceAddr, nnAddr);
         return namenode;
       }
     };

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa4b6fbe/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
index 311d5a6..98450f6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
@@ -124,8 +124,6 @@ public class TestBlockRecovery {
   private final static long RECOVERY_ID = 3000L;
   private final static String CLUSTER_ID = "testClusterID";
   private final static String POOL_ID = "BP-TEST";
-  private final static InetSocketAddress NN_ADDR = new InetSocketAddress(
-      "localhost", 5020);
   private final static long BLOCK_ID = 1000L;
   private final static long GEN_STAMP = 2000L;
   private final static long BLOCK_LEN = 3000L;
@@ -188,7 +186,7 @@ public class TestBlockRecovery {
     }
     conf.setInt(CommonConfigurationKeys.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 0);
     FileSystem.setDefaultUri(conf,
-        "hdfs://" + NN_ADDR.getHostName() + ":" + NN_ADDR.getPort());
+        "hdfs://localhost:5020");
     ArrayList<StorageLocation> locations = new ArrayList<StorageLocation>();
     File dataDir = new File(DATA_DIR);
     FileUtil.fullyDelete(dataDir);
@@ -231,7 +229,7 @@ public class TestBlockRecovery {
       @Override
       DatanodeProtocolClientSideTranslatorPB connectToNN(
           InetSocketAddress nnAddr) throws IOException {
-        Assert.assertEquals(NN_ADDR, nnAddr);
+        Assert.assertEquals("localhost:9840", nnAddr.toString());
         return namenode;
       }
     };

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa4b6fbe/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetricsLogger.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetricsLogger.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetricsLogger.java
index 32fda37..bee6c1d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetricsLogger.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetricsLogger.java
@@ -61,11 +61,16 @@ import com.google.common.base.Supplier;
 public class TestDataNodeMetricsLogger {
   static final Log LOG = LogFactory.getLog(TestDataNodeMetricsLogger.class);
 
+  @Rule
+  public Timeout globalTimeout = new Timeout(120_000);
+
   private static final String DATA_DIR = MiniDFSCluster.getBaseDirectory()
       + "data";
 
   private final static InetSocketAddress NN_ADDR = new InetSocketAddress(
       "localhost", 5020);
+  private final static InetSocketAddress NN_SERVICE_ADDR =
+      new InetSocketAddress("localhost", 5021);
 
   private DataNode dn;
 
@@ -86,10 +91,13 @@ public class TestDataNodeMetricsLogger {
     conf.set(DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_KEY, "0.0.0.0:0");
     conf.set(DFSConfigKeys.DFS_DATANODE_IPC_ADDRESS_KEY, "0.0.0.0:0");
     conf.setInt(CommonConfigurationKeys.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 0);
+    conf.set(DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY,
+        NN_SERVICE_ADDR.getHostName() + ":" + NN_SERVICE_ADDR.getPort());
     conf.setInt(DFS_DATANODE_METRICS_LOGGER_PERIOD_SECONDS_KEY,
         enableMetricsLogging ? 1 : 0); // If enabled, log early and log often
 
-    dn = InternalDataNodeTestUtils.startDNWithMockNN(conf, NN_ADDR, DATA_DIR);
+    dn = InternalDataNodeTestUtils.startDNWithMockNN(
+        conf, NN_ADDR, NN_SERVICE_ADDR, DATA_DIR);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa4b6fbe/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMultipleRegistrations.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMultipleRegistrations.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMultipleRegistrations.java
index 8e1e236..25650fd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMultipleRegistrations.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMultipleRegistrations.java
@@ -109,16 +109,16 @@ public class TestDataNodeMultipleRegistrations {
       BPOfferService bpos2 = dn.getAllBpOs().get(1);
 
       // The order of bpos is not guaranteed, so fix the order
-      if (getNNSocketAddress(bpos1).equals(nn2.getNameNodeAddress())) {
+      if (getNNSocketAddress(bpos1).equals(nn2.getServiceRpcAddress())) {
         BPOfferService tmp = bpos1;
         bpos1 = bpos2;
         bpos2 = tmp;
       }
 
       assertEquals("wrong nn address", getNNSocketAddress(bpos1),
-          nn1.getNameNodeAddress());
+          nn1.getServiceRpcAddress());
       assertEquals("wrong nn address", getNNSocketAddress(bpos2),
-          nn2.getNameNodeAddress());
+          nn2.getServiceRpcAddress());
       assertEquals("wrong bpid", bpos1.getBlockPoolId(), bpid1);
       assertEquals("wrong bpid", bpos2.getBlockPoolId(), bpid2);
       assertEquals("wrong cid", dn.getClusterId(), cid1);
@@ -182,7 +182,7 @@ public class TestDataNodeMultipleRegistrations {
 
       assertEquals("wrong nn address",
           getNNSocketAddress(bpos1),
-          nn1.getNameNodeAddress());
+          nn1.getServiceRpcAddress());
       assertEquals("wrong bpid", bpos1.getBlockPoolId(), bpid1);
       assertEquals("wrong cid", dn.getClusterId(), cid1);
       cluster.shutdown();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa4b6fbe/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeReconfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeReconfiguration.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeReconfiguration.java
index 1dfd3c3..884c93d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeReconfiguration.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeReconfiguration.java
@@ -51,8 +51,10 @@ public class TestDataNodeReconfiguration {
   private static final Log LOG = LogFactory.getLog(TestBlockRecovery.class);
   private static final String DATA_DIR = MiniDFSCluster.getBaseDirectory()
       + "data";
-  private final static InetSocketAddress NN_ADDR = new InetSocketAddress(
-      "localhost", 5020);
+  private final static InetSocketAddress NN_ADDR =
+      new InetSocketAddress("localhost", 5020);
+  private final static InetSocketAddress NN_SERVICE_ADDR =
+      new InetSocketAddress("localhost", 5021);
   private final int NUM_NAME_NODE = 1;
   private final int NUM_DATA_NODE = 10;
   private MiniDFSCluster cluster;
@@ -99,10 +101,13 @@ public class TestDataNodeReconfiguration {
     conf.set(DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_KEY, "0.0.0.0:0");
     conf.set(DFSConfigKeys.DFS_DATANODE_IPC_ADDRESS_KEY, "0.0.0.0:0");
     conf.setInt(CommonConfigurationKeys.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 0);
+    conf.set(DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY,
+        NN_SERVICE_ADDR.getHostName() + ":" + NN_SERVICE_ADDR.getPort());
 
     DataNode[] result = new DataNode[numDateNode];
     for (int i = 0; i < numDateNode; i++) {
-      result[i] = InternalDataNodeTestUtils.startDNWithMockNN(conf, NN_ADDR, DATA_DIR);
+      result[i] = InternalDataNodeTestUtils.startDNWithMockNN(
+          conf, NN_ADDR, NN_SERVICE_ADDR, DATA_DIR);
     }
     return result;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa4b6fbe/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeProtocolRetryPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeProtocolRetryPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeProtocolRetryPolicy.java
index bb1d9ef..5218021 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeProtocolRetryPolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeProtocolRetryPolicy.java
@@ -78,8 +78,6 @@ public class TestDatanodeProtocolRetryPolicy {
   ArrayList<StorageLocation> locations = new ArrayList<StorageLocation>();
   private final static String CLUSTER_ID = "testClusterID";
   private final static String POOL_ID = "BP-TEST";
-  private final static InetSocketAddress NN_ADDR = new InetSocketAddress(
-      "localhost", 5020);
   private static DatanodeRegistration datanodeRegistration =
       DFSTestUtil.getLocalDatanodeRegistration();
 
@@ -101,7 +99,7 @@ public class TestDatanodeProtocolRetryPolicy {
     conf.set(DFSConfigKeys.DFS_DATANODE_IPC_ADDRESS_KEY, "0.0.0.0:0");
     conf.setInt(CommonConfigurationKeys.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 0);
     FileSystem.setDefaultUri(conf,
-        "hdfs://" + NN_ADDR.getHostName() + ":" + NN_ADDR.getPort());
+        "hdfs://localhost:5020");
     File dataDir = new File(DATA_DIR);
     FileUtil.fullyDelete(dataDir);
     dataDir.mkdirs();
@@ -228,7 +226,7 @@ public class TestDatanodeProtocolRetryPolicy {
       @Override
       DatanodeProtocolClientSideTranslatorPB connectToNN(
           InetSocketAddress nnAddr) throws IOException {
-        Assert.assertEquals(NN_ADDR, nnAddr);
+        Assert.assertEquals("localhost:9840", nnAddr.toString());
         return namenode;
       }
     };

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa4b6fbe/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestRefreshNamenodes.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestRefreshNamenodes.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestRefreshNamenodes.java
index f8594ca..37d1b57 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestRefreshNamenodes.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestRefreshNamenodes.java
@@ -44,6 +44,11 @@ public class TestRefreshNamenodes {
   private final int nnPort3 = 2227;
   private final int nnPort4 = 2230;
 
+  private final int nnServicePort1 = 2222;
+  private final int nnServicePort2 = 2225;
+  private final int nnServicePort3 = 2228;
+  private final int nnServicePort4 = 2231;
+
   @Test
   public void testRefreshNamenodes() throws IOException {
     // Start cluster with a single NN and DN
@@ -52,7 +57,9 @@ public class TestRefreshNamenodes {
     try {
       MiniDFSNNTopology topology = new MiniDFSNNTopology()
         .addNameservice(new NSConf("ns1").addNN(
-            new NNConf(null).setIpcPort(nnPort1)))
+            new NNConf(null)
+                .setIpcPort(nnPort1)
+                .setServicePort(nnServicePort1)))
         .setFederation(true);
       cluster = new MiniDFSCluster.Builder(conf)
         .nnTopology(topology)
@@ -61,20 +68,20 @@ public class TestRefreshNamenodes {
       DataNode dn = cluster.getDataNodes().get(0);
       assertEquals(1, dn.getAllBpOs().size());
 
-      cluster.addNameNode(conf, nnPort2);
+      cluster.addNameNode(conf, nnPort2, nnServicePort2);
       assertEquals(2, dn.getAllBpOs().size());
 
-      cluster.addNameNode(conf, nnPort3);
+      cluster.addNameNode(conf, nnPort3, nnServicePort3);
       assertEquals(3, dn.getAllBpOs().size());
 
-      cluster.addNameNode(conf, nnPort4);
+      cluster.addNameNode(conf, nnPort4, nnServicePort4);
 
       // Ensure a BPOfferService in the datanodes corresponds to
       // a namenode in the cluster
       Set<InetSocketAddress> nnAddrsFromCluster = Sets.newHashSet();
       for (int i = 0; i < 4; i++) {
         assertTrue(nnAddrsFromCluster.add(
-            cluster.getNameNode(i).getNameNodeAddress()));
+            cluster.getNameNode(i).getServiceRpcAddress()));
       }
       
       Set<InetSocketAddress> nnAddrsFromDN = Sets.newHashSet();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa4b6fbe/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java
index 10d9f11..5c58e0a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java
@@ -54,6 +54,7 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.log4j.Level;
 import org.junit.Before;
+import org.junit.Ignore;
 import org.junit.Test;
 
 import com.google.common.base.Supplier;
@@ -61,6 +62,7 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
 
+@Ignore("Temporarily disabling the BackupNode unit test.")
 public class TestBackupNode {
   public static final Log LOG = LogFactory.getLog(TestBackupNode.class);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa4b6fbe/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
index 2e49674..4282c22 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
@@ -1364,9 +1364,9 @@ public class TestCheckpoint {
       Configuration snConf1 = new HdfsConfiguration(cluster.getConfiguration(0));
       Configuration snConf2 = new HdfsConfiguration(cluster.getConfiguration(1));
       InetSocketAddress nn1RpcAddress = cluster.getNameNode(0)
-          .getNameNodeAddress();
+          .getServiceRpcAddress();
       InetSocketAddress nn2RpcAddress = cluster.getNameNode(1)
-          .getNameNodeAddress();
+          .getServiceRpcAddress();
       String nn1 = nn1RpcAddress.getHostName() + ":" + nn1RpcAddress.getPort();
       String nn2 = nn2RpcAddress.getHostName() + ":" + nn2RpcAddress.getPort();
 
@@ -1923,6 +1923,7 @@ public class TestCheckpoint {
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
           .format(true).build();
       int origPort = cluster.getNameNodePort();
+      int origServicePort = cluster.getNameNodeServicePort();
       int origHttpPort = cluster.getNameNode().getHttpAddress().getPort();
       Configuration snnConf = new Configuration(conf);
       File checkpointDir = new File(MiniDFSCluster.getBaseDirectory(),
@@ -1949,6 +1950,7 @@ public class TestCheckpoint {
       cluster = new MiniDFSCluster.Builder(conf)
           .numDataNodes(0)
           .nameNodePort(origPort)
+          .nameNodeServicePort(origServicePort)
           .nameNodeHttpPort(origHttpPort)
           .format(true).build();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa4b6fbe/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
index 937bb61..95a391b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
@@ -661,12 +661,15 @@ public class TestNameNodeMXBean {
     for (int i = 0; i < 5; i++) {
       try{
         // Have to specify IPC ports so the NNs can talk to each other.
-        int[] ports = ServerSocketUtil.getPorts(2);
+        int[] ports = ServerSocketUtil.getPorts(4);
         MiniDFSNNTopology topology = new MiniDFSNNTopology()
             .addNameservice(new MiniDFSNNTopology.NSConf("ns1")
-                .addNN(new MiniDFSNNTopology.NNConf("nn1").setIpcPort(ports[0]))
-                .addNN(
-                    new MiniDFSNNTopology.NNConf("nn2").setIpcPort(ports[1])));
+                .addNN(new MiniDFSNNTopology.NNConf("nn1")
+                    .setIpcPort(ports[0])
+                    .setServicePort(ports[1]))
+                .addNN(new MiniDFSNNTopology.NNConf("nn2")
+                    .setIpcPort(ports[2])
+                    .setServicePort(ports[3])));
 
         cluster = new MiniDFSCluster.Builder(conf)
             .nnTopology(topology).numDataNodes(0)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa4b6fbe/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMetricsLogger.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMetricsLogger.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMetricsLogger.java
index 9a0e67c..d7216c0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMetricsLogger.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMetricsLogger.java
@@ -110,6 +110,7 @@ public class TestNameNodeMetricsLogger {
       throws IOException {
     Configuration conf = new HdfsConfiguration();
     conf.set(FS_DEFAULT_NAME_KEY, "hdfs://localhost:0");
+    conf.set(DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, "0.0.0.0:0");
     conf.set(DFS_NAMENODE_HTTP_ADDRESS_KEY, "0.0.0.0:0");
     conf.setInt(DFS_NAMENODE_METRICS_LOGGER_PERIOD_SECONDS_KEY,
         enableMetricsLogging ? 1 : 0);  // If enabled, log early and log often

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa4b6fbe/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestValidateConfigurationSettings.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestValidateConfigurationSettings.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestValidateConfigurationSettings.java
index 0cf1fed..981785a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestValidateConfigurationSettings.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestValidateConfigurationSettings.java
@@ -125,6 +125,8 @@ public class TestValidateConfigurationSettings {
     // Set ephemeral ports 
     conf.set(DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY,
         "127.0.0.1:0");
+    conf.set(DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY,
+        "127.0.0.1:0");
     conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY,
         "127.0.0.1:0");
     

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa4b6fbe/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java
index 169bbee..a367167 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java
@@ -43,7 +43,6 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
-import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.test.GenericTestUtils;


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


[21/50] [abbrv] hadoop git commit: HDFS-12350. Support meta tags in configs. Contributed by Ajay Kumar.

Posted by as...@apache.org.
HDFS-12350. Support meta tags in configs. Contributed by Ajay Kumar.


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

Branch: refs/heads/YARN-5972
Commit: a4cd101934ae5a5cad9663de872fb4ecee0d7560
Parents: 83449ab
Author: Anu Engineer <ae...@apache.org>
Authored: Thu Sep 7 12:40:09 2017 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Thu Sep 7 12:40:09 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/conf/Configuration.java   | 116 ++++++++++++++++++-
 .../org/apache/hadoop/conf/CorePropertyTag.java |  37 ++++++
 .../org/apache/hadoop/conf/HDFSPropertyTag.java |  41 +++++++
 .../org/apache/hadoop/conf/PropertyTag.java     |  30 +++++
 .../org/apache/hadoop/conf/YarnPropertyTag.java |  39 +++++++
 .../apache/hadoop/conf/TestConfiguration.java   | 112 +++++++++++++++++-
 6 files changed, 373 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4cd1019/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
index a1a40b9..a339dac 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
@@ -259,7 +259,18 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
    */
   private static final WeakHashMap<Configuration,Object> REGISTRY = 
     new WeakHashMap<Configuration,Object>();
-  
+
+  /**
+   * Map to register all classes holding property tag enums.
+   */
+  private static final Map<String, Class>
+      REGISTERED_TAG_CLASS = new HashMap<>();
+  /**
+   * Map to hold properties by there tag groupings.
+   */
+  private final Map<PropertyTag, Properties> propertyTagsMap =
+      new ConcurrentHashMap<>();
+
   /**
    * List of default Resources. Resources are loaded in the order of the list 
    * entries
@@ -738,6 +749,12 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
   public Configuration(boolean loadDefaults) {
     this.loadDefaults = loadDefaults;
     updatingResource = new ConcurrentHashMap<String, String[]>();
+
+    // Register all classes holding property tags with
+    REGISTERED_TAG_CLASS.put("core", CorePropertyTag.class);
+    REGISTERED_TAG_CLASS.put("hdfs", HDFSPropertyTag.class);
+    REGISTERED_TAG_CLASS.put("yarn", YarnPropertyTag.class);
+
     synchronized(Configuration.class) {
       REGISTRY.put(this, null);
     }
@@ -765,6 +782,8 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
      this.finalParameters = Collections.newSetFromMap(
          new ConcurrentHashMap<String, Boolean>());
      this.finalParameters.addAll(other.finalParameters);
+     this.REGISTERED_TAG_CLASS.putAll(other.REGISTERED_TAG_CLASS);
+     this.propertyTagsMap.putAll(other.propertyTagsMap);
    }
    
     synchronized(Configuration.class) {
@@ -2823,6 +2842,10 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
               } else if ("source".equals(propertyAttr)) {
                 confSource.add(StringInterner.weakIntern(
                     reader.getAttributeValue(i)));
+              } else if ("tag".equals(propertyAttr)) {
+                //Read tags and put them in propertyTagsMap
+                readTagFromConfig(reader.getAttributeValue(i), confName,
+                    confValue, confSource);
               }
             }
             break;
@@ -2830,6 +2853,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
           case "value":
           case "final":
           case "source":
+          case "tag":
             parseToken = true;
             token.setLength(0);
             break;
@@ -2911,6 +2935,13 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
           case "source":
             confSource.add(StringInterner.weakIntern(token.toString()));
             break;
+          case "tag":
+            if (token.length() > 0) {
+              //Read tags and put them in propertyTagsMap
+              readTagFromConfig(token.toString(), confName,
+                  confValue, confSource);
+            }
+            break;
           case "include":
             if (fallbackAllowed && !fallbackEntered) {
               throw new IOException("Fetch fail on include for '"
@@ -2962,6 +2993,48 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     }
   }
 
+  private void readTagFromConfig(String attributeValue, String confName, String
+      confValue, List<String> confSource) {
+    for (String tagStr : attributeValue.split(",")) {
+      tagStr = tagStr.trim();
+      try {
+        if (confSource.size() > 0) {
+          for (String source : confSource) {
+            PropertyTag tag1 = this.getPropertyTag(tagStr,
+                source.split("-")[0]);
+            if (propertyTagsMap.containsKey(tag1)) {
+              propertyTagsMap.get(tag1)
+                  .setProperty(confName, confValue);
+            } else {
+              Properties props = new Properties();
+              props.setProperty(confName, confValue);
+              propertyTagsMap.put(tag1, props);
+            }
+          }
+        } else {
+          //If no source is set try to find tag in CorePropertyTag
+          if (propertyTagsMap
+              .containsKey(CorePropertyTag.valueOf(tagStr)
+              )) {
+            propertyTagsMap.get(CorePropertyTag.valueOf(tagStr))
+                .setProperty(confName, confValue);
+          } else {
+            Properties props = new Properties();
+            props.setProperty(confName, confValue);
+            propertyTagsMap.put(CorePropertyTag.valueOf(tagStr),
+                props);
+          }
+        }
+      } catch (IllegalArgumentException iae) {
+        //Log the invalid tag and continue to parse rest of the
+        // properties.
+        LOG.info("Invalid tag '" + tagStr + "' found for "
+            + "property:" + confName, iae);
+      }
+
+    }
+  }
+
   private void overlay(Properties to, Properties from) {
     for (Entry<Object, Object> entry: from.entrySet()) {
       to.put(entry.getKey(), entry.getValue());
@@ -3438,4 +3511,45 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     }
     return false;
   }
+
+  /**
+   * Get all properties belonging to tag.
+   * @return Properties with matching properties
+   */
+  public Properties getAllPropertiesByTag(final PropertyTag tag) {
+    Properties props = new Properties();
+    if (propertyTagsMap.containsKey(tag)) {
+      props.putAll(propertyTagsMap.get(tag));
+    }
+    return props;
+  }
+
+  /**
+   * Get all properties belonging to list of input tags. Calls
+   * getAllPropertiesByTag internally.
+   *
+   * @return Properties with all matching properties
+   */
+  public Properties getAllPropertiesByTags(final List<PropertyTag> tagList) {
+    Properties prop = new Properties();
+    for (PropertyTag tag : tagList) {
+      prop.putAll(this.getAllPropertiesByTag(tag));
+    }
+    return prop;
+  }
+
+  /**
+   * Get Property tag Enum corresponding to given source.
+   *
+   * @param tagStr String representation of Enum
+   * @param group Group to which enum belongs.Ex hdfs,yarn
+   * @return Properties with all matching properties
+   */
+  private PropertyTag getPropertyTag(String tagStr, String group) {
+    PropertyTag tag = null;
+    if (REGISTERED_TAG_CLASS.containsKey(group)) {
+      tag = (PropertyTag) Enum.valueOf(REGISTERED_TAG_CLASS.get(group), tagStr);
+    }
+    return tag;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4cd1019/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/CorePropertyTag.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/CorePropertyTag.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/CorePropertyTag.java
new file mode 100644
index 0000000..54a75b8
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/CorePropertyTag.java
@@ -0,0 +1,37 @@
+/**
+ * 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.conf;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/***************************************************************************
+ * Enum for tagging hadoop core properties according to there usage.
+ * CorePropertyTag implements the
+ * {@link org.apache.hadoop.conf.PropertyTag} interface,
+ ***************************************************************************/
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public enum CorePropertyTag implements PropertyTag {
+  CORE,
+  REQUIRED,
+  PERFORMANCE,
+  CLIENT,
+  SERVER,
+  SECURITY,
+  DEBUG
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4cd1019/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/HDFSPropertyTag.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/HDFSPropertyTag.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/HDFSPropertyTag.java
new file mode 100644
index 0000000..02dfb86
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/HDFSPropertyTag.java
@@ -0,0 +1,41 @@
+/**
+ * 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.conf;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/***************************************************************************
+ * Enum for tagging hdfs properties according to there usage or application.
+ * HDFSPropertyTag implements the
+ * {@link org.apache.hadoop.conf.PropertyTag} interface,
+ ***************************************************************************/
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public enum HDFSPropertyTag implements PropertyTag {
+  HDFS,
+  NAMENODE,
+  DATANODE,
+  REQUIRED,
+  SECURITY,
+  KERBEROS,
+  PERFORMANCE,
+  CLIENT,
+  SERVER,
+  DEBUG,
+  DEPRICATED
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4cd1019/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/PropertyTag.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/PropertyTag.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/PropertyTag.java
new file mode 100644
index 0000000..df8d4f9
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/PropertyTag.java
@@ -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.
+ */
+package org.apache.hadoop.conf;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/***********************************************************
+ * PropertyTag is used for creating extendable property tag Enums.
+ * Property tags will group related properties together.
+ ***********************************************************/
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface PropertyTag {
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4cd1019/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/YarnPropertyTag.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/YarnPropertyTag.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/YarnPropertyTag.java
new file mode 100644
index 0000000..e7a9c79
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/YarnPropertyTag.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.conf;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/***************************************************************************
+ * Enum for tagging yarn properties according to there usage or application.
+ * YarnPropertyTag implements the
+ * {@link org.apache.hadoop.conf.PropertyTag} interface,
+ ***************************************************************************/
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public enum YarnPropertyTag implements PropertyTag {
+  YARN,
+  RESOURCEMANAGER,
+  SECURITY,
+  KERBEROS,
+  PERFORMANCE,
+  CLIENT,
+  REQUIRED,
+  SERVER,
+  DEBUG
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4cd1019/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
index b41a807..4cd1666 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
@@ -38,6 +38,7 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Properties;
 import java.util.Random;
 import java.util.Set;
 import java.util.regex.Pattern;
@@ -53,7 +54,6 @@ import static org.junit.Assert.assertArrayEquals;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration.IntegerRanges;
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IOUtils;
@@ -135,6 +135,7 @@ public class TestConfiguration {
 
   private void endConfig() throws IOException{
     out.write("</configuration>\n");
+    out.flush();
     out.close();
   }
 
@@ -577,6 +578,34 @@ public class TestConfiguration {
     out.write("</property>\n");
   }
 
+  private void appendPropertyByTag(String name, String val, String tags,
+      String... sources) throws IOException {
+    appendPropertyByTag(name, val, false, tags, sources);
+  }
+
+  private void appendPropertyByTag(String name, String val, boolean isFinal,
+      String tag, String... sources) throws IOException {
+    out.write("<property>");
+    out.write("<name>");
+    out.write(name);
+    out.write("</name>");
+    out.write("<value>");
+    out.write(val);
+    out.write("</value>");
+    if (isFinal) {
+      out.write("<final>true</final>");
+    }
+    for (String s : sources) {
+      out.write("<source>");
+      out.write(s);
+      out.write("</source>");
+    }
+    out.write("<tag>");
+    out.write(tag);
+    out.write("</tag>");
+    out.write("</property>\n");
+  }
+
   void appendCompactFormatProperty(String name, String val) throws IOException {
     appendCompactFormatProperty(name, val, false);
   }
@@ -2215,4 +2244,85 @@ public class TestConfiguration {
       TestConfiguration.class.getName()
     });
   }
+
+  @Test
+  public void testGetAllPropertiesByTags() throws Exception {
+
+    out = new BufferedWriter(new FileWriter(CONFIG));
+    startConfig();
+    appendPropertyByTag("dfs.cblock.trace.io", "false", "DEBUG");
+    appendPropertyByTag("dfs.replication", "1", "PERFORMANCE,REQUIRED");
+    appendPropertyByTag("dfs.namenode.logging.level", "INFO", "CLIENT,DEBUG");
+    endConfig();
+
+    Path fileResource = new Path(CONFIG);
+    conf.addResource(fileResource);
+    conf.getProps();
+
+    List<PropertyTag> tagList = new ArrayList<>();
+    tagList.add(CorePropertyTag.REQUIRED);
+    tagList.add(CorePropertyTag.PERFORMANCE);
+    tagList.add(CorePropertyTag.DEBUG);
+    tagList.add(CorePropertyTag.CLIENT);
+
+    Properties properties = conf.getAllPropertiesByTags(tagList);
+    assertEq(3, properties.size());
+    assertEq(true, properties.containsKey("dfs.namenode.logging.level"));
+    assertEq(true, properties.containsKey("dfs.replication"));
+    assertEq(true, properties.containsKey("dfs.cblock.trace.io"));
+    assertEq(false, properties.containsKey("namenode.host"));
+  }
+
+  @Test
+  public void testGetAllPropertiesWithSourceByTags() throws Exception {
+
+    out = new BufferedWriter(new FileWriter(CONFIG));
+    startConfig();
+    appendPropertyByTag("dfs.cblock.trace.io", "false", "DEBUG",
+        "hdfs-default.xml", "core-site.xml");
+    appendPropertyByTag("dfs.replication", "1", "PERFORMANCE,HDFS",
+        "hdfs-default.xml");
+    appendPropertyByTag("yarn.resourcemanager.work-preserving-recovery"
+        + ".enabled", "INFO", "CLIENT,DEBUG", "yarn-default.xml", "yarn-site"
+        + ".xml");
+    endConfig();
+
+    Path fileResource = new Path(CONFIG);
+    conf.addResource(fileResource);
+    conf.getProps();
+
+    List<PropertyTag> tagList = new ArrayList<>();
+    tagList.add(CorePropertyTag.REQUIRED);
+
+    Properties properties;
+    properties = conf.getAllPropertiesByTags(tagList);
+    assertNotEquals(3, properties.size());
+
+    tagList.add(HDFSPropertyTag.DEBUG);
+    tagList.add(YarnPropertyTag.CLIENT);
+    tagList.add(HDFSPropertyTag.PERFORMANCE);
+    tagList.add(HDFSPropertyTag.HDFS);
+    properties = conf.getAllPropertiesByTags(tagList);
+    assertEq(3, properties.size());
+
+    assertEq(true, properties.containsKey("dfs.cblock.trace.io"));
+    assertEq(true, properties.containsKey("dfs.replication"));
+    assertEq(true, properties
+        .containsKey("yarn.resourcemanager.work-preserving-recovery.enabled"));
+    assertEq(false, properties.containsKey("namenode.host"));
+
+    tagList.clear();
+    tagList.add(HDFSPropertyTag.DEBUG);
+    properties = conf.getAllPropertiesByTags(tagList);
+    assertEq(true, properties.containsKey("dfs.cblock.trace.io"));
+    assertEq(false, properties.containsKey("yarn.resourcemanager"
+        + ".work-preserving-recovery"));
+
+    tagList.clear();
+    tagList.add(YarnPropertyTag.DEBUG);
+    properties = conf.getAllPropertiesByTags(tagList);
+    assertEq(false, properties.containsKey("dfs.cblock.trace.io"));
+    assertEq(true, properties.containsKey("yarn.resourcemanager"
+        + ".work-preserving-recovery.enabled"));
+  }
 }


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


[48/50] [abbrv] hadoop git commit: YARN-6799. Remove the duplicated code in CGroupsHandlerImp.java. (Contributed by Weiyuan via Yufei Gu)

Posted by as...@apache.org.
YARN-6799. Remove the duplicated code in CGroupsHandlerImp.java. (Contributed by Weiyuan via Yufei Gu)


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

Branch: refs/heads/YARN-5972
Commit: 5ab21dfe9c8e14585db71e02bcb021a526507f98
Parents: 722ee84
Author: Yufei Gu <yu...@apache.org>
Authored: Sun Sep 10 23:17:48 2017 -0700
Committer: Yufei Gu <yu...@apache.org>
Committed: Sun Sep 10 23:17:59 2017 -0700

----------------------------------------------------------------------
 .../linux/resources/CGroupsHandlerImpl.java              | 11 -----------
 1 file changed, 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5ab21dfe/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 44ae12b..f37dfd3 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
@@ -367,17 +367,6 @@ class CGroupsHandlerImpl implements CGroupsHandler {
     if (enableCGroupMount) {
       // We have a controller that needs to be mounted
       mountCGroupController(controller);
-    } else {
-      String controllerPath = getControllerPath(controller);
-
-      if (controllerPath == null) {
-        throw new ResourceHandlerException(
-            String.format("Controller %s not mounted."
-                + " You either need to mount it with %s"
-                + " or mount cgroups before launching Yarn",
-                controller.getName(), YarnConfiguration.
-                NM_LINUX_CONTAINER_CGROUPS_MOUNT));
-      }
     }
 
     // We are working with a pre-mounted contoller


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


[40/50] [abbrv] hadoop git commit: MAPREDUCE-6953. Skip the testcase testJobWithChangePriority if FairScheduler is used (pbacsko via rkanter)

Posted by as...@apache.org.
MAPREDUCE-6953. Skip the testcase testJobWithChangePriority if FairScheduler is used (pbacsko via rkanter)


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

Branch: refs/heads/YARN-5972
Commit: a323f73bae65483652de290ef819268850803a17
Parents: a466185
Author: Robert Kanter <rk...@apache.org>
Authored: Fri Sep 8 13:14:23 2017 -0700
Committer: Robert Kanter <rk...@apache.org>
Committed: Fri Sep 8 13:14:36 2017 -0700

----------------------------------------------------------------------
 .../test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java  | 7 ++++++-
 1 file changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a323f73b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java
index 274f405..22cb530 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java
@@ -95,11 +95,13 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.log4j.Level;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Assert;
+import org.junit.Assume;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -402,6 +404,10 @@ public class TestMRJobs {
 
   @Test(timeout = 3000000)
   public void testJobWithChangePriority() throws Exception {
+    Configuration sleepConf = new Configuration(mrCluster.getConfig());
+    // Assumption can be removed when FS priority support is implemented
+    Assume.assumeFalse(sleepConf.get(YarnConfiguration.RM_SCHEDULER)
+            .equals(FairScheduler.class.getCanonicalName()));
 
     if (!(new File(MiniMRYarnCluster.APPJAR)).exists()) {
       LOG.info("MRAppJar " + MiniMRYarnCluster.APPJAR
@@ -409,7 +415,6 @@ public class TestMRJobs {
       return;
     }
 
-    Configuration sleepConf = new Configuration(mrCluster.getConfig());
     // set master address to local to test that local mode applied if framework
     // equals local
     sleepConf.set(MRConfig.MASTER_ADDRESS, "local");


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


[19/50] [abbrv] hadoop git commit: HADOOP-14520. WASB: Block compaction for Azure Block Blobs. Contributed by Georgi Chalakov

Posted by as...@apache.org.
HADOOP-14520. WASB: Block compaction for Azure Block Blobs.
Contributed by Georgi Chalakov


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

Branch: refs/heads/YARN-5972
Commit: 13eda5000304099d1145631f9be13ce8a00b600d
Parents: d77ed23
Author: Steve Loughran <st...@apache.org>
Authored: Thu Sep 7 18:35:03 2017 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Thu Sep 7 18:35:03 2017 +0100

----------------------------------------------------------------------
 .../fs/azure/AzureNativeFileSystemStore.java    |   73 +-
 .../hadoop/fs/azure/BlockBlobAppendStream.java  | 1301 +++++++++++-------
 .../hadoop/fs/azure/NativeAzureFileSystem.java  |   77 +-
 .../hadoop/fs/azure/NativeFileSystemStore.java  |    5 +-
 .../fs/azure/SecureStorageInterfaceImpl.java    |   10 +-
 .../hadoop/fs/azure/SelfRenewingLease.java      |   10 +-
 .../hadoop/fs/azure/StorageInterface.java       |    3 +-
 .../hadoop/fs/azure/StorageInterfaceImpl.java   |   12 +-
 .../fs/azure/SyncableDataOutputStream.java      |   11 +
 .../hadoop-azure/src/site/markdown/index.md     |   34 +
 .../hadoop/fs/azure/MockStorageInterface.java   |    3 +-
 .../azure/TestAzureConcurrentOutOfBandIo.java   |    6 +-
 ...estNativeAzureFileSystemBlockCompaction.java |  266 ++++
 .../src/test/resources/log4j.properties         |    1 +
 14 files changed, 1273 insertions(+), 539 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/13eda500/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
index bd8ac68..639862f 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
@@ -203,6 +203,23 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
   private Set<String> pageBlobDirs;
 
   /**
+   * Configuration key to indicate the set of directories in WASB where we
+   * should store files as block blobs with block compaction enabled.
+   *
+   * Entries can be directory paths relative to the container (e.g. "/path") or
+   * fully qualified wasb:// URIs (e.g.
+   * wasb://container@example.blob.core.windows.net/path)
+   */
+  public static final String KEY_BLOCK_BLOB_WITH_COMPACTION_DIRECTORIES =
+          "fs.azure.block.blob.with.compaction.dir";
+
+  /**
+   * The set of directories where we should store files as block blobs with
+   * block compaction enabled.
+   */
+  private Set<String> blockBlobWithCompationDirs;
+
+  /**
    * Configuration key to indicate the set of directories in WASB where
    * we should do atomic folder rename synchronized with createNonRecursive.
    */
@@ -527,6 +544,12 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
     // User-agent
     userAgentId = conf.get(USER_AGENT_ID_KEY, USER_AGENT_ID_DEFAULT);
 
+    // Extract the directories that should contain block blobs with compaction
+    blockBlobWithCompationDirs = getDirectorySet(
+        KEY_BLOCK_BLOB_WITH_COMPACTION_DIRECTORIES);
+    LOG.debug("Block blobs with compaction directories:  {}",
+        setToString(blockBlobWithCompationDirs));
+
     // Extract directories that should have atomic rename applied.
     atomicRenameDirs = getDirectorySet(KEY_ATOMIC_RENAME_DIRECTORIES);
     String hbaseRoot;
@@ -1165,6 +1188,17 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
   }
 
   /**
+   * Checks if the given key in Azure Storage should be stored as a block blobs
+   * with compaction enabled instead of normal block blob.
+   *
+   * @param key blob name
+   * @return true, if the file is in directory with block compaction enabled.
+   */
+  public boolean isBlockBlobWithCompactionKey(String key) {
+    return isKeyForDirectorySet(key, blockBlobWithCompationDirs);
+  }
+
+  /**
    * Checks if the given key in Azure storage should have synchronized
    * atomic folder rename createNonRecursive implemented.
    */
@@ -1356,7 +1390,9 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
   }
 
   @Override
-  public DataOutputStream storefile(String key, PermissionStatus permissionStatus)
+  public DataOutputStream storefile(String keyEncoded,
+                                    PermissionStatus permissionStatus,
+                                    String key)
       throws AzureException {
     try {
 
@@ -1417,12 +1453,26 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
 
       // Get the blob reference from the store's container and
       // return it.
-      CloudBlobWrapper blob = getBlobReference(key);
+      CloudBlobWrapper blob = getBlobReference(keyEncoded);
       storePermissionStatus(blob, permissionStatus);
 
       // Create the output stream for the Azure blob.
       //
-      OutputStream outputStream = openOutputStream(blob);
+      OutputStream outputStream;
+
+      if (isBlockBlobWithCompactionKey(key)) {
+        BlockBlobAppendStream blockBlobOutputStream = new BlockBlobAppendStream(
+            (CloudBlockBlobWrapper) blob,
+            keyEncoded,
+            this.uploadBlockSizeBytes,
+            true,
+            getInstrumentedContext());
+
+        outputStream = blockBlobOutputStream;
+      } else {
+        outputStream = openOutputStream(blob);
+      }
+
       DataOutputStream dataOutStream = new SyncableDataOutputStream(outputStream);
       return dataOutStream;
     } catch (Exception e) {
@@ -2869,10 +2919,21 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
 
       CloudBlobWrapper blob =  this.container.getBlockBlobReference(key);
 
-      BlockBlobAppendStream appendStream = new BlockBlobAppendStream((CloudBlockBlobWrapper) blob, key, bufferSize, getInstrumentedContext());
-      appendStream.initialize();
+      OutputStream outputStream;
+
+      BlockBlobAppendStream blockBlobOutputStream = new BlockBlobAppendStream(
+          (CloudBlockBlobWrapper) blob,
+          key,
+          bufferSize,
+          isBlockBlobWithCompactionKey(key),
+          getInstrumentedContext());
+
+      outputStream = blockBlobOutputStream;
+
+      DataOutputStream dataOutStream = new SyncableDataOutputStream(
+          outputStream);
 
-      return new DataOutputStream(appendStream);
+      return dataOutStream;
     } catch(Exception ex) {
       throw new AzureException(ex);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13eda500/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/BlockBlobAppendStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/BlockBlobAppendStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/BlockBlobAppendStream.java
index afb9379..84342cd 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/BlockBlobAppendStream.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/BlockBlobAppendStream.java
@@ -22,122 +22,256 @@ import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.OutputStream;
+import java.nio.ByteBuffer;
 import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
-import java.util.Calendar;
-import java.util.HashMap;
-import java.util.Locale;
+import java.util.Iterator;
 import java.util.List;
 import java.util.UUID;
 import java.util.Random;
-import java.util.TimeZone;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.Semaphore;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.commons.lang.StringUtils;
 
 import org.apache.hadoop.fs.FSExceptionMessages;
 import org.apache.commons.codec.binary.Base64;
+import org.apache.hadoop.fs.StreamCapabilities;
+import org.apache.hadoop.fs.Syncable;
 import org.apache.hadoop.fs.azure.StorageInterface.CloudBlockBlobWrapper;
-import org.eclipse.jetty.util.log.Log;
+import org.apache.hadoop.io.ElasticByteBufferPool;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.microsoft.azure.storage.AccessCondition;
 import com.microsoft.azure.storage.OperationContext;
 import com.microsoft.azure.storage.StorageException;
+import com.microsoft.azure.storage.StorageErrorCodeStrings;
 import com.microsoft.azure.storage.blob.BlobRequestOptions;
 import com.microsoft.azure.storage.blob.BlockEntry;
 import com.microsoft.azure.storage.blob.BlockListingFilter;
+import com.microsoft.azure.storage.blob.BlockSearchMode;
+
+import static org.apache.hadoop.fs.StreamCapabilities.StreamCapability.HFLUSH;
+import static org.apache.hadoop.fs.StreamCapabilities.StreamCapability.HSYNC;
 
 /**
- * Stream object that implememnts append for Block Blobs in WASB.
+ * Stream object that implements append for Block Blobs in WASB.
+ *
+ * The stream object implements hflush/hsync and block compaction. Block
+ * compaction is the process of replacing a sequence of small blocks with one
+ * big block. Azure Block blobs supports up to 50000 blocks and every
+ * hflush/hsync generates one block. When the number of blocks is above 32000,
+ * the process of compaction decreases the total number of blocks, if possible.
+ * If compaction is disabled, hflush/hsync are empty functions.
+ *
+ * The stream object uses background threads for uploading the blocks and the
+ * block blob list. Blocks can be uploaded concurrently. However, when the block
+ * list is uploaded, block uploading should stop. If a block is uploaded before
+ * the block list and the block id is not in the list, the block will be lost.
+ * If the block is uploaded after the block list and the block id is in the
+ * list, the block list upload will fail. The exclusive access for the block
+ * list upload is managed by uploadingSemaphore.
  */
-public class BlockBlobAppendStream extends OutputStream {
 
+public class BlockBlobAppendStream extends OutputStream implements Syncable,
+    StreamCapabilities {
+
+  /**
+   * The name of the blob/file.
+   */
   private final String key;
-  private final int bufferSize;
-  private ByteArrayOutputStream outBuffer;
-  private final CloudBlockBlobWrapper blob;
-  private final OperationContext opContext;
 
   /**
-   * Variable to track if the stream has been closed.
+   * This variable tracks if this is new blob or existing one.
+   */
+  private boolean blobExist;
+
+  /**
+   * When the blob exist, to to prevent concurrent write we take a lease.
+   * Taking a lease is not necessary for new blobs.
    */
-  private boolean closed = false;
+  private SelfRenewingLease lease = null;
 
   /**
-   * Variable to track if the append lease is released.
+   * The support for process of compaction is optional.
    */
+  private final boolean compactionEnabled;
 
-  private volatile boolean leaseFreed;
+  /**
+   * The number of blocks above each block compaction is triggered.
+   */
+  private static final int DEFAULT_ACTIVATE_COMPACTION_BLOCK_COUNT = 32000;
 
   /**
-   * Variable to track if the append stream has been
-   * initialized.
+   * The number of blocks above each block compaction is triggered.
    */
+  private int activateCompactionBlockCount
+      = DEFAULT_ACTIVATE_COMPACTION_BLOCK_COUNT;
 
-  private boolean initialized = false;
+  /**
+   * The size of the output buffer. Writes store the data in outBuffer until
+   * either the size is above maxBlockSize or hflush/hsync is called.
+   */
+  private final AtomicInteger maxBlockSize;
 
   /**
-   *  Last IOException encountered
+   * The current buffer where writes are stored.
    */
-  private volatile IOException lastError = null;
+  private ByteBuffer outBuffer;
 
   /**
-   * List to keep track of the uncommitted azure storage
-   * block ids
+   * The size of the blob that has been successfully stored in the Azure Blob
+   * service.
    */
-  private final List<BlockEntry> uncommittedBlockEntries;
+  private final AtomicLong committedBlobLength = new AtomicLong(0);
 
-  private static final int UNSET_BLOCKS_COUNT = -1;
+  /**
+   * Position of last block in the blob.
+   */
+  private volatile long blobLength = 0;
 
   /**
-   * Variable to hold the next block id to be used for azure
-   * storage blocks.
+   * Minutes waiting before the close operation timed out.
    */
-  private long nextBlockCount = UNSET_BLOCKS_COUNT;
+  private static final int CLOSE_UPLOAD_DELAY = 10;
 
   /**
-   * Variable to hold the block id prefix to be used for azure
-   * storage blocks from azure-storage-java sdk version 4.2.0 onwards
+   * Keep alive time for the threadpool.
    */
-  private String blockIdPrefix = null;
+  private static final int THREADPOOL_KEEP_ALIVE = 30;
+  /**
+   * Azure Block Blob used for the stream.
+   */
+  private final CloudBlockBlobWrapper blob;
+
+  /**
+   * Azure Storage operation context.
+   */
+  private final OperationContext opContext;
+
+  /**
+   * Commands send from client calls to the background thread pool.
+   */
+  private abstract class UploadCommand {
+
+    // the blob offset for the command
+    private final long commandBlobOffset;
+
+    // command completion latch
+    private final CountDownLatch completed = new CountDownLatch(1);
+
+    UploadCommand(long offset) {
+      this.commandBlobOffset = offset;
+    }
+
+    long getCommandBlobOffset() {
+      return commandBlobOffset;
+    }
+
+    void await() throws InterruptedException {
+      completed.await();
+    }
+
+    void awaitAsDependent() throws InterruptedException {
+      await();
+    }
+
+    void setCompleted() {
+      completed.countDown();
+    }
 
-  private final Random sequenceGenerator = new Random();
+    void execute() throws InterruptedException, IOException {}
+
+    void dump() {}
+  }
+
+  /**
+   * The list of recent commands. Before block list is committed, all the block
+   * listed in the list must be uploaded. activeBlockCommands is used for
+   * enumerating the blocks and waiting on the latch until the block is
+   * uploaded.
+   */
+  private final ConcurrentLinkedQueue<UploadCommand> activeBlockCommands
+      = new ConcurrentLinkedQueue<>();
+
+  /**
+   * Variable to track if the stream has been closed.
+   */
+  private volatile boolean closed = false;
+
+  /**
+   *  First IOException encountered.
+   */
+  private final AtomicReference<IOException> firstError
+          = new AtomicReference<>();
+
+  /**
+   * Flag set when the first error has been thrown.
+   */
+  private boolean firstErrorThrown = false;
 
   /**
-   *  Time to wait to renew lease in milliseconds
+   * Semaphore for serializing block uploads with NativeAzureFileSystem.
+   *
+   * The semaphore starts with number of permits equal to the number of block
+   * upload threads. Each block upload thread needs one permit to start the
+   * upload. The put block list acquires all the permits before the block list
+   * is committed.
    */
-  private static final int LEASE_RENEWAL_PERIOD = 10000;
+  private final Semaphore uploadingSemaphore = new Semaphore(
+      MAX_NUMBER_THREADS_IN_THREAD_POOL,
+      true);
 
   /**
-   *  Number of times to retry for lease renewal
+   * Queue storing buffers with the size of the Azure block ready for
+   * reuse. The pool allows reusing the blocks instead of allocating new
+   * blocks. After the data is sent to the service, the buffer is returned
+   * back to the queue
    */
-  private static final int MAX_LEASE_RENEWAL_RETRY_COUNT = 3;
+  private final ElasticByteBufferPool poolReadyByteBuffers
+          = new ElasticByteBufferPool();
 
   /**
-   *  Time to wait before retrying to set the lease
+   * The blob's block list.
    */
-  private static final int LEASE_RENEWAL_RETRY_SLEEP_PERIOD = 500;
+  private final List<BlockEntry> blockEntries = new ArrayList<>(
+      DEFAULT_CAPACITY_BLOCK_ENTRIES);
+
+  private static final int DEFAULT_CAPACITY_BLOCK_ENTRIES = 1024;
 
   /**
-   *  Metadata key used on the blob to indicate append lease is active
+   * The uncommitted blob's block list.
    */
-  public static final String APPEND_LEASE = "append_lease";
+  private final ConcurrentLinkedDeque<BlockEntry> uncommittedBlockEntries
+      = new ConcurrentLinkedDeque<>();
+
+  /**
+   * Variable to hold the next block id to be used for azure storage blocks.
+   */
+  private static final int UNSET_BLOCKS_COUNT = -1;
+  private long nextBlockCount = UNSET_BLOCKS_COUNT;
 
   /**
-   * Timeout value for the append lease in millisecs. If the lease is not
-   * renewed within 30 seconds then another thread can acquire the append lease
-   * on the blob
+   * Variable to hold the block id prefix to be used for azure storage blocks.
    */
-  public static final int APPEND_LEASE_TIMEOUT = 30000;
+  private String blockIdPrefix = null;
 
   /**
-   *  Metdata key used on the blob to indicate last modified time of append lease
+   *  Maximum number of threads in block upload thread pool.
    */
-  public static final String APPEND_LEASE_LAST_MODIFIED = "append_lease_last_modified";
+  private static final int MAX_NUMBER_THREADS_IN_THREAD_POOL = 4;
 
   /**
    * Number of times block upload needs is retried.
@@ -145,17 +279,33 @@ public class BlockBlobAppendStream extends OutputStream {
   private static final int MAX_BLOCK_UPLOAD_RETRIES = 3;
 
   /**
-   * Wait time between block upload retries in millisecs.
+   * Wait time between block upload retries in milliseconds.
    */
   private static final int BLOCK_UPLOAD_RETRY_INTERVAL = 1000;
 
-  private static final Logger LOG = LoggerFactory.getLogger(BlockBlobAppendStream.class);
+  /**
+   * Logger.
+   */
+  private static final Logger LOG =
+          LoggerFactory.getLogger(BlockBlobAppendStream.class);
 
+  /**
+   * The absolute maximum of blocks for a blob. It includes committed and
+   * temporary blocks.
+   */
   private static final int MAX_BLOCK_COUNT = 100000;
 
+  /**
+   * The upload thread pool executor.
+   */
   private ThreadPoolExecutor ioThreadPool;
 
   /**
+   * Azure Storage access conditions for the blob.
+   */
+  private final AccessCondition accessCondition = new AccessCondition();
+
+  /**
    * Atomic integer to provide thread id for thread names for uploader threads.
    */
   private final AtomicInteger threadSequenceNumber;
@@ -163,106 +313,123 @@ public class BlockBlobAppendStream extends OutputStream {
   /**
    * Prefix to be used for thread names for uploader threads.
    */
-  private static final String THREAD_ID_PREFIX = "BlockBlobAppendStream";
-
-  private static final String UTC_STR = "UTC";
+  private static final String THREAD_ID_PREFIX = "append-blockblob";
 
+  /**
+   * BlockBlobAppendStream constructor.
+   *
+   * @param blob
+   *          Azure Block Blob
+   * @param aKey
+   *          blob's name
+   * @param bufferSize
+   *          the maximum size of a blob block.
+   * @param compactionEnabled
+   *          is the compaction process enabled for this blob
+   * @param opContext
+   *          Azure Store operation context for the blob
+   * @throws IOException
+   *           if an I/O error occurs. In particular, an IOException may be
+   *           thrown if the output stream cannot be used for append operations
+   */
   public BlockBlobAppendStream(final CloudBlockBlobWrapper blob,
-      final String aKey, final int bufferSize, final OperationContext opContext)
+                               final String aKey,
+                               final int bufferSize,
+                               final boolean compactionEnabled,
+                               final OperationContext opContext)
           throws IOException {
 
-    if (null == aKey || 0 == aKey.length()) {
-      throw new IllegalArgumentException(
-          "Illegal argument: The key string is null or empty");
-    }
-
-    if (0 >= bufferSize) {
-      throw new IllegalArgumentException(
-          "Illegal argument bufferSize cannot be zero or negative");
-    }
-
+    Preconditions.checkArgument(StringUtils.isNotEmpty(aKey));
+    Preconditions.checkArgument(bufferSize >= 0);
 
     this.blob = blob;
     this.opContext = opContext;
     this.key = aKey;
-    this.bufferSize = bufferSize;
+    this.maxBlockSize = new AtomicInteger(bufferSize);
     this.threadSequenceNumber = new AtomicInteger(0);
     this.blockIdPrefix = null;
-    setBlocksCountAndBlockIdPrefix();
+    this.compactionEnabled = compactionEnabled;
+    this.blobExist = true;
+    this.outBuffer = poolReadyByteBuffers.getBuffer(false, maxBlockSize.get());
 
-    this.outBuffer = new ByteArrayOutputStream(bufferSize);
-    this.uncommittedBlockEntries = new ArrayList<BlockEntry>();
-
-    // Acquire append lease on the blob.
     try {
-      //Set the append lease if the value of the append lease is false
-      if (!updateBlobAppendMetadata(true, false)) {
-        LOG.error("Unable to set Append Lease on the Blob : {} "
-            + "Possibly because another client already has a create or append stream open on the Blob", key);
-        throw new IOException("Unable to set Append lease on the Blob. "
-            + "Possibly because another client already had an append stream open on the Blob.");
-      }
-    } catch (StorageException ex) {
-      LOG.error("Encountered Storage exception while acquiring append "
-          + "lease on blob : {}. Storage Exception : {} ErrorCode : {}",
-          key, ex, ex.getErrorCode());
+      // download the block list
+      blockEntries.addAll(
+          blob.downloadBlockList(
+              BlockListingFilter.COMMITTED,
+              new BlobRequestOptions(),
+              opContext));
+
+      blobLength = blob.getProperties().getLength();
+
+      committedBlobLength.set(blobLength);
 
-      throw new IOException(ex);
+      // Acquiring lease on the blob.
+      lease = new SelfRenewingLease(blob, true);
+      accessCondition.setLeaseID(lease.getLeaseID());
+
+    } catch (StorageException ex) {
+      if (ex.getErrorCode().equals(StorageErrorCodeStrings.BLOB_NOT_FOUND)) {
+        blobExist = false;
+      }
+      else if (ex.getErrorCode().equals(
+              StorageErrorCodeStrings.LEASE_ALREADY_PRESENT)) {
+        throw new AzureException(
+                "Unable to set Append lease on the Blob: " + ex, ex);
+      }
+      else {
+        LOG.debug(
+            "Encountered storage exception."
+                + " StorageException : {} ErrorCode : {}",
+            ex,
+            ex.getErrorCode());
+        throw new AzureException(ex);
+      }
     }
 
-    leaseFreed = false;
+    setBlocksCountAndBlockIdPrefix(blockEntries);
+
+    this.ioThreadPool = new ThreadPoolExecutor(
+        MAX_NUMBER_THREADS_IN_THREAD_POOL,
+        MAX_NUMBER_THREADS_IN_THREAD_POOL,
+        THREADPOOL_KEEP_ALIVE,
+        TimeUnit.SECONDS,
+        new LinkedBlockingQueue<>(),
+        new UploaderThreadFactory());
   }
 
   /**
-   * Helper method that starts an Append Lease renewer thread and the
-   * thread pool.
+   * Set payload size of the stream.
+   * It is intended to be used for unit testing purposes only.
    */
-  public synchronized void initialize() {
-
-    if (initialized) {
-      return;
-    }
-    /*
-     * Start the thread for  Append lease renewer.
-     */
-    Thread appendLeaseRenewer = new Thread(new AppendRenewer());
-    appendLeaseRenewer.setDaemon(true);
-    appendLeaseRenewer.setName(String.format("%s-AppendLeaseRenewer", key));
-    appendLeaseRenewer.start();
-
-    /*
-     * Parameters to ThreadPoolExecutor:
-     * corePoolSize : the number of threads to keep in the pool, even if they are idle,
-     *                unless allowCoreThreadTimeOut is set
-     * maximumPoolSize : the maximum number of threads to allow in the pool
-     * keepAliveTime - when the number of threads is greater than the core,
-     *                 this is the maximum time that excess idle threads will
-     *                 wait for new tasks before terminating.
-     * unit - the time unit for the keepAliveTime argument
-     * workQueue - the queue to use for holding tasks before they are executed
-     *  This queue will hold only the Runnable tasks submitted by the execute method.
-     */
-    this.ioThreadPool = new ThreadPoolExecutor(4, 4, 2, TimeUnit.SECONDS,
-        new LinkedBlockingQueue<Runnable>(), new UploaderThreadFactory());
+  @VisibleForTesting
+  synchronized void setMaxBlockSize(int size) {
+    maxBlockSize.set(size);
 
-    initialized = true;
+    // it is for testing only so we can abandon the previously allocated
+    // payload
+    this.outBuffer = ByteBuffer.allocate(maxBlockSize.get());
   }
 
   /**
-   * Get the blob name.
-   *
-   * @return String Blob name.
+   * Set compaction parameters.
+   * It is intended to be used for unit testing purposes only.
    */
-  public String getKey() {
-    return key;
+  @VisibleForTesting
+  void setCompactionBlockCount(int activationCount) {
+    activateCompactionBlockCount = activationCount;
   }
 
   /**
-   * Get the backing blob.
-   * @return buffer size of the stream.
+   * Get the list of block entries. It is used for testing purposes only.
+   * @return List of block entries.
    */
-  public int getBufferSize() {
-    return bufferSize;
+  @VisibleForTesting
+  List<BlockEntry> getBlockList() throws StorageException, IOException {
+    return blob.downloadBlockList(
+        BlockListingFilter.COMMITTED,
+        new BlobRequestOptions(),
+        opContext);
   }
 
   /**
@@ -283,21 +450,6 @@ public class BlockBlobAppendStream extends OutputStream {
   }
 
   /**
-   * Writes b.length bytes from the specified byte array to this output stream.
-   *
-   * @param data
-   *          the byte array to write.
-   *
-   * @throws IOException
-   *           if an I/O error occurs. In particular, an IOException may be
-   *           thrown if the output stream has been closed.
-   */
-  @Override
-  public void write(final byte[] data) throws IOException {
-    write(data, 0, data.length);
-  }
-
-  /**
    * Writes length bytes from the specified byte array starting at offset to
    * this output stream.
    *
@@ -312,529 +464,678 @@ public class BlockBlobAppendStream extends OutputStream {
    *           thrown if the output stream has been closed.
    */
   @Override
-  public void write(final byte[] data, final int offset, final int length)
+  public synchronized void write(final byte[] data, int offset, int length)
       throws IOException {
+    Preconditions.checkArgument(data != null, "null data");
 
     if (offset < 0 || length < 0 || length > data.length - offset) {
-      throw new IndexOutOfBoundsException("write API in append stream called with invalid arguments");
-    }
-
-    writeInternal(data, offset, length);
-  }
-
-  @Override
-  public synchronized void close() throws IOException {
-
-    if (!initialized) {
-      throw new IOException("Trying to close an uninitialized Append stream");
+      throw new IndexOutOfBoundsException();
     }
 
     if (closed) {
-      return;
+      throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
     }
 
-    if (leaseFreed) {
-      throw new IOException(String.format("Attempting to close an append stream on blob : %s "
-          + " that does not have lease on the Blob. Failing close", key));
-    }
+    while (outBuffer.remaining() < length) {
+
+      int remaining = outBuffer.remaining();
+      outBuffer.put(data, offset, remaining);
+
+      // upload payload to azure storage
+      addBlockUploadCommand();
 
-    if (outBuffer.size() > 0) {
-      uploadBlockToStorage(outBuffer.toByteArray());
+      offset += remaining;
+      length -= remaining;
     }
 
-    ioThreadPool.shutdown();
+    outBuffer.put(data, offset, length);
+  }
 
-    try {
-      if (!ioThreadPool.awaitTermination(10, TimeUnit.MINUTES)) {
-        LOG.error("Time out occurred while waiting for IO request to finish in append"
-            + " for blob : {}", key);
-        NativeAzureFileSystemHelper.logAllLiveStackTraces();
-        throw new IOException("Timed out waiting for IO requests to finish");
-      }
-    } catch(InterruptedException intrEx) {
 
-      // Restore the interrupted status
-      Thread.currentThread().interrupt();
-      LOG.error("Upload block operation in append interrupted for blob {}. Failing close", key);
-      throw new IOException("Append Commit interrupted.");
-    }
+  /**
+   * Flushes this output stream and forces any buffered output bytes to be
+   * written out. If any data remains in the payload it is committed to the
+   * service. Data is queued for writing and forced out to the service
+   * before the call returns.
+   */
+  @Override
+  public void flush() throws IOException {
 
-    // Calling commit after all blocks are succesfully uploaded.
-    if (lastError == null) {
-      commitAppendBlocks();
+    if (closed) {
+      // calling close() after the stream is closed starts with call to flush()
+      return;
     }
 
-    // Perform cleanup.
-    cleanup();
+    addBlockUploadCommand();
 
-    if (lastError != null) {
-      throw lastError;
+    if (committedBlobLength.get() < blobLength) {
+      try {
+        // wait until the block list is committed
+        addFlushCommand().await();
+      } catch (InterruptedException ie) {
+        Thread.currentThread().interrupt();
+      }
     }
   }
 
   /**
-   * Helper method that cleans up the append stream.
+   * Force all data in the output stream to be written to Azure storage.
+   * Wait to return until this is complete.
    */
-  private synchronized void cleanup() {
-
-    closed = true;
-
-    try {
-      // Set the value of append lease to false if the value is set to true.
-        updateBlobAppendMetadata(false, true);
-    } catch(StorageException ex) {
-      LOG.debug("Append metadata update on the Blob : {} encountered Storage Exception : {} "
-          + "Error Code : {}",
-          key, ex, ex.getErrorCode());
-      lastError = new IOException(ex);
+  @Override
+  public void hsync() throws IOException {
+    // when block compaction is disabled, hsync is empty function
+    if (compactionEnabled) {
+      flush();
     }
+  }
 
-    leaseFreed = true;
+  /**
+   * Force all data in the output stream to be written to Azure storage.
+   * Wait to return until this is complete.
+   */
+  @Override
+  public void hflush() throws IOException {
+    // when block compaction is disabled, hflush is empty function
+    if (compactionEnabled) {
+      flush();
+    }
   }
 
   /**
-   * Method to commit all the uncommited blocks to azure storage.
-   * If the commit fails then blocks are automatically cleaned up
-   * by Azure storage.
-   * @throws IOException
+   * The Synchronization capabilities of this stream depend upon the compaction
+   * policy.
+   * @param capability string to query the stream support for.
+   * @return true for hsync and hflush when compaction is enabled.
    */
-  private synchronized void commitAppendBlocks() throws IOException {
+  @Override
+  public boolean hasCapability(String capability) {
+    return compactionEnabled
+        && (capability.equalsIgnoreCase(HSYNC.getValue())
+        || capability.equalsIgnoreCase((HFLUSH.getValue())));
+  }
 
-    SelfRenewingLease lease = null;
+  /**
+   * Force all data in the output stream to be written to Azure storage.
+   * Wait to return until this is complete. Close the access to the stream and
+   * shutdown the upload thread pool.
+   * If the blob was created, its lease will be released.
+   * Any error encountered caught in threads and stored will be rethrown here
+   * after cleanup.
+   */
+  @Override
+  public synchronized void close() throws IOException {
 
-    try {
-      if (uncommittedBlockEntries.size() > 0) {
+    LOG.debug("close {} ", key);
 
-        //Acquiring lease on the blob.
-        lease = new SelfRenewingLease(blob);
+    if (closed) {
+      return;
+    }
 
-        // Downloading existing blocks
-        List<BlockEntry> blockEntries =  blob.downloadBlockList(BlockListingFilter.COMMITTED,
-            new BlobRequestOptions(), opContext);
+    // Upload the last block regardless of compactionEnabled flag
+    flush();
 
-        // Adding uncommitted blocks.
-        blockEntries.addAll(uncommittedBlockEntries);
+    // Initiates an orderly shutdown in which previously submitted tasks are
+    // executed.
+    ioThreadPool.shutdown();
 
-        AccessCondition accessCondition = new AccessCondition();
-        accessCondition.setLeaseID(lease.getLeaseID());
-        blob.commitBlockList(blockEntries, accessCondition, new BlobRequestOptions(), opContext);
-        uncommittedBlockEntries.clear();
+    try {
+      // wait up to CLOSE_UPLOAD_DELAY minutes to upload all the blocks
+      if (!ioThreadPool.awaitTermination(CLOSE_UPLOAD_DELAY, TimeUnit.MINUTES)) {
+        LOG.error("Time out occurred while close() is waiting for IO request to"
+            + " finish in append"
+            + " for blob : {}",
+            key);
+        NativeAzureFileSystemHelper.logAllLiveStackTraces();
+        throw new AzureException("Timed out waiting for IO requests to finish");
       }
-    } catch(StorageException ex) {
-      LOG.error("Storage exception encountered during block commit phase of append for blob"
-          + " : {} Storage Exception : {} Error Code: {}", key, ex, ex.getErrorCode());
-      throw new IOException("Encountered Exception while committing append blocks", ex);
-    } finally {
-      if (lease != null) {
+    } catch(InterruptedException ex) {
+      Thread.currentThread().interrupt();
+    }
+
+    // release the lease
+    if (firstError.get() == null && blobExist) {
         try {
           lease.free();
-        } catch(StorageException ex) {
-          LOG.debug("Exception encountered while releasing lease for "
-              + "blob : {} StorageException : {} ErrorCode : {}", key, ex, ex.getErrorCode());
-          // Swallowing exception here as the lease is cleaned up by the SelfRenewingLease object.
+        } catch (StorageException ex) {
+          LOG.debug("Lease free update blob {} encountered Storage Exception:"
+              + " {} Error Code : {}",
+              key,
+              ex,
+              ex.getErrorCode());
+          maybeSetFirstError(new AzureException(ex));
         }
-      }
+    }
+
+    closed = true;
+
+    // finally, throw the first exception raised if it has not
+    // been thrown elsewhere.
+    if (firstError.get() != null && !firstErrorThrown) {
+      throw firstError.get();
     }
   }
 
   /**
-   * Helper method used to generate the blockIDs. The algorithm used is similar to the Azure
-   * storage SDK.
+   * Helper method used to generate the blockIDs. The algorithm used is similar
+   * to the Azure storage SDK.
    */
-  private void setBlocksCountAndBlockIdPrefix() throws IOException {
+  private void setBlocksCountAndBlockIdPrefix(List<BlockEntry> blockEntries) {
 
-    try {
+    if (nextBlockCount == UNSET_BLOCKS_COUNT && blockIdPrefix == null) {
 
-      if (nextBlockCount == UNSET_BLOCKS_COUNT && blockIdPrefix==null) {
+      Random sequenceGenerator = new Random();
 
-        List<BlockEntry> blockEntries =
-            blob.downloadBlockList(BlockListingFilter.COMMITTED, new BlobRequestOptions(), opContext);
+      String blockZeroBlockId = (!blockEntries.isEmpty())
+          ? blockEntries.get(0).getId()
+          : "";
+      String prefix = UUID.randomUUID().toString() + "-";
+      String sampleNewerVersionBlockId = generateNewerVersionBlockId(prefix,
+          0);
 
-        String blockZeroBlockId = (blockEntries.size() > 0) ? blockEntries.get(0).getId() : "";
-        String prefix = UUID.randomUUID().toString() + "-";
-        String sampleNewerVersionBlockId = generateNewerVersionBlockId(prefix, 0);
+      if (!blockEntries.isEmpty()
+          && blockZeroBlockId.length() < sampleNewerVersionBlockId.length()) {
 
-        if (blockEntries.size() > 0 && blockZeroBlockId.length() < sampleNewerVersionBlockId.length()) {
+        // If blob has already been created with 2.2.0, append subsequent blocks
+        // with older version (2.2.0) blockId compute nextBlockCount, the way it
+        // was done before; and don't use blockIdPrefix
+        this.blockIdPrefix = "";
+        nextBlockCount = (long) (sequenceGenerator.nextInt(Integer.MAX_VALUE))
+            + sequenceGenerator.nextInt(
+                Integer.MAX_VALUE - MAX_BLOCK_COUNT);
+        nextBlockCount += blockEntries.size();
 
-          // If blob has already been created with 2.2.0, append subsequent blocks with older version (2.2.0) blockId
-          // compute nextBlockCount, the way it was done before; and don't use blockIdPrefix
-          this.blockIdPrefix = "";
-          nextBlockCount = (long) (sequenceGenerator.nextInt(Integer.MAX_VALUE))
-              + sequenceGenerator.nextInt(Integer.MAX_VALUE - MAX_BLOCK_COUNT);
-          nextBlockCount += blockEntries.size();
-
-        } else {
-
-          // If there are no existing blocks, create the first block with newer version (4.2.0) blockId
-          // If blob has already been created with 4.2.0, append subsequent blocks with newer version (4.2.0) blockId
-          this.blockIdPrefix = prefix;
-          nextBlockCount = blockEntries.size();
-
-        }
+      } else {
 
+        // If there are no existing blocks, create the first block with newer
+        // version (4.2.0) blockId. If blob has already been created with 4.2.0,
+        // append subsequent blocks with newer version (4.2.0) blockId
+        this.blockIdPrefix = prefix;
+        nextBlockCount = blockEntries.size();
       }
-
-    } catch (StorageException ex) {
-      LOG.debug("Encountered storage exception during setting next Block Count and BlockId prefix."
-          + " StorageException : {} ErrorCode : {}", ex, ex.getErrorCode());
-      throw new IOException(ex);
     }
   }
 
   /**
-   * Helper method that generates the next block id for uploading a block to azure storage.
+   * Helper method that generates the next block id for uploading a block to
+   * azure storage.
    * @return String representing the block ID generated.
-   * @throws IOException
+   * @throws IOException if the stream is in invalid state
    */
   private String generateBlockId() throws IOException {
 
-    if (nextBlockCount == UNSET_BLOCKS_COUNT) {
-      throw new IOException("Append Stream in invalid state. nextBlockCount not set correctly");
-    }
-
-    if (this.blockIdPrefix == null) {
-      throw new IOException("Append Stream in invalid state. blockIdPrefix not set correctly");
-    }
-
-    if (!this.blockIdPrefix.equals("")) {
-
-      return generateNewerVersionBlockId(this.blockIdPrefix, nextBlockCount++);
-
-    } else {
-
-      return generateOlderVersionBlockId(nextBlockCount++);
-
+    if (nextBlockCount == UNSET_BLOCKS_COUNT || blockIdPrefix == null) {
+      throw new AzureException(
+            "Append Stream in invalid state. nextBlockCount not set correctly");
     }
 
+    return (!blockIdPrefix.isEmpty())
+        ? generateNewerVersionBlockId(blockIdPrefix, nextBlockCount++)
+        : generateOlderVersionBlockId(nextBlockCount++);
   }
 
   /**
-   * Helper method that generates an older (2.2.0) version blockId
+   * Helper method that generates an older (2.2.0) version blockId.
    * @return String representing the block ID generated.
    */
   private String generateOlderVersionBlockId(long id) {
 
-    byte[] blockIdInBytes = getBytesFromLong(id);
-    return new String(Base64.encodeBase64(blockIdInBytes), StandardCharsets.UTF_8);
+    byte[] blockIdInBytes = new byte[8];
+    for (int m = 0; m < 8; m++) {
+      blockIdInBytes[7 - m] = (byte) ((id >> (8 * m)) & 0xFF);
+    }
+
+    return new String(
+            Base64.encodeBase64(blockIdInBytes),
+            StandardCharsets.UTF_8);
   }
 
   /**
-   * Helper method that generates an newer (4.2.0) version blockId
+   * Helper method that generates an newer (4.2.0) version blockId.
    * @return String representing the block ID generated.
    */
   private String generateNewerVersionBlockId(String prefix, long id) {
 
     String blockIdSuffix  = String.format("%06d", id);
-    byte[] blockIdInBytes = (prefix + blockIdSuffix).getBytes(StandardCharsets.UTF_8);
+    byte[] blockIdInBytes =
+            (prefix + blockIdSuffix).getBytes(StandardCharsets.UTF_8);
     return new String(Base64.encodeBase64(blockIdInBytes), StandardCharsets.UTF_8);
   }
 
   /**
-   * Returns a byte array that represents the data of a <code>long</code> value. This
-   * utility method is copied from com.microsoft.azure.storage.core.Utility class.
-   * This class is marked as internal, hence we clone the method here and not express
-   * dependency on the Utility Class
-   *
-   * @param value
-   *            The value from which the byte array will be returned.
-   *
-   * @return A byte array that represents the data of the specified <code>long</code> value.
+   * This is shared between upload block Runnable and CommitBlockList. The
+   * method captures retry logic
+   * @param blockId block name
+   * @param dataPayload block content
    */
-  private static byte[] getBytesFromLong(final long value) {
+  private void writeBlockRequestInternal(String blockId,
+                                         ByteBuffer dataPayload,
+                                         boolean bufferPoolBuffer) {
+    IOException lastLocalException = null;
+
+    int uploadRetryAttempts = 0;
+    while (uploadRetryAttempts < MAX_BLOCK_UPLOAD_RETRIES) {
+      try {
+        long startTime = System.nanoTime();
+
+        blob.uploadBlock(blockId, accessCondition, new ByteArrayInputStream(
+            dataPayload.array()), dataPayload.position(),
+            new BlobRequestOptions(), opContext);
 
-    final byte[] tempArray = new byte[8];
+        LOG.debug("upload block finished for {} ms. block {} ",
+            TimeUnit.NANOSECONDS.toMillis(
+                System.nanoTime() - startTime), blockId);
+        break;
+
+      } catch(Exception ioe) {
+        LOG.debug("Encountered exception during uploading block for Blob {}"
+            + " Exception : {}", key, ioe);
+        uploadRetryAttempts++;
+        lastLocalException = new AzureException(
+            "Encountered Exception while uploading block: " + ioe, ioe);
+        try {
+          Thread.sleep(
+              BLOCK_UPLOAD_RETRY_INTERVAL * (uploadRetryAttempts + 1));
+        } catch(InterruptedException ie) {
+          Thread.currentThread().interrupt();
+          break;
+        }
+      }
+    }
 
-    for (int m = 0; m < 8; m++) {
-      tempArray[7 - m] = (byte) ((value >> (8 * m)) & 0xFF);
+    if (bufferPoolBuffer) {
+      poolReadyByteBuffers.putBuffer(dataPayload);
     }
 
-    return tempArray;
+    if (uploadRetryAttempts == MAX_BLOCK_UPLOAD_RETRIES) {
+      maybeSetFirstError(lastLocalException);
+    }
   }
 
   /**
-   * Helper method that creates a thread to upload a block to azure storage.
-   * @param payload
-   * @throws IOException
+   * Set {@link #firstError} to the exception if it is not already set.
+   * @param exception exception to save
    */
-  private synchronized void uploadBlockToStorage(byte[] payload)
-      throws IOException {
-
-    // upload payload to azure storage
-    String blockId = generateBlockId();
-
-    // Since uploads of the Azure storage are done in parallel threads, we go ahead
-    // add the blockId in the uncommitted list. If the upload of the block fails
-    // we don't commit the blockIds.
-    BlockEntry blockEntry = new BlockEntry(blockId);
-    blockEntry.setSize(payload.length);
-    uncommittedBlockEntries.add(blockEntry);
-    ioThreadPool.execute(new WriteRequest(payload, blockId));
+  private void maybeSetFirstError(IOException exception) {
+    firstError.compareAndSet(null, exception);
   }
 
 
   /**
-   * Helper method to updated the Blob metadata during Append lease operations.
-   * Blob metadata is updated to holdLease value only if the current lease
-   * status is equal to testCondition and the last update on the blob metadata
-   * is less that 30 secs old.
-   * @param holdLease
-   * @param testCondition
-   * @return true if the updated lease operation was successful or false otherwise
-   * @throws StorageException
+   * Throw the first error caught if it has not been raised already
+   * @throws IOException if one is caught and needs to be thrown.
    */
-  private boolean updateBlobAppendMetadata(boolean holdLease, boolean testCondition)
-      throws StorageException {
-
-    SelfRenewingLease lease = null;
-    StorageException lastStorageException = null;
-    int leaseRenewalRetryCount = 0;
-
-    /*
-     * Updating the Blob metadata honours following algorithm based on
-     *  1) If the append lease metadata is present
-     *  2) Last updated time of the append lease
-     *  3) Previous value of the Append lease metadata.
-     *
-     * The algorithm:
-     *  1) If append lease metadata is not part of the Blob. In this case
-     *     this is the first client to Append so we update the metadata.
-     *  2) If append lease metadata is present and timeout has occurred.
-     *     In this case irrespective of what the value of the append lease is we update the metadata.
-     *  3) If append lease metadata is present and is equal to testCondition value (passed as parameter)
-     *     and timeout has not occurred, we update the metadata.
-     *  4) If append lease metadata is present and is not equal to testCondition value (passed as parameter)
-     *     and timeout has not occurred, we do not update metadata and return false.
-     *
-     */
-    while (leaseRenewalRetryCount < MAX_LEASE_RENEWAL_RETRY_COUNT) {
-
-      lastStorageException = null;
-
-      synchronized(this) {
-        try {
-
-          final Calendar currentCalendar = Calendar
-              .getInstance(Locale.US);
-          currentCalendar.setTimeZone(TimeZone.getTimeZone(UTC_STR));
-          long currentTime = currentCalendar.getTime().getTime();
+  private void maybeThrowFirstError() throws IOException {
+    if (firstError.get() != null) {
+      firstErrorThrown = true;
+      throw firstError.get();
+    }
+  }
 
-          // Acquire lease on the blob.
-          lease = new SelfRenewingLease(blob);
+  /**
+   * Write block list. The method captures retry logic
+   */
+  private void writeBlockListRequestInternal() {
 
-          blob.downloadAttributes(opContext);
-          HashMap<String, String> metadata = blob.getMetadata();
+    IOException lastLocalException = null;
 
-          if (metadata.containsKey(APPEND_LEASE)
-              && currentTime - Long.parseLong(
-                  metadata.get(APPEND_LEASE_LAST_MODIFIED)) <= BlockBlobAppendStream.APPEND_LEASE_TIMEOUT
-              && !metadata.get(APPEND_LEASE).equals(Boolean.toString(testCondition))) {
-            return false;
-          }
+    int uploadRetryAttempts = 0;
+    while (uploadRetryAttempts < MAX_BLOCK_UPLOAD_RETRIES) {
+      try {
 
-          metadata.put(APPEND_LEASE, Boolean.toString(holdLease));
-          metadata.put(APPEND_LEASE_LAST_MODIFIED, Long.toString(currentTime));
-          blob.setMetadata(metadata);
-          AccessCondition accessCondition = new AccessCondition();
-          accessCondition.setLeaseID(lease.getLeaseID());
-          blob.uploadMetadata(accessCondition, null, opContext);
-          return true;
+        long startTime = System.nanoTime();
 
-        } catch (StorageException ex) {
-
-          lastStorageException = ex;
-          LOG.debug("Lease renewal for Blob : {} encountered Storage Exception : {} "
-              + "Error Code : {}",
-              key, ex, ex.getErrorCode());
-          leaseRenewalRetryCount++;
-
-        } finally {
-
-          if (lease != null) {
-            try {
-              lease.free();
-            } catch(StorageException ex) {
-              LOG.debug("Encountered Storage exception while releasing lease for Blob {} "
-                  + "during Append  metadata operation. Storage Exception {} "
-                  + "Error Code : {} ", key, ex, ex.getErrorCode());
-            } finally {
-              lease = null;
-            }
-          }
-        }
-      }
+        blob.commitBlockList(blockEntries, accessCondition,
+            new BlobRequestOptions(), opContext);
 
-      if (leaseRenewalRetryCount == MAX_LEASE_RENEWAL_RETRY_COUNT) {
-        throw lastStorageException;
-      } else {
+        LOG.debug("Upload block list took {} ms for blob {} ",
+                TimeUnit.NANOSECONDS.toMillis(
+                    System.nanoTime() - startTime), key);
+        break;
+
+      } catch(Exception ioe) {
+        LOG.debug("Encountered exception during uploading block for Blob {}"
+            + " Exception : {}", key, ioe);
+        uploadRetryAttempts++;
+        lastLocalException = new AzureException(
+            "Encountered Exception while uploading block: " + ioe, ioe);
         try {
-          Thread.sleep(LEASE_RENEWAL_RETRY_SLEEP_PERIOD);
-        } catch(InterruptedException ex) {
-          LOG.debug("Blob append metadata updated method interrupted");
+          Thread.sleep(
+              BLOCK_UPLOAD_RETRY_INTERVAL * (uploadRetryAttempts + 1));
+        } catch(InterruptedException ie) {
           Thread.currentThread().interrupt();
+          break;
         }
       }
     }
 
-    // The code should not enter here because the while loop will
-    // always be executed and if the while loop is executed we
-    // would returning from the while loop.
-    return false;
+    if (uploadRetryAttempts == MAX_BLOCK_UPLOAD_RETRIES) {
+      maybeSetFirstError(lastLocalException);
+    }
   }
 
   /**
-   * This is the only method that should be writing to outBuffer to maintain consistency of the outBuffer.
-   * @param data
-   * @param offset
-   * @param length
-   * @throws IOException
+   * A ThreadFactory that creates uploader thread with
+   * meaningful names helpful for debugging purposes.
    */
-  private synchronized void writeInternal(final byte[] data, final int offset, final int length)
-      throws IOException {
+  class UploaderThreadFactory implements ThreadFactory {
 
-    if (!initialized) {
-      throw new IOException("Trying to write to an un-initialized Append stream");
+    @Override
+    public Thread newThread(Runnable r) {
+      Thread t = new Thread(r);
+      t.setName(String.format("%s-%d", THREAD_ID_PREFIX,
+          threadSequenceNumber.getAndIncrement()));
+      return t;
     }
+  }
 
-    if (closed) {
-      throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
-    }
+  /**
+   * Upload block commands.
+   */
+  private class UploadBlockCommand extends UploadCommand {
 
-    if (leaseFreed) {
-      throw new IOException(String.format("Write called on a append stream not holding lease. Failing Write"));
-    }
+    // the block content for upload
+    private final ByteBuffer payload;
 
-    byte[] currentData = new byte[length];
-    System.arraycopy(data, offset, currentData, 0, length);
+    // description of the block
+    private final BlockEntry entry;
 
-    // check to see if the data to be appended exceeds the
-    // buffer size. If so we upload a block to azure storage.
-    while ((outBuffer.size() + currentData.length) > bufferSize) {
+    UploadBlockCommand(String blockId, ByteBuffer payload) {
 
-      byte[] payload = new byte[bufferSize];
+      super(blobLength);
 
-      // Add data from the existing buffer
-      System.arraycopy(outBuffer.toByteArray(), 0, payload, 0, outBuffer.size());
+      BlockEntry blockEntry = new BlockEntry(blockId);
+      blockEntry.setSize(payload.position());
+      blockEntry.setSearchMode(BlockSearchMode.LATEST);
 
-      // Updating the available size in the payload
-      int availableSpaceInPayload = bufferSize - outBuffer.size();
+      this.payload = payload;
+      this.entry = blockEntry;
 
-      // Adding data from the current call
-      System.arraycopy(currentData, 0, payload, outBuffer.size(), availableSpaceInPayload);
+      uncommittedBlockEntries.add(blockEntry);
+    }
 
-      uploadBlockToStorage(payload);
+    /**
+     * Execute command.
+     */
+    void execute() throws InterruptedException {
+
+      uploadingSemaphore.acquire(1);
+      writeBlockRequestInternal(entry.getId(), payload, true);
+      uploadingSemaphore.release(1);
 
-      // updating the currentData buffer
-      byte[] tempBuffer = new byte[currentData.length - availableSpaceInPayload];
-      System.arraycopy(currentData, availableSpaceInPayload,
-          tempBuffer, 0, currentData.length - availableSpaceInPayload);
-      currentData = tempBuffer;
-      outBuffer = new ByteArrayOutputStream(bufferSize);
     }
 
-    outBuffer.write(currentData);
+    void dump() {
+      LOG.debug("upload block {} size: {} for blob {}",
+          entry.getId(),
+          entry.getSize(),
+          key);
+    }
   }
 
   /**
-   * Runnable instance that uploads the block of data to azure storage.
-   *
-   *
+   * Upload blob block list commands.
    */
-  private class WriteRequest implements Runnable {
-    private final byte[] dataPayload;
-    private final String blockId;
+  private class UploadBlockListCommand extends UploadCommand {
+
+    private BlockEntry lastBlock = null;
+
+    UploadBlockListCommand() {
+      super(blobLength);
 
-    public WriteRequest(byte[] dataPayload, String blockId) {
-      this.dataPayload = dataPayload;
-      this.blockId = blockId;
+      if (!uncommittedBlockEntries.isEmpty()) {
+        lastBlock = uncommittedBlockEntries.getLast();
+      }
     }
 
-    @Override
-    public void run() {
+    void awaitAsDependent() throws InterruptedException {
+      // empty. later commit block does not need to wait previous commit block
+      // lists.
+    }
 
-      int uploadRetryAttempts = 0;
-      IOException lastLocalException = null;
-      while (uploadRetryAttempts < MAX_BLOCK_UPLOAD_RETRIES) {
-        try {
+    void dump() {
+      LOG.debug("commit block list with {} blocks for blob {}",
+          uncommittedBlockEntries.size(), key);
+    }
+
+    /**
+     * Execute command.
+     */
+    public void execute() throws InterruptedException, IOException {
+
+      if (committedBlobLength.get() >= getCommandBlobOffset()) {
+        LOG.debug("commit already applied for {}", key);
+        return;
+      }
+
+      if (lastBlock == null) {
+        LOG.debug("nothing to commit for {}", key);
+        return;
+      }
+
+      LOG.debug("active commands: {} for {}", activeBlockCommands.size(), key);
+
+      for (UploadCommand activeCommand : activeBlockCommands) {
+        if (activeCommand.getCommandBlobOffset() < getCommandBlobOffset()) {
+          activeCommand.dump();
+          activeCommand.awaitAsDependent();
+        } else {
+          break;
+        }
+      }
+
+      // stop all uploads until the block list is committed
+      uploadingSemaphore.acquire(MAX_NUMBER_THREADS_IN_THREAD_POOL);
+
+      BlockEntry uncommittedBlock;
+      do  {
+        uncommittedBlock = uncommittedBlockEntries.poll();
+        blockEntries.add(uncommittedBlock);
+      } while (uncommittedBlock != lastBlock);
+
+      if (blockEntries.size() > activateCompactionBlockCount) {
+        LOG.debug("Block compaction: activated with {} blocks for {}",
+            blockEntries.size(), key);
+
+        // Block compaction
+        long startCompaction = System.nanoTime();
+        blockCompaction();
+        LOG.debug("Block compaction finished for {} ms with {} blocks for {}",
+                TimeUnit.NANOSECONDS.toMillis(
+                    System.nanoTime() - startCompaction),
+                blockEntries.size(), key);
+      }
+
+      writeBlockListRequestInternal();
+
+      uploadingSemaphore.release(MAX_NUMBER_THREADS_IN_THREAD_POOL);
 
-          blob.uploadBlock(blockId, new ByteArrayInputStream(dataPayload),
-              dataPayload.length, new BlobRequestOptions(), opContext);
+      // remove blocks previous commands
+      for (Iterator<UploadCommand> it = activeBlockCommands.iterator();
+           it.hasNext();) {
+        UploadCommand activeCommand = it.next();
+        if (activeCommand.getCommandBlobOffset() <= getCommandBlobOffset()) {
+          it.remove();
+        } else {
           break;
-        } catch(Exception ioe) {
-          Log.getLog().debug("Encountered exception during uploading block for Blob : {} Exception : {}", key, ioe);
-          uploadRetryAttempts++;
-          lastLocalException = new IOException("Encountered Exception while uploading block", ioe);
-          try {
-            Thread.sleep(BLOCK_UPLOAD_RETRY_INTERVAL);
-          } catch(InterruptedException ie) {
-            Thread.currentThread().interrupt();
-            break;
+        }
+      }
+
+      committedBlobLength.set(getCommandBlobOffset());
+    }
+
+    /**
+     * Internal output stream with read access to the internal buffer.
+     */
+    private class ByteArrayOutputStreamInternal extends ByteArrayOutputStream {
+
+      ByteArrayOutputStreamInternal(int size) {
+        super(size);
+      }
+
+      byte[] getByteArray() {
+        return buf;
+      }
+    }
+
+    /**
+     * Block compaction process.
+     *
+     * Block compaction is only enabled when the number of blocks exceeds
+     * activateCompactionBlockCount. The algorithm searches for the longest
+     * segment [b..e) where (e-b) > 2 && |b| + |b+1| ... |e-1| < maxBlockSize
+     * such that size(b1) + size(b2) + ... + size(bn) < maximum-block-size.
+     * It then downloads the blocks in the sequence, concatenates the data to
+     * form a single block, uploads this new block, and updates the block
+     * list to replace the sequence of blocks with the new block.
+     */
+    private void blockCompaction() throws IOException {
+      //current segment [segmentBegin, segmentEnd) and file offset/size of the
+      // current segment
+      int segmentBegin = 0, segmentEnd = 0;
+      long segmentOffsetBegin = 0, segmentOffsetEnd = 0;
+
+      //longest segment [maxSegmentBegin, maxSegmentEnd) and file offset/size of
+      // the longest segment
+      int maxSegmentBegin = 0, maxSegmentEnd = 0;
+      long maxSegmentOffsetBegin = 0, maxSegmentOffsetEnd = 0;
+
+      for (BlockEntry block : blockEntries) {
+        segmentEnd++;
+        segmentOffsetEnd += block.getSize();
+        if (segmentOffsetEnd - segmentOffsetBegin > maxBlockSize.get()) {
+          if (segmentEnd - segmentBegin > 2) {
+            if (maxSegmentEnd - maxSegmentBegin < segmentEnd - segmentBegin) {
+              maxSegmentBegin = segmentBegin;
+              maxSegmentEnd = segmentEnd;
+              maxSegmentOffsetBegin = segmentOffsetBegin;
+              maxSegmentOffsetEnd = segmentOffsetEnd - block.getSize();
+            }
           }
+          segmentBegin = segmentEnd - 1;
+          segmentOffsetBegin = segmentOffsetEnd - block.getSize();
         }
       }
 
-      if (uploadRetryAttempts == MAX_BLOCK_UPLOAD_RETRIES) {
-        lastError = lastLocalException;
+      if (maxSegmentEnd - maxSegmentBegin > 1) {
+
+        LOG.debug("Block compaction: {} blocks for {}",
+            maxSegmentEnd - maxSegmentBegin, key);
+
+        // download synchronously all the blocks from the azure storage
+        ByteArrayOutputStreamInternal blockOutputStream
+            = new ByteArrayOutputStreamInternal(maxBlockSize.get());
+
+        try {
+          long length = maxSegmentOffsetEnd - maxSegmentOffsetBegin;
+          blob.downloadRange(maxSegmentOffsetBegin, length, blockOutputStream,
+              new BlobRequestOptions(), opContext);
+        } catch(StorageException ex) {
+          LOG.error(
+              "Storage exception encountered during block compaction phase"
+                  + " : {} Storage Exception : {} Error Code: {}",
+              key, ex, ex.getErrorCode());
+          throw new AzureException(
+              "Encountered Exception while committing append blocks " + ex, ex);
+        }
+
+        // upload synchronously new block to the azure storage
+        String blockId = generateBlockId();
+
+        ByteBuffer byteBuffer = ByteBuffer.wrap(
+            blockOutputStream.getByteArray());
+        byteBuffer.position(blockOutputStream.size());
+
+        writeBlockRequestInternal(blockId, byteBuffer, false);
+
+        // replace blocks from the longest segment with new block id
+        blockEntries.subList(maxSegmentBegin + 1, maxSegmentEnd - 1).clear();
+        BlockEntry newBlock = blockEntries.get(maxSegmentBegin);
+        newBlock.setId(blockId);
+        newBlock.setSearchMode(BlockSearchMode.LATEST);
+        newBlock.setSize(maxSegmentOffsetEnd - maxSegmentOffsetBegin);
       }
     }
   }
 
   /**
-   * A ThreadFactory that creates uploader thread with
-   * meaningful names helpful for debugging purposes.
+   * Prepare block upload command and queue the command in thread pool executor.
    */
-  class UploaderThreadFactory implements ThreadFactory {
+  private synchronized void addBlockUploadCommand() throws IOException {
+
+    maybeThrowFirstError();
+
+    if (blobExist && lease.isFreed()) {
+      throw new AzureException(String.format(
+          "Attempting to upload a block on blob : %s "
+              + " that does not have lease on the Blob. Failing upload", key));
+    }
+
+    int blockSize = outBuffer.position();
+    if (blockSize > 0) {
+      UploadCommand command = new UploadBlockCommand(generateBlockId(),
+          outBuffer);
+      activeBlockCommands.add(command);
+
+      blobLength += blockSize;
+      outBuffer = poolReadyByteBuffers.getBuffer(false, maxBlockSize.get());
+
+      ioThreadPool.execute(new WriteRequest(command));
 
-    @Override
-    public Thread newThread(Runnable r) {
-      Thread t = new Thread(r);
-      t.setName(String.format("%s-%s-%d", THREAD_ID_PREFIX, key,
-          threadSequenceNumber.getAndIncrement()));
-      return t;
     }
   }
 
   /**
-   * A deamon thread that renews the Append lease on the blob.
-   * The thread sleeps for LEASE_RENEWAL_PERIOD time before renewing
-   * the lease. If an error is encountered while renewing the lease
-   * then an lease is released by this thread, which fails all other
-   * operations.
+   * Prepare block list commit command and queue the command in thread pool
+   * executor.
    */
-  private class AppendRenewer implements Runnable {
+  private synchronized UploadCommand addFlushCommand() throws IOException {
 
-    @Override
-    public void run() {
+    maybeThrowFirstError();
 
-      while (!leaseFreed) {
+    if (blobExist && lease.isFreed()) {
+      throw new AzureException(
+          String.format("Attempting to upload block list on blob : %s"
+              + " that does not have lease on the Blob. Failing upload", key));
+    }
 
-        try {
-          Thread.sleep(LEASE_RENEWAL_PERIOD);
-        } catch (InterruptedException ie) {
-          LOG.debug("Appender Renewer thread interrupted");
-          Thread.currentThread().interrupt();
-        }
+    UploadCommand command = new UploadBlockListCommand();
+    activeBlockCommands.add(command);
 
-        Log.getLog().debug("Attempting to renew append lease on {}", key);
+    ioThreadPool.execute(new WriteRequest(command));
 
-        try {
-          if (!leaseFreed) {
-            // Update the blob metadata to renew the append lease
-            if (!updateBlobAppendMetadata(true, true)) {
-              LOG.error("Unable to re-acquire append lease on the Blob {} ", key);
-              leaseFreed = true;
-            }
-          }
-        } catch (StorageException ex) {
+    return command;
+  }
 
-          LOG.debug("Lease renewal for Blob : {} encountered "
-              + "Storage Exception : {} Error Code : {}", key, ex, ex.getErrorCode());
+  /**
+   * Runnable instance that uploads the block of data to azure storage.
+   */
+  private class WriteRequest implements Runnable {
+    private final UploadCommand command;
 
-          // We swallow the exception here because if the blob metadata is not updated for
-          // APPEND_LEASE_TIMEOUT period, another thread would be able to detect this and
-          // continue forward if it needs to append.
-          leaseFreed = true;
-        }
+    WriteRequest(UploadCommand command) {
+      this.command = command;
+    }
+
+    @Override
+    public void run() {
+
+      try {
+        command.dump();
+        long startTime = System.nanoTime();
+        command.execute();
+        command.setCompleted();
+        LOG.debug("command finished for {} ms",
+            TimeUnit.NANOSECONDS.toMillis(
+                    System.nanoTime() - startTime));
+      } catch (InterruptedException ie) {
+        Thread.currentThread().interrupt();
+      } catch (Exception ex) {
+        LOG.debug(
+                "Encountered exception during execution of command for Blob :"
+                        + " {} Exception : {}", key, ex);
+        firstError.compareAndSet(null, new AzureException(ex));
       }
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13eda500/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
index 0bde124..280c0e0 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
@@ -62,6 +62,8 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Seekable;
+import org.apache.hadoop.fs.StreamCapabilities;
+import org.apache.hadoop.fs.Syncable;
 import org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation;
 import org.apache.hadoop.fs.azure.metrics.AzureFileSystemMetricsSystem;
 import org.apache.hadoop.fs.azure.security.Constants;
@@ -352,9 +354,9 @@ public class NativeAzureFileSystem extends FileSystem {
     }
 
     /**
-     * This is an exact copy of org.codehaus.jettison.json.JSONObject.quote 
+     * This is an exact copy of org.codehaus.jettison.json.JSONObject.quote
      * method.
-     * 
+     *
      * Produce a string in double quotes with backslash sequences in all the
      * right places. A backslash will be inserted within </, allowing JSON
      * text to be delivered in HTML. In JSON text, a string cannot contain a
@@ -947,11 +949,11 @@ public class NativeAzureFileSystem extends FileSystem {
     }
   }
 
-  private class NativeAzureFsOutputStream extends OutputStream {
-    // We should not override flush() to actually close current block and flush
-    // to DFS, this will break applications that assume flush() is a no-op.
-    // Applications are advised to use Syncable.hflush() for that purpose.
-    // NativeAzureFsOutputStream needs to implement Syncable if needed.
+  /**
+   * Azure output stream; wraps an inner stream of different types.
+   */
+  public class NativeAzureFsOutputStream extends OutputStream
+      implements Syncable, StreamCapabilities {
     private String key;
     private String keyEncoded;
     private OutputStream out;
@@ -983,6 +985,48 @@ public class NativeAzureFileSystem extends FileSystem {
       setEncodedKey(anEncodedKey);
     }
 
+    /**
+     * Get a reference to the wrapped output stream.
+     *
+     * @return the underlying output stream
+     */
+    @InterfaceAudience.LimitedPrivate({"HDFS"})
+    public OutputStream getOutStream() {
+      return out;
+    }
+
+    @Override  // Syncable
+    public void hflush() throws IOException {
+      if (out instanceof Syncable) {
+        ((Syncable) out).hflush();
+      } else {
+        flush();
+      }
+    }
+
+    @Override  // Syncable
+    public void hsync() throws IOException {
+      if (out instanceof Syncable) {
+        ((Syncable) out).hsync();
+      } else {
+        flush();
+      }
+    }
+
+    /**
+     * Propagate probe of stream capabilities to nested stream
+     * (if supported), else return false.
+     * @param capability string to query the stream support for.
+     * @return true if the nested stream supports the specific capability.
+     */
+    @Override // StreamCapability
+    public boolean hasCapability(String capability) {
+      if (out instanceof StreamCapabilities) {
+        return ((StreamCapabilities) out).hasCapability(capability);
+      }
+      return false;
+    }
+
     @Override
     public synchronized void close() throws IOException {
       if (out != null) {
@@ -990,8 +1034,11 @@ public class NativeAzureFileSystem extends FileSystem {
         // before returning to the caller.
         //
         out.close();
-        restoreKey();
-        out = null;
+        try {
+          restoreKey();
+        } finally {
+          out = null;
+        }
       }
     }
 
@@ -1045,10 +1092,10 @@ public class NativeAzureFileSystem extends FileSystem {
     /**
      * Writes <code>len</code> from the specified byte array starting at offset
      * <code>off</code> to the output stream. The general contract for write(b,
-     * off, len) is that some of the bytes in the array <code>
-     * b</code b> are written to the output stream in order; element
-     * <code>b[off]</code> is the first byte written and
-     * <code>b[off+len-1]</code> is the last byte written by this operation.
+     * off, len) is that some of the bytes in the array <code>b</code>
+     * are written to the output stream in order; element <code>b[off]</code>
+     * is the first byte written and <code>b[off+len-1]</code> is the last
+     * byte written by this operation.
      * 
      * @param b
      *          Byte array to be written.
@@ -1749,7 +1796,7 @@ public class NativeAzureFileSystem extends FileSystem {
     OutputStream bufOutStream;
     if (store.isPageBlobKey(key)) {
       // Store page blobs directly in-place without renames.
-      bufOutStream = store.storefile(key, permissionStatus);
+      bufOutStream = store.storefile(key, permissionStatus, key);
     } else {
       // This is a block blob, so open the output blob stream based on the
       // encoded key.
@@ -1777,7 +1824,7 @@ public class NativeAzureFileSystem extends FileSystem {
       // these
       // blocks.
       bufOutStream = new NativeAzureFsOutputStream(store.storefile(
-          keyEncoded, permissionStatus), key, keyEncoded);
+          keyEncoded, permissionStatus, key), key, keyEncoded);
     }
     // Construct the data output stream from the buffered output stream.
     FSDataOutputStream fsOut = new FSDataOutputStream(bufOutStream, statistics);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13eda500/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeFileSystemStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeFileSystemStore.java
index 1c7309f..57a729d 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeFileSystemStore.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeFileSystemStore.java
@@ -50,8 +50,9 @@ interface NativeFileSystemStore {
 
   InputStream retrieve(String key, long byteRangeStart) throws IOException;
 
-  DataOutputStream storefile(String key, PermissionStatus permissionStatus)
-      throws AzureException;
+  DataOutputStream storefile(String keyEncoded,
+      PermissionStatus permissionStatus,
+      String key) throws AzureException;
 
   boolean isPageBlobKey(String key);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13eda500/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SecureStorageInterfaceImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SecureStorageInterfaceImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SecureStorageInterfaceImpl.java
index 5dbb6bc..7c2722e 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SecureStorageInterfaceImpl.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SecureStorageInterfaceImpl.java
@@ -519,7 +519,7 @@ public class SecureStorageInterfaceImpl extends StorageInterface {
 
     @Override
     public SelfRenewingLease acquireLease() throws StorageException {
-      return new SelfRenewingLease(this);
+      return new SelfRenewingLease(this, false);
     }
   }
 
@@ -557,10 +557,12 @@ public class SecureStorageInterfaceImpl extends StorageInterface {
     }
 
     @Override
-    public void uploadBlock(String blockId, InputStream sourceStream,
+    public void uploadBlock(String blockId, AccessCondition accessCondition,
+        InputStream sourceStream,
         long length, BlobRequestOptions options,
         OperationContext opContext) throws IOException, StorageException {
-      ((CloudBlockBlob) getBlob()).uploadBlock(blockId, sourceStream, length, null, options, opContext);
+      ((CloudBlockBlob) getBlob()).uploadBlock(blockId, sourceStream, length,
+          accessCondition, options, opContext);
     }
 
     @Override
@@ -593,4 +595,4 @@ public class SecureStorageInterfaceImpl extends StorageInterface {
           null, options, opContext);
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13eda500/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SelfRenewingLease.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SelfRenewingLease.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SelfRenewingLease.java
index 00d5e99..10956f7 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SelfRenewingLease.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SelfRenewingLease.java
@@ -30,6 +30,8 @@ import com.microsoft.azure.storage.blob.CloudBlob;
 
 import java.util.concurrent.atomic.AtomicInteger;
 
+import static com.microsoft.azure.storage.StorageErrorCodeStrings.LEASE_ALREADY_PRESENT;
+
 /**
  * An Azure blob lease that automatically renews itself indefinitely
  * using a background thread. Use it to synchronize distributed processes,
@@ -66,7 +68,7 @@ public class SelfRenewingLease {
   @VisibleForTesting
   static final int LEASE_ACQUIRE_RETRY_INTERVAL = 2000;
 
-  public SelfRenewingLease(CloudBlobWrapper blobWrapper)
+  public SelfRenewingLease(CloudBlobWrapper blobWrapper, boolean throwIfPresent)
       throws StorageException {
 
     this.leaseFreed = false;
@@ -79,10 +81,14 @@ public class SelfRenewingLease {
         leaseID = blob.acquireLease(LEASE_TIMEOUT, null);
       } catch (StorageException e) {
 
+        if (throwIfPresent && e.getErrorCode().equals(LEASE_ALREADY_PRESENT)) {
+          throw e;
+        }
+
         // Throw again if we don't want to keep waiting.
         // We expect it to be that the lease is already present,
         // or in some cases that the blob does not exist.
-        if (!"LeaseAlreadyPresent".equals(e.getErrorCode())) {
+        if (!LEASE_ALREADY_PRESENT.equals(e.getErrorCode())) {
           LOG.info(
             "Caught exception when trying to get lease on blob "
             + blobWrapper.getUri().toString() + ". " + e.getMessage());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13eda500/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterface.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterface.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterface.java
index 8b6b082..e03d731 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterface.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterface.java
@@ -665,6 +665,7 @@ abstract class StorageInterface {
      *
      * @param blockId      A String that represents the Base-64 encoded block ID. Note for a given blob
      *                     the length of all Block IDs must be identical.
+     * @param accessCondition An {@link AccessCondition} object that represents the access conditions for the blob.
      * @param sourceStream An {@link InputStream} object that represents the input stream to write to the
      *                     block blob.
      * @param length       A long which represents the length, in bytes, of the stream data,
@@ -678,7 +679,7 @@ abstract class StorageInterface {
      * @throws IOException  If an I/O error occurred.
      * @throws StorageException If a storage service error occurred.
      */
-    void uploadBlock(String blockId, InputStream sourceStream,
+    void uploadBlock(String blockId, AccessCondition accessCondition, InputStream sourceStream,
         long length, BlobRequestOptions options,
         OperationContext opContext) throws IOException, StorageException;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13eda500/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterfaceImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterfaceImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterfaceImpl.java
index d3d0370..41a4dbb 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterfaceImpl.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterfaceImpl.java
@@ -277,7 +277,7 @@ class StorageInterfaceImpl extends StorageInterface {
 
       return new CloudBlockBlobWrapperImpl(container.getBlockBlobReference(relativePath));
     }
-    
+
     @Override
     public CloudBlobWrapper getPageBlobReference(String relativePath)
         throws URISyntaxException, StorageException {
@@ -286,7 +286,7 @@ class StorageInterfaceImpl extends StorageInterface {
     }
 
   }
-  
+
   abstract static class CloudBlobWrapperImpl implements CloudBlobWrapper {
     private final CloudBlob blob;
 
@@ -441,10 +441,10 @@ class StorageInterfaceImpl extends StorageInterface {
 
     @Override
     public SelfRenewingLease acquireLease() throws StorageException {
-      return new SelfRenewingLease(this);
+      return new SelfRenewingLease(this, false);
     }
   }
-  
+
 
   //
   // CloudBlockBlobWrapperImpl
@@ -479,10 +479,10 @@ class StorageInterfaceImpl extends StorageInterface {
     }
 
     @Override
-    public void uploadBlock(String blockId, InputStream sourceStream,
+    public void uploadBlock(String blockId, AccessCondition accessCondition, InputStream sourceStream,
         long length, BlobRequestOptions options,
         OperationContext opContext) throws IOException, StorageException {
-      ((CloudBlockBlob) getBlob()).uploadBlock(blockId, sourceStream, length, null, options, opContext);
+      ((CloudBlockBlob) getBlob()).uploadBlock(blockId, sourceStream, length, accessCondition, options, opContext);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13eda500/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SyncableDataOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SyncableDataOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SyncableDataOutputStream.java
index a52fdb7..fc8796b 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SyncableDataOutputStream.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SyncableDataOutputStream.java
@@ -24,6 +24,7 @@ import java.io.OutputStream;
 
 import org.apache.hadoop.fs.StreamCapabilities;
 import org.apache.hadoop.fs.Syncable;
+import org.apache.hadoop.classification.InterfaceAudience;
 
 /**
  * Support the Syncable interface on top of a DataOutputStream.
@@ -38,6 +39,16 @@ public class SyncableDataOutputStream extends DataOutputStream
     super(out);
   }
 
+  /**
+   * Get a reference to the wrapped output stream.
+   *
+   * @return the underlying output stream
+   */
+  @InterfaceAudience.LimitedPrivate({"HDFS"})
+  public OutputStream getOutStream() {
+    return out;
+  }
+
   @Override
   public boolean hasCapability(String capability) {
     if (out instanceof StreamCapabilities) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13eda500/hadoop-tools/hadoop-azure/src/site/markdown/index.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/site/markdown/index.md b/hadoop-tools/hadoop-azure/src/site/markdown/index.md
index 758650d..466bf0b 100644
--- a/hadoop-tools/hadoop-azure/src/site/markdown/index.md
+++ b/hadoop-tools/hadoop-azure/src/site/markdown/index.md
@@ -153,6 +153,40 @@ line argument:
 
 ```
 
+### Block Blob with Compaction Support and Configuration
+
+Block blobs are the default kind of blob and are good for most big-data use
+cases. However, block blobs have strict limit of 50,000 blocks per blob.
+To prevent reaching the limit WASB, by default, does not upload new block to
+the service after every `hflush()` or `hsync()`.
+
+For most of the cases, combining data from multiple `write()` calls in
+blocks of 4Mb is a good optimization. But, in others cases, like HBase log files,
+every call to `hflush()` or `hsync()` must upload the data to the service.
+
+Block blobs with compaction upload the data to the cloud service after every
+`hflush()`/`hsync()`. To mitigate the limit of 50000 blocks, `hflush()
+`/`hsync()` runs once compaction process, if number of blocks in the blob
+is above 32,000.
+
+Block compaction search and replaces a sequence of small blocks with one big
+block. That means there is associated cost with block compaction: reading
+small blocks back to the client and writing it again as one big block.
+
+In order to have the files you create be block blobs with block compaction
+enabled, the client must set the configuration variable
+`fs.azure.block.blob.with.compaction.dir` to a comma-separated list of
+folder names.
+
+For example:
+
+```xml
+<property>
+  <name>fs.azure.block.blob.with.compaction.dir</name>
+  <value>/hbase/WALs,/data/myblobfiles</value>
+</property>
+```
+
 ### Page Blob Support and Configuration
 
 The Azure Blob Storage interface for Hadoop supports two kinds of blobs,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13eda500/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockStorageInterface.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockStorageInterface.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockStorageInterface.java
index 4f26d9f..e0ae7b4 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockStorageInterface.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockStorageInterface.java
@@ -551,7 +551,8 @@ public class MockStorageInterface extends StorageInterface {
       throw new UnsupportedOperationException("downloadBlockList not used in Mock Tests");
     }
     @Override
-    public void uploadBlock(String blockId, InputStream sourceStream,
+    public void uploadBlock(String blockId, AccessCondition accessCondition,
+        InputStream sourceStream,
         long length, BlobRequestOptions options,
         OperationContext opContext) throws IOException, StorageException {
       throw new UnsupportedOperationException("uploadBlock not used in Mock Tests");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13eda500/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestAzureConcurrentOutOfBandIo.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestAzureConcurrentOutOfBandIo.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestAzureConcurrentOutOfBandIo.java
index 7ea7534..a10a366 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestAzureConcurrentOutOfBandIo.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestAzureConcurrentOutOfBandIo.java
@@ -107,7 +107,8 @@ public class TestAzureConcurrentOutOfBandIo {
           //
           outputStream = writerStorageAccount.getStore().storefile(
               key,
-              new PermissionStatus("", "", FsPermission.getDefault()));
+              new PermissionStatus("", "", FsPermission.getDefault()),
+              key);
 
           Arrays.fill(dataBlockWrite, (byte) (i % 256));
           for (int j = 0; j < NUMBER_OF_BLOCKS; j++) {
@@ -141,7 +142,8 @@ public class TestAzureConcurrentOutOfBandIo {
    // reading.  This eliminates the race between the reader and writer threads.
    OutputStream outputStream = testAccount.getStore().storefile(
        "WASB_String.txt",
-       new PermissionStatus("", "", FsPermission.getDefault()));
+       new PermissionStatus("", "", FsPermission.getDefault()),
+           "WASB_String.txt");
    Arrays.fill(dataBlockWrite, (byte) 255);
    for (int i = 0; i < NUMBER_OF_BLOCKS; i++) {
      outputStream.write(dataBlockWrite);


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


[06/50] [abbrv] hadoop git commit: YARN-7144. Log Aggregation controller should not swallow the exceptions when it calls closeWriter and closeReader. Contributed by Xuan Gong.

Posted by as...@apache.org.
YARN-7144. Log Aggregation controller should not swallow the exceptions when it calls closeWriter and closeReader. Contributed by Xuan Gong.


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

Branch: refs/heads/YARN-5972
Commit: 22de9449f8aa72c5b0bb586b8253390773502189
Parents: dd81494
Author: Junping Du <ju...@apache.org>
Authored: Wed Sep 6 14:53:31 2017 -0700
Committer: Junping Du <ju...@apache.org>
Committed: Wed Sep 6 14:53:31 2017 -0700

----------------------------------------------------------------------
 .../logaggregation/AggregatedLogFormat.java     | 27 +++++++++-----------
 .../LogAggregationFileController.java           | 10 ++++----
 2 files changed, 17 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/22de9449/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
index 9bec147..af3066e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
@@ -44,11 +44,8 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.regex.Pattern;
-import org.apache.commons.io.IOUtils;
 import org.apache.commons.io.input.BoundedInputStream;
 import org.apache.commons.io.output.WriterOutputStream;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.commons.math3.util.Pair;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
@@ -61,6 +58,7 @@ import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.SecureIOUtils;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.file.tfile.TFile;
@@ -71,7 +69,8 @@ import org.apache.hadoop.yarn.api.records.LogAggregationContext;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.util.Times;
-
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Predicate;
 import com.google.common.collect.Iterables;
@@ -81,7 +80,8 @@ import com.google.common.collect.Sets;
 @Evolving
 public class AggregatedLogFormat {
 
-  private static final Log LOG = LogFactory.getLog(AggregatedLogFormat.class);
+  private final static Logger LOG = LoggerFactory.getLogger(
+      AggregatedLogFormat.class);
   private static final LogKey APPLICATION_ACL_KEY = new LogKey("APPLICATION_ACL");
   private static final LogKey APPLICATION_OWNER_KEY = new LogKey("APPLICATION_OWNER");
   private static final LogKey VERSION_KEY = new LogKey("VERSION");
@@ -247,7 +247,7 @@ public class AggregatedLogFormat {
           in = secureOpenFile(logFile);
         } catch (IOException e) {
           logErrorMessage(logFile, e);
-          IOUtils.closeQuietly(in);
+          IOUtils.cleanupWithLogger(LOG, in);
           continue;
         }
 
@@ -285,7 +285,7 @@ public class AggregatedLogFormat {
           String message = logErrorMessage(logFile, e);
           out.write(message.getBytes(Charset.forName("UTF-8")));
         } finally {
-          IOUtils.closeQuietly(in);
+          IOUtils.cleanupWithLogger(LOG, in);
         }
       }
     }
@@ -555,7 +555,7 @@ public class AggregatedLogFormat {
       } catch (Exception e) {
         LOG.warn("Exception closing writer", e);
       } finally {
-        IOUtils.closeQuietly(this.fsDataOStream);
+        IOUtils.cleanupWithLogger(LOG, this.fsDataOStream);
       }
     }
   }
@@ -603,7 +603,7 @@ public class AggregatedLogFormat {
         }
         return null;
       } finally {
-        IOUtils.closeQuietly(ownerScanner);
+        IOUtils.cleanupWithLogger(LOG, ownerScanner);
       }
     }
 
@@ -649,7 +649,7 @@ public class AggregatedLogFormat {
         }
         return acls;
       } finally {
-        IOUtils.closeQuietly(aclScanner);
+        IOUtils.cleanupWithLogger(LOG, aclScanner);
       }
     }
 
@@ -774,8 +774,7 @@ public class AggregatedLogFormat {
           }
         }
       } finally {
-        IOUtils.closeQuietly(ps);
-        IOUtils.closeQuietly(os);
+        IOUtils.cleanupWithLogger(LOG, ps, os);
       }
     }
 
@@ -1001,9 +1000,7 @@ public class AggregatedLogFormat {
     }
 
     public void close() {
-      IOUtils.closeQuietly(scanner);
-      IOUtils.closeQuietly(reader);
-      IOUtils.closeQuietly(fsDataIStream);
+      IOUtils.cleanupWithLogger(LOG, scanner, reader, fsDataIStream);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/22de9449/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationFileController.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationFileController.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationFileController.java
index aafdb66..39f3dc3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationFileController.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationFileController.java
@@ -37,9 +37,6 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import org.apache.commons.io.IOUtils;
-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.conf.Configuration;
@@ -47,6 +44,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -56,6 +54,8 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.logaggregation.LogAggregationUtils;
 import org.apache.hadoop.yarn.webapp.View.ViewContext;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock.Block;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogKey;
 import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogValue;
 import org.apache.hadoop.yarn.logaggregation.ContainerLogMeta;
@@ -68,7 +68,7 @@ import org.apache.hadoop.yarn.logaggregation.ContainerLogsRequest;
 @Unstable
 public abstract class LogAggregationFileController {
 
-  private static final Log LOG = LogFactory.getLog(
+  private static final Logger LOG = LoggerFactory.getLogger(
       LogAggregationFileController.class);
 
   /*
@@ -193,7 +193,7 @@ public abstract class LogAggregationFileController {
 
   protected void closePrintStream(OutputStream out) {
     if (out != System.out) {
-      IOUtils.closeQuietly(out);
+      IOUtils.cleanupWithLogger(LOG, out);
     }
   }
 


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


[12/50] [abbrv] hadoop git commit: YARN-5328. Plan/ResourceAllocation data structure enhancements required to support recurring reservations in ReservationSystem.

Posted by as...@apache.org.
YARN-5328. Plan/ResourceAllocation data structure enhancements required to support recurring reservations in ReservationSystem.


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

Branch: refs/heads/YARN-5972
Commit: b6e7d1369690eaf50ce9ea7968f91a72ecb74de0
Parents: e3345e9
Author: Subru Krishnan <su...@apache.org>
Authored: Wed Sep 6 16:46:01 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Wed Sep 6 16:46:01 2017 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/conf/YarnConfiguration.java     |   6 +
 .../yarn/conf/TestYarnConfigurationFields.java  |   4 +-
 .../reservation/AbstractReservationSystem.java  |  90 ++--
 .../AbstractSchedulerPlanFollower.java          | 183 ++++----
 .../reservation/InMemoryPlan.java               | 400 ++++++++++++-----
 .../InMemoryReservationAllocation.java          |  36 +-
 .../reservation/NoOverCommitPolicy.java         |   2 +-
 .../PeriodicRLESparseResourceAllocation.java    | 130 ++++--
 .../resourcemanager/reservation/PlanEdit.java   |  24 +-
 .../resourcemanager/reservation/PlanView.java   |  94 ++--
 .../RLESparseResourceAllocation.java            | 112 ++---
 .../reservation/ReservationAllocation.java      |  60 ++-
 .../reservation/ReservationInputValidator.java  | 134 +++---
 .../reservation/ReservationSystem.java          |   6 +-
 .../reservation/SharingPolicy.java              |  13 +-
 .../reservation/planning/Planner.java           |   2 +-
 .../reservation/planning/PlanningAlgorithm.java |   2 +-
 .../reservation/planning/StageAllocator.java    |   2 +-
 .../planning/StageAllocatorGreedy.java          |   2 +-
 .../planning/StageAllocatorGreedyRLE.java       |   5 +-
 .../planning/StageAllocatorLowCostAligned.java  |   4 +-
 .../reservation/ReservationSystemTestUtil.java  | 135 +++---
 .../reservation/TestInMemoryPlan.java           | 431 ++++++++++++-------
 ...TestPeriodicRLESparseResourceAllocation.java | 109 +++--
 .../TestRLESparseResourceAllocation.java        | 122 +++---
 .../planning/TestSimpleCapacityReplanner.java   |   8 +-
 26 files changed, 1340 insertions(+), 776 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/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 4944821..27ca957 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
@@ -262,6 +262,12 @@ public class YarnConfiguration extends Configuration {
   public static final long DEFAULT_RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP =
       1000L;
 
+  /** The maximum periodicity for the Reservation System. */
+  public static final String RM_RESERVATION_SYSTEM_MAX_PERIODICITY =
+      RM_PREFIX + "reservation-system.max-periodicity";
+  public static final long DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY =
+      86400000L;
+
   /**
    * Enable periodic monitor threads.
    * @see #RM_SCHEDULER_MONITOR_POLICIES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/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 bd7bf93..1d3111c 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
@@ -33,7 +33,7 @@ import org.apache.hadoop.conf.TestConfigurationFieldsBase;
  */
 public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
 
-  @SuppressWarnings("deprecation")
+  @SuppressWarnings({"deprecation", "methodlength"})
   @Override
   public void initializeMemberVariables() {
     xmlFilename = new String("yarn-default.xml");
@@ -69,6 +69,8 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
     configurationPropsToSkipCompare.add(YarnConfiguration
         .YARN_SECURITY_SERVICE_AUTHORIZATION_COLLECTOR_NODEMANAGER_PROTOCOL);
     configurationPropsToSkipCompare.add(YarnConfiguration.CURATOR_LEADER_ELECTOR);
+    configurationPropsToSkipCompare
+        .add(YarnConfiguration.RM_RESERVATION_SYSTEM_MAX_PERIODICITY);
 
     // Federation default configs to be ignored
     configurationPropsToSkipCompare

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.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/reservation/AbstractReservationSystem.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.java
index 5ef4912..5b8772c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.java
@@ -18,6 +18,17 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
 import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
@@ -46,17 +57,6 @@ import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
 /**
  * This is the implementation of {@link ReservationSystem} based on the
  * {@link ResourceScheduler}
@@ -66,8 +66,8 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 public abstract class AbstractReservationSystem extends AbstractService
     implements ReservationSystem {
 
-  private static final Logger LOG = LoggerFactory
-      .getLogger(AbstractReservationSystem.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(AbstractReservationSystem.class);
 
   // private static final String DEFAULT_CAPACITY_SCHEDULER_PLAN
 
@@ -103,6 +103,8 @@ public abstract class AbstractReservationSystem extends AbstractService
 
   private boolean isRecoveryEnabled = false;
 
+  private long maxPeriodicity;
+
   /**
    * Construct the service.
    * 
@@ -143,36 +145,41 @@ public abstract class AbstractReservationSystem extends AbstractService
     this.conf = conf;
     scheduler = rmContext.getScheduler();
     // Get the plan step size
-    planStepSize =
-        conf.getTimeDuration(
-            YarnConfiguration.RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP,
-            YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP,
-            TimeUnit.MILLISECONDS);
+    planStepSize = conf.getTimeDuration(
+        YarnConfiguration.RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP,
+        YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP,
+        TimeUnit.MILLISECONDS);
     if (planStepSize < 0) {
       planStepSize =
           YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP;
     }
+    maxPeriodicity =
+        conf.getLong(YarnConfiguration.RM_RESERVATION_SYSTEM_MAX_PERIODICITY,
+            YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY);
+    if (maxPeriodicity <= 0) {
+      maxPeriodicity =
+          YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY;
+    }
     // Create a plan corresponding to every reservable queue
     Set<String> planQueueNames = scheduler.getPlanQueues();
     for (String planQueueName : planQueueNames) {
       Plan plan = initializePlan(planQueueName);
       plans.put(planQueueName, plan);
     }
-    isRecoveryEnabled = conf.getBoolean(
-        YarnConfiguration.RECOVERY_ENABLED,
+    isRecoveryEnabled = conf.getBoolean(YarnConfiguration.RECOVERY_ENABLED,
         YarnConfiguration.DEFAULT_RM_RECOVERY_ENABLED);
 
     if (conf.getBoolean(YarnConfiguration.YARN_RESERVATION_ACL_ENABLE,
-            YarnConfiguration.DEFAULT_YARN_RESERVATION_ACL_ENABLE) &&
-                    conf.getBoolean(YarnConfiguration.YARN_ACL_ENABLE,
-                            YarnConfiguration.DEFAULT_YARN_ACL_ENABLE)) {
+        YarnConfiguration.DEFAULT_YARN_RESERVATION_ACL_ENABLE)
+        && conf.getBoolean(YarnConfiguration.YARN_ACL_ENABLE,
+            YarnConfiguration.DEFAULT_YARN_ACL_ENABLE)) {
       reservationsACLsManager = new ReservationsACLsManager(scheduler, conf);
     }
   }
 
   private void loadPlan(String planName,
       Map<ReservationId, ReservationAllocationStateProto> reservations)
-          throws PlanningException {
+      throws PlanningException {
     Plan plan = plans.get(planName);
     Resource minAllocation = getMinAllocation();
     ResourceCalculator rescCalculator = getResourceCalculator();
@@ -248,8 +255,8 @@ public abstract class AbstractReservationSystem extends AbstractService
       Class<?> planFollowerPolicyClazz =
           conf.getClassByName(planFollowerPolicyClassName);
       if (PlanFollower.class.isAssignableFrom(planFollowerPolicyClazz)) {
-        return (PlanFollower) ReflectionUtils.newInstance(
-            planFollowerPolicyClazz, conf);
+        return (PlanFollower) ReflectionUtils
+            .newInstance(planFollowerPolicyClazz, conf);
       } else {
         throw new YarnRuntimeException("Class: " + planFollowerPolicyClassName
             + " not instance of " + PlanFollower.class.getCanonicalName());
@@ -257,7 +264,8 @@ public abstract class AbstractReservationSystem extends AbstractService
     } catch (ClassNotFoundException e) {
       throw new YarnRuntimeException(
           "Could not instantiate PlanFollowerPolicy: "
-              + planFollowerPolicyClassName, e);
+              + planFollowerPolicyClassName,
+          e);
     }
   }
 
@@ -371,9 +379,8 @@ public abstract class AbstractReservationSystem extends AbstractService
   public ReservationId getNewReservationId() {
     writeLock.lock();
     try {
-      ReservationId resId =
-          ReservationId.newInstance(ResourceManager.getClusterTimeStamp(),
-              resCounter.incrementAndGet());
+      ReservationId resId = ReservationId.newInstance(
+          ResourceManager.getClusterTimeStamp(), resCounter.incrementAndGet());
       LOG.info("Allocated new reservationId: " + resId);
       return resId;
     } finally {
@@ -390,8 +397,11 @@ public abstract class AbstractReservationSystem extends AbstractService
    * Get the default reservation system corresponding to the scheduler
    * 
    * @param scheduler the scheduler for which the reservation system is required
+   *
+   * @return the {@link ReservationSystem} based on the configured scheduler
    */
-  public static String getDefaultReservationSystem(ResourceScheduler scheduler) {
+  public static String getDefaultReservationSystem(
+      ResourceScheduler scheduler) {
     if (scheduler instanceof CapacityScheduler) {
       return CapacityReservationSystem.class.getName();
     } else if (scheduler instanceof FairScheduler) {
@@ -409,12 +419,11 @@ public abstract class AbstractReservationSystem extends AbstractService
     Resource maxAllocation = getMaxAllocation();
     ResourceCalculator rescCalc = getResourceCalculator();
     Resource totCap = getPlanQueueCapacity(planQueueName);
-    Plan plan =
-        new InMemoryPlan(getRootQueueMetrics(), adPolicy,
-            getAgent(planQueuePath), totCap, planStepSize, rescCalc,
-            minAllocation, maxAllocation, planQueueName,
-            getReplanner(planQueuePath), getReservationSchedulerConfiguration()
-            .getMoveOnExpiry(planQueuePath), rmContext);
+    Plan plan = new InMemoryPlan(getRootQueueMetrics(), adPolicy,
+        getAgent(planQueuePath), totCap, planStepSize, rescCalc, minAllocation,
+        maxAllocation, planQueueName, getReplanner(planQueuePath),
+        getReservationSchedulerConfiguration().getMoveOnExpiry(planQueuePath),
+        maxPeriodicity, rmContext);
     LOG.info("Initialized plan {} based on reservable queue {}",
         plan.toString(), planQueueName);
     return plan;
@@ -477,8 +486,8 @@ public abstract class AbstractReservationSystem extends AbstractService
       Class<?> admissionPolicyClazz =
           conf.getClassByName(admissionPolicyClassName);
       if (SharingPolicy.class.isAssignableFrom(admissionPolicyClazz)) {
-        return (SharingPolicy) ReflectionUtils.newInstance(
-            admissionPolicyClazz, conf);
+        return (SharingPolicy) ReflectionUtils.newInstance(admissionPolicyClazz,
+            conf);
       } else {
         throw new YarnRuntimeException("Class: " + admissionPolicyClassName
             + " not instance of " + SharingPolicy.class.getCanonicalName());
@@ -493,8 +502,7 @@ public abstract class AbstractReservationSystem extends AbstractService
     return this.reservationsACLsManager;
   }
 
-  protected abstract ReservationSchedulerConfiguration
-      getReservationSchedulerConfiguration();
+  protected abstract ReservationSchedulerConfiguration getReservationSchedulerConfiguration();
 
   protected abstract String getPlanQueuePath(String planQueueName);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractSchedulerPlanFollower.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/reservation/AbstractSchedulerPlanFollower.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractSchedulerPlanFollower.java
index 90357e3..9b6a0b0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractSchedulerPlanFollower.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractSchedulerPlanFollower.java
@@ -18,6 +18,14 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -33,24 +41,17 @@ import org.apache.hadoop.yarn.util.resource.Resources;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
 public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
-  private static final Logger LOG = LoggerFactory
-      .getLogger(AbstractSchedulerPlanFollower.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(AbstractSchedulerPlanFollower.class);
 
   protected Collection<Plan> plans = new ArrayList<Plan>();
   protected YarnScheduler scheduler;
   protected Clock clock;
 
   @Override
-  public void init(Clock clock, ResourceScheduler sched, Collection<Plan> plans) {
+  public void init(Clock clock, ResourceScheduler sched,
+      Collection<Plan> plans) {
     this.clock = clock;
     this.scheduler = sched;
     this.plans.addAll(plans);
@@ -71,7 +72,7 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
 
   @Override
   public synchronized void synchronizePlan(Plan plan, boolean shouldReplan) {
-     String planQueueName = plan.getQueueName();
+    String planQueueName = plan.getQueueName();
     if (LOG.isDebugEnabled()) {
       LOG.debug("Running plan follower edit policy for plan: " + planQueueName);
     }
@@ -82,12 +83,14 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
       now += step - (now % step);
     }
     Queue planQueue = getPlanQueue(planQueueName);
-    if (planQueue == null) return;
+    if (planQueue == null) {
+      return;
+    }
 
     // first we publish to the plan the current availability of resources
     Resource clusterResources = scheduler.getClusterResource();
-    Resource planResources = getPlanResources(plan, planQueue,
-        clusterResources);
+    Resource planResources =
+        getPlanResources(plan, planQueue, clusterResources);
     Set<ReservationAllocation> currentReservations =
         plan.getReservationsAtTime(now);
     Set<String> curReservationNames = new HashSet<String>();
@@ -95,12 +98,11 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
     int numRes = getReservedResources(now, currentReservations,
         curReservationNames, reservedResources);
     // create the default reservation queue if it doesnt exist
-    String defReservationId = getReservationIdFromQueueName(planQueueName) +
-        ReservationConstants.DEFAULT_QUEUE_SUFFIX;
-    String defReservationQueue = getReservationQueueName(planQueueName,
-        defReservationId);
-    createDefaultReservationQueue(planQueueName, planQueue,
-        defReservationId);
+    String defReservationId = getReservationIdFromQueueName(planQueueName)
+        + ReservationConstants.DEFAULT_QUEUE_SUFFIX;
+    String defReservationQueue =
+        getReservationQueueName(planQueueName, defReservationId);
+    createDefaultReservationQueue(planQueueName, planQueue, defReservationId);
     curReservationNames.add(defReservationId);
     // if the resources dedicated to this plan has shrunk invoke replanner
     boolean shouldResize = false;
@@ -149,10 +151,8 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
       // sort allocations from the one giving up the most resources, to the
       // one asking for the most avoid order-of-operation errors that
       // temporarily violate 100% capacity bound
-      List<ReservationAllocation> sortedAllocations =
-          sortByDelta(
-              new ArrayList<ReservationAllocation>(currentReservations), now,
-              plan);
+      List<ReservationAllocation> sortedAllocations = sortByDelta(
+          new ArrayList<ReservationAllocation>(currentReservations), now, plan);
       for (ReservationAllocation res : sortedAllocations) {
         String currResId = res.getReservationId().toString();
         if (curReservationNames.contains(currResId)) {
@@ -163,10 +163,9 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
         if (planResources.getMemorySize() > 0
             && planResources.getVirtualCores() > 0) {
           if (shouldResize) {
-            capToAssign =
-                calculateReservationToPlanProportion(
-                    plan.getResourceCalculator(), planResources,
-                    reservedResources, capToAssign);
+            capToAssign = calculateReservationToPlanProportion(
+                plan.getResourceCalculator(), planResources, reservedResources,
+                capToAssign);
           }
           targetCapacity =
               calculateReservationToPlanRatio(plan.getResourceCalculator(),
@@ -185,7 +184,8 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
           maxCapacity = targetCapacity;
         }
         try {
-          setQueueEntitlement(planQueueName, currResId, targetCapacity, maxCapacity);
+          setQueueEntitlement(planQueueName, currResId, targetCapacity,
+              maxCapacity);
         } catch (YarnException e) {
           LOG.warn("Exception while trying to size reservation for plan: {}",
               currResId, planQueueName, e);
@@ -196,9 +196,10 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
     // compute the default queue capacity
     float defQCap = 1.0f - totalAssignedCapacity;
     if (LOG.isDebugEnabled()) {
-      LOG.debug("PlanFollowerEditPolicyTask: total Plan Capacity: {} "
-          + "currReservation: {} default-queue capacity: {}", planResources,
-          numRes, defQCap);
+      LOG.debug(
+          "PlanFollowerEditPolicyTask: total Plan Capacity: {} "
+              + "currReservation: {} default-queue capacity: {}",
+          planResources, numRes, defQCap);
     }
     // set the default queue to eat-up all remaining capacity
     try {
@@ -225,12 +226,11 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
   }
 
   protected void setQueueEntitlement(String planQueueName, String currResId,
-      float targetCapacity,
-      float maxCapacity) throws YarnException {
-    String reservationQueueName = getReservationQueueName(planQueueName,
-        currResId);
-    scheduler.setEntitlement(reservationQueueName, new QueueEntitlement(
-        targetCapacity, maxCapacity));
+      float targetCapacity, float maxCapacity) throws YarnException {
+    String reservationQueueName =
+        getReservationQueueName(planQueueName, currResId);
+    scheduler.setEntitlement(reservationQueueName,
+        new QueueEntitlement(targetCapacity, maxCapacity));
   }
 
   // Schedulers have different ways of naming queues. See YARN-2773
@@ -244,14 +244,21 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
    * Then move all apps in the set of queues to the parent plan queue's default
    * reservation queue if move is enabled. Finally cleanups the queue by killing
    * any apps (if move is disabled or move failed) and removing the queue
+   *
+   * @param planQueueName the name of {@code PlanQueue}
+   * @param shouldMove flag to indicate if any running apps should be moved or
+   *          killed
+   * @param toRemove the remnant apps to clean up
+   * @param defReservationQueue the default {@code ReservationQueue} of the
+   *          {@link Plan}
    */
-  protected void cleanupExpiredQueues(String planQueueName,
-      boolean shouldMove, Set<String> toRemove, String defReservationQueue) {
+  protected void cleanupExpiredQueues(String planQueueName, boolean shouldMove,
+      Set<String> toRemove, String defReservationQueue) {
     for (String expiredReservationId : toRemove) {
       try {
         // reduce entitlement to 0
-        String expiredReservation = getReservationQueueName(planQueueName,
-            expiredReservationId);
+        String expiredReservation =
+            getReservationQueueName(planQueueName, expiredReservationId);
         setQueueEntitlement(planQueueName, expiredReservation, 0.0f, 0.0f);
         if (shouldMove) {
           moveAppsInQueueSync(expiredReservation, defReservationQueue);
@@ -275,7 +282,7 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
    * reservation queue in a synchronous fashion
    */
   private void moveAppsInQueueSync(String expiredReservation,
-                                   String defReservationQueue) {
+      String defReservationQueue) {
     List<ApplicationAttemptId> activeApps =
         scheduler.getAppsInQueue(expiredReservation);
     if (activeApps.isEmpty()) {
@@ -287,16 +294,16 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
         scheduler.moveApplication(app.getApplicationId(), defReservationQueue);
       } catch (YarnException e) {
         LOG.warn(
-            "Encountered unexpected error during migration of application: {}" +
-                " from reservation: {}",
+            "Encountered unexpected error during migration of application: {}"
+                + " from reservation: {}",
             app, expiredReservation, e);
       }
     }
   }
 
-  protected int getReservedResources(long now, Set<ReservationAllocation>
-      currentReservations, Set<String> curReservationNames,
-                                     Resource reservedResources) {
+  protected int getReservedResources(long now,
+      Set<ReservationAllocation> currentReservations,
+      Set<String> curReservationNames, Resource reservedResources) {
     int numRes = 0;
     if (currentReservations != null) {
       numRes = currentReservations.size();
@@ -312,23 +319,30 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
    * Sort in the order from the least new amount of resources asked (likely
    * negative) to the highest. This prevents "order-of-operation" errors related
    * to exceeding 100% capacity temporarily.
+   *
+   * @param currentReservations the currently active reservations
+   * @param now the current time
+   * @param plan the {@link Plan} that is being considered
+   *
+   * @return the sorted list of {@link ReservationAllocation}s
    */
   protected List<ReservationAllocation> sortByDelta(
       List<ReservationAllocation> currentReservations, long now, Plan plan) {
-    Collections.sort(currentReservations, new ReservationAllocationComparator(
-        now, this, plan));
+    Collections.sort(currentReservations,
+        new ReservationAllocationComparator(now, this, plan));
     return currentReservations;
   }
 
   /**
-   * Get queue associated with reservable queue named
-   * @param planQueueName Name of the reservable queue
+   * Get queue associated with reservable queue named.
+   *
+   * @param planQueueName name of the reservable queue
    * @return queue associated with the reservable queue
    */
   protected abstract Queue getPlanQueue(String planQueueName);
 
   /**
-   * Resizes reservations based on currently available resources
+   * Resizes reservations based on currently available resources.
    */
   private Resource calculateReservationToPlanProportion(
       ResourceCalculator rescCalculator, Resource availablePlanResources,
@@ -338,7 +352,7 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
   }
 
   /**
-   * Calculates ratio of reservationResources to planResources
+   * Calculates ratio of reservationResources to planResources.
    */
   private float calculateReservationToPlanRatio(
       ResourceCalculator rescCalculator, Resource clusterResources,
@@ -348,7 +362,7 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
   }
 
   /**
-   * Check if plan resources are less than expected reservation resources
+   * Check if plan resources are less than expected reservation resources.
    */
   private boolean arePlanResourcesLessThanReservations(
       ResourceCalculator rescCalculator, Resource clusterResources,
@@ -358,38 +372,56 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
   }
 
   /**
-   * Get a list of reservation queues for this planQueue
+   * Get a list of reservation queues for this planQueue.
+   *
+   * @param planQueue the queue for the current {@link Plan}
+   *
+   * @return the queues corresponding to the reservations
    */
   protected abstract List<? extends Queue> getChildReservationQueues(
       Queue planQueue);
 
   /**
-   * Add a new reservation queue for reservation currResId for this planQueue
+   * Add a new reservation queue for reservation currResId for this planQueue.
    */
-  protected abstract void addReservationQueue(
-      String planQueueName, Queue queue, String currResId);
+  protected abstract void addReservationQueue(String planQueueName, Queue queue,
+      String currResId);
 
   /**
-   * Creates the default reservation queue for use when no reservation is
-   * used for applications submitted to this planQueue
+   * Creates the default reservation queue for use when no reservation is used
+   * for applications submitted to this planQueue.
+   *
+   * @param planQueueName name of the reservable queue
+   * @param queue the queue for the current {@link Plan}
+   * @param defReservationQueue name of the default {@code ReservationQueue}
    */
-  protected abstract void createDefaultReservationQueue(
-      String planQueueName, Queue queue, String defReservationQueue);
+  protected abstract void createDefaultReservationQueue(String planQueueName,
+      Queue queue, String defReservationQueue);
 
   /**
-   * Get plan resources for this planQueue
+   * Get plan resources for this planQueue.
+   *
+   * @param plan the current {@link Plan} being considered
+   * @param clusterResources the resources available in the cluster
+   *
+   * @return the resources allocated to the specified {@link Plan}
    */
-  protected abstract Resource getPlanResources(
-      Plan plan, Queue queue, Resource clusterResources);
+  protected abstract Resource getPlanResources(Plan plan, Queue queue,
+      Resource clusterResources);
 
   /**
    * Get reservation queue resources if it exists otherwise return null.
+   *
+   * @param plan the current {@link Plan} being considered
+   * @param reservationId the identifier of the reservation
+   *
+   * @return the resources allocated to the specified reservation
    */
   protected abstract Resource getReservationQueueResourceIfExists(Plan plan,
       ReservationId reservationId);
 
-  private static class ReservationAllocationComparator implements
-      Comparator<ReservationAllocation> {
+  private static class ReservationAllocationComparator
+      implements Comparator<ReservationAllocation> {
     AbstractSchedulerPlanFollower planFollower;
     long now;
     Plan plan;
@@ -404,14 +436,12 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
     private Resource getUnallocatedReservedResources(
         ReservationAllocation reservation) {
       Resource resResource;
-      Resource reservationResource = planFollower
-          .getReservationQueueResourceIfExists
-              (plan, reservation.getReservationId());
+      Resource reservationResource =
+          planFollower.getReservationQueueResourceIfExists(plan,
+              reservation.getReservationId());
       if (reservationResource != null) {
-        resResource =
-            Resources.subtract(
-                reservation.getResourcesAtTime(now),
-                reservationResource);
+        resResource = Resources.subtract(reservation.getResourcesAtTime(now),
+            reservationResource);
       } else {
         resResource = reservation.getResourcesAtTime(now);
       }
@@ -428,4 +458,3 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
     }
   }
 }
-

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryPlan.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/reservation/InMemoryPlan.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryPlan.java
index 783fd09..9eb1820 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryPlan.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryPlan.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
  * 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.
@@ -33,9 +33,10 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.server.resourcemanager.reservation.RLESparseResourceAllocation.RLEOperator;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.RLESparseResourceAllocation.RLEOperator;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.planning.Planner;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.planning.ReservationAgent;
@@ -64,9 +65,14 @@ public class InMemoryPlan implements Plan {
 
   private RLESparseResourceAllocation rleSparseVector;
 
+  private PeriodicRLESparseResourceAllocation periodicRle;
+
   private Map<String, RLESparseResourceAllocation> userResourceAlloc =
       new HashMap<String, RLESparseResourceAllocation>();
 
+  private Map<String, RLESparseResourceAllocation> userPeriodicResourceAlloc =
+      new HashMap<String, RLESparseResourceAllocation>();
+
   private Map<String, RLESparseResourceAllocation> userActiveReservationCount =
       new HashMap<String, RLESparseResourceAllocation>();
 
@@ -96,15 +102,27 @@ public class InMemoryPlan implements Plan {
       String queueName, Planner replanner, boolean getMoveOnExpiry,
       RMContext rmContext) {
     this(queueMetrics, policy, agent, totalCapacity, step, resCalc, minAlloc,
-        maxAlloc, queueName, replanner, getMoveOnExpiry, rmContext,
-        new UTCClock());
+        maxAlloc, queueName, replanner, getMoveOnExpiry,
+        YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY,
+        rmContext);
   }
 
   public InMemoryPlan(QueueMetrics queueMetrics, SharingPolicy policy,
       ReservationAgent agent, Resource totalCapacity, long step,
       ResourceCalculator resCalc, Resource minAlloc, Resource maxAlloc,
       String queueName, Planner replanner, boolean getMoveOnExpiry,
-      RMContext rmContext, Clock clock) {
+      long maxPeriodicty, RMContext rmContext) {
+    this(queueMetrics, policy, agent, totalCapacity, step, resCalc, minAlloc,
+        maxAlloc, queueName, replanner, getMoveOnExpiry, maxPeriodicty,
+        rmContext, new UTCClock());
+  }
+
+  @SuppressWarnings("checkstyle:parameternumber")
+  public InMemoryPlan(QueueMetrics queueMetrics, SharingPolicy policy,
+      ReservationAgent agent, Resource totalCapacity, long step,
+      ResourceCalculator resCalc, Resource minAlloc, Resource maxAlloc,
+      String queueName, Planner replanner, boolean getMoveOnExpiry,
+      long maxPeriodicty, RMContext rmContext, Clock clock) {
     this.queueMetrics = queueMetrics;
     this.policy = policy;
     this.agent = agent;
@@ -114,6 +132,8 @@ public class InMemoryPlan implements Plan {
     this.minAlloc = minAlloc;
     this.maxAlloc = maxAlloc;
     this.rleSparseVector = new RLESparseResourceAllocation(resCalc);
+    this.periodicRle =
+        new PeriodicRLESparseResourceAllocation(resCalc, maxPeriodicty);
     this.queueName = queueName;
     this.replanner = replanner;
     this.getMoveOnExpiry = getMoveOnExpiry;
@@ -126,6 +146,39 @@ public class InMemoryPlan implements Plan {
     return queueMetrics;
   }
 
+  private RLESparseResourceAllocation getUserRLEResourceAllocation(String user,
+      long period) {
+    RLESparseResourceAllocation resAlloc = null;
+    if (period > 0) {
+      if (userPeriodicResourceAlloc.containsKey(user)) {
+        resAlloc = userPeriodicResourceAlloc.get(user);
+      } else {
+        resAlloc = new PeriodicRLESparseResourceAllocation(resCalc,
+            periodicRle.getTimePeriod());
+        userPeriodicResourceAlloc.put(user, resAlloc);
+      }
+    } else {
+      if (userResourceAlloc.containsKey(user)) {
+        resAlloc = userResourceAlloc.get(user);
+      } else {
+        resAlloc = new RLESparseResourceAllocation(resCalc);
+        userResourceAlloc.put(user, resAlloc);
+      }
+    }
+    return resAlloc;
+  }
+
+  private void gcUserRLEResourceAllocation(String user, long period) {
+    if (period > 0) {
+      if (userPeriodicResourceAlloc.get(user).isEmpty()) {
+        userPeriodicResourceAlloc.remove(user);
+      }
+    } else {
+      if (userResourceAlloc.get(user).isEmpty()) {
+        userResourceAlloc.remove(user);
+      }
+    }
+  }
 
   private void incrementAllocation(ReservationAllocation reservation) {
     assert (readWriteLock.isWriteLockedByCurrentThread());
@@ -133,11 +186,10 @@ public class InMemoryPlan implements Plan {
         reservation.getAllocationRequests();
     // check if we have encountered the user earlier and if not add an entry
     String user = reservation.getUser();
-    RLESparseResourceAllocation resAlloc = userResourceAlloc.get(user);
-    if (resAlloc == null) {
-      resAlloc = new RLESparseResourceAllocation(resCalc);
-      userResourceAlloc.put(user, resAlloc);
-    }
+    long period = reservation.getPeriodicity();
+    RLESparseResourceAllocation resAlloc =
+        getUserRLEResourceAllocation(user, period);
+
     RLESparseResourceAllocation resCount = userActiveReservationCount.get(user);
     if (resCount == null) {
       resCount = new RLESparseResourceAllocation(resCalc);
@@ -149,14 +201,43 @@ public class InMemoryPlan implements Plan {
 
     for (Map.Entry<ReservationInterval, Resource> r : allocationRequests
         .entrySet()) {
-      resAlloc.addInterval(r.getKey(), r.getValue());
-      rleSparseVector.addInterval(r.getKey(), r.getValue());
-      if (Resources.greaterThan(resCalc, totalCapacity, r.getValue(),
-          ZERO_RESOURCE)) {
-        earliestActive = Math.min(earliestActive, r.getKey().getStartTime());
-        latestActive = Math.max(latestActive, r.getKey().getEndTime());
+
+      if (period > 0L) {
+        for (int i = 0; i < periodicRle.getTimePeriod() / period; i++) {
+
+          long rStart = r.getKey().getStartTime() + i * period;
+          long rEnd = r.getKey().getEndTime() + i * period;
+
+          // handle wrap-around
+          if (rEnd > periodicRle.getTimePeriod()) {
+            long diff = rEnd - periodicRle.getTimePeriod();
+            rEnd = periodicRle.getTimePeriod();
+            ReservationInterval newInterval = new ReservationInterval(0, diff);
+            periodicRle.addInterval(newInterval, r.getValue());
+            resAlloc.addInterval(newInterval, r.getValue());
+          }
+
+          ReservationInterval newInterval =
+              new ReservationInterval(rStart, rEnd);
+          periodicRle.addInterval(newInterval, r.getValue());
+          resAlloc.addInterval(newInterval, r.getValue());
+        }
+
+      } else {
+        rleSparseVector.addInterval(r.getKey(), r.getValue());
+        resAlloc.addInterval(r.getKey(), r.getValue());
+        if (Resources.greaterThan(resCalc, totalCapacity, r.getValue(),
+            ZERO_RESOURCE)) {
+          earliestActive = Math.min(earliestActive, r.getKey().getStartTime());
+          latestActive = Math.max(latestActive, r.getKey().getEndTime());
+        }
       }
     }
+    // periodic reservations are active from start time and good till cancelled
+    if (period > 0L) {
+      earliestActive = reservation.getStartTime();
+      latestActive = Long.MAX_VALUE;
+    }
     resCount.addInterval(new ReservationInterval(earliestActive, latestActive),
         Resource.newInstance(1, 1));
   }
@@ -166,27 +247,55 @@ public class InMemoryPlan implements Plan {
     Map<ReservationInterval, Resource> allocationRequests =
         reservation.getAllocationRequests();
     String user = reservation.getUser();
-    RLESparseResourceAllocation resAlloc = userResourceAlloc.get(user);
+    long period = reservation.getPeriodicity();
+    RLESparseResourceAllocation resAlloc =
+        getUserRLEResourceAllocation(user, period);
 
     long earliestActive = Long.MAX_VALUE;
     long latestActive = Long.MIN_VALUE;
     for (Map.Entry<ReservationInterval, Resource> r : allocationRequests
         .entrySet()) {
-      resAlloc.removeInterval(r.getKey(), r.getValue());
-      rleSparseVector.removeInterval(r.getKey(), r.getValue());
-      if (Resources.greaterThan(resCalc, totalCapacity, r.getValue(),
-          ZERO_RESOURCE)) {
-        earliestActive = Math.min(earliestActive, r.getKey().getStartTime());
-        latestActive = Math.max(latestActive, r.getKey().getEndTime());
+      if (period > 0L) {
+        for (int i = 0; i < periodicRle.getTimePeriod() / period; i++) {
+
+          long rStart = r.getKey().getStartTime() + i * period;
+          long rEnd = r.getKey().getEndTime() + i * period;
+
+          // handle wrap-around
+          if (rEnd > periodicRle.getTimePeriod()) {
+            long diff = rEnd - periodicRle.getTimePeriod();
+            rEnd = periodicRle.getTimePeriod();
+            ReservationInterval newInterval = new ReservationInterval(0, diff);
+            periodicRle.removeInterval(newInterval, r.getValue());
+            resAlloc.removeInterval(newInterval, r.getValue());
+          }
+
+          ReservationInterval newInterval =
+              new ReservationInterval(rStart, rEnd);
+          periodicRle.removeInterval(newInterval, r.getValue());
+          resAlloc.removeInterval(newInterval, r.getValue());
+        }
+      } else {
+        rleSparseVector.removeInterval(r.getKey(), r.getValue());
+        resAlloc.removeInterval(r.getKey(), r.getValue());
+        if (Resources.greaterThan(resCalc, totalCapacity, r.getValue(),
+            ZERO_RESOURCE)) {
+          earliestActive = Math.min(earliestActive, r.getKey().getStartTime());
+          latestActive = Math.max(latestActive, r.getKey().getEndTime());
+        }
       }
     }
-    if (resAlloc.isEmpty()) {
-      userResourceAlloc.remove(user);
-    }
+    gcUserRLEResourceAllocation(user, period);
 
     RLESparseResourceAllocation resCount = userActiveReservationCount.get(user);
-    resCount.removeInterval(new ReservationInterval(earliestActive,
-        latestActive), Resource.newInstance(1, 1));
+    // periodic reservations are active from start time and good till cancelled
+    if (period > 0L) {
+      earliestActive = reservation.getStartTime();
+      latestActive = Long.MAX_VALUE;
+    }
+    resCount.removeInterval(
+        new ReservationInterval(earliestActive, latestActive),
+        Resource.newInstance(1, 1));
     if (resCount.isEmpty()) {
       userActiveReservationCount.remove(user);
     }
@@ -198,9 +307,9 @@ public class InMemoryPlan implements Plan {
       if (currentReservations != null) {
         Set<ReservationAllocation> flattenedReservations =
             new TreeSet<ReservationAllocation>();
-        for (Set<InMemoryReservationAllocation> reservationEntries :
-            currentReservations.values()) {
-          flattenedReservations.addAll(reservationEntries);
+        for (Set<InMemoryReservationAllocation> res : currentReservations
+            .values()) {
+          flattenedReservations.addAll(res);
         }
         return flattenedReservations;
       } else {
@@ -218,19 +327,16 @@ public class InMemoryPlan implements Plan {
     InMemoryReservationAllocation inMemReservation =
         (InMemoryReservationAllocation) reservation;
     if (inMemReservation.getUser() == null) {
-      String errMsg =
-          "The specified Reservation with ID "
-              + inMemReservation.getReservationId()
-              + " is not mapped to any user";
+      String errMsg = "The specified Reservation with ID "
+          + inMemReservation.getReservationId() + " is not mapped to any user";
       LOG.error(errMsg);
       throw new IllegalArgumentException(errMsg);
     }
     writeLock.lock();
     try {
       if (reservationTable.containsKey(inMemReservation.getReservationId())) {
-        String errMsg =
-            "The specified Reservation with ID "
-                + inMemReservation.getReservationId() + " already exists";
+        String errMsg = "The specified Reservation with ID "
+            + inMemReservation.getReservationId() + " already exists";
         LOG.error(errMsg);
         throw new IllegalArgumentException(errMsg);
       }
@@ -246,9 +352,8 @@ public class InMemoryPlan implements Plan {
               getQueueName(), inMemReservation.getReservationId().toString());
         }
       }
-      ReservationInterval searchInterval =
-          new ReservationInterval(inMemReservation.getStartTime(),
-              inMemReservation.getEndTime());
+      ReservationInterval searchInterval = new ReservationInterval(
+          inMemReservation.getStartTime(), inMemReservation.getEndTime());
       Set<InMemoryReservationAllocation> reservations =
           currentReservations.get(searchInterval);
       if (reservations == null) {
@@ -280,9 +385,8 @@ public class InMemoryPlan implements Plan {
       ReservationId resId = reservation.getReservationId();
       ReservationAllocation currReservation = getReservationById(resId);
       if (currReservation == null) {
-        String errMsg =
-            "The specified Reservation with ID " + resId
-                + " does not exist in the plan";
+        String errMsg = "The specified Reservation with ID " + resId
+            + " does not exist in the plan";
         LOG.error(errMsg);
         throw new IllegalArgumentException(errMsg);
       }
@@ -318,9 +422,8 @@ public class InMemoryPlan implements Plan {
 
   private boolean removeReservation(ReservationAllocation reservation) {
     assert (readWriteLock.isWriteLockedByCurrentThread());
-    ReservationInterval searchInterval =
-        new ReservationInterval(reservation.getStartTime(),
-            reservation.getEndTime());
+    ReservationInterval searchInterval = new ReservationInterval(
+        reservation.getStartTime(), reservation.getEndTime());
     Set<InMemoryReservationAllocation> reservations =
         currentReservations.get(searchInterval);
     if (reservations != null) {
@@ -337,16 +440,15 @@ public class InMemoryPlan implements Plan {
         currentReservations.remove(searchInterval);
       }
     } else {
-      String errMsg =
-          "The specified Reservation with ID " + reservation.getReservationId()
-              + " does not exist in the plan";
+      String errMsg = "The specified Reservation with ID "
+          + reservation.getReservationId() + " does not exist in the plan";
       LOG.error(errMsg);
       throw new IllegalArgumentException(errMsg);
     }
     reservationTable.remove(reservation.getReservationId());
     decrementAllocation(reservation);
     LOG.info("Sucessfully deleted reservation: {} in plan.",
-            reservation.getReservationId());
+        reservation.getReservationId());
     return true;
   }
 
@@ -356,9 +458,8 @@ public class InMemoryPlan implements Plan {
     try {
       ReservationAllocation reservation = getReservationById(reservationID);
       if (reservation == null) {
-        String errMsg =
-            "The specified Reservation with ID " + reservationID
-                + " does not exist in the plan";
+        String errMsg = "The specified Reservation with ID " + reservationID
+            + " does not exist in the plan";
         LOG.error(errMsg);
         throw new IllegalArgumentException(errMsg);
       }
@@ -453,66 +554,90 @@ public class InMemoryPlan implements Plan {
       long start, long end) {
     readLock.lock();
     try {
+      // merge periodic and non-periodic allocations
       RLESparseResourceAllocation userResAlloc = userResourceAlloc.get(user);
+      RLESparseResourceAllocation userPeriodicResAlloc =
+          userPeriodicResourceAlloc.get(user);
 
+      if (userResAlloc != null && userPeriodicResAlloc != null) {
+        return RLESparseResourceAllocation.merge(resCalc, totalCapacity,
+            userResAlloc, userPeriodicResAlloc, RLEOperator.add, start, end);
+      }
       if (userResAlloc != null) {
         return userResAlloc.getRangeOverlapping(start, end);
-      } else {
-        return new RLESparseResourceAllocation(resCalc);
       }
+      if (userPeriodicResAlloc != null) {
+        return userPeriodicResAlloc.getRangeOverlapping(start, end);
+      }
+    } catch (PlanningException e) {
+      LOG.warn("Exception while trying to merge periodic"
+          + " and non-periodic user allocations: {}", e.getMessage(), e);
     } finally {
       readLock.unlock();
     }
+    return new RLESparseResourceAllocation(resCalc);
   }
 
   @Override
   public Resource getTotalCommittedResources(long t) {
     readLock.lock();
     try {
-      return rleSparseVector.getCapacityAtTime(t);
+      return Resources.add(rleSparseVector.getCapacityAtTime(t),
+          periodicRle.getCapacityAtTime(t));
     } finally {
       readLock.unlock();
     }
   }
 
   @Override
-  public Set<ReservationAllocation> getReservations(ReservationId
-                    reservationID, ReservationInterval interval) {
+  public Set<ReservationAllocation> getReservations(ReservationId reservationID,
+      ReservationInterval interval) {
     return getReservations(reservationID, interval, null);
   }
 
   @Override
-  public Set<ReservationAllocation> getReservations(ReservationId
-                    reservationID, ReservationInterval interval, String user) {
+  public Set<ReservationAllocation> getReservations(ReservationId reservationID,
+      ReservationInterval interval, String user) {
     if (reservationID != null) {
       ReservationAllocation allocation = getReservationById(reservationID);
-      if (allocation == null){
+      if (allocation == null) {
         return Collections.emptySet();
       }
       return Collections.singleton(allocation);
     }
 
-    long startTime = interval == null? 0 : interval.getStartTime();
-    long endTime = interval == null? Long.MAX_VALUE : interval.getEndTime();
+    long startTime = interval == null ? 0 : interval.getStartTime();
+    long endTime = interval == null ? Long.MAX_VALUE : interval.getEndTime();
 
     ReservationInterval searchInterval =
-            new ReservationInterval(endTime, Long.MAX_VALUE);
+        new ReservationInterval(endTime, Long.MAX_VALUE);
     readLock.lock();
     try {
-      SortedMap<ReservationInterval, Set<InMemoryReservationAllocation>>
-            reservations = currentReservations.headMap(searchInterval, true);
-      if (!reservations.isEmpty()) {
-        Set<ReservationAllocation> flattenedReservations =
-                new HashSet<>();
-        for (Set<InMemoryReservationAllocation> reservationEntries :
-                reservations.values()) {
-          for (InMemoryReservationAllocation res : reservationEntries) {
-            if (res.getEndTime() > startTime) {
-              if (user != null && !user.isEmpty()
-                      && !res.getUser().equals(user)) {
-                continue;
+      SortedMap<ReservationInterval, Set<InMemoryReservationAllocation>> res =
+          currentReservations.headMap(searchInterval, true);
+      if (!res.isEmpty()) {
+        Set<ReservationAllocation> flattenedReservations = new HashSet<>();
+        for (Set<InMemoryReservationAllocation> resEntries : res.values()) {
+          for (InMemoryReservationAllocation reservation : resEntries) {
+            // validate user
+            if (user != null && !user.isEmpty()
+                && !reservation.getUser().equals(user)) {
+              continue;
+            }
+            // handle periodic reservations
+            long period = reservation.getPeriodicity();
+            if (period > 0) {
+              long t = endTime % period;
+              // check for both contained and wrap-around reservations
+              if ((t - startTime) * (t - endTime)
+                  * (startTime - endTime) >= 0) {
+                flattenedReservations.add(reservation);
+              }
+            } else {
+              // check for non-periodic reservations
+              if (reservation.getEndTime() > startTime) {
+                flattenedReservations.add(reservation);
               }
-              flattenedReservations.add(res);
             }
           }
         }
@@ -550,36 +675,82 @@ public class InMemoryPlan implements Plan {
 
   @Override
   public RLESparseResourceAllocation getAvailableResourceOverTime(String user,
-      ReservationId oldId, long start, long end) throws PlanningException {
+      ReservationId oldId, long start, long end, long period)
+      throws PlanningException {
     readLock.lock();
     try {
-      // create RLE of totCapacity
-      TreeMap<Long, Resource> totAvailable = new TreeMap<Long, Resource>();
-      totAvailable.put(start, Resources.clone(totalCapacity));
-      RLESparseResourceAllocation totRLEAvail =
-          new RLESparseResourceAllocation(totAvailable, resCalc);
-
-      // subtract used from available
-      RLESparseResourceAllocation netAvailable;
-
-      netAvailable =
-          RLESparseResourceAllocation.merge(resCalc,
-              Resources.clone(totalCapacity), totRLEAvail, rleSparseVector,
-              RLEOperator.subtractTestNonNegative, start, end);
-
-      // add back in old reservation used resources if any
-      ReservationAllocation old = reservationTable.get(oldId);
-      if (old != null) {
-        netAvailable =
-            RLESparseResourceAllocation.merge(resCalc,
-                Resources.clone(totalCapacity), netAvailable,
-                old.getResourcesOverTime(), RLEOperator.add, start, end);
+
+      // for non-periodic return simple available resources
+      if (period == 0) {
+
+        // create RLE of totCapacity
+        TreeMap<Long, Resource> totAvailable = new TreeMap<Long, Resource>();
+        totAvailable.put(start, Resources.clone(totalCapacity));
+        RLESparseResourceAllocation totRLEAvail =
+            new RLESparseResourceAllocation(totAvailable, resCalc);
+
+        // subtract used from available
+        RLESparseResourceAllocation netAvailable;
+
+        netAvailable = RLESparseResourceAllocation.merge(resCalc,
+            Resources.clone(totalCapacity), totRLEAvail, rleSparseVector,
+            RLEOperator.subtractTestNonNegative, start, end);
+
+        // remove periodic component
+        netAvailable = RLESparseResourceAllocation.merge(resCalc,
+            Resources.clone(totalCapacity), netAvailable, periodicRle,
+            RLEOperator.subtractTestNonNegative, start, end);
+
+        // add back in old reservation used resources if any
+        ReservationAllocation old = reservationTable.get(oldId);
+        if (old != null) {
+
+          RLESparseResourceAllocation addBackPrevious =
+              old.getResourcesOverTime(start, end);
+          netAvailable = RLESparseResourceAllocation.merge(resCalc,
+              Resources.clone(totalCapacity), netAvailable, addBackPrevious,
+              RLEOperator.add, start, end);
+        }
+        // lower it if this is needed by the sharing policy
+        netAvailable = getSharingPolicy().availableResources(netAvailable, this,
+            user, oldId, start, end);
+        return netAvailable;
+      } else {
+
+        if (periodicRle.getTimePeriod() % period != 0) {
+          throw new PlanningException("The reservation periodicity (" + period
+              + ") must be" + "an exact divider of the system maxPeriod ("
+              + periodicRle.getTimePeriod() + ")");
+        }
+
+        // find the minimum resources available among all the instances that fit
+        // in the LCM
+        long numInstInLCM = periodicRle.getTimePeriod() / period;
+
+        RLESparseResourceAllocation minOverLCM =
+            getAvailableResourceOverTime(user, oldId, start, end, 0);
+        for (int i = 1; i < numInstInLCM; i++) {
+
+          long rStart = start + i * period;
+          long rEnd = end + i * period;
+
+          // recursive invocation of non-periodic range (to pick raw-info)
+          RLESparseResourceAllocation snapShot =
+              getAvailableResourceOverTime(user, oldId, rStart, rEnd, 0);
+
+          // time-align on start
+          snapShot.shift(-(i * period));
+
+          // pick the minimum amount of resources in each time interval
+          minOverLCM =
+              RLESparseResourceAllocation.merge(resCalc, getTotalCapacity(),
+                  minOverLCM, snapShot, RLEOperator.min, start, end);
+
+        }
+
+        return minOverLCM;
+
       }
-      // lower it if this is needed by the sharing policy
-      netAvailable =
-          getSharingPolicy().availableResources(netAvailable, this, user,
-              oldId, start, end);
-      return netAvailable;
     } finally {
       readLock.unlock();
     }
@@ -637,7 +808,7 @@ public class InMemoryPlan implements Plan {
   public String toCumulativeString() {
     readLock.lock();
     try {
-      return rleSparseVector.toString();
+      return rleSparseVector.toString() + "\n" + periodicRle.toString();
     } finally {
       readLock.unlock();
     }
@@ -689,11 +860,18 @@ public class InMemoryPlan implements Plan {
   }
 
   @Override
-  public RLESparseResourceAllocation getCumulativeLoadOverTime(
-      long start, long end) {
+  public RLESparseResourceAllocation getCumulativeLoadOverTime(long start,
+      long end) throws PlanningException {
     readLock.lock();
     try {
-      return rleSparseVector.getRangeOverlapping(start, end);
+
+      RLESparseResourceAllocation ret =
+          rleSparseVector.getRangeOverlapping(start, end);
+      ret = RLESparseResourceAllocation.merge(resCalc, totalCapacity, ret,
+          periodicRle.getRangeOverlapping(start, end), RLEOperator.add, start,
+          end);
+
+      return ret;
     } finally {
       readLock.unlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryReservationAllocation.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/reservation/InMemoryReservationAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryReservationAllocation.java
index 69fd43f..00c8e44 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryReservationAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryReservationAllocation.java
@@ -42,6 +42,7 @@ public class InMemoryReservationAllocation implements ReservationAllocation {
   private final Map<ReservationInterval, Resource> allocationRequests;
   private boolean hasGang = false;
   private long acceptedAt = -1;
+  private long periodicity = 0;
 
   private RLESparseResourceAllocation resourcesOverTime;
 
@@ -67,9 +68,16 @@ public class InMemoryReservationAllocation implements ReservationAllocation {
     this.allocationRequests = allocations;
     this.planName = planName;
     this.hasGang = hasGang;
-    resourcesOverTime = new RLESparseResourceAllocation(calculator);
-    for (Map.Entry<ReservationInterval, Resource> r : allocations
-        .entrySet()) {
+    if (contract != null && contract.getRecurrenceExpression() != null) {
+      this.periodicity = Long.parseLong(contract.getRecurrenceExpression());
+    }
+    if (periodicity > 0) {
+      resourcesOverTime =
+          new PeriodicRLESparseResourceAllocation(calculator, periodicity);
+    } else {
+      resourcesOverTime = new RLESparseResourceAllocation(calculator);
+    }
+    for (Map.Entry<ReservationInterval, Resource> r : allocations.entrySet()) {
       resourcesOverTime.addInterval(r.getKey(), r.getValue());
     }
   }
@@ -133,17 +141,33 @@ public class InMemoryReservationAllocation implements ReservationAllocation {
   }
 
   @Override
-  public RLESparseResourceAllocation getResourcesOverTime(){
+  public RLESparseResourceAllocation getResourcesOverTime() {
     return resourcesOverTime;
   }
 
   @Override
+  public RLESparseResourceAllocation getResourcesOverTime(long start,
+      long end) {
+    return resourcesOverTime.getRangeOverlapping(start, end);
+  }
+
+  @Override
+  public long getPeriodicity() {
+    return periodicity;
+  }
+
+  @Override
+  public void setPeriodicity(long period) {
+    periodicity = period;
+  }
+
+  @Override
   public String toString() {
     StringBuilder sBuf = new StringBuilder();
     sBuf.append(getReservationId()).append(" user:").append(getUser())
         .append(" startTime: ").append(getStartTime()).append(" endTime: ")
-        .append(getEndTime()).append(" alloc:\n[")
-        .append(resourcesOverTime.toString()).append("] ");
+        .append(getEndTime()).append(" Periodiciy: ").append(periodicity)
+        .append(" alloc:\n[").append(resourcesOverTime.toString()).append("] ");
     return sBuf.toString();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/NoOverCommitPolicy.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/reservation/NoOverCommitPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/NoOverCommitPolicy.java
index 55f1d00..49d4702 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/NoOverCommitPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/NoOverCommitPolicy.java
@@ -40,7 +40,7 @@ public class NoOverCommitPolicy implements SharingPolicy {
 
     RLESparseResourceAllocation available = plan.getAvailableResourceOverTime(
         reservation.getUser(), reservation.getReservationId(),
-        reservation.getStartTime(), reservation.getEndTime());
+        reservation.getStartTime(), reservation.getEndTime(), 0);
 
     // test the reservation does not exceed what is available
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PeriodicRLESparseResourceAllocation.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/reservation/PeriodicRLESparseResourceAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PeriodicRLESparseResourceAllocation.java
index 8e3be8b3..7bc44f5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PeriodicRLESparseResourceAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PeriodicRLESparseResourceAllocation.java
@@ -18,47 +18,94 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
- * This data structure stores a periodic RLESparseResourceAllocation.
+ * This data structure stores a periodic {@link RLESparseResourceAllocation}.
  * Default period is 1 day (86400000ms).
  */
-public class PeriodicRLESparseResourceAllocation extends
-    RLESparseResourceAllocation {
+public class PeriodicRLESparseResourceAllocation
+    extends RLESparseResourceAllocation {
 
   // Log
-  private static final Logger LOG = LoggerFactory
-      .getLogger(PeriodicRLESparseResourceAllocation.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(PeriodicRLESparseResourceAllocation.class);
 
   private long timePeriod;
 
   /**
    * Constructor.
    *
-   * @param rleVector {@link RLESparseResourceAllocation} with the run-length
-              encoded data.
+   * @param resourceCalculator {@link ResourceCalculator} the resource
+   *          calculator to use.
    * @param timePeriod Time period in milliseconds.
    */
   public PeriodicRLESparseResourceAllocation(
-      RLESparseResourceAllocation rleVector, Long timePeriod) {
-    super(rleVector.getCumulative(), rleVector.getResourceCalculator());
+      ResourceCalculator resourceCalculator, Long timePeriod) {
+    super(resourceCalculator);
     this.timePeriod = timePeriod;
   }
 
   /**
    * Constructor. Default time period set to 1 day.
    *
+   * @param resourceCalculator {@link ResourceCalculator} the resource
+   *          calculator to use..
+   */
+  public PeriodicRLESparseResourceAllocation(
+      ResourceCalculator resourceCalculator) {
+    this(resourceCalculator,
+        YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY);
+  }
+
+  /**
+   * Constructor.
+   *
    * @param rleVector {@link RLESparseResourceAllocation} with the run-length
-              encoded data.
+   *          encoded data.
+   * @param timePeriod Time period in milliseconds.
    */
+  @VisibleForTesting
   public PeriodicRLESparseResourceAllocation(
-      RLESparseResourceAllocation rleVector) {
-    this(rleVector, 86400000L);
+      RLESparseResourceAllocation rleVector, Long timePeriod) {
+    super(rleVector.getCumulative(), rleVector.getResourceCalculator());
+    this.timePeriod = timePeriod;
+
+    // make sure the PeriodicRLE is zero-based, and handles wrap-around
+    long delta = (getEarliestStartTime() % timePeriod - getEarliestStartTime());
+    shift(delta);
+
+    List<Long> toRemove = new ArrayList<>();
+    Map<Long, Resource> toAdd = new TreeMap<>();
+
+    for (Map.Entry<Long, Resource> entry : cumulativeCapacity.entrySet()) {
+      if (entry.getKey() > timePeriod) {
+        toRemove.add(entry.getKey());
+        if (entry.getValue() != null) {
+          toAdd.put(timePeriod, entry.getValue());
+          long prev = entry.getKey() % timePeriod;
+          toAdd.put(prev, this.getCapacityAtTime(prev));
+          toAdd.put(0L, entry.getValue());
+        }
+      }
+    }
+    for (Long l : toRemove) {
+      cumulativeCapacity.remove(l);
+    }
+    cumulativeCapacity.putAll(toAdd);
   }
 
   /**
@@ -78,24 +125,25 @@ public class PeriodicRLESparseResourceAllocation extends
    * The interval may include 0, but the end time must be strictly less than
    * timePeriod.
    *
-   * @param interval {@link ReservationInterval} to which the specified
-   *          resource is to be added.
+   * @param interval {@link ReservationInterval} to which the specified resource
+   *          is to be added.
    * @param resource {@link Resource} to be added to the interval specified.
    * @return true if addition is successful, false otherwise
    */
-  public boolean addInterval(ReservationInterval interval,
-      Resource resource) {
+  public boolean addInterval(ReservationInterval interval, Resource resource) {
     long startTime = interval.getStartTime();
     long endTime = interval.getEndTime();
+
     if (startTime >= 0 && endTime > startTime && endTime <= timePeriod) {
       return super.addInterval(interval, resource);
     } else {
-      LOG.info("Cannot set capacity beyond end time: " + timePeriod);
+      LOG.info("Cannot set capacity beyond end time: " + timePeriod + " was ("
+          + interval.toString() + ")");
       return false;
     }
   }
 
-   /**
+  /**
    * Removes a resource for the specified interval.
    *
    * @param interval the {@link ReservationInterval} for which the resource is
@@ -103,14 +151,15 @@ public class PeriodicRLESparseResourceAllocation extends
    * @param resource the {@link Resource} to be removed.
    * @return true if removal is successful, false otherwise
    */
-  public boolean removeInterval(
-      ReservationInterval interval, Resource resource) {
+  public boolean removeInterval(ReservationInterval interval,
+      Resource resource) {
     long startTime = interval.getStartTime();
     long endTime = interval.getEndTime();
     // If the resource to be subtracted is less than the minimum resource in
     // the range, abort removal to avoid negative capacity.
-    if (!Resources.fitsIn(
-        resource, super.getMinimumCapacityInInterval(interval))) {
+    // TODO revesit decrementing endTime
+    if (!Resources.fitsIn(resource, getMinimumCapacityInInterval(
+        new ReservationInterval(startTime, endTime - 1)))) {
       LOG.info("Request to remove more resources than what is available");
       return false;
     }
@@ -125,17 +174,16 @@ public class PeriodicRLESparseResourceAllocation extends
   /**
    * Get maximum capacity at periodic offsets from the specified time.
    *
-   * @param tick UTC time base from which offsets are specified for finding
-   *          the maximum capacity.
-   * @param period periodic offset at which capacities are evaluted.
+   * @param tick UTC time base from which offsets are specified for finding the
+   *          maximum capacity.
+   * @param period periodic offset at which capacities are evaluated.
    * @return the maximum {@link Resource} across the specified time instants.
    * @return true if removal is successful, false otherwise
    */
   public Resource getMaximumPeriodicCapacity(long tick, long period) {
     Resource maxResource;
     if (period < timePeriod) {
-      maxResource =
-          super.getMaximumPeriodicCapacity(tick % timePeriod, period);
+      maxResource = super.getMaximumPeriodicCapacity(tick % timePeriod, period);
     } else {
       // if period is greater than the length of PeriodicRLESparseAllocation,
       // only a single value exists in this interval.
@@ -164,4 +212,30 @@ public class PeriodicRLESparseResourceAllocation extends
     return ret.toString();
   }
 
+  @Override
+  public RLESparseResourceAllocation getRangeOverlapping(long start, long end) {
+    NavigableMap<Long, Resource> unrolledMap = new TreeMap<>();
+    readLock.lock();
+    try {
+      long relativeStart = (start >= 0) ? start % timePeriod : 0;
+      NavigableMap<Long, Resource> cumulativeMap = this.getCumulative();
+      Long previous = cumulativeMap.floorKey(relativeStart);
+      previous = (previous != null) ? previous : 0;
+      for (long i = 0; i <= (end - start) / timePeriod; i++) {
+        for (Map.Entry<Long, Resource> e : cumulativeMap.entrySet()) {
+          long curKey = e.getKey() + (i * timePeriod);
+          if (curKey >= previous && (start + curKey - relativeStart) <= end) {
+            unrolledMap.put(curKey, e.getValue());
+          }
+        }
+      }
+      RLESparseResourceAllocation rle =
+          new RLESparseResourceAllocation(unrolledMap, getResourceCalculator());
+      rle.shift(start - relativeStart);
+      return rle;
+    } finally {
+      readLock.unlock();
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanEdit.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/reservation/PlanEdit.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanEdit.java
index 504a250..9afa324 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanEdit.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanEdit.java
@@ -28,54 +28,58 @@ import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.Plan
 public interface PlanEdit extends PlanContext, PlanView {
 
   /**
-   * Add a new {@link ReservationAllocation} to the plan
+   * Add a new {@link ReservationAllocation} to the plan.
    * 
    * @param reservation the {@link ReservationAllocation} to be added to the
    *          plan
    * @param isRecovering flag to indicate if reservation is being added as part
    *          of failover or not
    * @return true if addition is successful, false otherwise
+   * @throws PlanningException if addition is unsuccessful
    */
-  public boolean addReservation(ReservationAllocation reservation,
+  boolean addReservation(ReservationAllocation reservation,
       boolean isRecovering) throws PlanningException;
 
   /**
    * Updates an existing {@link ReservationAllocation} in the plan. This is
-   * required for re-negotiation
+   * required for re-negotiation.
    * 
    * @param reservation the {@link ReservationAllocation} to be updated the plan
    * @return true if update is successful, false otherwise
+   * @throws PlanningException if update is unsuccessful
    */
-  public boolean updateReservation(ReservationAllocation reservation)
+  boolean updateReservation(ReservationAllocation reservation)
       throws PlanningException;
 
   /**
    * Delete an existing {@link ReservationAllocation} from the plan identified
    * uniquely by its {@link ReservationId}. This will generally be used for
-   * garbage collection
+   * garbage collection.
    * 
    * @param reservationID the {@link ReservationAllocation} to be deleted from
    *          the plan identified uniquely by its {@link ReservationId}
    * @return true if delete is successful, false otherwise
+   * @throws PlanningException if deletion is unsuccessful
    */
-  public boolean deleteReservation(ReservationId reservationID)
+  boolean deleteReservation(ReservationId reservationID)
       throws PlanningException;
 
   /**
    * Method invoked to garbage collect old reservations. It cleans up expired
-   * reservations that have fallen out of the sliding archival window
+   * reservations that have fallen out of the sliding archival window.
    * 
    * @param tick the current time from which the archival window is computed
+   * @throws PlanningException if archival is unsuccessful
    */
-  public void archiveCompletedReservations(long tick) throws PlanningException;
+  void archiveCompletedReservations(long tick) throws PlanningException;
 
   /**
    * Sets the overall capacity in terms of {@link Resource} assigned to this
-   * plan
+   * plan.
    * 
    * @param capacity the overall capacity in terms of {@link Resource} assigned
    *          to this plan
    */
-  public void setTotalCapacity(Resource capacity);
+  void setTotalCapacity(Resource capacity);
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanView.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/reservation/PlanView.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanView.java
index 2767993..4035f68 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanView.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanView.java
@@ -17,50 +17,50 @@
  */
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
+import java.util.Set;
+
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
 
-import java.util.Set;
-
 /**
  * This interface provides a read-only view on the allocations made in this
  * plan. This methods are used for example by {@code ReservationAgent}s to
  * determine the free resources in a certain point in time, and by
  * PlanFollowerPolicy to publish this plan to the scheduler.
  */
-public interface PlanView extends PlanContext {
+interface PlanView extends PlanContext {
 
   /**
    * Return a set of {@link ReservationAllocation} identified by the user who
    * made the reservation.
    *
    * @param reservationID the unqiue id to identify the
-   * {@link ReservationAllocation}
+   *          {@link ReservationAllocation}
    * @param interval the time interval used to retrieve the reservation
-   *                 allocations from. Only reservations with start time no
-   *                 greater than the interval end time, and end time no less
-   *                 than the interval start time will be selected.
+   *          allocations from. Only reservations with start time no greater
+   *          than the interval end time, and end time no less than the interval
+   *          start time will be selected.
    * @param user the user to retrieve the reservation allocation from.
    * @return a set of {@link ReservationAllocation} identified by the user who
-   * made the reservation
+   *         made the reservation
    */
-  Set<ReservationAllocation> getReservations(ReservationId
-                    reservationID, ReservationInterval interval, String user);
+  Set<ReservationAllocation> getReservations(ReservationId reservationID,
+      ReservationInterval interval, String user);
 
   /**
    * Return a set of {@link ReservationAllocation} identified by any user.
    *
    * @param reservationID the unqiue id to identify the
-   * {@link ReservationAllocation}
+   *          {@link ReservationAllocation}
    * @param interval the time interval used to retrieve the reservation
-   *                 allocations from. Only reservations with start time no
-   *                 greater than the interval end time, and end time no less
-   *                 than the interval start time will be selected.
+   *          allocations from. Only reservations with start time no greater
+   *          than the interval end time, and end time no less than the interval
+   *          start time will be selected.
    * @return a set of {@link ReservationAllocation} identified by any user
    */
   Set<ReservationAllocation> getReservations(ReservationId reservationID,
-                    ReservationInterval interval);
+      ReservationInterval interval);
 
   /**
    * Return a {@link ReservationAllocation} identified by its
@@ -70,7 +70,7 @@ public interface PlanView extends PlanContext {
    *          {@link ReservationAllocation}
    * @return {@link ReservationAllocation} identified by the specified id
    */
-  public ReservationAllocation getReservationById(ReservationId reservationID);
+  ReservationAllocation getReservationById(ReservationId reservationID);
 
   /**
    * Return a set of {@link ReservationAllocation} that belongs to a certain
@@ -78,11 +78,10 @@ public interface PlanView extends PlanContext {
    *
    * @param user the user being considered
    * @param t the instant in time being considered
-   * @return set of active {@link ReservationAllocation}s for this
-   *         user at this time
+   * @return set of active {@link ReservationAllocation}s for this user at this
+   *         time
    */
-  public Set<ReservationAllocation> getReservationByUserAtTime(String user,
-      long t);
+  Set<ReservationAllocation> getReservationByUserAtTime(String user, long t);
 
   /**
    * Gets all the active reservations at the specified point of time
@@ -91,14 +90,14 @@ public interface PlanView extends PlanContext {
    *          requested
    * @return set of active reservations at the specified time
    */
-  public Set<ReservationAllocation> getReservationsAtTime(long tick);
+  Set<ReservationAllocation> getReservationsAtTime(long tick);
 
   /**
    * Gets all the reservations in the plan
    * 
    * @return set of all reservations handled by this Plan
    */
-  public Set<ReservationAllocation> getAllReservations();
+  Set<ReservationAllocation> getAllReservations();
 
   /**
    * Returns the total {@link Resource} reserved for all users at the specified
@@ -126,61 +125,68 @@ public interface PlanView extends PlanContext {
    * 
    * @return the time (UTC in ms) at which the first reservation starts
    */
-  public long getEarliestStartTime();
+  long getEarliestStartTime();
 
   /**
    * Returns the time (UTC in ms) at which the last reservation terminates
    *
    * @return the time (UTC in ms) at which the last reservation terminates
    */
-  public long getLastEndTime();
+  long getLastEndTime();
 
   /**
    * This method returns the amount of resources available to a given user
    * (optionally if removing a certain reservation) over the start-end time
-   * range.
+   * range. If the request is periodic (period is non-zero) we return the
+   * minimum amount of resources available to periodic reservations (in all
+   * "period" windows within the system maxPeriod / LCM).
    *
-   * @param user
-   * @param oldId
-   * @param start
-   * @param end
+   * @param user the user being considered
+   * @param oldId the identifier of the existing reservation
+   * @param start start of the time interval.
+   * @param end end of the time interval.
+   * @param period the ms periodicty for this request (loop and pick min till
+   *          maxPeriodicity)
    * @return a view of the plan as it is available to this user
-   * @throws PlanningException
+   * @throws PlanningException if operation is unsuccessful
    */
-  public RLESparseResourceAllocation getAvailableResourceOverTime(String user,
-      ReservationId oldId, long start, long end) throws PlanningException;
+  RLESparseResourceAllocation getAvailableResourceOverTime(String user,
+      ReservationId oldId, long start, long end, long period)
+      throws PlanningException;
 
   /**
    * This method returns a RLE encoded view of the user reservation count
    * utilization between start and end time.
    *
-   * @param user
-   * @param start
-   * @param end
+   * @param user the user being considered
+   * @param start start of the time interval.
+   * @param end end of the time interval.
    * @return RLE encoded view of reservation used over time
    */
-  public RLESparseResourceAllocation getReservationCountForUserOverTime(
-      String user, long start, long end);
+  RLESparseResourceAllocation getReservationCountForUserOverTime(String user,
+      long start, long end);
 
   /**
    * This method returns a RLE encoded view of the user reservation utilization
    * between start and end time.
    *
-   * @param user
-   * @param start
-   * @param end
+   * @param user the user being considered
+   * @param start start of the time interval.
+   * @param end end of the time interval.
    * @return RLE encoded view of resources used over time
    */
-  public RLESparseResourceAllocation getConsumptionForUserOverTime(String user,
+  RLESparseResourceAllocation getConsumptionForUserOverTime(String user,
       long start, long end);
 
   /**
    * Get the cumulative load over a time interval.
    *
-   * @param start Start of the time interval.
-   * @param end End of the time interval.
+   * @param start start of the time interval.
+   * @param end end of the time interval.
    * @return RLE sparse allocation.
+   * @throws PlanningException if operation is unsuccessful
    */
-  RLESparseResourceAllocation getCumulativeLoadOverTime(long start, long end);
+  RLESparseResourceAllocation getCumulativeLoadOverTime(long start, long end)
+      throws PlanningException;
 
 }


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


[28/50] [abbrv] hadoop git commit: HDFS-12349. Improve log message when it could not alloc enough blocks for EC. (lei)

Posted by as...@apache.org.
HDFS-12349. Improve log message when it could not alloc enough blocks for EC. (lei)


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

Branch: refs/heads/YARN-5972
Commit: 3e6d0ca2b2f79bfa87faa7bbd46d814a48334fbd
Parents: 3b3be35
Author: Lei Xu <le...@apache.org>
Authored: Thu Sep 7 18:01:37 2017 -0700
Committer: Lei Xu <le...@apache.org>
Committed: Thu Sep 7 18:01:37 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/DFSStripedOutputStream.java     | 17 ++++++------
 .../server/blockmanagement/BlockManager.java    | 26 ++++++++++++------
 .../hdfs/server/namenode/FSDirWriteFileOp.java  | 14 ++++++++--
 .../TestDFSStripedOutputStreamWithFailure.java  | 29 ++++++++++----------
 .../hdfs/server/namenode/TestDeadDatanode.java  |  4 +--
 5 files changed, 54 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3e6d0ca2/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
index 7f05338..09dc181 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
@@ -260,6 +260,7 @@ public class DFSStripedOutputStream extends DFSOutputStream
 
   private final Coordinator coordinator;
   private final CellBuffers cellBuffers;
+  private final ErasureCodingPolicy ecPolicy;
   private final RawErasureEncoder encoder;
   private final List<StripedDataStreamer> streamers;
   private final DFSPacket[] currentPackets; // current Packet of each streamer
@@ -286,7 +287,7 @@ public class DFSStripedOutputStream extends DFSOutputStream
       LOG.debug("Creating DFSStripedOutputStream for " + src);
     }
 
-    final ErasureCodingPolicy ecPolicy = stat.getErasureCodingPolicy();
+    ecPolicy = stat.getErasureCodingPolicy();
     final int numParityBlocks = ecPolicy.getNumParityUnits();
     cellSize = ecPolicy.getCellSize();
     numDataBlocks = ecPolicy.getNumDataUnits();
@@ -478,11 +479,6 @@ public class DFSStripedOutputStream extends DFSOutputStream
     final LocatedBlock lb = addBlock(excludedNodes, dfsClient, src,
         currentBlockGroup, fileId, favoredNodes, getAddBlockFlags());
     assert lb.isStriped();
-    if (lb.getLocations().length < numDataBlocks) {
-      throw new IOException("Failed to get " + numDataBlocks
-          + " nodes from namenode: blockGroupSize= " + numAllBlocks
-          + ", blocks.length= " + lb.getLocations().length);
-    }
     // assign the new block to the current block group
     currentBlockGroup = lb.getBlock();
     blockGroupIndex++;
@@ -494,11 +490,16 @@ public class DFSStripedOutputStream extends DFSOutputStream
       StripedDataStreamer si = getStripedDataStreamer(i);
       assert si.isHealthy();
       if (blocks[i] == null) {
+        // allocBlock() should guarantee that all data blocks are successfully
+        // allocated.
+        assert i >= numDataBlocks;
         // Set exception and close streamer as there is no block locations
         // found for the parity block.
-        LOG.warn("Failed to get block location for parity block, index=" + i);
+        LOG.warn("Cannot allocate parity block(index={}, policy={}). " +
+            "Not enough datanodes? Excluded nodes={}", i,  ecPolicy.getName(),
+            excludedNodes);
         si.getLastException().set(
-            new IOException("Failed to get following block, i=" + i));
+            new IOException("Failed to get parity block, index=" + i));
         si.getErrorState().setInternalError();
         si.close(true);
       } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3e6d0ca2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index f4e5cb4..40c249d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -2057,6 +2057,7 @@ public class BlockManager implements BlockStatsMXBean {
       final List<String> favoredNodes,
       final byte storagePolicyID,
       final BlockType blockType,
+      final ErasureCodingPolicy ecPolicy,
       final EnumSet<AddBlockFlag> flags) throws IOException {
     List<DatanodeDescriptor> favoredDatanodeDescriptors = 
         getDatanodeDescriptors(favoredNodes);
@@ -2067,14 +2068,23 @@ public class BlockManager implements BlockStatsMXBean {
     final DatanodeStorageInfo[] targets = blockplacement.chooseTarget(src,
         numOfReplicas, client, excludedNodes, blocksize, 
         favoredDatanodeDescriptors, storagePolicy, flags);
-    if (targets.length < minReplication) {
-      throw new IOException("File " + src + " could only be replicated to "
-          + targets.length + " nodes instead of minReplication (="
-          + minReplication + ").  There are "
-          + getDatanodeManager().getNetworkTopology().getNumOfLeaves()
-          + " datanode(s) running and "
-          + (excludedNodes == null? "no": excludedNodes.size())
-          + " node(s) are excluded in this operation.");
+
+    final String ERROR_MESSAGE = "File %s could only be written to %d of " +
+        "the %d %s. There are %d datanode(s) running and %s "
+        + "node(s) are excluded in this operation.";
+    if (blockType == BlockType.CONTIGUOUS && targets.length < minReplication) {
+      throw new IOException(String.format(ERROR_MESSAGE, src,
+          targets.length, minReplication, "minReplication", minReplication,
+          getDatanodeManager().getNetworkTopology().getNumOfLeaves(),
+          (excludedNodes == null? "no": excludedNodes.size())));
+    } else if (blockType == BlockType.STRIPED &&
+        targets.length < ecPolicy.getNumDataUnits()) {
+      throw new IOException(
+          String.format(ERROR_MESSAGE, src, targets.length,
+              ecPolicy.getNumDataUnits(),
+              String.format("required nodes for %s", ecPolicy.getName()),
+              getDatanodeManager().getNetworkTopology().getNumOfLeaves(),
+              (excludedNodes == null ? "no" : excludedNodes.size())));
     }
     return targets;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3e6d0ca2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
index 012e916..fc5ac9e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
@@ -201,7 +201,7 @@ class FSDirWriteFileOp {
     }
     storagePolicyID = pendingFile.getStoragePolicyID();
     return new ValidateAddBlockResult(blockSize, numTargets, storagePolicyID,
-                                      clientMachine, blockType);
+                                      clientMachine, blockType, ecPolicy);
   }
 
   static LocatedBlock makeLocatedBlock(FSNamesystem fsn, BlockInfo blk,
@@ -286,7 +286,7 @@ class FSDirWriteFileOp {
     return bm.chooseTarget4NewBlock(src, r.numTargets, clientNode,
                                     excludedNodesSet, r.blockSize,
                                     favoredNodesList, r.storagePolicyID,
-                                    r.blockType, flags);
+                                    r.blockType, r.ecPolicy, flags);
   }
 
   /**
@@ -836,15 +836,23 @@ class FSDirWriteFileOp {
     final byte storagePolicyID;
     final String clientMachine;
     final BlockType blockType;
+    final ErasureCodingPolicy ecPolicy;
 
     ValidateAddBlockResult(
         long blockSize, int numTargets, byte storagePolicyID,
-        String clientMachine, BlockType blockType) {
+        String clientMachine, BlockType blockType,
+        ErasureCodingPolicy ecPolicy) {
       this.blockSize = blockSize;
       this.numTargets = numTargets;
       this.storagePolicyID = storagePolicyID;
       this.clientMachine = clientMachine;
       this.blockType = blockType;
+      this.ecPolicy = ecPolicy;
+
+      if (blockType == BlockType.STRIPED) {
+        Preconditions.checkArgument(ecPolicy != null,
+            "ecPolicy is not specified for striped block");
+      }
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3e6d0ca2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
index f63a353..f770673 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.io.erasurecode.ErasureCodeNative;
 import org.apache.hadoop.io.erasurecode.rawcoder.NativeRSRawErasureCoderFactory;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.test.LambdaTestUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.log4j.Level;
 import org.junit.Assert;
@@ -282,7 +283,7 @@ public class TestDFSStripedOutputStreamWithFailure {
 
   @Test(timeout = 90000)
   public void testAddBlockWhenNoSufficientDataBlockNumOfNodes()
-      throws IOException {
+      throws Exception {
     HdfsConfiguration conf = new HdfsConfiguration();
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
     try {
@@ -301,20 +302,18 @@ public class TestDFSStripedOutputStreamWithFailure {
           DatanodeReportType.LIVE);
       assertEquals("Mismatches number of live Dns ", numDatanodes, info.length);
       final Path dirFile = new Path(dir, "ecfile");
-      FSDataOutputStream out;
-      try {
-        out = dfs.create(dirFile, true);
-        out.write("something".getBytes());
-        out.flush();
-        out.close();
-        Assert.fail("Failed to validate available dns against blkGroupSize");
-      } catch (IOException ioe) {
-        // expected
-        GenericTestUtils.assertExceptionContains("Failed to get " +
-            dataBlocks + " nodes from namenode: blockGroupSize= " +
-            (dataBlocks + parityBlocks) + ", blocks.length= " +
-            numDatanodes, ioe);
-      }
+      LambdaTestUtils.intercept(
+          IOException.class,
+          "File " + dirFile + " could only be written to " +
+              numDatanodes + " of the " + dataBlocks + " required nodes for " +
+              getEcPolicy().getName(),
+          () -> {
+            try (FSDataOutputStream out = dfs.create(dirFile, true)) {
+              out.write("something".getBytes());
+              out.flush();
+            }
+            return 0;
+          });
     } finally {
       tearDown();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3e6d0ca2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
index 74be90c..b6c1318 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
@@ -175,8 +175,8 @@ public class TestDeadDatanode {
     // choose the targets, but local node should not get selected as this is not
     // part of the cluster anymore
     DatanodeStorageInfo[] results = bm.chooseTarget4NewBlock("/hello", 3,
-        clientNode, new HashSet<Node>(), 256 * 1024 * 1024L, null, (byte) 7,
-        BlockType.CONTIGUOUS, null);
+        clientNode, new HashSet<>(), 256 * 1024 * 1024L, null, (byte) 7,
+        BlockType.CONTIGUOUS, null, null);
     for (DatanodeStorageInfo datanodeStorageInfo : results) {
       assertFalse("Dead node should not be choosen", datanodeStorageInfo
           .getDatanodeDescriptor().equals(clientNode));


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


[39/50] [abbrv] hadoop git commit: HADOOP-14844. Remove requirement to specify TenantGuid for MSI Token Provider. Contributed by Atul Sikaria.

Posted by as...@apache.org.
HADOOP-14844. Remove requirement to specify TenantGuid for MSI Token Provider. Contributed by Atul Sikaria.


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

Branch: refs/heads/YARN-5972
Commit: a4661850c1e0794baf493a468191e12681d68ab4
Parents: c35510a
Author: John Zhuge <jz...@apache.org>
Authored: Fri Sep 8 11:51:03 2017 -0700
Committer: John Zhuge <jz...@apache.org>
Committed: Fri Sep 8 11:51:03 2017 -0700

----------------------------------------------------------------------
 .../src/main/resources/core-default.xml         | 16 +++------------
 hadoop-tools/hadoop-azure-datalake/pom.xml      |  2 +-
 .../org/apache/hadoop/fs/adl/AdlConfKeys.java   |  1 -
 .../org/apache/hadoop/fs/adl/AdlFileSystem.java |  4 +---
 .../src/site/markdown/index.md                  | 21 ++++++--------------
 .../hadoop/fs/adl/TestAzureADTokenProvider.java |  4 ----
 6 files changed, 11 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4661850/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
index 23739b0..269a13c 100644
--- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
+++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
@@ -2717,8 +2717,7 @@
       fs.adl.oauth2.credential, and fs.adl.oauth2.refresh.url.
       The RefreshToken type requires property fs.adl.oauth2.client.id and
       fs.adl.oauth2.refresh.token.
-      The MSI type requires properties fs.adl.oauth2.msi.port and
-      fs.adl.oauth2.msi.tenantguid.
+      The MSI type reads optional property fs.adl.oauth2.msi.port, if specified.
       The DeviceCode type requires property
       fs.adl.oauth2.devicecode.clientapp.id.
       The Custom type requires property fs.adl.oauth2.access.token.provider.
@@ -2762,17 +2761,8 @@
     <value></value>
     <description>
       The localhost port for the MSI token service. This is the port specified
-      when creating the Azure VM.
-      Used by MSI token provider.
-    </description>
-  </property>
-
-  <property>
-    <name>fs.adl.oauth2.msi.tenantguid</name>
-    <value></value>
-    <description>
-      The tenant guid for the Azure AAD tenant under which the azure data lake
-      store account is created.
+      when creating the Azure VM. The default, if this setting is not specified,
+      is 50342.
       Used by MSI token provider.
     </description>
   </property>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4661850/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 b06c774..e32f9fc 100644
--- a/hadoop-tools/hadoop-azure-datalake/pom.xml
+++ b/hadoop-tools/hadoop-azure-datalake/pom.xml
@@ -110,7 +110,7 @@
     <dependency>
       <groupId>com.microsoft.azure</groupId>
       <artifactId>azure-data-lake-store-sdk</artifactId>
-      <version>2.2.2</version>
+      <version>2.2.3</version>
     </dependency>
     <!--  ENDS HERE-->
     <dependency>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4661850/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlConfKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlConfKeys.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlConfKeys.java
index f77d981..790902c 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlConfKeys.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlConfKeys.java
@@ -56,7 +56,6 @@ public final class AdlConfKeys {
 
   // MSI Auth Configuration
   public static final String MSI_PORT = "fs.adl.oauth2.msi.port";
-  public static final String MSI_TENANT_GUID = "fs.adl.oauth2.msi.tenantguid";
 
   // DeviceCode Auth configuration
   public static final String DEVICE_CODE_CLIENT_APP_ID =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4661850/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java
index a5e31e1..a496595 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java
@@ -290,9 +290,7 @@ public class AdlFileSystem extends FileSystem {
 
   private AccessTokenProvider getMsiBasedTokenProvider(
           Configuration conf) throws IOException {
-    int port = Integer.parseInt(getNonEmptyVal(conf, MSI_PORT));
-    String tenantGuid = getPasswordString(conf, MSI_TENANT_GUID);
-    return new MsiTokenProvider(port, tenantGuid);
+    return new MsiTokenProvider(conf.getInt(MSI_PORT, -1));
   }
 
   private AccessTokenProvider getDeviceCodeTokenProvider(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4661850/hadoop-tools/hadoop-azure-datalake/src/site/markdown/index.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/site/markdown/index.md b/hadoop-tools/hadoop-azure-datalake/src/site/markdown/index.md
index e34da36..ca79321 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/site/markdown/index.md
+++ b/hadoop-tools/hadoop-azure-datalake/src/site/markdown/index.md
@@ -164,15 +164,11 @@ Identity extension within the VM. The advantage of doing this is that the
 credentials are managed by the extension, and do not have to be put into
 core-site.xml.
 
-To use MSI, the following two steps are needed:
-1. Modify the VM deployment template to specify the port number of the token
- service exposed to localhost by the identity extension in the VM.
-2. Get your Azure ActiveDirectory Tenant ID:
-   1. Go to [the portal](https://portal.azure.com)
-   2. Under services in left nav, look for Azure Active Directory and click on it.
-   3. Click on Properties
-   4. Note down the GUID shown under "Directory ID" - this is your AAD tenant ID
-
+To use MSI, modify the VM deployment template to use the identity extension. Note the
+port number you specified in the template: this is the port number for the REST endpoint
+of the token service exposed to localhost by the identity extension in the VM. The default
+recommended port number is 50342 - if the recommended port number is used, then the msi.port
+setting below can be omitted in the configuration.
 
 ##### Configure core-site.xml
 Add the following properties to your `core-site.xml`
@@ -185,12 +181,7 @@ Add the following properties to your `core-site.xml`
 
 <property>
   <name>fs.adl.oauth2.msi.port</name>
-  <value>PORT NUMBER FROM STEP 1 ABOVE</value>
-</property>
-
-<property>
-  <name>fs.adl.oauth2.msi.TenantGuid</name>
-  <value>AAD TENANT ID GUID FROM STEP 2 ABOVE</value>
+  <value>PORT NUMBER FROM ABOVE (if different from the default of 50342)</value>
 </property>
 ```
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4661850/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestAzureADTokenProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestAzureADTokenProvider.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestAzureADTokenProvider.java
index 929b33a..12c2e3f 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestAzureADTokenProvider.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestAzureADTokenProvider.java
@@ -43,8 +43,6 @@ import static org.apache.hadoop.fs.adl.AdlConfKeys
 import static org.apache.hadoop.fs.adl.AdlConfKeys
     .AZURE_AD_TOKEN_PROVIDER_TYPE_KEY;
 import static org.apache.hadoop.fs.adl.AdlConfKeys.DEVICE_CODE_CLIENT_APP_ID;
-import static org.apache.hadoop.fs.adl.AdlConfKeys.MSI_PORT;
-import static org.apache.hadoop.fs.adl.AdlConfKeys.MSI_TENANT_GUID;
 import static org.apache.hadoop.fs.adl.TokenProviderType.*;
 import static org.junit.Assert.assertEquals;
 
@@ -107,8 +105,6 @@ public class TestAzureADTokenProvider {
           throws IOException, URISyntaxException {
     Configuration conf = new Configuration();
     conf.setEnum(AZURE_AD_TOKEN_PROVIDER_TYPE_KEY, MSI);
-    conf.set(MSI_PORT, "54321");
-    conf.set(MSI_TENANT_GUID, "TENANT_GUID");
 
     URI uri = new URI("adl://localhost:8080");
     AdlFileSystem fileSystem = new AdlFileSystem();


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


[10/50] [abbrv] hadoop git commit: YARN-5328. Plan/ResourceAllocation data structure enhancements required to support recurring reservations in ReservationSystem.

Posted by as...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryPlan.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/reservation/TestInMemoryPlan.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryPlan.java
index bc98e2f..03569d4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryPlan.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryPlan.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.reservation.planning.Planne
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.planning.ReservationAgent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.util.Clock;
+import org.apache.hadoop.yarn.util.UTCClock;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.junit.After;
@@ -47,6 +48,10 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
+/**
+ * Testing the class {@link InMemoryPlan}.
+ */
+@SuppressWarnings("checkstyle:nowhitespaceafter")
 public class TestInMemoryPlan {
 
   private String user = "yarn";
@@ -62,6 +67,7 @@ public class TestInMemoryPlan {
   private ReservationAgent agent;
   private Planner replanner;
   private RMContext context;
+  private long maxPeriodicity;
 
   @Before
   public void setUp() throws PlanningException {
@@ -72,7 +78,7 @@ public class TestInMemoryPlan {
 
     clock = mock(Clock.class);
     queueMetrics = mock(QueueMetrics.class);
-    policy = mock(SharingPolicy.class);
+    policy = new NoOverCommitPolicy();
     replanner = mock(Planner.class);
 
     when(clock.getTime()).thenReturn(1L);
@@ -95,15 +101,41 @@ public class TestInMemoryPlan {
 
   @Test
   public void testAddReservation() {
-    Plan plan =
-        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-            ReservationSystemTestUtil.getNewReservationId();
+        ReservationSystemTestUtil.getNewReservationId();
     int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    ReservationAllocation rAllocation = createReservationAllocation
-            (reservationID, start, alloc);
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
+    Assert.assertNull(plan.getReservationById(reservationID));
+    try {
+      plan.addReservation(rAllocation, false);
+    } catch (PlanningException e) {
+      Assert.fail(e.getMessage());
+    }
+    doAssertions(plan, rAllocation);
+    checkAllocation(plan, alloc, start, 0);
+  }
+
+  @Test
+  public void testAddPeriodicReservation() throws PlanningException {
+
+    maxPeriodicity = 100;
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, maxPeriodicity,
+        context, new UTCClock());
+
+    ReservationId reservationID =
+        ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = { 10, 50 };
+    int start = 10;
+    long period = 20;
+    ReservationAllocation rAllocation = createReservationAllocation(
+        reservationID, start, alloc, String.valueOf(period));
+    // use periodicity of 1hr
+    rAllocation.setPeriodicity(period);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -111,32 +143,54 @@ public class TestInMemoryPlan {
       Assert.fail(e.getMessage());
     }
     doAssertions(plan, rAllocation);
-    checkAllocation(plan, alloc, start);
+    checkAllocation(plan, alloc, start, period);
+
+    RLESparseResourceAllocation available =
+        plan.getAvailableResourceOverTime(user, reservationID, 150, 330, 50);
+    System.out.println(available);
   }
 
-  private void checkAllocation(Plan plan, int[] alloc, int start) {
+  private void checkAllocation(Plan plan, int[] alloc, int start,
+      long periodicity) {
+    long end = start + alloc.length;
+    if (periodicity > 0) {
+      end = end + maxPeriodicity;
+    }
     RLESparseResourceAllocation userCons =
-        plan.getConsumptionForUserOverTime(user, start, start + alloc.length);
+        plan.getConsumptionForUserOverTime(user, start, end * 3);
 
     for (int i = 0; i < alloc.length; i++) {
-      Assert.assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
-          plan.getTotalCommittedResources(start + i));
-      Assert.assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
-          userCons.getCapacityAtTime(start + i));
+      // only one instance for non-periodic reservation
+      if (periodicity <= 0) {
+        Assert.assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
+            plan.getTotalCommittedResources(start + i));
+        Assert.assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
+            userCons.getCapacityAtTime(start + i));
+      } else {
+        // periodic reservations should repeat
+        long y = 0;
+        Resource res = Resource.newInstance(1024 * (alloc[i]), (alloc[i]));
+        while (y <= end * 2) {
+          Assert.assertEquals("At time: " + start + i + y, res,
+              plan.getTotalCommittedResources(start + i + y));
+          Assert.assertEquals(" At time: " + (start + i + y), res,
+              userCons.getCapacityAtTime(start + i + y));
+          y = y + periodicity;
+        }
+      }
     }
   }
 
   @Test
   public void testAddEmptyReservation() {
-    Plan plan =
-        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-            ReservationSystemTestUtil.getNewReservationId();
+        ReservationSystemTestUtil.getNewReservationId();
     int[] alloc = {};
     int start = 100;
-    ReservationAllocation rAllocation = createReservationAllocation
-            (reservationID, start, alloc);
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -148,15 +202,14 @@ public class TestInMemoryPlan {
   @Test
   public void testAddReservationAlreadyExists() {
     // First add a reservation
-    Plan plan =
-        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-            ReservationSystemTestUtil.getNewReservationId();
+        ReservationSystemTestUtil.getNewReservationId();
     int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    ReservationAllocation rAllocation = createReservationAllocation
-            (reservationID, start, alloc);
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -164,7 +217,7 @@ public class TestInMemoryPlan {
       Assert.fail(e.getMessage());
     }
     doAssertions(plan, rAllocation);
-    checkAllocation(plan, alloc, start);
+    checkAllocation(plan, alloc, start, 0);
 
     // Try to add it again
     try {
@@ -180,16 +233,15 @@ public class TestInMemoryPlan {
 
   @Test
   public void testUpdateReservation() {
-    Plan plan =
-        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
         ReservationSystemTestUtil.getNewReservationId();
     // First add a reservation
     int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    ReservationAllocation rAllocation = createReservationAllocation
-            (reservationID, start, alloc);
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -210,8 +262,8 @@ public class TestInMemoryPlan {
     // Now update it
     start = 110;
     int[] updatedAlloc = { 0, 5, 10, 10, 5, 0 };
-    rAllocation = createReservationAllocation(reservationID, start,
-            updatedAlloc, true);
+    rAllocation =
+        createReservationAllocation(reservationID, start, updatedAlloc, true);
     try {
       plan.updateReservation(rAllocation);
     } catch (PlanningException e) {
@@ -219,32 +271,71 @@ public class TestInMemoryPlan {
     }
     doAssertions(plan, rAllocation);
 
-    userCons =
-        plan.getConsumptionForUserOverTime(user, start, start
-            + updatedAlloc.length);
+    userCons = plan.getConsumptionForUserOverTime(user, start,
+        start + updatedAlloc.length);
 
     for (int i = 0; i < updatedAlloc.length; i++) {
-      Assert.assertEquals(
-     Resource.newInstance(1024 * (updatedAlloc[i] + i), updatedAlloc[i]
-              + i), plan.getTotalCommittedResources(start + i));
-      Assert.assertEquals(
-          Resource.newInstance(1024 * (updatedAlloc[i] + i), updatedAlloc[i]
-              + i), userCons.getCapacityAtTime(start + i));
+      Assert.assertEquals(Resource.newInstance(1024 * (updatedAlloc[i] + i),
+          updatedAlloc[i] + i), plan.getTotalCommittedResources(start + i));
+      Assert.assertEquals(Resource.newInstance(1024 * (updatedAlloc[i] + i),
+          updatedAlloc[i] + i), userCons.getCapacityAtTime(start + i));
+    }
+  }
+
+  @Test
+  public void testUpdatePeriodicReservation() {
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    // First add a reservation
+    ReservationId reservationID =
+        ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = { 10, 20 };
+    int start = 100;
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
+    // use periodicity of 1hr
+    long period = 3600000;
+    rAllocation.getReservationDefinition()
+        .setRecurrenceExpression(String.valueOf(period));
+    rAllocation.setPeriodicity(period);
+    Assert.assertNull(plan.getReservationById(reservationID));
+    try {
+      plan.addReservation(rAllocation, false);
+    } catch (PlanningException e) {
+      Assert.fail(e.getMessage());
+    }
+    System.out.println(plan.toString());
+    doAssertions(plan, rAllocation);
+    checkAllocation(plan, alloc, start, period);
+
+    // Now update it
+    start = 110;
+    int[] updatedAlloc = { 30, 40, 50 };
+    rAllocation =
+        createReservationAllocation(reservationID, start, updatedAlloc);
+    rAllocation.getReservationDefinition()
+        .setRecurrenceExpression(String.valueOf(period));
+    rAllocation.setPeriodicity(period);
+    try {
+      plan.updateReservation(rAllocation);
+    } catch (PlanningException e) {
+      Assert.fail(e.getMessage());
     }
+    doAssertions(plan, rAllocation);
+    checkAllocation(plan, updatedAlloc, start, period);
   }
 
   @Test
   public void testUpdateNonExistingReservation() {
-    Plan plan =
-        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
         ReservationSystemTestUtil.getNewReservationId();
     // Try to update a reservation without adding
     int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
     ReservationAllocation rAllocation =
-            createReservationAllocation(reservationID, start, alloc);
+        createReservationAllocation(reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.updateReservation(rAllocation);
@@ -260,15 +351,14 @@ public class TestInMemoryPlan {
   @Test
   public void testDeleteReservation() {
     // First add a reservation
-    Plan plan =
-        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
         ReservationSystemTestUtil.getNewReservationId();
     int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
     ReservationAllocation rAllocation =
-            createReservationAllocation(reservationID, start, alloc, true);
+        createReservationAllocation(reservationID, start, alloc, true);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -307,10 +397,46 @@ public class TestInMemoryPlan {
   }
 
   @Test
+  public void testDeletePeriodicReservation() {
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    // First add a reservation
+    ReservationId reservationID =
+        ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = { 10, 20 };
+    int start = 100;
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
+    // use periodicity of 1hr
+    long period = 3600000;
+    rAllocation.getReservationDefinition()
+        .setRecurrenceExpression(String.valueOf(period));
+    rAllocation.setPeriodicity(period);
+    Assert.assertNull(plan.getReservationById(reservationID));
+    try {
+      plan.addReservation(rAllocation, false);
+    } catch (PlanningException e) {
+      Assert.fail(e.getMessage());
+    }
+    System.out.println(plan.toString());
+    doAssertions(plan, rAllocation);
+    checkAllocation(plan, alloc, start, period);
+
+    // Now delete it
+    try {
+      plan.deleteReservation(reservationID);
+    } catch (PlanningException e) {
+      Assert.fail(e.getMessage());
+    }
+    Assert.assertNull(plan.getReservationById(reservationID));
+    System.out.print(plan);
+    checkAllocation(plan, new int[] { 0, 0 }, start, period);
+  }
+
+  @Test
   public void testDeleteNonExistingReservation() {
-    Plan plan =
-        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
         ReservationSystemTestUtil.getNewReservationId();
     // Try to delete a reservation without adding
@@ -328,8 +454,9 @@ public class TestInMemoryPlan {
 
   @Test
   public void testArchiveCompletedReservations() {
+    SharingPolicy sharingPolicy = mock(SharingPolicy.class);
     Plan plan =
-        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        new InMemoryPlan(queueMetrics, sharingPolicy, agent, totalCapacity, 1L,
             resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID1 =
         ReservationSystemTestUtil.getNewReservationId();
@@ -337,7 +464,7 @@ public class TestInMemoryPlan {
     int[] alloc1 = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
     ReservationAllocation rAllocation =
-            createReservationAllocation(reservationID1, start, alloc1);
+        createReservationAllocation(reservationID1, start, alloc1);
     Assert.assertNull(plan.getReservationById(reservationID1));
     try {
       plan.addReservation(rAllocation, false);
@@ -345,15 +472,14 @@ public class TestInMemoryPlan {
       Assert.fail(e.getMessage());
     }
     doAssertions(plan, rAllocation);
-    checkAllocation(plan, alloc1, start);
-
+    checkAllocation(plan, alloc1, start, 0);
 
     // Now add another one
     ReservationId reservationID2 =
         ReservationSystemTestUtil.getNewReservationId();
     int[] alloc2 = { 0, 5, 10, 5, 0 };
     rAllocation =
-            createReservationAllocation(reservationID2, start, alloc2, true);
+        createReservationAllocation(reservationID2, start, alloc2, true);
     Assert.assertNull(plan.getReservationById(reservationID2));
     try {
       plan.addReservation(rAllocation, false);
@@ -367,16 +493,18 @@ public class TestInMemoryPlan {
 
     for (int i = 0; i < alloc2.length; i++) {
       Assert.assertEquals(
-          Resource.newInstance(1024 * (alloc1[i] + alloc2[i] + i), alloc1[i]
-              + alloc2[i] + i), plan.getTotalCommittedResources(start + i));
+          Resource.newInstance(1024 * (alloc1[i] + alloc2[i] + i),
+              alloc1[i] + alloc2[i] + i),
+          plan.getTotalCommittedResources(start + i));
       Assert.assertEquals(
-          Resource.newInstance(1024 * (alloc1[i] + alloc2[i] + i), alloc1[i]
-              + alloc2[i] + i), userCons.getCapacityAtTime(start + i));
+          Resource.newInstance(1024 * (alloc1[i] + alloc2[i] + i),
+              alloc1[i] + alloc2[i] + i),
+          userCons.getCapacityAtTime(start + i));
     }
 
     // Now archive completed reservations
     when(clock.getTime()).thenReturn(106L);
-    when(policy.getValidWindow()).thenReturn(1L);
+    when(sharingPolicy.getValidWindow()).thenReturn(1L);
     try {
       // will only remove 2nd reservation as only that has fallen out of the
       // archival window
@@ -386,7 +514,7 @@ public class TestInMemoryPlan {
     }
     Assert.assertNotNull(plan.getReservationById(reservationID1));
     Assert.assertNull(plan.getReservationById(reservationID2));
-    checkAllocation(plan, alloc1, start);
+    checkAllocation(plan, alloc1, start, 0);
 
     when(clock.getTime()).thenReturn(107L);
     try {
@@ -411,15 +539,14 @@ public class TestInMemoryPlan {
 
   @Test
   public void testGetReservationsById() {
-    Plan plan =
-            new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-                    resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-            ReservationSystemTestUtil.getNewReservationId();
-    int[] alloc = {10, 10, 10, 10, 10, 10};
+        ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    ReservationAllocation rAllocation = createReservationAllocation
-            (reservationID, start, alloc);
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -429,31 +556,30 @@ public class TestInMemoryPlan {
 
     // Verify that get by reservation id works.
     Set<ReservationAllocation> rAllocations =
-            plan.getReservations(reservationID, null, "");
+        plan.getReservations(reservationID, null, "");
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation.compareTo(
-            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation
+        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
 
     // Verify that get by reservation id works even when time range
     // and user is invalid.
     ReservationInterval interval = new ReservationInterval(0, 0);
     rAllocations = plan.getReservations(reservationID, interval, "invalid");
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation.compareTo(
-            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation
+        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
   }
 
   @Test
   public void testGetReservationsByInvalidId() {
-    Plan plan =
-            new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-                    resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-            ReservationSystemTestUtil.getNewReservationId();
-    int[] alloc = {10, 10, 10, 10, 10, 10};
+        ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    ReservationAllocation rAllocation = createReservationAllocation
-            (reservationID, start, alloc);
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -463,23 +589,22 @@ public class TestInMemoryPlan {
 
     // If reservationId is null, then nothing is returned.
     ReservationId invalidReservationID =
-            ReservationSystemTestUtil.getNewReservationId();
+        ReservationSystemTestUtil.getNewReservationId();
     Set<ReservationAllocation> rAllocations =
-            plan.getReservations(invalidReservationID, null, "");
+        plan.getReservations(invalidReservationID, null, "");
     Assert.assertTrue(rAllocations.size() == 0);
   }
 
   @Test
   public void testGetReservationsByTimeInterval() {
-    Plan plan =
-            new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-                    resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-            ReservationSystemTestUtil.getNewReservationId();
-    int[] alloc = {10, 10, 10, 10, 10, 10};
+        ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    ReservationAllocation rAllocation = createReservationAllocation
-            (reservationID, start, alloc);
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -489,23 +614,24 @@ public class TestInMemoryPlan {
 
     // Verify that get by time interval works if the selection interval
     // completely overlaps with the allocation.
-    ReservationInterval interval = new ReservationInterval(rAllocation
-            .getStartTime(), rAllocation.getEndTime());
+    ReservationInterval interval = new ReservationInterval(
+        rAllocation.getStartTime(), rAllocation.getEndTime());
     Set<ReservationAllocation> rAllocations =
-            plan.getReservations(null, interval, "");
+        plan.getReservations(null, interval, "");
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation.compareTo(
-            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation
+        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
 
     // Verify that get by time interval works if the selection interval
     // falls within the allocation
     long duration = rAllocation.getEndTime() - rAllocation.getStartTime();
-    interval = new ReservationInterval(rAllocation.getStartTime() + duration
-            * (long)0.3, rAllocation.getEndTime() - duration * (long)0.3);
+    interval = new ReservationInterval(
+        rAllocation.getStartTime() + duration * (long) 0.3,
+        rAllocation.getEndTime() - duration * (long) 0.3);
     rAllocations = plan.getReservations(null, interval, "");
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation.compareTo(
-            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation
+        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
 
     // Verify that get by time interval selects 1 allocation if the end
     // time of the selection interval falls right at the start of the
@@ -513,13 +639,13 @@ public class TestInMemoryPlan {
     interval = new ReservationInterval(0, rAllocation.getStartTime());
     rAllocations = plan.getReservations(null, interval, "");
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation.compareTo(
-            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation
+        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
 
     // Verify that get by time interval selects no reservations if the start
     // time of the selection interval falls right at the end of the allocation.
-    interval = new ReservationInterval(rAllocation
-            .getEndTime(), Long.MAX_VALUE);
+    interval =
+        new ReservationInterval(rAllocation.getEndTime(), Long.MAX_VALUE);
     rAllocations = plan.getReservations(null, interval, "");
     Assert.assertTrue(rAllocations.size() == 0);
 
@@ -532,15 +658,14 @@ public class TestInMemoryPlan {
 
   @Test
   public void testGetReservationsAtTime() {
-    Plan plan =
-            new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-                    resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-            ReservationSystemTestUtil.getNewReservationId();
-    int[] alloc = {10, 10, 10, 10, 10, 10};
+        ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    ReservationAllocation rAllocation = createReservationAllocation
-            (reservationID, start, alloc);
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -549,23 +674,22 @@ public class TestInMemoryPlan {
     }
 
     Set<ReservationAllocation> rAllocations =
-            plan.getReservationsAtTime(rAllocation.getStartTime());
+        plan.getReservationsAtTime(rAllocation.getStartTime());
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation.compareTo(
-            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation
+        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
   }
 
   @Test
   public void testGetReservationsWithNoInput() {
-    Plan plan =
-            new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-                    resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-            ReservationSystemTestUtil.getNewReservationId();
-    int[] alloc = {10, 10, 10, 10, 10, 10};
+        ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    ReservationAllocation rAllocation = createReservationAllocation
-            (reservationID, start, alloc);
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -576,22 +700,21 @@ public class TestInMemoryPlan {
     // Verify that getReservations defaults to getting all reservations if no
     // reservationID, time interval, and user is provided,
     Set<ReservationAllocation> rAllocations =
-            plan.getReservations(null, null, "");
+        plan.getReservations(null, null, "");
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation.compareTo(
-            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation
+        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
   }
 
   @Test
   public void testGetReservationsWithNoReservation() {
-    Plan plan =
-            new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-                    resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     // Verify that get reservation returns no entries if no queries are made.
 
     ReservationInterval interval = new ReservationInterval(0, Long.MAX_VALUE);
     Set<ReservationAllocation> rAllocations =
-            plan.getReservations(null, interval, "");
+        plan.getReservations(null, interval, "");
     Assert.assertTrue(rAllocations.size() == 0);
   }
 
@@ -600,7 +723,9 @@ public class TestInMemoryPlan {
     Assert.assertNotNull(plan.getReservationById(reservationID));
     Assert.assertEquals(rAllocation, plan.getReservationById(reservationID));
     Assert.assertTrue(((InMemoryPlan) plan).getAllReservations().size() == 1);
-    Assert.assertEquals(rAllocation.getEndTime(), plan.getLastEndTime());
+    if (rAllocation.getPeriodicity() <= 0) {
+      Assert.assertEquals(rAllocation.getEndTime(), plan.getLastEndTime());
+    }
     Assert.assertEquals(totalCapacity, plan.getTotalCapacity());
     Assert.assertEquals(minAlloc, plan.getMinimumAllocation());
     Assert.assertEquals(maxAlloc, plan.getMaximumAllocation());
@@ -610,7 +735,8 @@ public class TestInMemoryPlan {
   }
 
   private ReservationDefinition createSimpleReservationDefinition(long arrival,
-      long deadline, long duration, Collection<ReservationRequest> resources) {
+      long deadline, long duration, Collection<ReservationRequest> resources,
+      String recurrenceExpression) {
     // create a request with a single atomic ask
     ReservationDefinition rDef = new ReservationDefinitionPBImpl();
     ReservationRequests reqs = new ReservationRequestsPBImpl();
@@ -619,6 +745,7 @@ public class TestInMemoryPlan {
     rDef.setReservationRequests(reqs);
     rDef.setArrival(arrival);
     rDef.setDeadline(deadline);
+    rDef.setRecurrenceExpression(recurrenceExpression);
     return rDef;
   }
 
@@ -633,31 +760,43 @@ public class TestInMemoryPlan {
       } else {
         numContainers = alloc[i];
       }
-      ReservationRequest rr =
-          ReservationRequest.newInstance(Resource.newInstance(1024, 1),
-              (numContainers));
+      ReservationRequest rr = ReservationRequest
+          .newInstance(Resource.newInstance(1024, 1), (numContainers));
       req.put(new ReservationInterval(startTime + i, startTime + i + 1), rr);
     }
     return req;
   }
 
-  private ReservationAllocation createReservationAllocation(ReservationId
-            reservationID, int start, int[] alloc) {
-    return createReservationAllocation(reservationID, start, alloc, false);
+  private ReservationAllocation createReservationAllocation(
+      ReservationId reservationID, int start, int[] alloc) {
+    return createReservationAllocation(reservationID, start, alloc, false, "0");
+  }
+
+  private ReservationAllocation createReservationAllocation(
+      ReservationId reservationID, int start, int[] alloc, boolean isStep) {
+    return createReservationAllocation(reservationID, start, alloc, isStep,
+        "0");
+  }
+
+  private ReservationAllocation createReservationAllocation(
+      ReservationId reservationID, int start, int[] alloc,
+      String recurrenceExp) {
+    return createReservationAllocation(reservationID, start, alloc, false,
+        recurrenceExp);
   }
 
-  private ReservationAllocation createReservationAllocation(ReservationId
-            reservationID, int start, int[] alloc, boolean isStep) {
+  private ReservationAllocation createReservationAllocation(
+      ReservationId reservationID, int start, int[] alloc, boolean isStep,
+      String recurrenceExp) {
     Map<ReservationInterval, ReservationRequest> allocations =
-            generateAllocation(start, alloc, isStep);
+        generateAllocation(start, alloc, isStep);
     ReservationDefinition rDef =
-            createSimpleReservationDefinition(start, start + alloc.length,
-                    alloc.length, allocations.values());
+        createSimpleReservationDefinition(start, start + alloc.length,
+            alloc.length, allocations.values(), recurrenceExp);
     Map<ReservationInterval, Resource> allocs =
-            ReservationSystemUtil.toResources(allocations);
+        ReservationSystemUtil.toResources(allocations);
     return new InMemoryReservationAllocation(reservationID, rDef, user,
-            planName,
-                    start, start + alloc.length, allocs, resCalc, minAlloc);
+        planName, start, start + alloc.length, allocs, resCalc, minAlloc);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestPeriodicRLESparseResourceAllocation.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/reservation/TestPeriodicRLESparseResourceAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestPeriodicRLESparseResourceAllocation.java
index 554eb58..457e2ee 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestPeriodicRLESparseResourceAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestPeriodicRLESparseResourceAllocation.java
@@ -19,26 +19,27 @@
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
 import org.junit.Assert;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * Testing the class PeriodicRLESparseResourceAllocation.
+ * Testing the class {@link PeriodicRLESparseResourceAllocation}.
  */
+@SuppressWarnings("checkstyle:nowhitespaceafter")
 public class TestPeriodicRLESparseResourceAllocation {
 
-  private static final Logger LOG = LoggerFactory
-      .getLogger(TestPeriodicRLESparseResourceAllocation.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestPeriodicRLESparseResourceAllocation.class);
 
   @Test
   public void testPeriodicCapacity() {
-    int[] alloc = {10, 7, 5, 2, 0};
-    long[] timeSteps = {0L, 5L, 10L, 15L, 19L};
-    RLESparseResourceAllocation rleSparseVector =
-        ReservationSystemTestUtil.generateRLESparseResourceAllocation(
-            alloc, timeSteps);
+    int[] alloc = { 10, 7, 5, 2, 0 };
+    long[] timeSteps = { 0L, 5L, 10L, 15L, 19L };
+    RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
+        .generateRLESparseResourceAllocation(alloc, timeSteps);
     PeriodicRLESparseResourceAllocation periodicVector =
         new PeriodicRLESparseResourceAllocation(rleSparseVector, 20L);
     LOG.info(periodicVector.toString());
@@ -54,43 +55,74 @@ public class TestPeriodicRLESparseResourceAllocation {
 
   @Test
   public void testMaxPeriodicCapacity() {
-    int[] alloc = {2, 5, 7, 10, 3, 4, 6, 8};
-    long[] timeSteps = {0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L};
-    RLESparseResourceAllocation rleSparseVector =
-        ReservationSystemTestUtil.generateRLESparseResourceAllocation(
-            alloc, timeSteps);
+    int[] alloc = { 2, 5, 7, 10, 3, 4, 6, 8 };
+    long[] timeSteps = { 0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L };
+    RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
+        .generateRLESparseResourceAllocation(alloc, timeSteps);
     PeriodicRLESparseResourceAllocation periodicVector =
         new PeriodicRLESparseResourceAllocation(rleSparseVector, 8L);
     LOG.info(periodicVector.toString());
-    Assert.assertEquals(
-        periodicVector.getMaximumPeriodicCapacity(0, 1),
+    Assert.assertEquals(periodicVector.getMaximumPeriodicCapacity(0, 1),
         Resource.newInstance(10, 10));
-    Assert.assertEquals(
-        periodicVector.getMaximumPeriodicCapacity(8, 2),
+    Assert.assertEquals(periodicVector.getMaximumPeriodicCapacity(8, 2),
         Resource.newInstance(7, 7));
-    Assert.assertEquals(
-        periodicVector.getMaximumPeriodicCapacity(16, 3),
+    Assert.assertEquals(periodicVector.getMaximumPeriodicCapacity(16, 3),
         Resource.newInstance(10, 10));
-    Assert.assertEquals(
-        periodicVector.getMaximumPeriodicCapacity(17, 4),
+    Assert.assertEquals(periodicVector.getMaximumPeriodicCapacity(17, 4),
         Resource.newInstance(5, 5));
-    Assert.assertEquals(
-        periodicVector.getMaximumPeriodicCapacity(32, 5),
+    Assert.assertEquals(periodicVector.getMaximumPeriodicCapacity(32, 5),
         Resource.newInstance(4, 4));
   }
 
   @Test
+  public void testMixPeriodicAndNonPeriodic() throws PlanningException {
+    int[] alloc = { 2, 5, 0 };
+    long[] timeSteps = { 1L, 2L, 3L };
+    RLESparseResourceAllocation tempPeriodic = ReservationSystemTestUtil
+        .generateRLESparseResourceAllocation(alloc, timeSteps);
+    PeriodicRLESparseResourceAllocation periodic =
+        new PeriodicRLESparseResourceAllocation(tempPeriodic, 10L);
+
+    int[] alloc2 = { 10, 10, 0 };
+    long[] timeSteps2 = { 12L, 13L, 14L };
+    RLESparseResourceAllocation nonPeriodic = ReservationSystemTestUtil
+        .generateRLESparseResourceAllocation(alloc2, timeSteps2);
+
+    RLESparseResourceAllocation merged =
+        RLESparseResourceAllocation.merge(nonPeriodic.getResourceCalculator(),
+            Resource.newInstance(100 * 1024, 100), periodic, nonPeriodic,
+            RLESparseResourceAllocation.RLEOperator.add, 2, 25);
+
+    Assert.assertEquals(Resource.newInstance(5, 5),
+        merged.getCapacityAtTime(2L));
+    Assert.assertEquals(Resource.newInstance(0, 0),
+        merged.getCapacityAtTime(3L));
+    Assert.assertEquals(Resource.newInstance(2, 2),
+        merged.getCapacityAtTime(11L));
+    Assert.assertEquals(Resource.newInstance(15, 15),
+        merged.getCapacityAtTime(12L));
+    Assert.assertEquals(Resource.newInstance(10, 10),
+        merged.getCapacityAtTime(13L));
+    Assert.assertEquals(Resource.newInstance(0, 0),
+        merged.getCapacityAtTime(14L));
+    Assert.assertEquals(Resource.newInstance(2, 2),
+        merged.getCapacityAtTime(21L));
+    Assert.assertEquals(Resource.newInstance(5, 5),
+        merged.getCapacityAtTime(22L));
+    Assert.assertEquals(Resource.newInstance(0, 0),
+        merged.getCapacityAtTime(23L));
+  }
+
+  @Test
   public void testSetCapacityInInterval() {
-    int[] alloc = {2, 5, 0};
-    long[] timeSteps = {1L, 2L, 3L};
-    RLESparseResourceAllocation rleSparseVector =
-        ReservationSystemTestUtil.generateRLESparseResourceAllocation(
-            alloc, timeSteps);
+    int[] alloc = { 2, 5, 0 };
+    long[] timeSteps = { 1L, 2L, 3L };
+    RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
+        .generateRLESparseResourceAllocation(alloc, timeSteps);
     PeriodicRLESparseResourceAllocation periodicVector =
         new PeriodicRLESparseResourceAllocation(rleSparseVector, 10L);
     ReservationInterval interval = new ReservationInterval(5L, 10L);
-    periodicVector.addInterval(
-        interval, Resource.newInstance(8, 8));
+    periodicVector.addInterval(interval, Resource.newInstance(8, 8));
     Assert.assertEquals(Resource.newInstance(8, 8),
         periodicVector.getCapacityAtTime(5L));
     Assert.assertEquals(Resource.newInstance(8, 8),
@@ -99,21 +131,20 @@ public class TestPeriodicRLESparseResourceAllocation {
         periodicVector.getCapacityAtTime(10L));
     Assert.assertEquals(Resource.newInstance(0, 0),
         periodicVector.getCapacityAtTime(0L));
-    Assert.assertFalse(periodicVector.addInterval(
-        new ReservationInterval(7L, 12L), Resource.newInstance(8, 8)));
+    // Assert.assertFalse(periodicVector.addInterval(
+    // new ReservationInterval(7L, 12L), Resource.newInstance(8, 8)));
   }
 
   public void testRemoveInterval() {
-    int[] alloc = {2, 5, 3, 4, 0};
-    long[] timeSteps = {1L, 3L, 5L, 7L, 9L};
-    RLESparseResourceAllocation rleSparseVector =
-        ReservationSystemTestUtil.generateRLESparseResourceAllocation(
-            alloc, timeSteps);
+    int[] alloc = { 2, 5, 3, 4, 0 };
+    long[] timeSteps = { 1L, 3L, 5L, 7L, 9L };
+    RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
+        .generateRLESparseResourceAllocation(alloc, timeSteps);
     PeriodicRLESparseResourceAllocation periodicVector =
         new PeriodicRLESparseResourceAllocation(rleSparseVector, 10L);
     ReservationInterval interval = new ReservationInterval(3L, 7L);
-    Assert.assertTrue(periodicVector.removeInterval(
-        interval, Resource.newInstance(3, 3)));
+    Assert.assertTrue(
+        periodicVector.removeInterval(interval, Resource.newInstance(3, 3)));
     Assert.assertEquals(Resource.newInstance(2, 2),
         periodicVector.getCapacityAtTime(1L));
     Assert.assertEquals(Resource.newInstance(2, 2),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.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/reservation/TestRLESparseResourceAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.java
index bfe46e1..0027ceb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.java
@@ -40,10 +40,14 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+/**
+ * Testing the class {@link RLESparseResourceAllocation}.
+ */
+@SuppressWarnings("checkstyle:nowhitespaceafter")
 public class TestRLESparseResourceAllocation {
 
-  private static final Logger LOG = LoggerFactory
-      .getLogger(TestRLESparseResourceAllocation.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestRLESparseResourceAllocation.class);
 
   @Test
   public void testMergeAdd() throws PlanningException {
@@ -196,7 +200,8 @@ public class TestRLESparseResourceAllocation {
       // Expected!
     }
 
-    // Testing that the subtractTestNonNegative detects problems even if only one
+    // Testing that the subtractTestNonNegative detects problems even if only
+    // one
     // of the resource dimensions is "<0"
     a.put(10L, Resource.newInstance(10, 5));
     b.put(11L, Resource.newInstance(5, 6));
@@ -286,9 +291,8 @@ public class TestRLESparseResourceAllocation {
   public void testRangeOverlapping() {
     ResourceCalculator resCalc = new DefaultResourceCalculator();
 
-    RLESparseResourceAllocation r =
-        new RLESparseResourceAllocation(resCalc);
-    int[] alloc = {10, 10, 10, 10, 10, 10};
+    RLESparseResourceAllocation r = new RLESparseResourceAllocation(resCalc);
+    int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
     Set<Entry<ReservationInterval, Resource>> inputs =
         generateAllocation(start, alloc, false).entrySet();
@@ -299,9 +303,9 @@ public class TestRLESparseResourceAllocation {
     long d = r.getLatestNonNullTime();
 
     // tries to trigger "out-of-range" bug
-    r =  r.getRangeOverlapping(s, d);
-    r = r.getRangeOverlapping(s-1, d-1);
-    r = r.getRangeOverlapping(s+1, d+1);
+    r = r.getRangeOverlapping(s, d);
+    r = r.getRangeOverlapping(s - 1, d - 1);
+    r = r.getRangeOverlapping(s + 1, d + 1);
   }
 
   @Test
@@ -370,25 +374,29 @@ public class TestRLESparseResourceAllocation {
     // Current bug prevents this to pass. The RLESparseResourceAllocation
     // does not handle removal of "partial"
     // allocations correctly.
-    Assert.assertEquals(102400, rleSparseVector.getCapacityAtTime(10)
-        .getMemorySize());
-    Assert.assertEquals(0, rleSparseVector.getCapacityAtTime(13).getMemorySize());
-    Assert.assertEquals(0, rleSparseVector.getCapacityAtTime(19).getMemorySize());
-    Assert.assertEquals(102400, rleSparseVector.getCapacityAtTime(21)
-        .getMemorySize());
-    Assert.assertEquals(2 * 102400, rleSparseVector.getCapacityAtTime(26)
-        .getMemorySize());
+    Assert.assertEquals(102400,
+        rleSparseVector.getCapacityAtTime(10).getMemorySize());
+    Assert.assertEquals(0,
+        rleSparseVector.getCapacityAtTime(13).getMemorySize());
+    Assert.assertEquals(0,
+        rleSparseVector.getCapacityAtTime(19).getMemorySize());
+    Assert.assertEquals(102400,
+        rleSparseVector.getCapacityAtTime(21).getMemorySize());
+    Assert.assertEquals(2 * 102400,
+        rleSparseVector.getCapacityAtTime(26).getMemorySize());
 
     ReservationInterval riRemove2 = new ReservationInterval(9, 13);
     rleSparseVector.removeInterval(riRemove2, rr);
     LOG.info(rleSparseVector.toString());
 
-    Assert.assertEquals(0, rleSparseVector.getCapacityAtTime(11).getMemorySize());
-    Assert.assertEquals(-102400, rleSparseVector.getCapacityAtTime(9)
-        .getMemorySize());
-    Assert.assertEquals(0, rleSparseVector.getCapacityAtTime(13).getMemorySize());
-    Assert.assertEquals(102400, rleSparseVector.getCapacityAtTime(20)
-        .getMemorySize());
+    Assert.assertEquals(0,
+        rleSparseVector.getCapacityAtTime(11).getMemorySize());
+    Assert.assertEquals(-102400,
+        rleSparseVector.getCapacityAtTime(9).getMemorySize());
+    Assert.assertEquals(0,
+        rleSparseVector.getCapacityAtTime(13).getMemorySize());
+    Assert.assertEquals(102400,
+        rleSparseVector.getCapacityAtTime(20).getMemorySize());
 
   }
 
@@ -500,7 +508,8 @@ public class TestRLESparseResourceAllocation {
     }
     mapAllocations = rleSparseVector.toIntervalMap();
     Assert.assertTrue(mapAllocations.size() == 5);
-    for (Entry<ReservationInterval, Resource> entry : mapAllocations.entrySet()) {
+    for (Entry<ReservationInterval, Resource> entry : mapAllocations
+        .entrySet()) {
       ReservationInterval interval = entry.getKey();
       Resource resource = entry.getValue();
       if (interval.getStartTime() == 101L) {
@@ -526,59 +535,46 @@ public class TestRLESparseResourceAllocation {
 
   @Test
   public void testMaxPeriodicCapacity() {
-    long[] timeSteps = {0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L};
-    int[] alloc = {2, 5, 7, 10, 3, 4, 6, 8};
-    RLESparseResourceAllocation rleSparseVector =
-        ReservationSystemTestUtil.generateRLESparseResourceAllocation(
-            alloc, timeSteps);
+    long[] timeSteps = { 0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L };
+    int[] alloc = { 2, 5, 7, 10, 3, 4, 6, 8 };
+    RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
+        .generateRLESparseResourceAllocation(alloc, timeSteps);
     LOG.info(rleSparseVector.toString());
-    Assert.assertEquals(
-        rleSparseVector.getMaximumPeriodicCapacity(0, 1),
+    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 1),
         Resource.newInstance(10, 10));
-    Assert.assertEquals(
-        rleSparseVector.getMaximumPeriodicCapacity(0, 2),
+    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 2),
         Resource.newInstance(7, 7));
-    Assert.assertEquals(
-        rleSparseVector.getMaximumPeriodicCapacity(0, 3),
+    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 3),
         Resource.newInstance(10, 10));
-    Assert.assertEquals(
-        rleSparseVector.getMaximumPeriodicCapacity(0, 4),
+    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 4),
         Resource.newInstance(3, 3));
-    Assert.assertEquals(
-        rleSparseVector.getMaximumPeriodicCapacity(0, 5),
+    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 5),
         Resource.newInstance(4, 4));
-    Assert.assertEquals(
-        rleSparseVector.getMaximumPeriodicCapacity(0, 5),
+    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 5),
         Resource.newInstance(4, 4));
-    Assert.assertEquals(
-        rleSparseVector.getMaximumPeriodicCapacity(7, 5),
+    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(7, 5),
         Resource.newInstance(8, 8));
-    Assert.assertEquals(
-        rleSparseVector.getMaximumPeriodicCapacity(10, 3),
+    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(10, 3),
         Resource.newInstance(0, 0));
   }
 
   @Test
   public void testGetMinimumCapacityInInterval() {
-    long[] timeSteps = {0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L};
-    int[] alloc = {2, 5, 7, 10, 3, 4, 0, 8};
-    RLESparseResourceAllocation rleSparseVector =
-        ReservationSystemTestUtil.generateRLESparseResourceAllocation(
-            alloc, timeSteps);
+    long[] timeSteps = { 0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L };
+    int[] alloc = { 2, 5, 7, 10, 3, 4, 0, 8 };
+    RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
+        .generateRLESparseResourceAllocation(alloc, timeSteps);
     LOG.info(rleSparseVector.toString());
-    Assert.assertEquals(
-        rleSparseVector.getMinimumCapacityInInterval(
-            new ReservationInterval(1L, 3L)), Resource.newInstance(5, 5));
-    Assert.assertEquals(
-        rleSparseVector.getMinimumCapacityInInterval(
-            new ReservationInterval(2L, 5L)), Resource.newInstance(3, 3));
-    Assert.assertEquals(
-        rleSparseVector.getMinimumCapacityInInterval(
-            new ReservationInterval(1L, 7L)), Resource.newInstance(0, 0));
+    Assert.assertEquals(rleSparseVector.getMinimumCapacityInInterval(
+        new ReservationInterval(1L, 3L)), Resource.newInstance(5, 5));
+    Assert.assertEquals(rleSparseVector.getMinimumCapacityInInterval(
+        new ReservationInterval(2L, 5L)), Resource.newInstance(3, 3));
+    Assert.assertEquals(rleSparseVector.getMinimumCapacityInInterval(
+        new ReservationInterval(1L, 7L)), Resource.newInstance(0, 0));
   }
 
-  private void setupArrays(
-      TreeMap<Long, Resource> a, TreeMap<Long, Resource> b) {
+  private void setupArrays(TreeMap<Long, Resource> a,
+      TreeMap<Long, Resource> b) {
     a.put(10L, Resource.newInstance(5, 5));
     a.put(20L, Resource.newInstance(10, 10));
     a.put(30L, Resource.newInstance(15, 15));
@@ -620,8 +616,8 @@ public class TestRLESparseResourceAllocation {
         numContainers = alloc[i];
       }
       req.put(new ReservationInterval(startTime + i, startTime + i + 1),
-          ReservationSystemUtil.toResource(ReservationRequest.newInstance(
-              Resource.newInstance(1024, 1), (numContainers))));
+          ReservationSystemUtil.toResource(ReservationRequest
+              .newInstance(Resource.newInstance(1024, 1), (numContainers))));
     }
     return req;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestSimpleCapacityReplanner.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/reservation/planning/TestSimpleCapacityReplanner.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestSimpleCapacityReplanner.java
index c4f94c2..ddd290d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestSimpleCapacityReplanner.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestSimpleCapacityReplanner.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.yarn.api.records.ReservationDefinition;
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.ReservationRequest;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.InMemoryPlan;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.InMemoryReservationAllocation;
@@ -78,9 +79,10 @@ public class TestSimpleCapacityReplanner {
     enf.init("blah", conf);
 
     // Initialize the plan with more resources
-    InMemoryPlan plan =
-        new InMemoryPlan(queueMetrics, policy, agent, clusterCapacity, step,
-            res, minAlloc, maxAlloc, "dedicated", enf, true, context, clock);
+    InMemoryPlan plan = new InMemoryPlan(queueMetrics, policy, agent,
+        clusterCapacity, step, res, minAlloc, maxAlloc, "dedicated", enf, true,
+        YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY,
+        context, clock);
 
     // add reservation filling the plan (separating them 1ms, so we are sure
     // s2 follows s1 on acceptance


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


[25/50] [abbrv] hadoop git commit: HDFS-12218. Rename split EC / replicated block metrics in BlockManager.

Posted by as...@apache.org.
HDFS-12218. Rename split EC / replicated block metrics in BlockManager.


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

Branch: refs/heads/YARN-5972
Commit: 40c2f31f8dd45bc94291535ad41ffe3cc30b5536
Parents: 52b894d
Author: Andrew Wang <wa...@apache.org>
Authored: Thu Sep 7 16:56:35 2017 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Thu Sep 7 16:56:35 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/protocol/BlocksStats.java       | 90 --------------------
 .../hadoop/hdfs/protocol/ClientProtocol.java    |  4 +-
 .../hadoop/hdfs/protocol/ECBlockGroupStats.java | 83 ++++++++++++++++++
 .../hdfs/protocol/ECBlockGroupsStats.java       | 83 ------------------
 .../hdfs/protocol/ReplicatedBlockStats.java     | 90 ++++++++++++++++++++
 .../ClientNamenodeProtocolTranslatorPB.java     |  8 +-
 .../hadoop/hdfs/protocolPB/PBHelperClient.java  | 38 ++++-----
 .../hdfs/server/namenode/FSNamesystem.java      | 13 +--
 .../hdfs/server/namenode/NameNodeRpcServer.java |  8 +-
 .../org/apache/hadoop/hdfs/tools/DFSAdmin.java  | 26 +++---
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     | 30 +++----
 11 files changed, 237 insertions(+), 236 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/40c2f31f/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BlocksStats.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BlocksStats.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BlocksStats.java
deleted file mode 100644
index 7eb30ca..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BlocksStats.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/**
- * 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.protocol;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * Get statistics pertaining to blocks of type {@link BlockType#CONTIGUOUS}
- * in the filesystem.
- * <p>
- * @see ClientProtocol#getBlocksStats()
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public final class BlocksStats {
-  private final long lowRedundancyBlocksStat;
-  private final long corruptBlocksStat;
-  private final long missingBlocksStat;
-  private final long missingReplicationOneBlocksStat;
-  private final long bytesInFutureBlocksStat;
-  private final long pendingDeletionBlocksStat;
-
-  public BlocksStats(long lowRedundancyBlocksStat,
-      long corruptBlocksStat, long missingBlocksStat,
-      long missingReplicationOneBlocksStat, long bytesInFutureBlocksStat,
-      long pendingDeletionBlocksStat) {
-    this.lowRedundancyBlocksStat = lowRedundancyBlocksStat;
-    this.corruptBlocksStat = corruptBlocksStat;
-    this.missingBlocksStat = missingBlocksStat;
-    this.missingReplicationOneBlocksStat = missingReplicationOneBlocksStat;
-    this.bytesInFutureBlocksStat = bytesInFutureBlocksStat;
-    this.pendingDeletionBlocksStat = pendingDeletionBlocksStat;
-  }
-
-  public long getLowRedundancyBlocksStat() {
-    return lowRedundancyBlocksStat;
-  }
-
-  public long getCorruptBlocksStat() {
-    return corruptBlocksStat;
-  }
-
-  public long getMissingReplicaBlocksStat() {
-    return missingBlocksStat;
-  }
-
-  public long getMissingReplicationOneBlocksStat() {
-    return missingReplicationOneBlocksStat;
-  }
-
-  public long getBytesInFutureBlocksStat() {
-    return bytesInFutureBlocksStat;
-  }
-
-  public long getPendingDeletionBlocksStat() {
-    return pendingDeletionBlocksStat;
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder statsBuilder = new StringBuilder();
-    statsBuilder.append("ReplicatedBlocksStats=[")
-        .append("LowRedundancyBlocks=").append(getLowRedundancyBlocksStat())
-        .append(", CorruptBlocks=").append(getCorruptBlocksStat())
-        .append(", MissingReplicaBlocks=").append(getMissingReplicaBlocksStat())
-        .append(", MissingReplicationOneBlocks=").append(
-            getMissingReplicationOneBlocksStat())
-        .append(", BytesInFutureBlocks=").append(getBytesInFutureBlocksStat())
-        .append(", PendingDeletionBlocks=").append(
-            getPendingDeletionBlocksStat())
-        .append("]");
-    return statsBuilder.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40c2f31f/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 b550467..6626e3b 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
@@ -778,14 +778,14 @@ public interface ClientProtocol {
    * in the filesystem.
    */
   @Idempotent
-  BlocksStats getBlocksStats() throws IOException;
+  ReplicatedBlockStats getBlocksStats() throws IOException;
 
   /**
    * Get statistics pertaining to blocks of type {@link BlockType#STRIPED}
    * in the filesystem.
    */
   @Idempotent
-  ECBlockGroupsStats getECBlockGroupsStats() throws IOException;
+  ECBlockGroupStats getECBlockGroupsStats() throws IOException;
 
   /**
    * Get a report on the system's current datanodes.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40c2f31f/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ECBlockGroupStats.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ECBlockGroupStats.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ECBlockGroupStats.java
new file mode 100644
index 0000000..7258c43
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ECBlockGroupStats.java
@@ -0,0 +1,83 @@
+/**
+ * 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.protocol;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Get statistics pertaining to blocks of type {@link BlockType#STRIPED}
+ * in the filesystem.
+ * <p>
+ * @see ClientProtocol#getECBlockGroupsStats()
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class ECBlockGroupStats {
+  private final long lowRedundancyBlockGroupsStat;
+  private final long corruptBlockGroupsStat;
+  private final long missingBlockGroupsStat;
+  private final long bytesInFutureBlockGroupsStat;
+  private final long pendingDeletionBlockGroupsStat;
+
+  public ECBlockGroupStats(long lowRedundancyBlockGroupsStat, long
+      corruptBlockGroupsStat, long missingBlockGroupsStat, long
+      bytesInFutureBlockGroupsStat, long pendingDeletionBlockGroupsStat) {
+    this.lowRedundancyBlockGroupsStat = lowRedundancyBlockGroupsStat;
+    this.corruptBlockGroupsStat = corruptBlockGroupsStat;
+    this.missingBlockGroupsStat = missingBlockGroupsStat;
+    this.bytesInFutureBlockGroupsStat = bytesInFutureBlockGroupsStat;
+    this.pendingDeletionBlockGroupsStat = pendingDeletionBlockGroupsStat;
+  }
+
+  public long getBytesInFutureBlockGroupsStat() {
+    return bytesInFutureBlockGroupsStat;
+  }
+
+  public long getCorruptBlockGroupsStat() {
+    return corruptBlockGroupsStat;
+  }
+
+  public long getLowRedundancyBlockGroupsStat() {
+    return lowRedundancyBlockGroupsStat;
+  }
+
+  public long getMissingBlockGroupsStat() {
+    return missingBlockGroupsStat;
+  }
+
+  public long getPendingDeletionBlockGroupsStat() {
+    return pendingDeletionBlockGroupsStat;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder statsBuilder = new StringBuilder();
+    statsBuilder.append("ECBlockGroupStats=[")
+        .append("LowRedundancyBlockGroups=").append(
+            getLowRedundancyBlockGroupsStat())
+        .append(", CorruptBlockGroups=").append(getCorruptBlockGroupsStat())
+        .append(", MissingBlockGroups=").append(getMissingBlockGroupsStat())
+        .append(", BytesInFutureBlockGroups=").append(
+            getBytesInFutureBlockGroupsStat())
+        .append(", PendingDeletionBlockGroups=").append(
+            getPendingDeletionBlockGroupsStat())
+        .append("]");
+    return statsBuilder.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40c2f31f/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ECBlockGroupsStats.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ECBlockGroupsStats.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ECBlockGroupsStats.java
deleted file mode 100644
index 80cf262..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ECBlockGroupsStats.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/**
- * 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.protocol;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * Get statistics pertaining to blocks of type {@link BlockType#STRIPED}
- * in the filesystem.
- * <p>
- * @see ClientProtocol#getECBlockGroupsStats()
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public final class ECBlockGroupsStats {
-  private final long lowRedundancyBlockGroupsStat;
-  private final long corruptBlockGroupsStat;
-  private final long missingBlockGroupsStat;
-  private final long bytesInFutureBlockGroupsStat;
-  private final long pendingDeletionBlockGroupsStat;
-
-  public ECBlockGroupsStats(long lowRedundancyBlockGroupsStat, long
-      corruptBlockGroupsStat, long missingBlockGroupsStat, long
-      bytesInFutureBlockGroupsStat, long pendingDeletionBlockGroupsStat) {
-    this.lowRedundancyBlockGroupsStat = lowRedundancyBlockGroupsStat;
-    this.corruptBlockGroupsStat = corruptBlockGroupsStat;
-    this.missingBlockGroupsStat = missingBlockGroupsStat;
-    this.bytesInFutureBlockGroupsStat = bytesInFutureBlockGroupsStat;
-    this.pendingDeletionBlockGroupsStat = pendingDeletionBlockGroupsStat;
-  }
-
-  public long getBytesInFutureBlockGroupsStat() {
-    return bytesInFutureBlockGroupsStat;
-  }
-
-  public long getCorruptBlockGroupsStat() {
-    return corruptBlockGroupsStat;
-  }
-
-  public long getLowRedundancyBlockGroupsStat() {
-    return lowRedundancyBlockGroupsStat;
-  }
-
-  public long getMissingBlockGroupsStat() {
-    return missingBlockGroupsStat;
-  }
-
-  public long getPendingDeletionBlockGroupsStat() {
-    return pendingDeletionBlockGroupsStat;
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder statsBuilder = new StringBuilder();
-    statsBuilder.append("ECBlockGroupsStats=[")
-        .append("LowRedundancyBlockGroups=").append(
-            getLowRedundancyBlockGroupsStat())
-        .append(", CorruptBlockGroups=").append(getCorruptBlockGroupsStat())
-        .append(", MissingBlockGroups=").append(getMissingBlockGroupsStat())
-        .append(", BytesInFutureBlockGroups=").append(
-            getBytesInFutureBlockGroupsStat())
-        .append(", PendingDeletionBlockGroups=").append(
-            getPendingDeletionBlockGroupsStat())
-        .append("]");
-    return statsBuilder.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40c2f31f/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ReplicatedBlockStats.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ReplicatedBlockStats.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ReplicatedBlockStats.java
new file mode 100644
index 0000000..c92dbc7
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ReplicatedBlockStats.java
@@ -0,0 +1,90 @@
+/**
+ * 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.protocol;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Get statistics pertaining to blocks of type {@link BlockType#CONTIGUOUS}
+ * in the filesystem.
+ * <p>
+ * @see ClientProtocol#getBlocksStats()
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class ReplicatedBlockStats {
+  private final long lowRedundancyBlocksStat;
+  private final long corruptBlocksStat;
+  private final long missingBlocksStat;
+  private final long missingReplicationOneBlocksStat;
+  private final long bytesInFutureBlocksStat;
+  private final long pendingDeletionBlocksStat;
+
+  public ReplicatedBlockStats(long lowRedundancyBlocksStat,
+      long corruptBlocksStat, long missingBlocksStat,
+      long missingReplicationOneBlocksStat, long bytesInFutureBlocksStat,
+      long pendingDeletionBlocksStat) {
+    this.lowRedundancyBlocksStat = lowRedundancyBlocksStat;
+    this.corruptBlocksStat = corruptBlocksStat;
+    this.missingBlocksStat = missingBlocksStat;
+    this.missingReplicationOneBlocksStat = missingReplicationOneBlocksStat;
+    this.bytesInFutureBlocksStat = bytesInFutureBlocksStat;
+    this.pendingDeletionBlocksStat = pendingDeletionBlocksStat;
+  }
+
+  public long getLowRedundancyBlocksStat() {
+    return lowRedundancyBlocksStat;
+  }
+
+  public long getCorruptBlocksStat() {
+    return corruptBlocksStat;
+  }
+
+  public long getMissingReplicaBlocksStat() {
+    return missingBlocksStat;
+  }
+
+  public long getMissingReplicationOneBlocksStat() {
+    return missingReplicationOneBlocksStat;
+  }
+
+  public long getBytesInFutureBlocksStat() {
+    return bytesInFutureBlocksStat;
+  }
+
+  public long getPendingDeletionBlocksStat() {
+    return pendingDeletionBlocksStat;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder statsBuilder = new StringBuilder();
+    statsBuilder.append("ReplicatedBlocksStats=[")
+        .append("LowRedundancyBlocks=").append(getLowRedundancyBlocksStat())
+        .append(", CorruptBlocks=").append(getCorruptBlocksStat())
+        .append(", MissingReplicaBlocks=").append(getMissingReplicaBlocksStat())
+        .append(", MissingReplicationOneBlocks=").append(
+            getMissingReplicationOneBlocksStat())
+        .append(", BytesInFutureBlocks=").append(getBytesInFutureBlocksStat())
+        .append(", PendingDeletionBlocks=").append(
+            getPendingDeletionBlocksStat())
+        .append("]");
+    return statsBuilder.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40c2f31f/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
index ec7d93f..53d8804 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
@@ -61,7 +61,7 @@ import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
-import org.apache.hadoop.hdfs.protocol.ECBlockGroupsStats;
+import org.apache.hadoop.hdfs.protocol.ECBlockGroupStats;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -73,7 +73,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.protocol.BlocksStats;
+import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
@@ -695,7 +695,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public BlocksStats getBlocksStats() throws IOException {
+  public ReplicatedBlockStats getBlocksStats() throws IOException {
     try {
       return PBHelperClient.convert(rpcProxy.getFsBlocksStats(null,
           VOID_GET_FS_REPLICABLOCKS_STATS_REQUEST));
@@ -705,7 +705,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public ECBlockGroupsStats getECBlockGroupsStats() throws IOException {
+  public ECBlockGroupStats getECBlockGroupsStats() throws IOException {
     try {
       return PBHelperClient.convert(rpcProxy.getFsECBlockGroupsStats(null,
           VOID_GET_FS_ECBLOCKGROUPS_STATS_REQUEST));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40c2f31f/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
index 04f9686..684ad70 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
@@ -76,7 +76,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo.DatanodeInfoBuilder;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
 import org.apache.hadoop.hdfs.protocol.DatanodeLocalInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
-import org.apache.hadoop.hdfs.protocol.ECBlockGroupsStats;
+import org.apache.hadoop.hdfs.protocol.ECBlockGroupStats;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyState;
@@ -92,7 +92,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
-import org.apache.hadoop.hdfs.protocol.BlocksStats;
+import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeStatus;
@@ -1810,17 +1810,17 @@ public class PBHelperClient {
     return result;
   }
 
-  public static BlocksStats convert(
+  public static ReplicatedBlockStats convert(
       GetFsBlocksStatsResponseProto res) {
-    return new BlocksStats(res.getLowRedundancy(),
+    return new ReplicatedBlockStats(res.getLowRedundancy(),
         res.getCorruptBlocks(), res.getMissingBlocks(),
         res.getMissingReplOneBlocks(), res.getBlocksInFuture(),
         res.getPendingDeletionBlocks());
   }
 
-  public static ECBlockGroupsStats convert(
+  public static ECBlockGroupStats convert(
       GetFsECBlockGroupsStatsResponseProto res) {
-    return new ECBlockGroupsStats(res.getLowRedundancy(),
+    return new ECBlockGroupStats(res.getLowRedundancy(),
         res.getCorruptBlocks(), res.getMissingBlocks(),
         res.getBlocksInFuture(), res.getPendingDeletionBlocks());
   }
@@ -2237,36 +2237,36 @@ public class PBHelperClient {
   }
 
   public static GetFsBlocksStatsResponseProto convert(
-      BlocksStats blocksStats) {
+      ReplicatedBlockStats replicatedBlockStats) {
     GetFsBlocksStatsResponseProto.Builder result =
         GetFsBlocksStatsResponseProto.newBuilder();
     result.setLowRedundancy(
-        blocksStats.getLowRedundancyBlocksStat());
+        replicatedBlockStats.getLowRedundancyBlocksStat());
     result.setCorruptBlocks(
-        blocksStats.getCorruptBlocksStat());
+        replicatedBlockStats.getCorruptBlocksStat());
     result.setMissingBlocks(
-        blocksStats.getMissingReplicaBlocksStat());
+        replicatedBlockStats.getMissingReplicaBlocksStat());
     result.setMissingReplOneBlocks(
-        blocksStats.getMissingReplicationOneBlocksStat());
+        replicatedBlockStats.getMissingReplicationOneBlocksStat());
     result.setBlocksInFuture(
-        blocksStats.getBytesInFutureBlocksStat());
+        replicatedBlockStats.getBytesInFutureBlocksStat());
     result.setPendingDeletionBlocks(
-        blocksStats.getPendingDeletionBlocksStat());
+        replicatedBlockStats.getPendingDeletionBlocksStat());
     return result.build();
   }
 
   public static GetFsECBlockGroupsStatsResponseProto convert(
-      ECBlockGroupsStats ecBlockGroupsStats) {
+      ECBlockGroupStats ecBlockGroupStats) {
     GetFsECBlockGroupsStatsResponseProto.Builder result =
         GetFsECBlockGroupsStatsResponseProto.newBuilder();
     result.setLowRedundancy(
-        ecBlockGroupsStats.getLowRedundancyBlockGroupsStat());
-    result.setCorruptBlocks(ecBlockGroupsStats.getCorruptBlockGroupsStat());
-    result.setMissingBlocks(ecBlockGroupsStats.getMissingBlockGroupsStat());
+        ecBlockGroupStats.getLowRedundancyBlockGroupsStat());
+    result.setCorruptBlocks(ecBlockGroupStats.getCorruptBlockGroupsStat());
+    result.setMissingBlocks(ecBlockGroupStats.getMissingBlockGroupsStat());
     result.setBlocksInFuture(
-        ecBlockGroupsStats.getBytesInFutureBlockGroupsStat());
+        ecBlockGroupStats.getBytesInFutureBlockGroupsStat());
     result.setPendingDeletionBlocks(
-        ecBlockGroupsStats.getPendingDeletionBlockGroupsStat());
+        ecBlockGroupStats.getPendingDeletionBlockGroupsStat());
     return result.build();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40c2f31f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index dedb11e..aada5bf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -89,8 +89,9 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY;
 import static org.apache.hadoop.hdfs.server.namenode.FSDirStatAndListingOp.*;
 
-import org.apache.hadoop.hdfs.protocol.BlocksStats;
-import org.apache.hadoop.hdfs.protocol.ECBlockGroupsStats;
+import org.apache.hadoop.crypto.key.KeyProvider.KeyVersion;
+import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
+import org.apache.hadoop.hdfs.protocol.ECBlockGroupStats;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
 import org.apache.hadoop.hdfs.server.namenode.metrics.ReplicatedBlocksMBean;
@@ -4082,8 +4083,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * <p>
    * @see ClientProtocol#getBlocksStats()
    */
-  BlocksStats getBlocksStats() {
-    return new BlocksStats(getLowRedundancyReplicatedBlocks(),
+  ReplicatedBlockStats getBlocksStats() {
+    return new ReplicatedBlockStats(getLowRedundancyReplicatedBlocks(),
         getCorruptReplicatedBlocks(), getMissingReplicatedBlocks(),
         getMissingReplicationOneBlocks(), getBytesInFutureReplicatedBlocks(),
         getPendingDeletionReplicatedBlocks());
@@ -4095,8 +4096,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * <p>
    * @see ClientProtocol#getECBlockGroupsStats()
    */
-  ECBlockGroupsStats getECBlockGroupsStats() {
-    return new ECBlockGroupsStats(getLowRedundancyECBlockGroups(),
+  ECBlockGroupStats getECBlockGroupsStats() {
+    return new ECBlockGroupStats(getLowRedundancyECBlockGroups(),
         getCorruptECBlockGroups(), getMissingECBlockGroups(),
         getBytesInFutureECBlockGroups(), getPendingDeletionECBlockGroups());
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40c2f31f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index 3fbb7bd..7b14226 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -98,7 +98,7 @@ import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
-import org.apache.hadoop.hdfs.protocol.ECBlockGroupsStats;
+import org.apache.hadoop.hdfs.protocol.ECBlockGroupStats;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -116,7 +116,7 @@ import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.QuotaByStorageTypeExceededException;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
-import org.apache.hadoop.hdfs.protocol.BlocksStats;
+import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
@@ -1163,14 +1163,14 @@ public class NameNodeRpcServer implements NamenodeProtocols {
   }
 
   @Override // ClientProtocol
-  public BlocksStats getBlocksStats() throws IOException {
+  public ReplicatedBlockStats getBlocksStats() throws IOException {
     checkNNStartup();
     namesystem.checkOperation(OperationCategory.READ);
     return namesystem.getBlocksStats();
   }
 
   @Override // ClientProtocol
-  public ECBlockGroupsStats getECBlockGroupsStats() throws IOException {
+  public ECBlockGroupStats getECBlockGroupsStats() throws IOException {
     checkNNStartup();
     namesystem.checkOperation(OperationCategory.READ);
     return namesystem.getECBlockGroupsStats();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40c2f31f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
index 88aafe2..a2bb2c05 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
@@ -66,13 +66,13 @@ import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.NameNodeProxies;
 import org.apache.hadoop.hdfs.NameNodeProxiesClient.ProxyAndInfo;
-import org.apache.hadoop.hdfs.protocol.BlocksStats;
+import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeLocalInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeVolumeInfo;
-import org.apache.hadoop.hdfs.protocol.ECBlockGroupsStats;
+import org.apache.hadoop.hdfs.protocol.ECBlockGroupStats;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
@@ -534,30 +534,30 @@ public class DFSAdmin extends FsShell {
      * minutes. Use "-metaSave" to list of all such blocks and accurate 
      * counts.
      */
-    BlocksStats blocksStats = dfs.getClient().getNamenode().getBlocksStats();
+    ReplicatedBlockStats replicatedBlockStats = dfs.getClient().getNamenode().getBlocksStats();
     System.out.println("Replicated Blocks:");
     System.out.println("\tUnder replicated blocks: " +
-        blocksStats.getLowRedundancyBlocksStat());
+        replicatedBlockStats.getLowRedundancyBlocksStat());
     System.out.println("\tBlocks with corrupt replicas: " +
-        blocksStats.getCorruptBlocksStat());
+        replicatedBlockStats.getCorruptBlocksStat());
     System.out.println("\tMissing blocks: " +
-        blocksStats.getMissingReplicaBlocksStat());
+        replicatedBlockStats.getMissingReplicaBlocksStat());
     System.out.println("\tMissing blocks (with replication factor 1): " +
-        blocksStats.getMissingReplicationOneBlocksStat());
+        replicatedBlockStats.getMissingReplicationOneBlocksStat());
     System.out.println("\tPending deletion blocks: " +
-        blocksStats.getPendingDeletionBlocksStat());
+        replicatedBlockStats.getPendingDeletionBlocksStat());
 
-    ECBlockGroupsStats ecBlockGroupsStats =
+    ECBlockGroupStats ecBlockGroupStats =
         dfs.getClient().getNamenode().getECBlockGroupsStats();
     System.out.println("Erasure Coded Block Groups: ");
     System.out.println("\tLow redundancy block groups: " +
-        ecBlockGroupsStats.getLowRedundancyBlockGroupsStat());
+        ecBlockGroupStats.getLowRedundancyBlockGroupsStat());
     System.out.println("\tBlock groups with corrupt internal blocks: " +
-        ecBlockGroupsStats.getCorruptBlockGroupsStat());
+        ecBlockGroupStats.getCorruptBlockGroupsStat());
     System.out.println("\tMissing block groups: " +
-        ecBlockGroupsStats.getMissingBlockGroupsStat());
+        ecBlockGroupStats.getMissingBlockGroupsStat());
     System.out.println("\tPending deletion block groups: " +
-        ecBlockGroupsStats.getPendingDeletionBlockGroupsStat());
+        ecBlockGroupStats.getPendingDeletionBlockGroupsStat());
 
     System.out.println();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40c2f31f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
index f3572ff..0926b44 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
@@ -117,8 +117,8 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.DatanodeInfoBuilder;
-import org.apache.hadoop.hdfs.protocol.ECBlockGroupsStats;
-import org.apache.hadoop.hdfs.protocol.BlocksStats;
+import org.apache.hadoop.hdfs.protocol.ECBlockGroupStats;
+import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
 import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -1667,36 +1667,36 @@ public class DFSTestUtil {
         cluster.getFileSystem(0).getUri(),
         ClientProtocol.class).getProxy();
     long[] aggregatedStats = cluster.getNameNode().getRpcServer().getStats();
-    BlocksStats blocksStats =
+    ReplicatedBlockStats replicatedBlockStats =
         client.getBlocksStats();
-    ECBlockGroupsStats ecBlockGroupsStats = client.getECBlockGroupsStats();
+    ECBlockGroupStats ecBlockGroupStats = client.getECBlockGroupsStats();
 
     assertEquals("Under replicated stats not matching!",
         aggregatedStats[ClientProtocol.GET_STATS_LOW_REDUNDANCY_IDX],
         aggregatedStats[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX]);
     assertEquals("Low redundancy stats not matching!",
         aggregatedStats[ClientProtocol.GET_STATS_LOW_REDUNDANCY_IDX],
-        blocksStats.getLowRedundancyBlocksStat() +
-            ecBlockGroupsStats.getLowRedundancyBlockGroupsStat());
+        replicatedBlockStats.getLowRedundancyBlocksStat() +
+            ecBlockGroupStats.getLowRedundancyBlockGroupsStat());
     assertEquals("Corrupt blocks stats not matching!",
         aggregatedStats[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX],
-        blocksStats.getCorruptBlocksStat() +
-            ecBlockGroupsStats.getCorruptBlockGroupsStat());
+        replicatedBlockStats.getCorruptBlocksStat() +
+            ecBlockGroupStats.getCorruptBlockGroupsStat());
     assertEquals("Missing blocks stats not matching!",
         aggregatedStats[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX],
-        blocksStats.getMissingReplicaBlocksStat() +
-            ecBlockGroupsStats.getMissingBlockGroupsStat());
+        replicatedBlockStats.getMissingReplicaBlocksStat() +
+            ecBlockGroupStats.getMissingBlockGroupsStat());
     assertEquals("Missing blocks with replication factor one not matching!",
         aggregatedStats[ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX],
-        blocksStats.getMissingReplicationOneBlocksStat());
+        replicatedBlockStats.getMissingReplicationOneBlocksStat());
     assertEquals("Bytes in future blocks stats not matching!",
         aggregatedStats[ClientProtocol.GET_STATS_BYTES_IN_FUTURE_BLOCKS_IDX],
-        blocksStats.getBytesInFutureBlocksStat() +
-            ecBlockGroupsStats.getBytesInFutureBlockGroupsStat());
+        replicatedBlockStats.getBytesInFutureBlocksStat() +
+            ecBlockGroupStats.getBytesInFutureBlockGroupsStat());
     assertEquals("Pending deletion blocks stats not matching!",
         aggregatedStats[ClientProtocol.GET_STATS_PENDING_DELETION_BLOCKS_IDX],
-        blocksStats.getPendingDeletionBlocksStat() +
-            ecBlockGroupsStats.getPendingDeletionBlockGroupsStat());
+        replicatedBlockStats.getPendingDeletionBlocksStat() +
+            ecBlockGroupStats.getPendingDeletionBlockGroupsStat());
   }
 
   /**


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


[29/50] [abbrv] hadoop git commit: YARN-6600. Introduce default and max lifetime of application at LeafQueue level. Contributed by Rohith Sharma K S.

Posted by as...@apache.org.
YARN-6600. Introduce default and max lifetime of application at LeafQueue level. 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/56d93d2e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/56d93d2e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/56d93d2e

Branch: refs/heads/YARN-5972
Commit: 56d93d2e39ead89bb79c4f4096554820dc77e84b
Parents: 3e6d0ca
Author: Sunil G <su...@apache.org>
Authored: Fri Sep 8 07:15:17 2017 +0530
Committer: Sunil G <su...@apache.org>
Committed: Fri Sep 8 07:15:17 2017 +0530

----------------------------------------------------------------------
 .../yarn/api/ApplicationClientProtocol.java     |   2 +-
 .../UpdateApplicationTimeoutsResponse.java      |  19 ++++
 .../hadoop/yarn/client/cli/ApplicationCLI.java  |  16 ++-
 .../hadoop/yarn/client/cli/TestYarnCLI.java     |  22 ++--
 ...UpdateApplicationTimeoutsResponsePBImpl.java | 108 +++++++++++++++++++
 .../conf/capacity-scheduler.xml                 |  35 ++++++
 .../server/resourcemanager/ClientRMService.java |   5 +-
 .../server/resourcemanager/RMAppManager.java    |  30 +++++-
 .../server/resourcemanager/rmapp/RMAppImpl.java |   2 +
 .../scheduler/AbstractYarnScheduler.java        |  11 ++
 .../scheduler/YarnScheduler.java                |  20 ++++
 .../scheduler/capacity/CapacityScheduler.java   |  43 ++++++++
 .../CapacitySchedulerConfiguration.java         |  26 +++++
 .../scheduler/capacity/LeafQueue.java           |  30 +++++-
 .../rmapp/TestApplicationLifetimeMonitor.java   |  56 +++++++++-
 .../capacity/TestCapacityScheduler.java         |  90 ++++++++++++++++
 .../src/site/markdown/CapacityScheduler.md      |  10 ++
 17 files changed, 502 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/56d93d2e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java
index 394454f..6d39366 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java
@@ -578,7 +578,7 @@ public interface ApplicationClientProtocol extends ApplicationBaseProtocol {
    * <b>Note:</b> If application timeout value is less than or equal to current
    * time then update application throws YarnException.
    * @param request to set ApplicationTimeouts of an application
-   * @return an empty response that the update has completed successfully.
+   * @return a response with updated timeouts.
    * @throws YarnException if update request has empty values or application is
    *           in completing states.
    * @throws IOException on IO failures

http://git-wip-us.apache.org/repos/asf/hadoop/blob/56d93d2e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/UpdateApplicationTimeoutsResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/UpdateApplicationTimeoutsResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/UpdateApplicationTimeoutsResponse.java
index bd02bb8..3770eb4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/UpdateApplicationTimeoutsResponse.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/UpdateApplicationTimeoutsResponse.java
@@ -22,6 +22,7 @@ import java.util.Map;
 
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
@@ -43,4 +44,22 @@ public abstract class UpdateApplicationTimeoutsResponse {
         Records.newRecord(UpdateApplicationTimeoutsResponse.class);
     return response;
   }
+
+  /**
+   * Get <code>ApplicationTimeouts</code> of the application. Timeout value is
+   * in ISO8601 standard with format <b>yyyy-MM-dd'T'HH:mm:ss.SSSZ</b>.
+   * @return all <code>ApplicationTimeouts</code> of the application.
+   */
+  public abstract Map<ApplicationTimeoutType, String> getApplicationTimeouts();
+
+  /**
+   * Set the <code>ApplicationTimeouts</code> for the application. Timeout value
+   * is absolute. Timeout value should meet ISO8601 format. Support ISO8601
+   * format is <b>yyyy-MM-dd'T'HH:mm:ss.SSSZ</b>. All pre-existing Map entries
+   * are cleared before adding the new Map.
+   * @param applicationTimeouts <code>ApplicationTimeouts</code>s for the
+   *          application
+   */
+  public abstract void setApplicationTimeouts(
+      Map<ApplicationTimeoutType, String> applicationTimeouts);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/56d93d2e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
index 893348a..5f6b300 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationTimeoutsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationTimeoutsResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -359,10 +360,21 @@ public class ApplicationCLI extends YarnCLI {
         + timeoutType.toString() + " of an application " + applicationId);
     UpdateApplicationTimeoutsRequest request = UpdateApplicationTimeoutsRequest
         .newInstance(appId, Collections.singletonMap(timeoutType, newTimeout));
-    client.updateApplicationTimeouts(request);
+    UpdateApplicationTimeoutsResponse updateApplicationTimeouts =
+        client.updateApplicationTimeouts(request);
+    String updatedTimeout =
+        updateApplicationTimeouts.getApplicationTimeouts().get(timeoutType);
+
+    if (timeoutType.equals(ApplicationTimeoutType.LIFETIME)
+        && !newTimeout.equals(updatedTimeout)) {
+      sysout.println("Updated lifetime of an application  " + applicationId
+          + " to queue max/default lifetime." + " New expiry time is "
+          + updatedTimeout);
+      return;
+    }
     sysout.println(
         "Successfully updated " + timeoutType.toString() + " of an application "
-            + applicationId + ". New expiry time is " + newTimeout);
+            + applicationId + ". New expiry time is " + updatedTimeout);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/56d93d2e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
index 3c35b9c..13730f1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
@@ -48,6 +48,7 @@ import org.apache.commons.cli.Options;
 import org.apache.commons.lang.time.DateFormatUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationTimeoutsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationTimeoutsResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -2148,17 +2149,16 @@ public class TestYarnCLI {
     ApplicationCLI cli = createAndGetAppCLI();
     ApplicationId applicationId = ApplicationId.newInstance(1234, 6);
 
-    ApplicationReport appReport = ApplicationReport.newInstance(applicationId,
-        ApplicationAttemptId.newInstance(applicationId, 1), "user", "queue",
-        "appname", "host", 124, null, YarnApplicationState.RUNNING,
-        "diagnostics", "url", 0, 0, FinalApplicationStatus.UNDEFINED, null,
-        "N/A", 0.53789f, "YARN", null);
-    ApplicationTimeout timeout = ApplicationTimeout
-        .newInstance(ApplicationTimeoutType.LIFETIME, "N/A", -1);
-    appReport.setApplicationTimeouts(
-        Collections.singletonMap(timeout.getTimeoutType(), timeout));
-    when(client.getApplicationReport(any(ApplicationId.class)))
-        .thenReturn(appReport);
+    UpdateApplicationTimeoutsResponse response =
+        mock(UpdateApplicationTimeoutsResponse.class);
+    String formatISO8601 =
+        Times.formatISO8601(System.currentTimeMillis() + 5 * 1000);
+    when(response.getApplicationTimeouts()).thenReturn(Collections
+        .singletonMap(ApplicationTimeoutType.LIFETIME, formatISO8601));
+
+    when(client
+        .updateApplicationTimeouts(any(UpdateApplicationTimeoutsRequest.class)))
+            .thenReturn(response);
 
     int result = cli.run(new String[] { "application", "-appId",
         applicationId.toString(), "-updateLifetime", "10" });

http://git-wip-us.apache.org/repos/asf/hadoop/blob/56d93d2e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/UpdateApplicationTimeoutsResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/UpdateApplicationTimeoutsResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/UpdateApplicationTimeoutsResponsePBImpl.java
index 74f1715..0c94f97 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/UpdateApplicationTimeoutsResponsePBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/UpdateApplicationTimeoutsResponsePBImpl.java
@@ -18,10 +18,19 @@
 
 package org.apache.hadoop.yarn.api.protocolrecords.impl.pb;
 
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationTimeoutsResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType;
+import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
+import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationUpdateTimeoutMapProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.UpdateApplicationTimeoutsResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.UpdateApplicationTimeoutsResponseProtoOrBuilder;
 
 import com.google.protobuf.TextFormat;
 
@@ -33,6 +42,7 @@ public class UpdateApplicationTimeoutsResponsePBImpl
       UpdateApplicationTimeoutsResponseProto.getDefaultInstance();
   UpdateApplicationTimeoutsResponseProto.Builder builder = null;
   boolean viaProto = false;
+  private Map<ApplicationTimeoutType, String> applicationTimeouts = null;
 
   public UpdateApplicationTimeoutsResponsePBImpl() {
     builder = UpdateApplicationTimeoutsResponseProto.newBuilder();
@@ -45,11 +55,34 @@ public class UpdateApplicationTimeoutsResponsePBImpl
   }
 
   public UpdateApplicationTimeoutsResponseProto getProto() {
+    mergeLocalToProto();
     proto = viaProto ? proto : builder.build();
     viaProto = true;
     return proto;
   }
 
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = UpdateApplicationTimeoutsResponseProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.applicationTimeouts != null) {
+      addApplicationTimeouts();
+    }
+  }
+
   @Override
   public int hashCode() {
     return getProto().hashCode();
@@ -70,4 +103,79 @@ public class UpdateApplicationTimeoutsResponsePBImpl
   public String toString() {
     return TextFormat.shortDebugString(getProto());
   }
+
+  @Override
+  public Map<ApplicationTimeoutType, String> getApplicationTimeouts() {
+    initApplicationTimeout();
+    return this.applicationTimeouts;
+  }
+
+  private void initApplicationTimeout() {
+    if (this.applicationTimeouts != null) {
+      return;
+    }
+    UpdateApplicationTimeoutsResponseProtoOrBuilder p =
+        viaProto ? proto : builder;
+    List<ApplicationUpdateTimeoutMapProto> lists =
+        p.getApplicationTimeoutsList();
+    this.applicationTimeouts =
+        new HashMap<ApplicationTimeoutType, String>(lists.size());
+    for (ApplicationUpdateTimeoutMapProto timeoutProto : lists) {
+      this.applicationTimeouts.put(
+          ProtoUtils
+              .convertFromProtoFormat(timeoutProto.getApplicationTimeoutType()),
+          timeoutProto.getExpireTime());
+    }
+  }
+
+  @Override
+  public void setApplicationTimeouts(
+      Map<ApplicationTimeoutType, String> appTimeouts) {
+    if (appTimeouts == null) {
+      return;
+    }
+    initApplicationTimeout();
+    this.applicationTimeouts.clear();
+    this.applicationTimeouts.putAll(appTimeouts);
+  }
+
+  private void addApplicationTimeouts() {
+    maybeInitBuilder();
+    builder.clearApplicationTimeouts();
+    if (applicationTimeouts == null) {
+      return;
+    }
+    Iterable<? extends ApplicationUpdateTimeoutMapProto> values =
+        new Iterable<ApplicationUpdateTimeoutMapProto>() {
+
+          @Override
+          public Iterator<ApplicationUpdateTimeoutMapProto> iterator() {
+            return new Iterator<ApplicationUpdateTimeoutMapProto>() {
+              private Iterator<ApplicationTimeoutType> iterator =
+                  applicationTimeouts.keySet().iterator();
+
+              @Override
+              public boolean hasNext() {
+                return iterator.hasNext();
+              }
+
+              @Override
+              public ApplicationUpdateTimeoutMapProto next() {
+                ApplicationTimeoutType key = iterator.next();
+                return ApplicationUpdateTimeoutMapProto.newBuilder()
+                    .setExpireTime(applicationTimeouts.get(key))
+                    .setApplicationTimeoutType(
+                        ProtoUtils.convertToProtoFormat(key))
+                    .build();
+              }
+
+              @Override
+              public void remove() {
+                throw new UnsupportedOperationException();
+              }
+            };
+          }
+        };
+    this.builder.addAllApplicationTimeouts(values);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/56d93d2e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/conf/capacity-scheduler.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/conf/capacity-scheduler.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/conf/capacity-scheduler.xml
index 785ed04..aca6c7c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/conf/capacity-scheduler.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/conf/capacity-scheduler.xml
@@ -106,6 +106,41 @@
     </description>
   </property>
 
+   <property>
+     <name>yarn.scheduler.capacity.root.default.maximum-application-lifetime
+     </name>
+     <value>-1</value>
+     <description>
+        Maximum lifetime of an application which is submitted to a queue
+        in seconds. Any value less than or equal to zero will be considered as
+        disabled.
+        This will be a hard time limit for all applications in this
+        queue. If positive value is configured then any application submitted
+        to this queue will be killed after exceeds the configured lifetime.
+        User can also specify lifetime per application basis in
+        application submission context. But user lifetime will be
+        overridden if it exceeds queue maximum lifetime. It is point-in-time
+        configuration.
+        Note : Configuring too low value will result in killing application
+        sooner. This feature is applicable only for leaf queue.
+     </description>
+   </property>
+
+   <property>
+     <name>yarn.scheduler.capacity.root.default.default-application-lifetime
+     </name>
+     <value>-1</value>
+     <description>
+        Default lifetime of an application which is submitted to a queue
+        in seconds. Any value less than or equal to zero will be considered as
+        disabled.
+        If the user has not submitted application with lifetime value then this
+        value will be taken. It is point-in-time configuration.
+        Note : Default lifetime can't exceed maximum lifetime. This feature is
+        applicable only for leaf queue.
+     </description>
+   </property>
+
   <property>
     <name>yarn.scheduler.capacity.node-locality-delay</name>
     <value>40</value>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/56d93d2e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.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/ClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
index e6c25ad..df38893 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
@@ -1691,6 +1691,7 @@ public class ClientRMService extends AbstractService implements
         RMAuditLogger.logSuccess(callerUGI.getShortUserName(),
             AuditConstants.UPDATE_APP_TIMEOUTS, "ClientRMService",
             applicationId);
+        response.setApplicationTimeouts(applicationTimeouts);
         return response;
       }
       String msg =
@@ -1702,7 +1703,8 @@ public class ClientRMService extends AbstractService implements
     }
 
     try {
-      rmAppManager.updateApplicationTimeout(application, applicationTimeouts);
+      applicationTimeouts = rmAppManager.updateApplicationTimeout(application,
+          applicationTimeouts);
     } catch (YarnException ex) {
       RMAuditLogger.logFailure(callerUGI.getShortUserName(),
           AuditConstants.UPDATE_APP_TIMEOUTS, "UNKNOWN", "ClientRMService",
@@ -1712,6 +1714,7 @@ public class ClientRMService extends AbstractService implements
 
     RMAuditLogger.logSuccess(callerUGI.getShortUserName(),
         AuditConstants.UPDATE_APP_TIMEOUTS, "ClientRMService", applicationId);
+    response.setApplicationTimeouts(applicationTimeouts);
     return response;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/56d93d2e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.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/RMAppManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
index bcd1a9c..cb2828f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
@@ -66,6 +66,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
+import org.apache.hadoop.yarn.util.Times;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.util.concurrent.SettableFuture;
@@ -571,18 +572,41 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
   }
 
   // transaction method.
-  public void updateApplicationTimeout(RMApp app,
+  public Map<ApplicationTimeoutType, String> updateApplicationTimeout(RMApp app,
       Map<ApplicationTimeoutType, String> newTimeoutInISO8601Format)
       throws YarnException {
     ApplicationId applicationId = app.getApplicationId();
     synchronized (applicationId) {
       if (app.isAppInCompletedStates()) {
-        return;
+        return newTimeoutInISO8601Format;
       }
 
       Map<ApplicationTimeoutType, Long> newExpireTime = RMServerUtils
           .validateISO8601AndConvertToLocalTimeEpoch(newTimeoutInISO8601Format);
 
+      // validation is only for lifetime
+      Long updatedlifetimeInMillis =
+          newExpireTime.get(ApplicationTimeoutType.LIFETIME);
+      if (updatedlifetimeInMillis != null) {
+        long queueMaxLifetimeInSec =
+            scheduler.getMaximumApplicationLifetime(app.getQueue());
+
+        if (queueMaxLifetimeInSec > 0) {
+          if (updatedlifetimeInMillis > (app.getSubmitTime()
+              + queueMaxLifetimeInSec * 1000)) {
+            updatedlifetimeInMillis =
+                app.getSubmitTime() + queueMaxLifetimeInSec * 1000;
+            // cut off to maximum queue lifetime if update lifetime is exceeding
+            // queue lifetime.
+            newExpireTime.put(ApplicationTimeoutType.LIFETIME,
+                updatedlifetimeInMillis);
+
+            newTimeoutInISO8601Format.put(ApplicationTimeoutType.LIFETIME,
+                Times.formatISO8601(updatedlifetimeInMillis.longValue()));
+          }
+        }
+      }
+
       SettableFuture<Object> future = SettableFuture.create();
 
       Map<ApplicationTimeoutType, Long> currentExpireTimeouts =
@@ -605,6 +629,8 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
 
       // update in-memory
       ((RMAppImpl) app).updateApplicationTimeout(newExpireTime);
+
+      return newTimeoutInISO8601Format;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/56d93d2e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.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/rmapp/RMAppImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
index 13b0792..665f458 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
@@ -1225,6 +1225,8 @@ public class RMAppImpl implements RMApp, Recoverable {
 
       long applicationLifetime =
           app.getApplicationLifetime(ApplicationTimeoutType.LIFETIME);
+      applicationLifetime = app.scheduler
+          .checkAndGetApplicationLifetime(app.queue, applicationLifetime);
       if (applicationLifetime > 0) {
         // calculate next timeout value
         Long newTimeout =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/56d93d2e/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 2c27017..0c07b3e 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
@@ -1283,4 +1283,15 @@ public abstract class AbstractYarnScheduler
     this.rmContext.getDispatcher().getEventHandler()
         .handle(new ReleaseContainerEvent(container));
   }
+
+  @Override
+  public long checkAndGetApplicationLifetime(String queueName, long lifetime) {
+    // -1 indicates, lifetime is not configured.
+    return -1;
+  }
+
+  @Override
+  public long getMaximumApplicationLifetime(String queueName) {
+    return -1;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/56d93d2e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.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/YarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
index 08e0603..111998b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
@@ -385,4 +385,24 @@ public interface YarnScheduler extends EventHandler<SchedulerEvent> {
    * @return the normalized resource
    */
   Resource getNormalizedResource(Resource requestedResource);
+
+  /**
+   * Verify whether a submitted application lifetime is valid as per configured
+   * Queue lifetime.
+   * @param queueName Name of the Queue
+   * @param lifetime configured application lifetime
+   * @return valid lifetime as per queue
+   */
+  @Public
+  @Evolving
+  long checkAndGetApplicationLifetime(String queueName, long lifetime);
+
+  /**
+   * Get maximum lifetime for a queue.
+   * @param queueName to get lifetime
+   * @return maximum lifetime in seconds
+   */
+  @Public
+  @Evolving
+  long getMaximumApplicationLifetime(String queueName);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/56d93d2e/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 fde84c4..deec091 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
@@ -2564,4 +2564,47 @@ public class CapacityScheduler extends
       writeLock.unlock();
     }
   }
+
+  @Override
+  public long checkAndGetApplicationLifetime(String queueName,
+      long lifetimeRequestedByApp) {
+    try {
+      readLock.lock();
+      CSQueue queue = getQueue(queueName);
+      if (queue == null || !(queue instanceof LeafQueue)) {
+        return lifetimeRequestedByApp;
+      }
+
+      long defaultApplicationLifetime =
+          ((LeafQueue) queue).getDefaultApplicationLifetime();
+      long maximumApplicationLifetime =
+          ((LeafQueue) queue).getMaximumApplicationLifetime();
+
+      // check only for maximum, that's enough because default cann't
+      // exceed maximum
+      if (maximumApplicationLifetime <= 0) {
+        return lifetimeRequestedByApp;
+      }
+
+      if (lifetimeRequestedByApp <= 0) {
+        return defaultApplicationLifetime;
+      } else if (lifetimeRequestedByApp > maximumApplicationLifetime) {
+        return maximumApplicationLifetime;
+      }
+      return lifetimeRequestedByApp;
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  @Override
+  public long getMaximumApplicationLifetime(String queueName) {
+    CSQueue queue = getQueue(queueName);
+    if (queue == null || !(queue instanceof LeafQueue)) {
+      LOG.error("Unknown queue: " + queueName);
+      return -1;
+    }
+    // In seconds
+    return ((LeafQueue) queue).getMaximumApplicationLifetime();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/56d93d2e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.java
index 13b9ff6..3a519ec 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/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.java
@@ -1496,4 +1496,30 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
   public int getMaxAssignPerHeartbeat() {
     return getInt(MAX_ASSIGN_PER_HEARTBEAT, DEFAULT_MAX_ASSIGN_PER_HEARTBEAT);
   }
+
+  public static final String MAXIMUM_LIFETIME_SUFFIX =
+      "maximum-application-lifetime";
+
+  public static final String DEFAULT_LIFETIME_SUFFIX =
+      "default-application-lifetime";
+
+  public long getMaximumLifetimePerQueue(String queue) {
+    long maximumLifetimePerQueue = getLong(
+        getQueuePrefix(queue) + MAXIMUM_LIFETIME_SUFFIX, (long) UNDEFINED);
+    return maximumLifetimePerQueue;
+  }
+
+  public void setMaximumLifetimePerQueue(String queue, long maximumLifetime) {
+    setLong(getQueuePrefix(queue) + MAXIMUM_LIFETIME_SUFFIX, maximumLifetime);
+  }
+
+  public long getDefaultLifetimePerQueue(String queue) {
+    long maximumLifetimePerQueue = getLong(
+        getQueuePrefix(queue) + DEFAULT_LIFETIME_SUFFIX, (long) UNDEFINED);
+    return maximumLifetimePerQueue;
+  }
+
+  public void setDefaultLifetimePerQueue(String queue, long defaultLifetime) {
+    setLong(getQueuePrefix(queue) + DEFAULT_LIFETIME_SUFFIX, defaultLifetime);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/56d93d2e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.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/LeafQueue.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/LeafQueue.java
index d15431e..ffe0c0f 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/LeafQueue.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/LeafQueue.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.QueueState;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
@@ -130,6 +131,10 @@ public class LeafQueue extends AbstractCSQueue {
   List<AppPriorityACLGroup> priorityAcls =
       new ArrayList<AppPriorityACLGroup>();
 
+  // -1 indicates lifetime is disabled
+  private volatile long maxApplicationLifetime = -1;
+  private volatile long defaultApplicationLifetime = -1;
+
   @SuppressWarnings({ "unchecked", "rawtypes" })
   public LeafQueue(CapacitySchedulerContext cs,
       String queueName, CSQueue parent, CSQueue old) throws IOException {
@@ -238,6 +243,18 @@ public class LeafQueue extends AbstractCSQueue {
       defaultAppPriorityPerQueue = Priority.newInstance(
           conf.getDefaultApplicationPriorityConfPerQueue(getQueuePath()));
 
+      maxApplicationLifetime =
+          conf.getMaximumLifetimePerQueue((getQueuePath()));
+      defaultApplicationLifetime =
+          conf.getDefaultLifetimePerQueue((getQueuePath()));
+      if (defaultApplicationLifetime > maxApplicationLifetime) {
+        throw new YarnRuntimeException(
+            "Default lifetime" + defaultApplicationLifetime
+                + " can't exceed maximum lifetime " + maxApplicationLifetime);
+      }
+      defaultApplicationLifetime = defaultApplicationLifetime > 0
+          ? defaultApplicationLifetime : maxApplicationLifetime;
+
       // Validate leaf queue's user's weights.
       int queueUL = Math.min(100, conf.getUserLimit(getQueuePath()));
       for (Entry<String, Float> e : getUserWeights().entrySet()) {
@@ -293,7 +310,10 @@ public class LeafQueue extends AbstractCSQueue {
               + "reservationsContinueLooking = "
               + reservationsContinueLooking + "\n" + "preemptionDisabled = "
               + getPreemptionDisabled() + "\n" + "defaultAppPriorityPerQueue = "
-              + defaultAppPriorityPerQueue + "\npriority = " + priority);
+              + defaultAppPriorityPerQueue + "\npriority = " + priority
+              + "\nmaxLifetime = " + maxApplicationLifetime + " seconds"
+              + "\ndefaultLifetime = "
+              + defaultApplicationLifetime + " seconds");
     } finally {
       writeLock.unlock();
     }
@@ -2086,4 +2106,12 @@ public class LeafQueue extends AbstractCSQueue {
       this.userLimit = userLimit;
     }
   }
+
+  public long getMaximumApplicationLifetime() {
+    return maxApplicationLifetime;
+  }
+
+  public long getDefaultApplicationLifetime() {
+    return defaultApplicationLifetime;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/56d93d2e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestApplicationLifetimeMonitor.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/rmapp/TestApplicationLifetimeMonitor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestApplicationLifetimeMonitor.java
index f7e76bb..a7808d7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestApplicationLifetimeMonitor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestApplicationLifetimeMonitor.java
@@ -23,6 +23,7 @@ import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
@@ -50,6 +51,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationStateData;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
+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.util.Times;
 import org.apache.log4j.Level;
 import org.apache.log4j.LogManager;
@@ -67,6 +71,9 @@ public class TestApplicationLifetimeMonitor {
   @Before
   public void setup() throws IOException {
     conf = new YarnConfiguration();
+    // Always run for CS, since other scheduler do not support this.
+    conf.setClass(YarnConfiguration.RM_SCHEDULER,
+        CapacityScheduler.class, ResourceScheduler.class);
     Logger rootLogger = LogManager.getRootLogger();
     rootLogger.setLevel(Level.DEBUG);
     UserGroupInformation.setConfiguration(conf);
@@ -78,8 +85,15 @@ public class TestApplicationLifetimeMonitor {
   public void testApplicationLifetimeMonitor() throws Exception {
     MockRM rm = null;
     try {
+      long maxLifetime = 30L;
+      long defaultLifetime = 15L;
+
+      YarnConfiguration newConf =
+          new YarnConfiguration(setUpCSQueue(maxLifetime, defaultLifetime));
+      conf = new YarnConfiguration(newConf);
       rm = new MockRM(conf);
       rm.start();
+
       Priority appPriority = Priority.newInstance(0);
       MockNM nm1 = rm.registerNode("127.0.0.1:1234", 16 * 1024);
 
@@ -92,6 +106,13 @@ public class TestApplicationLifetimeMonitor {
       timeouts.put(ApplicationTimeoutType.LIFETIME, 20L);
       RMApp app2 = rm.submitApp(1024, appPriority, timeouts);
 
+      // user not set lifetime, so queue max lifetime will be considered.
+      RMApp app3 = rm.submitApp(1024, appPriority, Collections.emptyMap());
+
+      // asc lifetime exceeds queue max lifetime
+      timeouts.put(ApplicationTimeoutType.LIFETIME, 40L);
+      RMApp app4 = rm.submitApp(1024, appPriority, timeouts);
+
       nm1.nodeHeartbeat(true);
       // Send launch Event
       MockAM am1 =
@@ -103,8 +124,9 @@ public class TestApplicationLifetimeMonitor {
 
       Map<ApplicationTimeoutType, String> updateTimeout =
           new HashMap<ApplicationTimeoutType, String>();
-      long newLifetime = 10L;
-      // update 10L seconds more to timeout
+      long newLifetime = 40L;
+      // update 30L seconds more to timeout which is greater than queue max
+      // lifetime
       String formatISO8601 =
           Times.formatISO8601(System.currentTimeMillis() + newLifetime * 1000);
       updateTimeout.put(ApplicationTimeoutType.LIFETIME, formatISO8601);
@@ -142,8 +164,6 @@ public class TestApplicationLifetimeMonitor {
           !appTimeouts.isEmpty());
       ApplicationTimeout timeout =
           appTimeouts.get(ApplicationTimeoutType.LIFETIME);
-      Assert.assertEquals("Application timeout string is incorrect.",
-          formatISO8601, timeout.getExpiryTime());
       Assert.assertTrue("Application remaining time is incorrect",
           timeout.getRemainingTime() > 0);
 
@@ -152,6 +172,17 @@ public class TestApplicationLifetimeMonitor {
       Assert.assertTrue("Application killed before lifetime value",
           app2.getFinishTime() > afterUpdate);
 
+      rm.waitForState(app3.getApplicationId(), RMAppState.KILLED);
+
+      // app4 submitted exceeding queue max lifetime, so killed after queue max
+      // lifetime.
+      rm.waitForState(app4.getApplicationId(), RMAppState.KILLED);
+      long totalTimeRun = (app4.getFinishTime() - app4.getSubmitTime()) / 1000;
+      Assert.assertTrue("Application killed before lifetime value",
+          totalTimeRun > maxLifetime);
+      Assert.assertTrue(
+          "Application killed before lifetime value " + totalTimeRun,
+          totalTimeRun < maxLifetime + 10L);
     } finally {
       stopRM(rm);
     }
@@ -172,7 +203,7 @@ public class TestApplicationLifetimeMonitor {
     nm1.registerNode();
     nm1.nodeHeartbeat(true);
 
-    long appLifetime = 60L;
+    long appLifetime = 30L;
     Map<ApplicationTimeoutType, Long> timeouts =
         new HashMap<ApplicationTimeoutType, Long>();
     timeouts.put(ApplicationTimeoutType.LIFETIME, appLifetime);
@@ -305,6 +336,21 @@ public class TestApplicationLifetimeMonitor {
     }
   }
 
+  private CapacitySchedulerConfiguration setUpCSQueue(long maxLifetime,
+      long defaultLifetime) {
+    CapacitySchedulerConfiguration csConf =
+        new CapacitySchedulerConfiguration();
+    csConf.setQueues(CapacitySchedulerConfiguration.ROOT,
+        new String[] {"default"});
+    csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + ".default", 100);
+    csConf.setMaximumLifetimePerQueue(
+        CapacitySchedulerConfiguration.ROOT + ".default", maxLifetime);
+    csConf.setDefaultLifetimePerQueue(
+        CapacitySchedulerConfiguration.ROOT + ".default", defaultLifetime);
+
+    return csConf;
+  }
+
   private void stopRM(MockRM rm) {
     if (rm != null) {
       rm.stop();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/56d93d2e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.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/TestCapacityScheduler.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/TestCapacityScheduler.java
index a526222..a3e518d 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/TestCapacityScheduler.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/TestCapacityScheduler.java
@@ -4909,6 +4909,96 @@ public class TestCapacityScheduler {
     rm.stop();
   }
 
+  @Test(timeout = 30000)
+  public void testcheckAndGetApplicationLifetime() throws Exception {
+    long maxLifetime = 10;
+    long defaultLifetime = 5;
+    // positive integer value
+    CapacityScheduler cs = setUpCSQueue(maxLifetime, defaultLifetime);
+    Assert.assertEquals(maxLifetime,
+        cs.checkAndGetApplicationLifetime("default", 100));
+    Assert.assertEquals(9, cs.checkAndGetApplicationLifetime("default", 9));
+    Assert.assertEquals(defaultLifetime,
+        cs.checkAndGetApplicationLifetime("default", -1));
+    Assert.assertEquals(defaultLifetime,
+        cs.checkAndGetApplicationLifetime("default", 0));
+    Assert.assertEquals(maxLifetime,
+        cs.getMaximumApplicationLifetime("default"));
+
+    maxLifetime = -1;
+    defaultLifetime = -1;
+    // test for default values
+    cs = setUpCSQueue(maxLifetime, defaultLifetime);
+    Assert.assertEquals(100, cs.checkAndGetApplicationLifetime("default", 100));
+    Assert.assertEquals(defaultLifetime,
+        cs.checkAndGetApplicationLifetime("default", -1));
+    Assert.assertEquals(0, cs.checkAndGetApplicationLifetime("default", 0));
+    Assert.assertEquals(maxLifetime,
+        cs.getMaximumApplicationLifetime("default"));
+
+    maxLifetime = 10;
+    defaultLifetime = 10;
+    cs = setUpCSQueue(maxLifetime, defaultLifetime);
+    Assert.assertEquals(maxLifetime,
+        cs.checkAndGetApplicationLifetime("default", 100));
+    Assert.assertEquals(defaultLifetime,
+        cs.checkAndGetApplicationLifetime("default", -1));
+    Assert.assertEquals(defaultLifetime,
+        cs.checkAndGetApplicationLifetime("default", 0));
+    Assert.assertEquals(maxLifetime,
+        cs.getMaximumApplicationLifetime("default"));
+
+    maxLifetime = 0;
+    defaultLifetime = 0;
+    cs = setUpCSQueue(maxLifetime, defaultLifetime);
+    Assert.assertEquals(100, cs.checkAndGetApplicationLifetime("default", 100));
+    Assert.assertEquals(-1, cs.checkAndGetApplicationLifetime("default", -1));
+    Assert.assertEquals(0, cs.checkAndGetApplicationLifetime("default", 0));
+
+    maxLifetime = 10;
+    defaultLifetime = -1;
+    cs = setUpCSQueue(maxLifetime, defaultLifetime);
+    Assert.assertEquals(maxLifetime,
+        cs.checkAndGetApplicationLifetime("default", 100));
+    Assert.assertEquals(maxLifetime,
+        cs.checkAndGetApplicationLifetime("default", -1));
+    Assert.assertEquals(maxLifetime,
+        cs.checkAndGetApplicationLifetime("default", 0));
+
+    maxLifetime = 5;
+    defaultLifetime = 10;
+    try {
+      setUpCSQueue(maxLifetime, defaultLifetime);
+      Assert.fail("Expected to fails since maxLifetime < defaultLifetime.");
+    } catch (YarnRuntimeException ye) {
+      Assert.assertTrue(
+          ye.getMessage().contains("can't exceed maximum lifetime"));
+    }
+  }
+
+  private CapacityScheduler setUpCSQueue(long maxLifetime,
+      long defaultLifetime) {
+    CapacitySchedulerConfiguration csConf =
+        new CapacitySchedulerConfiguration();
+    csConf.setQueues(CapacitySchedulerConfiguration.ROOT,
+        new String[] {"default"});
+    csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + ".default", 100);
+    csConf.setMaximumLifetimePerQueue(
+        CapacitySchedulerConfiguration.ROOT + ".default", maxLifetime);
+    csConf.setDefaultLifetimePerQueue(
+        CapacitySchedulerConfiguration.ROOT + ".default", defaultLifetime);
+
+    YarnConfiguration conf = new YarnConfiguration(csConf);
+    CapacityScheduler cs = new CapacityScheduler();
+
+    RMContext rmContext = TestUtils.getMockRMContext();
+    cs.setConf(conf);
+    cs.setRMContext(rmContext);
+    cs.init(conf);
+
+    return cs;
+  }
+
   private void waitforNMRegistered(ResourceScheduler scheduler, int nodecount,
       int timesec) throws InterruptedException {
     long start = System.currentTimeMillis();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/56d93d2e/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 f1d4535..6bb8489 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
@@ -171,6 +171,16 @@ Example:
  </property>
 ```
 
+  * Queue lifetime for applications
+
+    The `CapacityScheduler` supports the following parameters to lifetime of an application:
+
+| Property | Description |
+|:---- |:---- |
+| `yarn.scheduler.capacity.<queue-path>.maximum-application-lifetime` | Maximum lifetime of an application which is submitted to a queue in seconds. Any value less than or equal to zero will be considered as disabled. This will be a hard time limit for all applications in this queue. If positive value is configured then any application submitted to this queue will be killed after exceeds the configured lifetime. User can also specify lifetime per application basis in application submission context. But user lifetime will be overridden if it exceeds queue maximum lifetime. It is point-in-time configuration. Note : Configuring too low value will result in killing application sooner. This feature is applicable only for leaf queue. |
+| `yarn.scheduler.capacity.root.<queue-path>.default-application-lifetime` | Default lifetime of an application which is submitted to a queue in seconds. Any value less than or equal to zero will be considered as disabled. If the user has not submitted application with lifetime value then this value will be taken. It is point-in-time configuration. Note : Default lifetime can't exceed maximum lifetime. This feature is applicable only for leaf queue.|
+
+
 ###Setup for application priority.
 
   Application priority works only along with FIFO ordering policy. Default ordering policy is FIFO.


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


[50/50] [abbrv] hadoop git commit: YARN-5216. Expose configurable preemption policy for OPPORTUNISTIC containers running on the NM. (Hitesh Sharma via asuresh)

Posted by as...@apache.org.
YARN-5216. Expose configurable preemption policy for OPPORTUNISTIC containers running on the NM. (Hitesh Sharma via asuresh)


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

Branch: refs/heads/YARN-5972
Commit: 026f877b9ba07e2564d583b5ed908cb72098ec44
Parents: 01eb8ce
Author: Arun Suresh <as...@apache.org>
Authored: Sat Dec 24 17:16:52 2016 -0800
Committer: Arun Suresh <as...@apache.org>
Committed: Sun Sep 10 23:28:03 2017 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/conf/YarnConfiguration.java     |   9 ++
 .../src/main/resources/yarn-default.xml         |   9 ++
 .../containermanager/container/Container.java   |   2 +
 .../container/ContainerImpl.java                |  32 ++++--
 .../scheduler/ContainerScheduler.java           |  85 ++++++++++++---
 .../TestContainerSchedulerQueuing.java          | 103 +++++++++++++++++++
 .../nodemanager/webapp/MockContainer.java       |   5 +
 7 files changed, 219 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/026f877b/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 be63233..4fe4400 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
@@ -1036,6 +1036,15 @@ public class YarnConfiguration extends Configuration {
       NM_PREFIX + "container-retry-minimum-interval-ms";
   public static final int DEFAULT_NM_CONTAINER_RETRY_MINIMUM_INTERVAL_MS = 1000;
 
+  /**
+   * Use container pause as the preemption policy over kill in the container
+   * queue at a NodeManager.
+   **/
+  public static final String NM_CONTAINER_QUEUING_USE_PAUSE_FOR_PREEMPTION =
+      NM_PREFIX + "opportunistic-containers-use-pause-for-preemption";
+  public static final boolean
+      DEFAULT_NM_CONTAINER_QUEUING_USE_PAUSE_FOR_PREEMPTION = false;
+
   /** Interval at which the delayed token removal thread runs */
   public static final String RM_DELAYED_DELEGATION_TOKEN_REMOVAL_INTERVAL_MS =
       RM_PREFIX + "delayed.delegation-token.removal-interval-ms";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/026f877b/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 afde222..af42845 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
@@ -3003,6 +3003,15 @@
 
   <property>
     <description>
+    Use container pause as the preemption policy over kill in the container
+    queue at a NodeManager.
+    </description>
+    <name>yarn.nodemanager.opportunistic-containers-use-pause-for-preemption</name>
+    <value>false</value>
+  </property>
+
+  <property>
+    <description>
     Error filename pattern, to identify the file in the container's
     Log directory which contain the container's error log. As error file
     redirection is done by client/AM and yarn will not be aware of the error

http://git-wip-us.apache.org/repos/asf/hadoop/blob/026f877b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.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/container/Container.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java
index ef5d72c..86f2554 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java
@@ -103,4 +103,6 @@ public interface Container extends EventHandler<ContainerEvent> {
    * @return Resource Mappings of the container
    */
   ResourceMappings getResourceMappings();
+
+  void sendPauseEvent(String description);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/026f877b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.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/container/ContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
index 7a12371..95ebfd5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
@@ -816,15 +816,22 @@ public class ContainerImpl implements Container {
   @SuppressWarnings("unchecked") // dispatcher not typed
   @Override
   public void sendLaunchEvent() {
-    ContainersLauncherEventType launcherEvent =
-        ContainersLauncherEventType.LAUNCH_CONTAINER;
-    if (recoveredStatus == RecoveredContainerStatus.LAUNCHED) {
-      // try to recover a container that was previously launched
-      launcherEvent = ContainersLauncherEventType.RECOVER_CONTAINER;
+    if (ContainerState.PAUSED == getContainerState()) {
+      dispatcher.getEventHandler().handle(
+          new ContainerResumeEvent(containerId,
+              "Container Resumed as some resources freed up"));
+    } else {
+      ContainersLauncherEventType launcherEvent =
+          ContainersLauncherEventType.LAUNCH_CONTAINER;
+      if (recoveredStatus == RecoveredContainerStatus.LAUNCHED) {
+        // try to recover a container that was previously launched
+        launcherEvent = ContainersLauncherEventType.RECOVER_CONTAINER;
+      }
+      containerLaunchStartTime = clock.getTime();
+      dispatcher.getEventHandler().handle(
+          new ContainersLauncherEvent(this, launcherEvent));
     }
-    containerLaunchStartTime = clock.getTime();
-    dispatcher.getEventHandler().handle(
-        new ContainersLauncherEvent(this, launcherEvent));
+
   }
 
   @SuppressWarnings("unchecked") // dispatcher not typed
@@ -844,6 +851,13 @@ public class ContainerImpl implements Container {
   }
 
   @SuppressWarnings("unchecked") // dispatcher not typed
+  @Override
+  public void sendPauseEvent(String description) {
+    dispatcher.getEventHandler().handle(
+        new ContainerPauseEvent(containerId, description));
+  }
+
+  @SuppressWarnings("unchecked") // dispatcher not typed
   private void sendRelaunchEvent() {
     ContainersLauncherEventType launcherEvent =
         ContainersLauncherEventType.RELAUNCH_CONTAINER;
@@ -1799,7 +1813,7 @@ public class ContainerImpl implements Container {
 
   /**
    * Transitions upon receiving PAUSE_CONTAINER.
-   * - RUNNING -> PAUSED
+   * - RUNNING -> PAUSING
    */
   @SuppressWarnings("unchecked") // dispatcher not typed
   static class PauseContainerTransition implements

http://git-wip-us.apache.org/repos/asf/hadoop/blob/026f877b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.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/scheduler/ContainerScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java
index 644bdae..e39d2c1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -34,6 +35,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Cont
 
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor
     .ChangeMonitoringContainerResourceEvent;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
 
 
@@ -74,7 +76,7 @@ public class ContainerScheduler extends AbstractService implements
       queuedOpportunisticContainers = new LinkedHashMap<>();
 
   // Used to keep track of containers that have been marked to be killed
-  // to make room for a guaranteed container.
+  // or paused to make room for a guaranteed container.
   private final Map<ContainerId, Container> oppContainersToKill =
       new HashMap<>();
 
@@ -98,6 +100,8 @@ public class ContainerScheduler extends AbstractService implements
   private final AsyncDispatcher dispatcher;
   private final NodeManagerMetrics metrics;
 
+  private Boolean usePauseEventForPreemption = false;
+
   /**
    * Instantiate a Container Scheduler.
    * @param context NodeManager Context.
@@ -112,6 +116,17 @@ public class ContainerScheduler extends AbstractService implements
             DEFAULT_NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH));
   }
 
+
+  @Override
+  public void serviceInit(Configuration conf) throws Exception {
+    super.serviceInit(conf);
+    this.usePauseEventForPreemption =
+        conf.getBoolean(
+            YarnConfiguration.NM_CONTAINER_QUEUING_USE_PAUSE_FOR_PREEMPTION,
+            YarnConfiguration.
+                DEFAULT_NM_CONTAINER_QUEUING_USE_PAUSE_FOR_PREEMPTION);
+  }
+
   @VisibleForTesting
   public ContainerScheduler(Context context, AsyncDispatcher dispatcher,
       NodeManagerMetrics metrics, int qLength) {
@@ -136,8 +151,9 @@ public class ContainerScheduler extends AbstractService implements
     case SCHEDULE_CONTAINER:
       scheduleContainer(event.getContainer());
       break;
+    case CONTAINER_PAUSED:
     case CONTAINER_COMPLETED:
-      onContainerCompleted(event.getContainer());
+      onResourcesReclaimed(event.getContainer());
       break;
     case UPDATE_CONTAINER:
       if (event instanceof UpdateContainerSchedulerEvent) {
@@ -203,9 +219,9 @@ public class ContainerScheduler extends AbstractService implements
             queuedGuaranteedContainers.put(containerId,
                 updateEvent.getContainer());
           }
-          //Kill opportunistic containers if any to make room for
+          //Kill/pause opportunistic containers if any to make room for
           // promotion request
-          killOpportunisticContainers(updateEvent.getContainer());
+          reclaimOpportunisticContainerResources(updateEvent.getContainer());
         } else {
           // Demotion of queued container.. Should not happen too often
           // since you should not find too many queued guaranteed
@@ -238,6 +254,12 @@ public class ContainerScheduler extends AbstractService implements
     return this.queuedOpportunisticContainers.size();
   }
 
+  @VisibleForTesting
+  public void setUsePauseEventForPreemption(
+      boolean usePauseEventForPreemption) {
+    this.usePauseEventForPreemption = usePauseEventForPreemption;
+  }
+
   public OpportunisticContainersStatus getOpportunisticContainersStatus() {
     this.opportunisticContainersStatus.setQueuedOpportContainers(
         getNumQueuedOpportunisticContainers());
@@ -252,7 +274,7 @@ public class ContainerScheduler extends AbstractService implements
     return this.opportunisticContainersStatus;
   }
 
-  private void onContainerCompleted(Container container) {
+  private void onResourcesReclaimed(Container container) {
     oppContainersToKill.remove(container.getContainerId());
 
     // This could be killed externally for eg. by the ContainerManager,
@@ -282,6 +304,24 @@ public class ContainerScheduler extends AbstractService implements
     // Start pending guaranteed containers, if resources available.
     boolean resourcesAvailable =
         startContainersFromQueue(queuedGuaranteedContainers.values());
+    // Resume opportunistic containers, if resource available.
+    if (resourcesAvailable) {
+      List<Container> pausedContainers = new ArrayList<Container>();
+      Map<ContainerId, Container> containers =
+          context.getContainers();
+      for (Map.Entry<ContainerId, Container>entry : containers.entrySet()) {
+        ContainerId contId = entry.getKey();
+        // Find containers that were not already started and are in paused state
+        if(false == runningContainers.containsKey(contId)) {
+          if(containers.get(contId).getContainerState()
+              == ContainerState.PAUSED) {
+            pausedContainers.add(containers.get(contId));
+          }
+        }
+      }
+      resourcesAvailable =
+          startContainersFromQueue(pausedContainers);
+    }
     // Start opportunistic containers, if resources available.
     if (resourcesAvailable) {
       startContainersFromQueue(queuedOpportunisticContainers.values());
@@ -378,7 +418,7 @@ public class ContainerScheduler extends AbstractService implements
       // if the guaranteed container is queued, we need to preempt opportunistic
       // containers for make room for it
       if (queuedGuaranteedContainers.containsKey(container.getContainerId())) {
-        killOpportunisticContainers(container);
+        reclaimOpportunisticContainerResources(container);
       }
     } else {
       // Given an opportunistic container, we first try to start as many queuing
@@ -396,19 +436,30 @@ public class ContainerScheduler extends AbstractService implements
     }
   }
 
-  private void killOpportunisticContainers(Container container) {
-    List<Container> extraOpportContainersToKill =
-        pickOpportunisticContainersToKill(container.getContainerId());
+  @SuppressWarnings("unchecked")
+  private void reclaimOpportunisticContainerResources(Container container) {
+    List<Container> extraOppContainersToReclaim =
+        pickOpportunisticContainersToReclaimResources(
+            container.getContainerId());
     // Kill the opportunistic containers that were chosen.
-    for (Container contToKill : extraOpportContainersToKill) {
-      contToKill.sendKillEvent(
-          ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER,
-          "Container Killed to make room for Guaranteed Container.");
-      oppContainersToKill.put(contToKill.getContainerId(), contToKill);
+    for (Container contToReclaim : extraOppContainersToReclaim) {
+      String preemptionAction = usePauseEventForPreemption == true ? "paused" :
+          "resumed";
       LOG.info(
-          "Opportunistic container {} will be killed in order to start the "
+          "Container {} will be {} to start the "
               + "execution of guaranteed container {}.",
-          contToKill.getContainerId(), container.getContainerId());
+          contToReclaim.getContainerId(), preemptionAction,
+          container.getContainerId());
+
+      if (usePauseEventForPreemption) {
+        contToReclaim.sendPauseEvent(
+            "Container Paused to make room for Guaranteed Container");
+      } else {
+        contToReclaim.sendKillEvent(
+            ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER,
+            "Container Killed to make room for Guaranteed Container.");
+      }
+      oppContainersToKill.put(contToReclaim.getContainerId(), contToReclaim);
     }
   }
 
@@ -423,7 +474,7 @@ public class ContainerScheduler extends AbstractService implements
     container.sendLaunchEvent();
   }
 
-  private List<Container> pickOpportunisticContainersToKill(
+  private List<Container> pickOpportunisticContainersToReclaimResources(
       ContainerId containerToStartId) {
     // The opportunistic containers that need to be killed for the
     // given container to start.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/026f877b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/TestContainerSchedulerQueuing.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/scheduler/TestContainerSchedulerQueuing.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/TestContainerSchedulerQueuing.java
index 9676568..f3fc724 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/TestContainerSchedulerQueuing.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/TestContainerSchedulerQueuing.java
@@ -23,6 +23,8 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
 
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -49,6 +51,7 @@ import org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.BaseContainerManagerTest;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorImpl;
@@ -124,18 +127,38 @@ public class TestContainerSchedulerQueuing extends BaseContainerManagerTest {
   @Override
   protected ContainerExecutor createContainerExecutor() {
     DefaultContainerExecutor exec = new DefaultContainerExecutor() {
+      ConcurrentMap<String, Boolean> oversleepMap =
+          new ConcurrentHashMap<String, Boolean>();
       @Override
       public int launchContainer(ContainerStartContext ctx)
           throws IOException, ConfigurationException {
+        oversleepMap.put(ctx.getContainer().getContainerId().toString(), false);
         if (delayContainers) {
           try {
             Thread.sleep(10000);
+            if(oversleepMap.get(ctx.getContainer().getContainerId().toString())
+                == true) {
+              Thread.sleep(10000);
+            }
           } catch (InterruptedException e) {
             // Nothing..
           }
         }
         return super.launchContainer(ctx);
       }
+
+      @Override
+      public void pauseContainer(Container container) {
+        // To mimic pausing we force the container to be in the PAUSED state
+        // a little longer by oversleeping.
+        oversleepMap.put(container.getContainerId().toString(), true);
+        LOG.info("Container was paused");
+      }
+
+      @Override
+      public void resumeContainer(Container container) {
+        LOG.info("Container was resumed");
+      }
     };
     exec.setConf(conf);
     return spy(exec);
@@ -506,6 +529,86 @@ public class TestContainerSchedulerQueuing extends BaseContainerManagerTest {
   }
 
   /**
+   * Submit two OPPORTUNISTIC and one GUARANTEED containers. The resources
+   * requests by each container as such that only one can run in parallel.
+   * Thus, the OPPORTUNISTIC container that started running, will be
+   * paused for the GUARANTEED container to start.
+   * Once the GUARANTEED container finishes its execution, the remaining
+   * OPPORTUNISTIC container will be executed.
+   * @throws Exception
+   */
+  @Test
+  public void testPauseOpportunisticForGuaranteedContainer() throws Exception {
+    containerManager.start();
+    containerManager.getContainerScheduler().
+        setUsePauseEventForPreemption(true);
+    ContainerLaunchContext containerLaunchContext =
+        recordFactory.newRecordInstance(ContainerLaunchContext.class);
+
+    List<StartContainerRequest> list = new ArrayList<>();
+    list.add(StartContainerRequest.newInstance(
+        containerLaunchContext,
+        createContainerToken(createContainerId(0), DUMMY_RM_IDENTIFIER,
+            context.getNodeId(),
+            user, BuilderUtils.newResource(2048, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.OPPORTUNISTIC)));
+
+    StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
+    containerManager.startContainers(allRequests);
+
+    BaseContainerManagerTest.waitForNMContainerState(containerManager,
+        createContainerId(0), ContainerState.RUNNING, 40);
+
+    list = new ArrayList<>();
+    list.add(StartContainerRequest.newInstance(
+        containerLaunchContext,
+        createContainerToken(createContainerId(1), DUMMY_RM_IDENTIFIER,
+            context.getNodeId(),
+            user, BuilderUtils.newResource(2048, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.GUARANTEED)));
+    allRequests =
+        StartContainersRequest.newInstance(list);
+
+    containerManager.startContainers(allRequests);
+
+    BaseContainerManagerTest.waitForNMContainerState(containerManager,
+        createContainerId(1), ContainerState.RUNNING, 40);
+
+    // Get container statuses. Container 0 should be paused, container 1
+    // should be running.
+    List<ContainerId> statList = new ArrayList<ContainerId>();
+    for (int i = 0; i < 2; i++) {
+      statList.add(createContainerId(i));
+    }
+    GetContainerStatusesRequest statRequest =
+        GetContainerStatusesRequest.newInstance(statList);
+    List<ContainerStatus> containerStatuses = containerManager
+        .getContainerStatuses(statRequest).getContainerStatuses();
+    for (ContainerStatus status : containerStatuses) {
+      if (status.getContainerId().equals(createContainerId(0))) {
+        Assert.assertTrue(status.getDiagnostics().contains(
+            "Container Paused to make room for Guaranteed Container"));
+      } else if (status.getContainerId().equals(createContainerId(1))) {
+        Assert.assertEquals(
+            org.apache.hadoop.yarn.api.records.ContainerState.RUNNING,
+            status.getState());
+      }
+      System.out.println("\nStatus : [" + status + "]\n");
+    }
+
+    // Make sure that the GUARANTEED container completes
+    BaseContainerManagerTest.waitForNMContainerState(containerManager,
+        createContainerId(1), ContainerState.DONE, 40);
+    // Make sure that the PAUSED opportunistic container resumes and
+    // starts running
+    BaseContainerManagerTest.waitForNMContainerState(containerManager,
+        createContainerId(0), ContainerState.DONE, 40);
+  }
+
+  /**
    * 1. Submit a long running GUARANTEED container to hog all NM resources.
    * 2. Submit 6 OPPORTUNISTIC containers, all of which will be queued.
    * 3. Update the Queue Limit to 2.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/026f877b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.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/webapp/MockContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java
index d435ba0..77ebd34 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java
@@ -245,4 +245,9 @@ public class MockContainer implements Container {
   public ResourceMappings getResourceMappings() {
     return null;
   }
+
+  @Override
+  public void sendPauseEvent(String description) {
+
+  }
 }


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


[09/50] [abbrv] hadoop git commit: Revert "Plan/ResourceAllocation data structure enhancements required to support recurring reservations in ReservationSystem."

Posted by as...@apache.org.
Revert "Plan/ResourceAllocation data structure enhancements required to support recurring reservations in ReservationSystem."

This reverts commit 7996eca7dcfaa1bdf970e32022274f2699bef8a1.


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

Branch: refs/heads/YARN-5972
Commit: e3345e985bff93c6c74a76747e45376c6027f42c
Parents: 22de944
Author: Subru Krishnan <su...@apache.org>
Authored: Wed Sep 6 16:39:23 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Wed Sep 6 16:39:23 2017 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/conf/YarnConfiguration.java     |   6 -
 .../yarn/conf/TestYarnConfigurationFields.java  |   4 +-
 .../reservation/AbstractReservationSystem.java  |  90 ++--
 .../AbstractSchedulerPlanFollower.java          | 183 ++++----
 .../reservation/InMemoryPlan.java               | 400 +++++------------
 .../InMemoryReservationAllocation.java          |  36 +-
 .../reservation/NoOverCommitPolicy.java         |   2 +-
 .../PeriodicRLESparseResourceAllocation.java    | 130 ++----
 .../resourcemanager/reservation/PlanEdit.java   |  24 +-
 .../resourcemanager/reservation/PlanView.java   |  94 ++--
 .../RLESparseResourceAllocation.java            | 115 +++--
 .../reservation/ReservationAllocation.java      |  60 +--
 .../reservation/ReservationInputValidator.java  | 134 +++---
 .../reservation/ReservationSystem.java          |   6 +-
 .../reservation/SharingPolicy.java              |  13 +-
 .../reservation/planning/Planner.java           |   2 +-
 .../reservation/planning/PlanningAlgorithm.java |   2 +-
 .../reservation/planning/StageAllocator.java    |   2 +-
 .../planning/StageAllocatorGreedy.java          |   2 +-
 .../planning/StageAllocatorGreedyRLE.java       |   5 +-
 .../planning/StageAllocatorLowCostAligned.java  |   4 +-
 .../reservation/ReservationSystemTestUtil.java  | 135 +++---
 .../reservation/TestInMemoryPlan.java           | 431 +++++++------------
 ...TestPeriodicRLESparseResourceAllocation.java | 109 ++---
 .../TestRLESparseResourceAllocation.java        | 122 +++---
 .../planning/TestSimpleCapacityReplanner.java   |   8 +-
 26 files changed, 777 insertions(+), 1342 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/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 27ca957..4944821 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
@@ -262,12 +262,6 @@ public class YarnConfiguration extends Configuration {
   public static final long DEFAULT_RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP =
       1000L;
 
-  /** The maximum periodicity for the Reservation System. */
-  public static final String RM_RESERVATION_SYSTEM_MAX_PERIODICITY =
-      RM_PREFIX + "reservation-system.max-periodicity";
-  public static final long DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY =
-      86400000L;
-
   /**
    * Enable periodic monitor threads.
    * @see #RM_SCHEDULER_MONITOR_POLICIES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/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 1d3111c..bd7bf93 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
@@ -33,7 +33,7 @@ import org.apache.hadoop.conf.TestConfigurationFieldsBase;
  */
 public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
 
-  @SuppressWarnings({"deprecation", "methodlength"})
+  @SuppressWarnings("deprecation")
   @Override
   public void initializeMemberVariables() {
     xmlFilename = new String("yarn-default.xml");
@@ -69,8 +69,6 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
     configurationPropsToSkipCompare.add(YarnConfiguration
         .YARN_SECURITY_SERVICE_AUTHORIZATION_COLLECTOR_NODEMANAGER_PROTOCOL);
     configurationPropsToSkipCompare.add(YarnConfiguration.CURATOR_LEADER_ELECTOR);
-    configurationPropsToSkipCompare
-        .add(YarnConfiguration.RM_RESERVATION_SYSTEM_MAX_PERIODICITY);
 
     // Federation default configs to be ignored
     configurationPropsToSkipCompare

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.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/reservation/AbstractReservationSystem.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.java
index 5b8772c..5ef4912 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.java
@@ -18,17 +18,6 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
 import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
@@ -57,6 +46,17 @@ import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
 /**
  * This is the implementation of {@link ReservationSystem} based on the
  * {@link ResourceScheduler}
@@ -66,8 +66,8 @@ import org.slf4j.LoggerFactory;
 public abstract class AbstractReservationSystem extends AbstractService
     implements ReservationSystem {
 
-  private static final Logger LOG =
-      LoggerFactory.getLogger(AbstractReservationSystem.class);
+  private static final Logger LOG = LoggerFactory
+      .getLogger(AbstractReservationSystem.class);
 
   // private static final String DEFAULT_CAPACITY_SCHEDULER_PLAN
 
@@ -103,8 +103,6 @@ public abstract class AbstractReservationSystem extends AbstractService
 
   private boolean isRecoveryEnabled = false;
 
-  private long maxPeriodicity;
-
   /**
    * Construct the service.
    * 
@@ -145,41 +143,36 @@ public abstract class AbstractReservationSystem extends AbstractService
     this.conf = conf;
     scheduler = rmContext.getScheduler();
     // Get the plan step size
-    planStepSize = conf.getTimeDuration(
-        YarnConfiguration.RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP,
-        YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP,
-        TimeUnit.MILLISECONDS);
+    planStepSize =
+        conf.getTimeDuration(
+            YarnConfiguration.RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP,
+            YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP,
+            TimeUnit.MILLISECONDS);
     if (planStepSize < 0) {
       planStepSize =
           YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP;
     }
-    maxPeriodicity =
-        conf.getLong(YarnConfiguration.RM_RESERVATION_SYSTEM_MAX_PERIODICITY,
-            YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY);
-    if (maxPeriodicity <= 0) {
-      maxPeriodicity =
-          YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY;
-    }
     // Create a plan corresponding to every reservable queue
     Set<String> planQueueNames = scheduler.getPlanQueues();
     for (String planQueueName : planQueueNames) {
       Plan plan = initializePlan(planQueueName);
       plans.put(planQueueName, plan);
     }
-    isRecoveryEnabled = conf.getBoolean(YarnConfiguration.RECOVERY_ENABLED,
+    isRecoveryEnabled = conf.getBoolean(
+        YarnConfiguration.RECOVERY_ENABLED,
         YarnConfiguration.DEFAULT_RM_RECOVERY_ENABLED);
 
     if (conf.getBoolean(YarnConfiguration.YARN_RESERVATION_ACL_ENABLE,
-        YarnConfiguration.DEFAULT_YARN_RESERVATION_ACL_ENABLE)
-        && conf.getBoolean(YarnConfiguration.YARN_ACL_ENABLE,
-            YarnConfiguration.DEFAULT_YARN_ACL_ENABLE)) {
+            YarnConfiguration.DEFAULT_YARN_RESERVATION_ACL_ENABLE) &&
+                    conf.getBoolean(YarnConfiguration.YARN_ACL_ENABLE,
+                            YarnConfiguration.DEFAULT_YARN_ACL_ENABLE)) {
       reservationsACLsManager = new ReservationsACLsManager(scheduler, conf);
     }
   }
 
   private void loadPlan(String planName,
       Map<ReservationId, ReservationAllocationStateProto> reservations)
-      throws PlanningException {
+          throws PlanningException {
     Plan plan = plans.get(planName);
     Resource minAllocation = getMinAllocation();
     ResourceCalculator rescCalculator = getResourceCalculator();
@@ -255,8 +248,8 @@ public abstract class AbstractReservationSystem extends AbstractService
       Class<?> planFollowerPolicyClazz =
           conf.getClassByName(planFollowerPolicyClassName);
       if (PlanFollower.class.isAssignableFrom(planFollowerPolicyClazz)) {
-        return (PlanFollower) ReflectionUtils
-            .newInstance(planFollowerPolicyClazz, conf);
+        return (PlanFollower) ReflectionUtils.newInstance(
+            planFollowerPolicyClazz, conf);
       } else {
         throw new YarnRuntimeException("Class: " + planFollowerPolicyClassName
             + " not instance of " + PlanFollower.class.getCanonicalName());
@@ -264,8 +257,7 @@ public abstract class AbstractReservationSystem extends AbstractService
     } catch (ClassNotFoundException e) {
       throw new YarnRuntimeException(
           "Could not instantiate PlanFollowerPolicy: "
-              + planFollowerPolicyClassName,
-          e);
+              + planFollowerPolicyClassName, e);
     }
   }
 
@@ -379,8 +371,9 @@ public abstract class AbstractReservationSystem extends AbstractService
   public ReservationId getNewReservationId() {
     writeLock.lock();
     try {
-      ReservationId resId = ReservationId.newInstance(
-          ResourceManager.getClusterTimeStamp(), resCounter.incrementAndGet());
+      ReservationId resId =
+          ReservationId.newInstance(ResourceManager.getClusterTimeStamp(),
+              resCounter.incrementAndGet());
       LOG.info("Allocated new reservationId: " + resId);
       return resId;
     } finally {
@@ -397,11 +390,8 @@ public abstract class AbstractReservationSystem extends AbstractService
    * Get the default reservation system corresponding to the scheduler
    * 
    * @param scheduler the scheduler for which the reservation system is required
-   *
-   * @return the {@link ReservationSystem} based on the configured scheduler
    */
-  public static String getDefaultReservationSystem(
-      ResourceScheduler scheduler) {
+  public static String getDefaultReservationSystem(ResourceScheduler scheduler) {
     if (scheduler instanceof CapacityScheduler) {
       return CapacityReservationSystem.class.getName();
     } else if (scheduler instanceof FairScheduler) {
@@ -419,11 +409,12 @@ public abstract class AbstractReservationSystem extends AbstractService
     Resource maxAllocation = getMaxAllocation();
     ResourceCalculator rescCalc = getResourceCalculator();
     Resource totCap = getPlanQueueCapacity(planQueueName);
-    Plan plan = new InMemoryPlan(getRootQueueMetrics(), adPolicy,
-        getAgent(planQueuePath), totCap, planStepSize, rescCalc, minAllocation,
-        maxAllocation, planQueueName, getReplanner(planQueuePath),
-        getReservationSchedulerConfiguration().getMoveOnExpiry(planQueuePath),
-        maxPeriodicity, rmContext);
+    Plan plan =
+        new InMemoryPlan(getRootQueueMetrics(), adPolicy,
+            getAgent(planQueuePath), totCap, planStepSize, rescCalc,
+            minAllocation, maxAllocation, planQueueName,
+            getReplanner(planQueuePath), getReservationSchedulerConfiguration()
+            .getMoveOnExpiry(planQueuePath), rmContext);
     LOG.info("Initialized plan {} based on reservable queue {}",
         plan.toString(), planQueueName);
     return plan;
@@ -486,8 +477,8 @@ public abstract class AbstractReservationSystem extends AbstractService
       Class<?> admissionPolicyClazz =
           conf.getClassByName(admissionPolicyClassName);
       if (SharingPolicy.class.isAssignableFrom(admissionPolicyClazz)) {
-        return (SharingPolicy) ReflectionUtils.newInstance(admissionPolicyClazz,
-            conf);
+        return (SharingPolicy) ReflectionUtils.newInstance(
+            admissionPolicyClazz, conf);
       } else {
         throw new YarnRuntimeException("Class: " + admissionPolicyClassName
             + " not instance of " + SharingPolicy.class.getCanonicalName());
@@ -502,7 +493,8 @@ public abstract class AbstractReservationSystem extends AbstractService
     return this.reservationsACLsManager;
   }
 
-  protected abstract ReservationSchedulerConfiguration getReservationSchedulerConfiguration();
+  protected abstract ReservationSchedulerConfiguration
+      getReservationSchedulerConfiguration();
 
   protected abstract String getPlanQueuePath(String planQueueName);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractSchedulerPlanFollower.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/reservation/AbstractSchedulerPlanFollower.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractSchedulerPlanFollower.java
index 9b6a0b0..90357e3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractSchedulerPlanFollower.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractSchedulerPlanFollower.java
@@ -18,14 +18,6 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -41,17 +33,24 @@ import org.apache.hadoop.yarn.util.resource.Resources;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
 public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
-  private static final Logger LOG =
-      LoggerFactory.getLogger(AbstractSchedulerPlanFollower.class);
+  private static final Logger LOG = LoggerFactory
+      .getLogger(AbstractSchedulerPlanFollower.class);
 
   protected Collection<Plan> plans = new ArrayList<Plan>();
   protected YarnScheduler scheduler;
   protected Clock clock;
 
   @Override
-  public void init(Clock clock, ResourceScheduler sched,
-      Collection<Plan> plans) {
+  public void init(Clock clock, ResourceScheduler sched, Collection<Plan> plans) {
     this.clock = clock;
     this.scheduler = sched;
     this.plans.addAll(plans);
@@ -72,7 +71,7 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
 
   @Override
   public synchronized void synchronizePlan(Plan plan, boolean shouldReplan) {
-    String planQueueName = plan.getQueueName();
+     String planQueueName = plan.getQueueName();
     if (LOG.isDebugEnabled()) {
       LOG.debug("Running plan follower edit policy for plan: " + planQueueName);
     }
@@ -83,14 +82,12 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
       now += step - (now % step);
     }
     Queue planQueue = getPlanQueue(planQueueName);
-    if (planQueue == null) {
-      return;
-    }
+    if (planQueue == null) return;
 
     // first we publish to the plan the current availability of resources
     Resource clusterResources = scheduler.getClusterResource();
-    Resource planResources =
-        getPlanResources(plan, planQueue, clusterResources);
+    Resource planResources = getPlanResources(plan, planQueue,
+        clusterResources);
     Set<ReservationAllocation> currentReservations =
         plan.getReservationsAtTime(now);
     Set<String> curReservationNames = new HashSet<String>();
@@ -98,11 +95,12 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
     int numRes = getReservedResources(now, currentReservations,
         curReservationNames, reservedResources);
     // create the default reservation queue if it doesnt exist
-    String defReservationId = getReservationIdFromQueueName(planQueueName)
-        + ReservationConstants.DEFAULT_QUEUE_SUFFIX;
-    String defReservationQueue =
-        getReservationQueueName(planQueueName, defReservationId);
-    createDefaultReservationQueue(planQueueName, planQueue, defReservationId);
+    String defReservationId = getReservationIdFromQueueName(planQueueName) +
+        ReservationConstants.DEFAULT_QUEUE_SUFFIX;
+    String defReservationQueue = getReservationQueueName(planQueueName,
+        defReservationId);
+    createDefaultReservationQueue(planQueueName, planQueue,
+        defReservationId);
     curReservationNames.add(defReservationId);
     // if the resources dedicated to this plan has shrunk invoke replanner
     boolean shouldResize = false;
@@ -151,8 +149,10 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
       // sort allocations from the one giving up the most resources, to the
       // one asking for the most avoid order-of-operation errors that
       // temporarily violate 100% capacity bound
-      List<ReservationAllocation> sortedAllocations = sortByDelta(
-          new ArrayList<ReservationAllocation>(currentReservations), now, plan);
+      List<ReservationAllocation> sortedAllocations =
+          sortByDelta(
+              new ArrayList<ReservationAllocation>(currentReservations), now,
+              plan);
       for (ReservationAllocation res : sortedAllocations) {
         String currResId = res.getReservationId().toString();
         if (curReservationNames.contains(currResId)) {
@@ -163,9 +163,10 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
         if (planResources.getMemorySize() > 0
             && planResources.getVirtualCores() > 0) {
           if (shouldResize) {
-            capToAssign = calculateReservationToPlanProportion(
-                plan.getResourceCalculator(), planResources, reservedResources,
-                capToAssign);
+            capToAssign =
+                calculateReservationToPlanProportion(
+                    plan.getResourceCalculator(), planResources,
+                    reservedResources, capToAssign);
           }
           targetCapacity =
               calculateReservationToPlanRatio(plan.getResourceCalculator(),
@@ -184,8 +185,7 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
           maxCapacity = targetCapacity;
         }
         try {
-          setQueueEntitlement(planQueueName, currResId, targetCapacity,
-              maxCapacity);
+          setQueueEntitlement(planQueueName, currResId, targetCapacity, maxCapacity);
         } catch (YarnException e) {
           LOG.warn("Exception while trying to size reservation for plan: {}",
               currResId, planQueueName, e);
@@ -196,10 +196,9 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
     // compute the default queue capacity
     float defQCap = 1.0f - totalAssignedCapacity;
     if (LOG.isDebugEnabled()) {
-      LOG.debug(
-          "PlanFollowerEditPolicyTask: total Plan Capacity: {} "
-              + "currReservation: {} default-queue capacity: {}",
-          planResources, numRes, defQCap);
+      LOG.debug("PlanFollowerEditPolicyTask: total Plan Capacity: {} "
+          + "currReservation: {} default-queue capacity: {}", planResources,
+          numRes, defQCap);
     }
     // set the default queue to eat-up all remaining capacity
     try {
@@ -226,11 +225,12 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
   }
 
   protected void setQueueEntitlement(String planQueueName, String currResId,
-      float targetCapacity, float maxCapacity) throws YarnException {
-    String reservationQueueName =
-        getReservationQueueName(planQueueName, currResId);
-    scheduler.setEntitlement(reservationQueueName,
-        new QueueEntitlement(targetCapacity, maxCapacity));
+      float targetCapacity,
+      float maxCapacity) throws YarnException {
+    String reservationQueueName = getReservationQueueName(planQueueName,
+        currResId);
+    scheduler.setEntitlement(reservationQueueName, new QueueEntitlement(
+        targetCapacity, maxCapacity));
   }
 
   // Schedulers have different ways of naming queues. See YARN-2773
@@ -244,21 +244,14 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
    * Then move all apps in the set of queues to the parent plan queue's default
    * reservation queue if move is enabled. Finally cleanups the queue by killing
    * any apps (if move is disabled or move failed) and removing the queue
-   *
-   * @param planQueueName the name of {@code PlanQueue}
-   * @param shouldMove flag to indicate if any running apps should be moved or
-   *          killed
-   * @param toRemove the remnant apps to clean up
-   * @param defReservationQueue the default {@code ReservationQueue} of the
-   *          {@link Plan}
    */
-  protected void cleanupExpiredQueues(String planQueueName, boolean shouldMove,
-      Set<String> toRemove, String defReservationQueue) {
+  protected void cleanupExpiredQueues(String planQueueName,
+      boolean shouldMove, Set<String> toRemove, String defReservationQueue) {
     for (String expiredReservationId : toRemove) {
       try {
         // reduce entitlement to 0
-        String expiredReservation =
-            getReservationQueueName(planQueueName, expiredReservationId);
+        String expiredReservation = getReservationQueueName(planQueueName,
+            expiredReservationId);
         setQueueEntitlement(planQueueName, expiredReservation, 0.0f, 0.0f);
         if (shouldMove) {
           moveAppsInQueueSync(expiredReservation, defReservationQueue);
@@ -282,7 +275,7 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
    * reservation queue in a synchronous fashion
    */
   private void moveAppsInQueueSync(String expiredReservation,
-      String defReservationQueue) {
+                                   String defReservationQueue) {
     List<ApplicationAttemptId> activeApps =
         scheduler.getAppsInQueue(expiredReservation);
     if (activeApps.isEmpty()) {
@@ -294,16 +287,16 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
         scheduler.moveApplication(app.getApplicationId(), defReservationQueue);
       } catch (YarnException e) {
         LOG.warn(
-            "Encountered unexpected error during migration of application: {}"
-                + " from reservation: {}",
+            "Encountered unexpected error during migration of application: {}" +
+                " from reservation: {}",
             app, expiredReservation, e);
       }
     }
   }
 
-  protected int getReservedResources(long now,
-      Set<ReservationAllocation> currentReservations,
-      Set<String> curReservationNames, Resource reservedResources) {
+  protected int getReservedResources(long now, Set<ReservationAllocation>
+      currentReservations, Set<String> curReservationNames,
+                                     Resource reservedResources) {
     int numRes = 0;
     if (currentReservations != null) {
       numRes = currentReservations.size();
@@ -319,30 +312,23 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
    * Sort in the order from the least new amount of resources asked (likely
    * negative) to the highest. This prevents "order-of-operation" errors related
    * to exceeding 100% capacity temporarily.
-   *
-   * @param currentReservations the currently active reservations
-   * @param now the current time
-   * @param plan the {@link Plan} that is being considered
-   *
-   * @return the sorted list of {@link ReservationAllocation}s
    */
   protected List<ReservationAllocation> sortByDelta(
       List<ReservationAllocation> currentReservations, long now, Plan plan) {
-    Collections.sort(currentReservations,
-        new ReservationAllocationComparator(now, this, plan));
+    Collections.sort(currentReservations, new ReservationAllocationComparator(
+        now, this, plan));
     return currentReservations;
   }
 
   /**
-   * Get queue associated with reservable queue named.
-   *
-   * @param planQueueName name of the reservable queue
+   * Get queue associated with reservable queue named
+   * @param planQueueName Name of the reservable queue
    * @return queue associated with the reservable queue
    */
   protected abstract Queue getPlanQueue(String planQueueName);
 
   /**
-   * Resizes reservations based on currently available resources.
+   * Resizes reservations based on currently available resources
    */
   private Resource calculateReservationToPlanProportion(
       ResourceCalculator rescCalculator, Resource availablePlanResources,
@@ -352,7 +338,7 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
   }
 
   /**
-   * Calculates ratio of reservationResources to planResources.
+   * Calculates ratio of reservationResources to planResources
    */
   private float calculateReservationToPlanRatio(
       ResourceCalculator rescCalculator, Resource clusterResources,
@@ -362,7 +348,7 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
   }
 
   /**
-   * Check if plan resources are less than expected reservation resources.
+   * Check if plan resources are less than expected reservation resources
    */
   private boolean arePlanResourcesLessThanReservations(
       ResourceCalculator rescCalculator, Resource clusterResources,
@@ -372,56 +358,38 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
   }
 
   /**
-   * Get a list of reservation queues for this planQueue.
-   *
-   * @param planQueue the queue for the current {@link Plan}
-   *
-   * @return the queues corresponding to the reservations
+   * Get a list of reservation queues for this planQueue
    */
   protected abstract List<? extends Queue> getChildReservationQueues(
       Queue planQueue);
 
   /**
-   * Add a new reservation queue for reservation currResId for this planQueue.
+   * Add a new reservation queue for reservation currResId for this planQueue
    */
-  protected abstract void addReservationQueue(String planQueueName, Queue queue,
-      String currResId);
+  protected abstract void addReservationQueue(
+      String planQueueName, Queue queue, String currResId);
 
   /**
-   * Creates the default reservation queue for use when no reservation is used
-   * for applications submitted to this planQueue.
-   *
-   * @param planQueueName name of the reservable queue
-   * @param queue the queue for the current {@link Plan}
-   * @param defReservationQueue name of the default {@code ReservationQueue}
+   * Creates the default reservation queue for use when no reservation is
+   * used for applications submitted to this planQueue
    */
-  protected abstract void createDefaultReservationQueue(String planQueueName,
-      Queue queue, String defReservationQueue);
+  protected abstract void createDefaultReservationQueue(
+      String planQueueName, Queue queue, String defReservationQueue);
 
   /**
-   * Get plan resources for this planQueue.
-   *
-   * @param plan the current {@link Plan} being considered
-   * @param clusterResources the resources available in the cluster
-   *
-   * @return the resources allocated to the specified {@link Plan}
+   * Get plan resources for this planQueue
    */
-  protected abstract Resource getPlanResources(Plan plan, Queue queue,
-      Resource clusterResources);
+  protected abstract Resource getPlanResources(
+      Plan plan, Queue queue, Resource clusterResources);
 
   /**
    * Get reservation queue resources if it exists otherwise return null.
-   *
-   * @param plan the current {@link Plan} being considered
-   * @param reservationId the identifier of the reservation
-   *
-   * @return the resources allocated to the specified reservation
    */
   protected abstract Resource getReservationQueueResourceIfExists(Plan plan,
       ReservationId reservationId);
 
-  private static class ReservationAllocationComparator
-      implements Comparator<ReservationAllocation> {
+  private static class ReservationAllocationComparator implements
+      Comparator<ReservationAllocation> {
     AbstractSchedulerPlanFollower planFollower;
     long now;
     Plan plan;
@@ -436,12 +404,14 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
     private Resource getUnallocatedReservedResources(
         ReservationAllocation reservation) {
       Resource resResource;
-      Resource reservationResource =
-          planFollower.getReservationQueueResourceIfExists(plan,
-              reservation.getReservationId());
+      Resource reservationResource = planFollower
+          .getReservationQueueResourceIfExists
+              (plan, reservation.getReservationId());
       if (reservationResource != null) {
-        resResource = Resources.subtract(reservation.getResourcesAtTime(now),
-            reservationResource);
+        resResource =
+            Resources.subtract(
+                reservation.getResourcesAtTime(now),
+                reservationResource);
       } else {
         resResource = reservation.getResourcesAtTime(now);
       }
@@ -458,3 +428,4 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
     }
   }
 }
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryPlan.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/reservation/InMemoryPlan.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryPlan.java
index 9eb1820..783fd09 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryPlan.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryPlan.java
@@ -6,9 +6,9 @@
  * 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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
+ *
+ *     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.
@@ -33,10 +33,9 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.RLESparseResourceAllocation.RLEOperator;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
-import org.apache.hadoop.yarn.server.resourcemanager.reservation.RLESparseResourceAllocation.RLEOperator;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.planning.Planner;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.planning.ReservationAgent;
@@ -65,14 +64,9 @@ public class InMemoryPlan implements Plan {
 
   private RLESparseResourceAllocation rleSparseVector;
 
-  private PeriodicRLESparseResourceAllocation periodicRle;
-
   private Map<String, RLESparseResourceAllocation> userResourceAlloc =
       new HashMap<String, RLESparseResourceAllocation>();
 
-  private Map<String, RLESparseResourceAllocation> userPeriodicResourceAlloc =
-      new HashMap<String, RLESparseResourceAllocation>();
-
   private Map<String, RLESparseResourceAllocation> userActiveReservationCount =
       new HashMap<String, RLESparseResourceAllocation>();
 
@@ -102,27 +96,15 @@ public class InMemoryPlan implements Plan {
       String queueName, Planner replanner, boolean getMoveOnExpiry,
       RMContext rmContext) {
     this(queueMetrics, policy, agent, totalCapacity, step, resCalc, minAlloc,
-        maxAlloc, queueName, replanner, getMoveOnExpiry,
-        YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY,
-        rmContext);
+        maxAlloc, queueName, replanner, getMoveOnExpiry, rmContext,
+        new UTCClock());
   }
 
   public InMemoryPlan(QueueMetrics queueMetrics, SharingPolicy policy,
       ReservationAgent agent, Resource totalCapacity, long step,
       ResourceCalculator resCalc, Resource minAlloc, Resource maxAlloc,
       String queueName, Planner replanner, boolean getMoveOnExpiry,
-      long maxPeriodicty, RMContext rmContext) {
-    this(queueMetrics, policy, agent, totalCapacity, step, resCalc, minAlloc,
-        maxAlloc, queueName, replanner, getMoveOnExpiry, maxPeriodicty,
-        rmContext, new UTCClock());
-  }
-
-  @SuppressWarnings("checkstyle:parameternumber")
-  public InMemoryPlan(QueueMetrics queueMetrics, SharingPolicy policy,
-      ReservationAgent agent, Resource totalCapacity, long step,
-      ResourceCalculator resCalc, Resource minAlloc, Resource maxAlloc,
-      String queueName, Planner replanner, boolean getMoveOnExpiry,
-      long maxPeriodicty, RMContext rmContext, Clock clock) {
+      RMContext rmContext, Clock clock) {
     this.queueMetrics = queueMetrics;
     this.policy = policy;
     this.agent = agent;
@@ -132,8 +114,6 @@ public class InMemoryPlan implements Plan {
     this.minAlloc = minAlloc;
     this.maxAlloc = maxAlloc;
     this.rleSparseVector = new RLESparseResourceAllocation(resCalc);
-    this.periodicRle =
-        new PeriodicRLESparseResourceAllocation(resCalc, maxPeriodicty);
     this.queueName = queueName;
     this.replanner = replanner;
     this.getMoveOnExpiry = getMoveOnExpiry;
@@ -146,39 +126,6 @@ public class InMemoryPlan implements Plan {
     return queueMetrics;
   }
 
-  private RLESparseResourceAllocation getUserRLEResourceAllocation(String user,
-      long period) {
-    RLESparseResourceAllocation resAlloc = null;
-    if (period > 0) {
-      if (userPeriodicResourceAlloc.containsKey(user)) {
-        resAlloc = userPeriodicResourceAlloc.get(user);
-      } else {
-        resAlloc = new PeriodicRLESparseResourceAllocation(resCalc,
-            periodicRle.getTimePeriod());
-        userPeriodicResourceAlloc.put(user, resAlloc);
-      }
-    } else {
-      if (userResourceAlloc.containsKey(user)) {
-        resAlloc = userResourceAlloc.get(user);
-      } else {
-        resAlloc = new RLESparseResourceAllocation(resCalc);
-        userResourceAlloc.put(user, resAlloc);
-      }
-    }
-    return resAlloc;
-  }
-
-  private void gcUserRLEResourceAllocation(String user, long period) {
-    if (period > 0) {
-      if (userPeriodicResourceAlloc.get(user).isEmpty()) {
-        userPeriodicResourceAlloc.remove(user);
-      }
-    } else {
-      if (userResourceAlloc.get(user).isEmpty()) {
-        userResourceAlloc.remove(user);
-      }
-    }
-  }
 
   private void incrementAllocation(ReservationAllocation reservation) {
     assert (readWriteLock.isWriteLockedByCurrentThread());
@@ -186,10 +133,11 @@ public class InMemoryPlan implements Plan {
         reservation.getAllocationRequests();
     // check if we have encountered the user earlier and if not add an entry
     String user = reservation.getUser();
-    long period = reservation.getPeriodicity();
-    RLESparseResourceAllocation resAlloc =
-        getUserRLEResourceAllocation(user, period);
-
+    RLESparseResourceAllocation resAlloc = userResourceAlloc.get(user);
+    if (resAlloc == null) {
+      resAlloc = new RLESparseResourceAllocation(resCalc);
+      userResourceAlloc.put(user, resAlloc);
+    }
     RLESparseResourceAllocation resCount = userActiveReservationCount.get(user);
     if (resCount == null) {
       resCount = new RLESparseResourceAllocation(resCalc);
@@ -201,43 +149,14 @@ public class InMemoryPlan implements Plan {
 
     for (Map.Entry<ReservationInterval, Resource> r : allocationRequests
         .entrySet()) {
-
-      if (period > 0L) {
-        for (int i = 0; i < periodicRle.getTimePeriod() / period; i++) {
-
-          long rStart = r.getKey().getStartTime() + i * period;
-          long rEnd = r.getKey().getEndTime() + i * period;
-
-          // handle wrap-around
-          if (rEnd > periodicRle.getTimePeriod()) {
-            long diff = rEnd - periodicRle.getTimePeriod();
-            rEnd = periodicRle.getTimePeriod();
-            ReservationInterval newInterval = new ReservationInterval(0, diff);
-            periodicRle.addInterval(newInterval, r.getValue());
-            resAlloc.addInterval(newInterval, r.getValue());
-          }
-
-          ReservationInterval newInterval =
-              new ReservationInterval(rStart, rEnd);
-          periodicRle.addInterval(newInterval, r.getValue());
-          resAlloc.addInterval(newInterval, r.getValue());
-        }
-
-      } else {
-        rleSparseVector.addInterval(r.getKey(), r.getValue());
-        resAlloc.addInterval(r.getKey(), r.getValue());
-        if (Resources.greaterThan(resCalc, totalCapacity, r.getValue(),
-            ZERO_RESOURCE)) {
-          earliestActive = Math.min(earliestActive, r.getKey().getStartTime());
-          latestActive = Math.max(latestActive, r.getKey().getEndTime());
-        }
+      resAlloc.addInterval(r.getKey(), r.getValue());
+      rleSparseVector.addInterval(r.getKey(), r.getValue());
+      if (Resources.greaterThan(resCalc, totalCapacity, r.getValue(),
+          ZERO_RESOURCE)) {
+        earliestActive = Math.min(earliestActive, r.getKey().getStartTime());
+        latestActive = Math.max(latestActive, r.getKey().getEndTime());
       }
     }
-    // periodic reservations are active from start time and good till cancelled
-    if (period > 0L) {
-      earliestActive = reservation.getStartTime();
-      latestActive = Long.MAX_VALUE;
-    }
     resCount.addInterval(new ReservationInterval(earliestActive, latestActive),
         Resource.newInstance(1, 1));
   }
@@ -247,55 +166,27 @@ public class InMemoryPlan implements Plan {
     Map<ReservationInterval, Resource> allocationRequests =
         reservation.getAllocationRequests();
     String user = reservation.getUser();
-    long period = reservation.getPeriodicity();
-    RLESparseResourceAllocation resAlloc =
-        getUserRLEResourceAllocation(user, period);
+    RLESparseResourceAllocation resAlloc = userResourceAlloc.get(user);
 
     long earliestActive = Long.MAX_VALUE;
     long latestActive = Long.MIN_VALUE;
     for (Map.Entry<ReservationInterval, Resource> r : allocationRequests
         .entrySet()) {
-      if (period > 0L) {
-        for (int i = 0; i < periodicRle.getTimePeriod() / period; i++) {
-
-          long rStart = r.getKey().getStartTime() + i * period;
-          long rEnd = r.getKey().getEndTime() + i * period;
-
-          // handle wrap-around
-          if (rEnd > periodicRle.getTimePeriod()) {
-            long diff = rEnd - periodicRle.getTimePeriod();
-            rEnd = periodicRle.getTimePeriod();
-            ReservationInterval newInterval = new ReservationInterval(0, diff);
-            periodicRle.removeInterval(newInterval, r.getValue());
-            resAlloc.removeInterval(newInterval, r.getValue());
-          }
-
-          ReservationInterval newInterval =
-              new ReservationInterval(rStart, rEnd);
-          periodicRle.removeInterval(newInterval, r.getValue());
-          resAlloc.removeInterval(newInterval, r.getValue());
-        }
-      } else {
-        rleSparseVector.removeInterval(r.getKey(), r.getValue());
-        resAlloc.removeInterval(r.getKey(), r.getValue());
-        if (Resources.greaterThan(resCalc, totalCapacity, r.getValue(),
-            ZERO_RESOURCE)) {
-          earliestActive = Math.min(earliestActive, r.getKey().getStartTime());
-          latestActive = Math.max(latestActive, r.getKey().getEndTime());
-        }
+      resAlloc.removeInterval(r.getKey(), r.getValue());
+      rleSparseVector.removeInterval(r.getKey(), r.getValue());
+      if (Resources.greaterThan(resCalc, totalCapacity, r.getValue(),
+          ZERO_RESOURCE)) {
+        earliestActive = Math.min(earliestActive, r.getKey().getStartTime());
+        latestActive = Math.max(latestActive, r.getKey().getEndTime());
       }
     }
-    gcUserRLEResourceAllocation(user, period);
+    if (resAlloc.isEmpty()) {
+      userResourceAlloc.remove(user);
+    }
 
     RLESparseResourceAllocation resCount = userActiveReservationCount.get(user);
-    // periodic reservations are active from start time and good till cancelled
-    if (period > 0L) {
-      earliestActive = reservation.getStartTime();
-      latestActive = Long.MAX_VALUE;
-    }
-    resCount.removeInterval(
-        new ReservationInterval(earliestActive, latestActive),
-        Resource.newInstance(1, 1));
+    resCount.removeInterval(new ReservationInterval(earliestActive,
+        latestActive), Resource.newInstance(1, 1));
     if (resCount.isEmpty()) {
       userActiveReservationCount.remove(user);
     }
@@ -307,9 +198,9 @@ public class InMemoryPlan implements Plan {
       if (currentReservations != null) {
         Set<ReservationAllocation> flattenedReservations =
             new TreeSet<ReservationAllocation>();
-        for (Set<InMemoryReservationAllocation> res : currentReservations
-            .values()) {
-          flattenedReservations.addAll(res);
+        for (Set<InMemoryReservationAllocation> reservationEntries :
+            currentReservations.values()) {
+          flattenedReservations.addAll(reservationEntries);
         }
         return flattenedReservations;
       } else {
@@ -327,16 +218,19 @@ public class InMemoryPlan implements Plan {
     InMemoryReservationAllocation inMemReservation =
         (InMemoryReservationAllocation) reservation;
     if (inMemReservation.getUser() == null) {
-      String errMsg = "The specified Reservation with ID "
-          + inMemReservation.getReservationId() + " is not mapped to any user";
+      String errMsg =
+          "The specified Reservation with ID "
+              + inMemReservation.getReservationId()
+              + " is not mapped to any user";
       LOG.error(errMsg);
       throw new IllegalArgumentException(errMsg);
     }
     writeLock.lock();
     try {
       if (reservationTable.containsKey(inMemReservation.getReservationId())) {
-        String errMsg = "The specified Reservation with ID "
-            + inMemReservation.getReservationId() + " already exists";
+        String errMsg =
+            "The specified Reservation with ID "
+                + inMemReservation.getReservationId() + " already exists";
         LOG.error(errMsg);
         throw new IllegalArgumentException(errMsg);
       }
@@ -352,8 +246,9 @@ public class InMemoryPlan implements Plan {
               getQueueName(), inMemReservation.getReservationId().toString());
         }
       }
-      ReservationInterval searchInterval = new ReservationInterval(
-          inMemReservation.getStartTime(), inMemReservation.getEndTime());
+      ReservationInterval searchInterval =
+          new ReservationInterval(inMemReservation.getStartTime(),
+              inMemReservation.getEndTime());
       Set<InMemoryReservationAllocation> reservations =
           currentReservations.get(searchInterval);
       if (reservations == null) {
@@ -385,8 +280,9 @@ public class InMemoryPlan implements Plan {
       ReservationId resId = reservation.getReservationId();
       ReservationAllocation currReservation = getReservationById(resId);
       if (currReservation == null) {
-        String errMsg = "The specified Reservation with ID " + resId
-            + " does not exist in the plan";
+        String errMsg =
+            "The specified Reservation with ID " + resId
+                + " does not exist in the plan";
         LOG.error(errMsg);
         throw new IllegalArgumentException(errMsg);
       }
@@ -422,8 +318,9 @@ public class InMemoryPlan implements Plan {
 
   private boolean removeReservation(ReservationAllocation reservation) {
     assert (readWriteLock.isWriteLockedByCurrentThread());
-    ReservationInterval searchInterval = new ReservationInterval(
-        reservation.getStartTime(), reservation.getEndTime());
+    ReservationInterval searchInterval =
+        new ReservationInterval(reservation.getStartTime(),
+            reservation.getEndTime());
     Set<InMemoryReservationAllocation> reservations =
         currentReservations.get(searchInterval);
     if (reservations != null) {
@@ -440,15 +337,16 @@ public class InMemoryPlan implements Plan {
         currentReservations.remove(searchInterval);
       }
     } else {
-      String errMsg = "The specified Reservation with ID "
-          + reservation.getReservationId() + " does not exist in the plan";
+      String errMsg =
+          "The specified Reservation with ID " + reservation.getReservationId()
+              + " does not exist in the plan";
       LOG.error(errMsg);
       throw new IllegalArgumentException(errMsg);
     }
     reservationTable.remove(reservation.getReservationId());
     decrementAllocation(reservation);
     LOG.info("Sucessfully deleted reservation: {} in plan.",
-        reservation.getReservationId());
+            reservation.getReservationId());
     return true;
   }
 
@@ -458,8 +356,9 @@ public class InMemoryPlan implements Plan {
     try {
       ReservationAllocation reservation = getReservationById(reservationID);
       if (reservation == null) {
-        String errMsg = "The specified Reservation with ID " + reservationID
-            + " does not exist in the plan";
+        String errMsg =
+            "The specified Reservation with ID " + reservationID
+                + " does not exist in the plan";
         LOG.error(errMsg);
         throw new IllegalArgumentException(errMsg);
       }
@@ -554,90 +453,66 @@ public class InMemoryPlan implements Plan {
       long start, long end) {
     readLock.lock();
     try {
-      // merge periodic and non-periodic allocations
       RLESparseResourceAllocation userResAlloc = userResourceAlloc.get(user);
-      RLESparseResourceAllocation userPeriodicResAlloc =
-          userPeriodicResourceAlloc.get(user);
 
-      if (userResAlloc != null && userPeriodicResAlloc != null) {
-        return RLESparseResourceAllocation.merge(resCalc, totalCapacity,
-            userResAlloc, userPeriodicResAlloc, RLEOperator.add, start, end);
-      }
       if (userResAlloc != null) {
         return userResAlloc.getRangeOverlapping(start, end);
+      } else {
+        return new RLESparseResourceAllocation(resCalc);
       }
-      if (userPeriodicResAlloc != null) {
-        return userPeriodicResAlloc.getRangeOverlapping(start, end);
-      }
-    } catch (PlanningException e) {
-      LOG.warn("Exception while trying to merge periodic"
-          + " and non-periodic user allocations: {}", e.getMessage(), e);
     } finally {
       readLock.unlock();
     }
-    return new RLESparseResourceAllocation(resCalc);
   }
 
   @Override
   public Resource getTotalCommittedResources(long t) {
     readLock.lock();
     try {
-      return Resources.add(rleSparseVector.getCapacityAtTime(t),
-          periodicRle.getCapacityAtTime(t));
+      return rleSparseVector.getCapacityAtTime(t);
     } finally {
       readLock.unlock();
     }
   }
 
   @Override
-  public Set<ReservationAllocation> getReservations(ReservationId reservationID,
-      ReservationInterval interval) {
+  public Set<ReservationAllocation> getReservations(ReservationId
+                    reservationID, ReservationInterval interval) {
     return getReservations(reservationID, interval, null);
   }
 
   @Override
-  public Set<ReservationAllocation> getReservations(ReservationId reservationID,
-      ReservationInterval interval, String user) {
+  public Set<ReservationAllocation> getReservations(ReservationId
+                    reservationID, ReservationInterval interval, String user) {
     if (reservationID != null) {
       ReservationAllocation allocation = getReservationById(reservationID);
-      if (allocation == null) {
+      if (allocation == null){
         return Collections.emptySet();
       }
       return Collections.singleton(allocation);
     }
 
-    long startTime = interval == null ? 0 : interval.getStartTime();
-    long endTime = interval == null ? Long.MAX_VALUE : interval.getEndTime();
+    long startTime = interval == null? 0 : interval.getStartTime();
+    long endTime = interval == null? Long.MAX_VALUE : interval.getEndTime();
 
     ReservationInterval searchInterval =
-        new ReservationInterval(endTime, Long.MAX_VALUE);
+            new ReservationInterval(endTime, Long.MAX_VALUE);
     readLock.lock();
     try {
-      SortedMap<ReservationInterval, Set<InMemoryReservationAllocation>> res =
-          currentReservations.headMap(searchInterval, true);
-      if (!res.isEmpty()) {
-        Set<ReservationAllocation> flattenedReservations = new HashSet<>();
-        for (Set<InMemoryReservationAllocation> resEntries : res.values()) {
-          for (InMemoryReservationAllocation reservation : resEntries) {
-            // validate user
-            if (user != null && !user.isEmpty()
-                && !reservation.getUser().equals(user)) {
-              continue;
-            }
-            // handle periodic reservations
-            long period = reservation.getPeriodicity();
-            if (period > 0) {
-              long t = endTime % period;
-              // check for both contained and wrap-around reservations
-              if ((t - startTime) * (t - endTime)
-                  * (startTime - endTime) >= 0) {
-                flattenedReservations.add(reservation);
-              }
-            } else {
-              // check for non-periodic reservations
-              if (reservation.getEndTime() > startTime) {
-                flattenedReservations.add(reservation);
+      SortedMap<ReservationInterval, Set<InMemoryReservationAllocation>>
+            reservations = currentReservations.headMap(searchInterval, true);
+      if (!reservations.isEmpty()) {
+        Set<ReservationAllocation> flattenedReservations =
+                new HashSet<>();
+        for (Set<InMemoryReservationAllocation> reservationEntries :
+                reservations.values()) {
+          for (InMemoryReservationAllocation res : reservationEntries) {
+            if (res.getEndTime() > startTime) {
+              if (user != null && !user.isEmpty()
+                      && !res.getUser().equals(user)) {
+                continue;
               }
+              flattenedReservations.add(res);
             }
           }
         }
@@ -675,82 +550,36 @@ public class InMemoryPlan implements Plan {
 
   @Override
   public RLESparseResourceAllocation getAvailableResourceOverTime(String user,
-      ReservationId oldId, long start, long end, long period)
-      throws PlanningException {
+      ReservationId oldId, long start, long end) throws PlanningException {
     readLock.lock();
     try {
-
-      // for non-periodic return simple available resources
-      if (period == 0) {
-
-        // create RLE of totCapacity
-        TreeMap<Long, Resource> totAvailable = new TreeMap<Long, Resource>();
-        totAvailable.put(start, Resources.clone(totalCapacity));
-        RLESparseResourceAllocation totRLEAvail =
-            new RLESparseResourceAllocation(totAvailable, resCalc);
-
-        // subtract used from available
-        RLESparseResourceAllocation netAvailable;
-
-        netAvailable = RLESparseResourceAllocation.merge(resCalc,
-            Resources.clone(totalCapacity), totRLEAvail, rleSparseVector,
-            RLEOperator.subtractTestNonNegative, start, end);
-
-        // remove periodic component
-        netAvailable = RLESparseResourceAllocation.merge(resCalc,
-            Resources.clone(totalCapacity), netAvailable, periodicRle,
-            RLEOperator.subtractTestNonNegative, start, end);
-
-        // add back in old reservation used resources if any
-        ReservationAllocation old = reservationTable.get(oldId);
-        if (old != null) {
-
-          RLESparseResourceAllocation addBackPrevious =
-              old.getResourcesOverTime(start, end);
-          netAvailable = RLESparseResourceAllocation.merge(resCalc,
-              Resources.clone(totalCapacity), netAvailable, addBackPrevious,
-              RLEOperator.add, start, end);
-        }
-        // lower it if this is needed by the sharing policy
-        netAvailable = getSharingPolicy().availableResources(netAvailable, this,
-            user, oldId, start, end);
-        return netAvailable;
-      } else {
-
-        if (periodicRle.getTimePeriod() % period != 0) {
-          throw new PlanningException("The reservation periodicity (" + period
-              + ") must be" + "an exact divider of the system maxPeriod ("
-              + periodicRle.getTimePeriod() + ")");
-        }
-
-        // find the minimum resources available among all the instances that fit
-        // in the LCM
-        long numInstInLCM = periodicRle.getTimePeriod() / period;
-
-        RLESparseResourceAllocation minOverLCM =
-            getAvailableResourceOverTime(user, oldId, start, end, 0);
-        for (int i = 1; i < numInstInLCM; i++) {
-
-          long rStart = start + i * period;
-          long rEnd = end + i * period;
-
-          // recursive invocation of non-periodic range (to pick raw-info)
-          RLESparseResourceAllocation snapShot =
-              getAvailableResourceOverTime(user, oldId, rStart, rEnd, 0);
-
-          // time-align on start
-          snapShot.shift(-(i * period));
-
-          // pick the minimum amount of resources in each time interval
-          minOverLCM =
-              RLESparseResourceAllocation.merge(resCalc, getTotalCapacity(),
-                  minOverLCM, snapShot, RLEOperator.min, start, end);
-
-        }
-
-        return minOverLCM;
-
+      // create RLE of totCapacity
+      TreeMap<Long, Resource> totAvailable = new TreeMap<Long, Resource>();
+      totAvailable.put(start, Resources.clone(totalCapacity));
+      RLESparseResourceAllocation totRLEAvail =
+          new RLESparseResourceAllocation(totAvailable, resCalc);
+
+      // subtract used from available
+      RLESparseResourceAllocation netAvailable;
+
+      netAvailable =
+          RLESparseResourceAllocation.merge(resCalc,
+              Resources.clone(totalCapacity), totRLEAvail, rleSparseVector,
+              RLEOperator.subtractTestNonNegative, start, end);
+
+      // add back in old reservation used resources if any
+      ReservationAllocation old = reservationTable.get(oldId);
+      if (old != null) {
+        netAvailable =
+            RLESparseResourceAllocation.merge(resCalc,
+                Resources.clone(totalCapacity), netAvailable,
+                old.getResourcesOverTime(), RLEOperator.add, start, end);
       }
+      // lower it if this is needed by the sharing policy
+      netAvailable =
+          getSharingPolicy().availableResources(netAvailable, this, user,
+              oldId, start, end);
+      return netAvailable;
     } finally {
       readLock.unlock();
     }
@@ -808,7 +637,7 @@ public class InMemoryPlan implements Plan {
   public String toCumulativeString() {
     readLock.lock();
     try {
-      return rleSparseVector.toString() + "\n" + periodicRle.toString();
+      return rleSparseVector.toString();
     } finally {
       readLock.unlock();
     }
@@ -860,18 +689,11 @@ public class InMemoryPlan implements Plan {
   }
 
   @Override
-  public RLESparseResourceAllocation getCumulativeLoadOverTime(long start,
-      long end) throws PlanningException {
+  public RLESparseResourceAllocation getCumulativeLoadOverTime(
+      long start, long end) {
     readLock.lock();
     try {
-
-      RLESparseResourceAllocation ret =
-          rleSparseVector.getRangeOverlapping(start, end);
-      ret = RLESparseResourceAllocation.merge(resCalc, totalCapacity, ret,
-          periodicRle.getRangeOverlapping(start, end), RLEOperator.add, start,
-          end);
-
-      return ret;
+      return rleSparseVector.getRangeOverlapping(start, end);
     } finally {
       readLock.unlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryReservationAllocation.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/reservation/InMemoryReservationAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryReservationAllocation.java
index 00c8e44..69fd43f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryReservationAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryReservationAllocation.java
@@ -42,7 +42,6 @@ public class InMemoryReservationAllocation implements ReservationAllocation {
   private final Map<ReservationInterval, Resource> allocationRequests;
   private boolean hasGang = false;
   private long acceptedAt = -1;
-  private long periodicity = 0;
 
   private RLESparseResourceAllocation resourcesOverTime;
 
@@ -68,16 +67,9 @@ public class InMemoryReservationAllocation implements ReservationAllocation {
     this.allocationRequests = allocations;
     this.planName = planName;
     this.hasGang = hasGang;
-    if (contract != null && contract.getRecurrenceExpression() != null) {
-      this.periodicity = Long.parseLong(contract.getRecurrenceExpression());
-    }
-    if (periodicity > 0) {
-      resourcesOverTime =
-          new PeriodicRLESparseResourceAllocation(calculator, periodicity);
-    } else {
-      resourcesOverTime = new RLESparseResourceAllocation(calculator);
-    }
-    for (Map.Entry<ReservationInterval, Resource> r : allocations.entrySet()) {
+    resourcesOverTime = new RLESparseResourceAllocation(calculator);
+    for (Map.Entry<ReservationInterval, Resource> r : allocations
+        .entrySet()) {
       resourcesOverTime.addInterval(r.getKey(), r.getValue());
     }
   }
@@ -141,33 +133,17 @@ public class InMemoryReservationAllocation implements ReservationAllocation {
   }
 
   @Override
-  public RLESparseResourceAllocation getResourcesOverTime() {
+  public RLESparseResourceAllocation getResourcesOverTime(){
     return resourcesOverTime;
   }
 
   @Override
-  public RLESparseResourceAllocation getResourcesOverTime(long start,
-      long end) {
-    return resourcesOverTime.getRangeOverlapping(start, end);
-  }
-
-  @Override
-  public long getPeriodicity() {
-    return periodicity;
-  }
-
-  @Override
-  public void setPeriodicity(long period) {
-    periodicity = period;
-  }
-
-  @Override
   public String toString() {
     StringBuilder sBuf = new StringBuilder();
     sBuf.append(getReservationId()).append(" user:").append(getUser())
         .append(" startTime: ").append(getStartTime()).append(" endTime: ")
-        .append(getEndTime()).append(" Periodiciy: ").append(periodicity)
-        .append(" alloc:\n[").append(resourcesOverTime.toString()).append("] ");
+        .append(getEndTime()).append(" alloc:\n[")
+        .append(resourcesOverTime.toString()).append("] ");
     return sBuf.toString();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/NoOverCommitPolicy.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/reservation/NoOverCommitPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/NoOverCommitPolicy.java
index 49d4702..55f1d00 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/NoOverCommitPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/NoOverCommitPolicy.java
@@ -40,7 +40,7 @@ public class NoOverCommitPolicy implements SharingPolicy {
 
     RLESparseResourceAllocation available = plan.getAvailableResourceOverTime(
         reservation.getUser(), reservation.getReservationId(),
-        reservation.getStartTime(), reservation.getEndTime(), 0);
+        reservation.getStartTime(), reservation.getEndTime());
 
     // test the reservation does not exceed what is available
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PeriodicRLESparseResourceAllocation.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/reservation/PeriodicRLESparseResourceAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PeriodicRLESparseResourceAllocation.java
index 7bc44f5..8e3be8b3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PeriodicRLESparseResourceAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PeriodicRLESparseResourceAllocation.java
@@ -18,94 +18,47 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.NavigableMap;
-import java.util.TreeMap;
-
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.annotations.VisibleForTesting;
-
 /**
- * This data structure stores a periodic {@link RLESparseResourceAllocation}.
+ * This data structure stores a periodic RLESparseResourceAllocation.
  * Default period is 1 day (86400000ms).
  */
-public class PeriodicRLESparseResourceAllocation
-    extends RLESparseResourceAllocation {
+public class PeriodicRLESparseResourceAllocation extends
+    RLESparseResourceAllocation {
 
   // Log
-  private static final Logger LOG =
-      LoggerFactory.getLogger(PeriodicRLESparseResourceAllocation.class);
+  private static final Logger LOG = LoggerFactory
+      .getLogger(PeriodicRLESparseResourceAllocation.class);
 
   private long timePeriod;
 
   /**
    * Constructor.
    *
-   * @param resourceCalculator {@link ResourceCalculator} the resource
-   *          calculator to use.
+   * @param rleVector {@link RLESparseResourceAllocation} with the run-length
+              encoded data.
    * @param timePeriod Time period in milliseconds.
    */
   public PeriodicRLESparseResourceAllocation(
-      ResourceCalculator resourceCalculator, Long timePeriod) {
-    super(resourceCalculator);
+      RLESparseResourceAllocation rleVector, Long timePeriod) {
+    super(rleVector.getCumulative(), rleVector.getResourceCalculator());
     this.timePeriod = timePeriod;
   }
 
   /**
    * Constructor. Default time period set to 1 day.
    *
-   * @param resourceCalculator {@link ResourceCalculator} the resource
-   *          calculator to use..
-   */
-  public PeriodicRLESparseResourceAllocation(
-      ResourceCalculator resourceCalculator) {
-    this(resourceCalculator,
-        YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY);
-  }
-
-  /**
-   * Constructor.
-   *
    * @param rleVector {@link RLESparseResourceAllocation} with the run-length
-   *          encoded data.
-   * @param timePeriod Time period in milliseconds.
+              encoded data.
    */
-  @VisibleForTesting
   public PeriodicRLESparseResourceAllocation(
-      RLESparseResourceAllocation rleVector, Long timePeriod) {
-    super(rleVector.getCumulative(), rleVector.getResourceCalculator());
-    this.timePeriod = timePeriod;
-
-    // make sure the PeriodicRLE is zero-based, and handles wrap-around
-    long delta = (getEarliestStartTime() % timePeriod - getEarliestStartTime());
-    shift(delta);
-
-    List<Long> toRemove = new ArrayList<>();
-    Map<Long, Resource> toAdd = new TreeMap<>();
-
-    for (Map.Entry<Long, Resource> entry : cumulativeCapacity.entrySet()) {
-      if (entry.getKey() > timePeriod) {
-        toRemove.add(entry.getKey());
-        if (entry.getValue() != null) {
-          toAdd.put(timePeriod, entry.getValue());
-          long prev = entry.getKey() % timePeriod;
-          toAdd.put(prev, this.getCapacityAtTime(prev));
-          toAdd.put(0L, entry.getValue());
-        }
-      }
-    }
-    for (Long l : toRemove) {
-      cumulativeCapacity.remove(l);
-    }
-    cumulativeCapacity.putAll(toAdd);
+      RLESparseResourceAllocation rleVector) {
+    this(rleVector, 86400000L);
   }
 
   /**
@@ -125,25 +78,24 @@ public class PeriodicRLESparseResourceAllocation
    * The interval may include 0, but the end time must be strictly less than
    * timePeriod.
    *
-   * @param interval {@link ReservationInterval} to which the specified resource
-   *          is to be added.
+   * @param interval {@link ReservationInterval} to which the specified
+   *          resource is to be added.
    * @param resource {@link Resource} to be added to the interval specified.
    * @return true if addition is successful, false otherwise
    */
-  public boolean addInterval(ReservationInterval interval, Resource resource) {
+  public boolean addInterval(ReservationInterval interval,
+      Resource resource) {
     long startTime = interval.getStartTime();
     long endTime = interval.getEndTime();
-
     if (startTime >= 0 && endTime > startTime && endTime <= timePeriod) {
       return super.addInterval(interval, resource);
     } else {
-      LOG.info("Cannot set capacity beyond end time: " + timePeriod + " was ("
-          + interval.toString() + ")");
+      LOG.info("Cannot set capacity beyond end time: " + timePeriod);
       return false;
     }
   }
 
-  /**
+   /**
    * Removes a resource for the specified interval.
    *
    * @param interval the {@link ReservationInterval} for which the resource is
@@ -151,15 +103,14 @@ public class PeriodicRLESparseResourceAllocation
    * @param resource the {@link Resource} to be removed.
    * @return true if removal is successful, false otherwise
    */
-  public boolean removeInterval(ReservationInterval interval,
-      Resource resource) {
+  public boolean removeInterval(
+      ReservationInterval interval, Resource resource) {
     long startTime = interval.getStartTime();
     long endTime = interval.getEndTime();
     // If the resource to be subtracted is less than the minimum resource in
     // the range, abort removal to avoid negative capacity.
-    // TODO revesit decrementing endTime
-    if (!Resources.fitsIn(resource, getMinimumCapacityInInterval(
-        new ReservationInterval(startTime, endTime - 1)))) {
+    if (!Resources.fitsIn(
+        resource, super.getMinimumCapacityInInterval(interval))) {
       LOG.info("Request to remove more resources than what is available");
       return false;
     }
@@ -174,16 +125,17 @@ public class PeriodicRLESparseResourceAllocation
   /**
    * Get maximum capacity at periodic offsets from the specified time.
    *
-   * @param tick UTC time base from which offsets are specified for finding the
-   *          maximum capacity.
-   * @param period periodic offset at which capacities are evaluated.
+   * @param tick UTC time base from which offsets are specified for finding
+   *          the maximum capacity.
+   * @param period periodic offset at which capacities are evaluted.
    * @return the maximum {@link Resource} across the specified time instants.
    * @return true if removal is successful, false otherwise
    */
   public Resource getMaximumPeriodicCapacity(long tick, long period) {
     Resource maxResource;
     if (period < timePeriod) {
-      maxResource = super.getMaximumPeriodicCapacity(tick % timePeriod, period);
+      maxResource =
+          super.getMaximumPeriodicCapacity(tick % timePeriod, period);
     } else {
       // if period is greater than the length of PeriodicRLESparseAllocation,
       // only a single value exists in this interval.
@@ -212,30 +164,4 @@ public class PeriodicRLESparseResourceAllocation
     return ret.toString();
   }
 
-  @Override
-  public RLESparseResourceAllocation getRangeOverlapping(long start, long end) {
-    NavigableMap<Long, Resource> unrolledMap = new TreeMap<>();
-    readLock.lock();
-    try {
-      long relativeStart = (start >= 0) ? start % timePeriod : 0;
-      NavigableMap<Long, Resource> cumulativeMap = this.getCumulative();
-      Long previous = cumulativeMap.floorKey(relativeStart);
-      previous = (previous != null) ? previous : 0;
-      for (long i = 0; i <= (end - start) / timePeriod; i++) {
-        for (Map.Entry<Long, Resource> e : cumulativeMap.entrySet()) {
-          long curKey = e.getKey() + (i * timePeriod);
-          if (curKey >= previous && (start + curKey - relativeStart) <= end) {
-            unrolledMap.put(curKey, e.getValue());
-          }
-        }
-      }
-      RLESparseResourceAllocation rle =
-          new RLESparseResourceAllocation(unrolledMap, getResourceCalculator());
-      rle.shift(start - relativeStart);
-      return rle;
-    } finally {
-      readLock.unlock();
-    }
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanEdit.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/reservation/PlanEdit.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanEdit.java
index 9afa324..504a250 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanEdit.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanEdit.java
@@ -28,58 +28,54 @@ import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.Plan
 public interface PlanEdit extends PlanContext, PlanView {
 
   /**
-   * Add a new {@link ReservationAllocation} to the plan.
+   * Add a new {@link ReservationAllocation} to the plan
    * 
    * @param reservation the {@link ReservationAllocation} to be added to the
    *          plan
    * @param isRecovering flag to indicate if reservation is being added as part
    *          of failover or not
    * @return true if addition is successful, false otherwise
-   * @throws PlanningException if addition is unsuccessful
    */
-  boolean addReservation(ReservationAllocation reservation,
+  public boolean addReservation(ReservationAllocation reservation,
       boolean isRecovering) throws PlanningException;
 
   /**
    * Updates an existing {@link ReservationAllocation} in the plan. This is
-   * required for re-negotiation.
+   * required for re-negotiation
    * 
    * @param reservation the {@link ReservationAllocation} to be updated the plan
    * @return true if update is successful, false otherwise
-   * @throws PlanningException if update is unsuccessful
    */
-  boolean updateReservation(ReservationAllocation reservation)
+  public boolean updateReservation(ReservationAllocation reservation)
       throws PlanningException;
 
   /**
    * Delete an existing {@link ReservationAllocation} from the plan identified
    * uniquely by its {@link ReservationId}. This will generally be used for
-   * garbage collection.
+   * garbage collection
    * 
    * @param reservationID the {@link ReservationAllocation} to be deleted from
    *          the plan identified uniquely by its {@link ReservationId}
    * @return true if delete is successful, false otherwise
-   * @throws PlanningException if deletion is unsuccessful
    */
-  boolean deleteReservation(ReservationId reservationID)
+  public boolean deleteReservation(ReservationId reservationID)
       throws PlanningException;
 
   /**
    * Method invoked to garbage collect old reservations. It cleans up expired
-   * reservations that have fallen out of the sliding archival window.
+   * reservations that have fallen out of the sliding archival window
    * 
    * @param tick the current time from which the archival window is computed
-   * @throws PlanningException if archival is unsuccessful
    */
-  void archiveCompletedReservations(long tick) throws PlanningException;
+  public void archiveCompletedReservations(long tick) throws PlanningException;
 
   /**
    * Sets the overall capacity in terms of {@link Resource} assigned to this
-   * plan.
+   * plan
    * 
    * @param capacity the overall capacity in terms of {@link Resource} assigned
    *          to this plan
    */
-  void setTotalCapacity(Resource capacity);
+  public void setTotalCapacity(Resource capacity);
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanView.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/reservation/PlanView.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanView.java
index 4035f68..2767993 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanView.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanView.java
@@ -17,50 +17,50 @@
  */
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
-import java.util.Set;
-
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
 
+import java.util.Set;
+
 /**
  * This interface provides a read-only view on the allocations made in this
  * plan. This methods are used for example by {@code ReservationAgent}s to
  * determine the free resources in a certain point in time, and by
  * PlanFollowerPolicy to publish this plan to the scheduler.
  */
-interface PlanView extends PlanContext {
+public interface PlanView extends PlanContext {
 
   /**
    * Return a set of {@link ReservationAllocation} identified by the user who
    * made the reservation.
    *
    * @param reservationID the unqiue id to identify the
-   *          {@link ReservationAllocation}
+   * {@link ReservationAllocation}
    * @param interval the time interval used to retrieve the reservation
-   *          allocations from. Only reservations with start time no greater
-   *          than the interval end time, and end time no less than the interval
-   *          start time will be selected.
+   *                 allocations from. Only reservations with start time no
+   *                 greater than the interval end time, and end time no less
+   *                 than the interval start time will be selected.
    * @param user the user to retrieve the reservation allocation from.
    * @return a set of {@link ReservationAllocation} identified by the user who
-   *         made the reservation
+   * made the reservation
    */
-  Set<ReservationAllocation> getReservations(ReservationId reservationID,
-      ReservationInterval interval, String user);
+  Set<ReservationAllocation> getReservations(ReservationId
+                    reservationID, ReservationInterval interval, String user);
 
   /**
    * Return a set of {@link ReservationAllocation} identified by any user.
    *
    * @param reservationID the unqiue id to identify the
-   *          {@link ReservationAllocation}
+   * {@link ReservationAllocation}
    * @param interval the time interval used to retrieve the reservation
-   *          allocations from. Only reservations with start time no greater
-   *          than the interval end time, and end time no less than the interval
-   *          start time will be selected.
+   *                 allocations from. Only reservations with start time no
+   *                 greater than the interval end time, and end time no less
+   *                 than the interval start time will be selected.
    * @return a set of {@link ReservationAllocation} identified by any user
    */
   Set<ReservationAllocation> getReservations(ReservationId reservationID,
-      ReservationInterval interval);
+                    ReservationInterval interval);
 
   /**
    * Return a {@link ReservationAllocation} identified by its
@@ -70,7 +70,7 @@ interface PlanView extends PlanContext {
    *          {@link ReservationAllocation}
    * @return {@link ReservationAllocation} identified by the specified id
    */
-  ReservationAllocation getReservationById(ReservationId reservationID);
+  public ReservationAllocation getReservationById(ReservationId reservationID);
 
   /**
    * Return a set of {@link ReservationAllocation} that belongs to a certain
@@ -78,10 +78,11 @@ interface PlanView extends PlanContext {
    *
    * @param user the user being considered
    * @param t the instant in time being considered
-   * @return set of active {@link ReservationAllocation}s for this user at this
-   *         time
+   * @return set of active {@link ReservationAllocation}s for this
+   *         user at this time
    */
-  Set<ReservationAllocation> getReservationByUserAtTime(String user, long t);
+  public Set<ReservationAllocation> getReservationByUserAtTime(String user,
+      long t);
 
   /**
    * Gets all the active reservations at the specified point of time
@@ -90,14 +91,14 @@ interface PlanView extends PlanContext {
    *          requested
    * @return set of active reservations at the specified time
    */
-  Set<ReservationAllocation> getReservationsAtTime(long tick);
+  public Set<ReservationAllocation> getReservationsAtTime(long tick);
 
   /**
    * Gets all the reservations in the plan
    * 
    * @return set of all reservations handled by this Plan
    */
-  Set<ReservationAllocation> getAllReservations();
+  public Set<ReservationAllocation> getAllReservations();
 
   /**
    * Returns the total {@link Resource} reserved for all users at the specified
@@ -125,68 +126,61 @@ interface PlanView extends PlanContext {
    * 
    * @return the time (UTC in ms) at which the first reservation starts
    */
-  long getEarliestStartTime();
+  public long getEarliestStartTime();
 
   /**
    * Returns the time (UTC in ms) at which the last reservation terminates
    *
    * @return the time (UTC in ms) at which the last reservation terminates
    */
-  long getLastEndTime();
+  public long getLastEndTime();
 
   /**
    * This method returns the amount of resources available to a given user
    * (optionally if removing a certain reservation) over the start-end time
-   * range. If the request is periodic (period is non-zero) we return the
-   * minimum amount of resources available to periodic reservations (in all
-   * "period" windows within the system maxPeriod / LCM).
+   * range.
    *
-   * @param user the user being considered
-   * @param oldId the identifier of the existing reservation
-   * @param start start of the time interval.
-   * @param end end of the time interval.
-   * @param period the ms periodicty for this request (loop and pick min till
-   *          maxPeriodicity)
+   * @param user
+   * @param oldId
+   * @param start
+   * @param end
    * @return a view of the plan as it is available to this user
-   * @throws PlanningException if operation is unsuccessful
+   * @throws PlanningException
    */
-  RLESparseResourceAllocation getAvailableResourceOverTime(String user,
-      ReservationId oldId, long start, long end, long period)
-      throws PlanningException;
+  public RLESparseResourceAllocation getAvailableResourceOverTime(String user,
+      ReservationId oldId, long start, long end) throws PlanningException;
 
   /**
    * This method returns a RLE encoded view of the user reservation count
    * utilization between start and end time.
    *
-   * @param user the user being considered
-   * @param start start of the time interval.
-   * @param end end of the time interval.
+   * @param user
+   * @param start
+   * @param end
    * @return RLE encoded view of reservation used over time
    */
-  RLESparseResourceAllocation getReservationCountForUserOverTime(String user,
-      long start, long end);
+  public RLESparseResourceAllocation getReservationCountForUserOverTime(
+      String user, long start, long end);
 
   /**
    * This method returns a RLE encoded view of the user reservation utilization
    * between start and end time.
    *
-   * @param user the user being considered
-   * @param start start of the time interval.
-   * @param end end of the time interval.
+   * @param user
+   * @param start
+   * @param end
    * @return RLE encoded view of resources used over time
    */
-  RLESparseResourceAllocation getConsumptionForUserOverTime(String user,
+  public RLESparseResourceAllocation getConsumptionForUserOverTime(String user,
       long start, long end);
 
   /**
    * Get the cumulative load over a time interval.
    *
-   * @param start start of the time interval.
-   * @param end end of the time interval.
+   * @param start Start of the time interval.
+   * @param end End of the time interval.
    * @return RLE sparse allocation.
-   * @throws PlanningException if operation is unsuccessful
    */
-  RLESparseResourceAllocation getCumulativeLoadOverTime(long start, long end)
-      throws PlanningException;
+  RLESparseResourceAllocation getCumulativeLoadOverTime(long start, long end);
 
 }


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


[26/50] [abbrv] hadoop git commit: HDFS-12218. Addendum. Rename split EC / replicated block metrics in BlockManager.

Posted by as...@apache.org.
HDFS-12218. Addendum. Rename split EC / replicated block metrics in BlockManager.


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

Branch: refs/heads/YARN-5972
Commit: 4e50dc976a92a9560630c87cfc4e4513916e5735
Parents: 40c2f31
Author: Andrew Wang <wa...@apache.org>
Authored: Thu Sep 7 16:57:19 2017 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Thu Sep 7 16:57:19 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/protocol/ClientProtocol.java    |  4 +-
 .../hadoop/hdfs/protocol/ECBlockGroupStats.java | 60 ++++++++--------
 .../hdfs/protocol/ReplicatedBlockStats.java     | 72 ++++++++++----------
 .../ClientNamenodeProtocolTranslatorPB.java     | 28 ++++----
 .../hadoop/hdfs/protocolPB/PBHelperClient.java  | 42 ++++++------
 .../src/main/proto/ClientNamenodeProtocol.proto | 16 ++---
 ...tNamenodeProtocolServerSideTranslatorPB.java | 20 +++---
 .../server/blockmanagement/BlockManager.java    | 22 +++---
 .../blockmanagement/CorruptReplicasMap.java     |  6 +-
 .../blockmanagement/InvalidateBlocks.java       | 52 +++++++-------
 .../blockmanagement/LowRedundancyBlocks.java    | 16 ++---
 .../hdfs/server/namenode/FSNamesystem.java      | 19 +++---
 .../hdfs/server/namenode/NameNodeRpcServer.java |  8 +--
 .../namenode/metrics/ECBlockGroupsMBean.java    |  4 +-
 .../org/apache/hadoop/hdfs/tools/DFSAdmin.java  | 25 +++----
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     | 30 ++++----
 .../TestComputeInvalidateWork.java              |  4 +-
 .../blockmanagement/TestCorruptReplicaInfo.java |  8 +--
 .../TestLowRedundancyBlockQueues.java           | 10 +--
 .../namenode/metrics/TestNameNodeMetrics.java   |  2 +-
 .../apache/hadoop/hdfs/tools/TestDFSAdmin.java  |  4 +-
 21 files changed, 226 insertions(+), 226 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4e50dc97/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 6626e3b..8d5503f 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
@@ -778,14 +778,14 @@ public interface ClientProtocol {
    * in the filesystem.
    */
   @Idempotent
-  ReplicatedBlockStats getBlocksStats() throws IOException;
+  ReplicatedBlockStats getReplicatedBlockStats() throws IOException;
 
   /**
    * Get statistics pertaining to blocks of type {@link BlockType#STRIPED}
    * in the filesystem.
    */
   @Idempotent
-  ECBlockGroupStats getECBlockGroupsStats() throws IOException;
+  ECBlockGroupStats getECBlockGroupStats() throws IOException;
 
   /**
    * Get a report on the system's current datanodes.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4e50dc97/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ECBlockGroupStats.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ECBlockGroupStats.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ECBlockGroupStats.java
index 7258c43..9a8ad8c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ECBlockGroupStats.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ECBlockGroupStats.java
@@ -24,45 +24,45 @@ import org.apache.hadoop.classification.InterfaceStability;
  * Get statistics pertaining to blocks of type {@link BlockType#STRIPED}
  * in the filesystem.
  * <p>
- * @see ClientProtocol#getECBlockGroupsStats()
+ * @see ClientProtocol#getECBlockGroupStats()
  */
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 public final class ECBlockGroupStats {
-  private final long lowRedundancyBlockGroupsStat;
-  private final long corruptBlockGroupsStat;
-  private final long missingBlockGroupsStat;
-  private final long bytesInFutureBlockGroupsStat;
-  private final long pendingDeletionBlockGroupsStat;
+  private final long lowRedundancyBlockGroups;
+  private final long corruptBlockGroups;
+  private final long missingBlockGroups;
+  private final long bytesInFutureBlockGroups;
+  private final long pendingDeletionBlocks;
 
-  public ECBlockGroupStats(long lowRedundancyBlockGroupsStat, long
-      corruptBlockGroupsStat, long missingBlockGroupsStat, long
-      bytesInFutureBlockGroupsStat, long pendingDeletionBlockGroupsStat) {
-    this.lowRedundancyBlockGroupsStat = lowRedundancyBlockGroupsStat;
-    this.corruptBlockGroupsStat = corruptBlockGroupsStat;
-    this.missingBlockGroupsStat = missingBlockGroupsStat;
-    this.bytesInFutureBlockGroupsStat = bytesInFutureBlockGroupsStat;
-    this.pendingDeletionBlockGroupsStat = pendingDeletionBlockGroupsStat;
+  public ECBlockGroupStats(long lowRedundancyBlockGroups,
+      long corruptBlockGroups, long missingBlockGroups,
+      long bytesInFutureBlockGroups, long pendingDeletionBlocks) {
+    this.lowRedundancyBlockGroups = lowRedundancyBlockGroups;
+    this.corruptBlockGroups = corruptBlockGroups;
+    this.missingBlockGroups = missingBlockGroups;
+    this.bytesInFutureBlockGroups = bytesInFutureBlockGroups;
+    this.pendingDeletionBlocks = pendingDeletionBlocks;
   }
 
-  public long getBytesInFutureBlockGroupsStat() {
-    return bytesInFutureBlockGroupsStat;
+  public long getBytesInFutureBlockGroups() {
+    return bytesInFutureBlockGroups;
   }
 
-  public long getCorruptBlockGroupsStat() {
-    return corruptBlockGroupsStat;
+  public long getCorruptBlockGroups() {
+    return corruptBlockGroups;
   }
 
-  public long getLowRedundancyBlockGroupsStat() {
-    return lowRedundancyBlockGroupsStat;
+  public long getLowRedundancyBlockGroups() {
+    return lowRedundancyBlockGroups;
   }
 
-  public long getMissingBlockGroupsStat() {
-    return missingBlockGroupsStat;
+  public long getMissingBlockGroups() {
+    return missingBlockGroups;
   }
 
-  public long getPendingDeletionBlockGroupsStat() {
-    return pendingDeletionBlockGroupsStat;
+  public long getPendingDeletionBlocks() {
+    return pendingDeletionBlocks;
   }
 
   @Override
@@ -70,13 +70,13 @@ public final class ECBlockGroupStats {
     StringBuilder statsBuilder = new StringBuilder();
     statsBuilder.append("ECBlockGroupStats=[")
         .append("LowRedundancyBlockGroups=").append(
-            getLowRedundancyBlockGroupsStat())
-        .append(", CorruptBlockGroups=").append(getCorruptBlockGroupsStat())
-        .append(", MissingBlockGroups=").append(getMissingBlockGroupsStat())
+            getLowRedundancyBlockGroups())
+        .append(", CorruptBlockGroups=").append(getCorruptBlockGroups())
+        .append(", MissingBlockGroups=").append(getMissingBlockGroups())
         .append(", BytesInFutureBlockGroups=").append(
-            getBytesInFutureBlockGroupsStat())
-        .append(", PendingDeletionBlockGroups=").append(
-            getPendingDeletionBlockGroupsStat())
+            getBytesInFutureBlockGroups())
+        .append(", PendingDeletionBlocks=").append(
+            getPendingDeletionBlocks())
         .append("]");
     return statsBuilder.toString();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4e50dc97/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ReplicatedBlockStats.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ReplicatedBlockStats.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ReplicatedBlockStats.java
index c92dbc7..49aaded 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ReplicatedBlockStats.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ReplicatedBlockStats.java
@@ -24,66 +24,66 @@ import org.apache.hadoop.classification.InterfaceStability;
  * Get statistics pertaining to blocks of type {@link BlockType#CONTIGUOUS}
  * in the filesystem.
  * <p>
- * @see ClientProtocol#getBlocksStats()
+ * @see ClientProtocol#getReplicatedBlockStats()
  */
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 public final class ReplicatedBlockStats {
-  private final long lowRedundancyBlocksStat;
-  private final long corruptBlocksStat;
-  private final long missingBlocksStat;
-  private final long missingReplicationOneBlocksStat;
-  private final long bytesInFutureBlocksStat;
-  private final long pendingDeletionBlocksStat;
+  private final long lowRedundancyBlocks;
+  private final long corruptBlocks;
+  private final long missingBlocks;
+  private final long missingReplicationOneBlocks;
+  private final long bytesInFutureBlocks;
+  private final long pendingDeletionBlocks;
 
-  public ReplicatedBlockStats(long lowRedundancyBlocksStat,
-      long corruptBlocksStat, long missingBlocksStat,
-      long missingReplicationOneBlocksStat, long bytesInFutureBlocksStat,
-      long pendingDeletionBlocksStat) {
-    this.lowRedundancyBlocksStat = lowRedundancyBlocksStat;
-    this.corruptBlocksStat = corruptBlocksStat;
-    this.missingBlocksStat = missingBlocksStat;
-    this.missingReplicationOneBlocksStat = missingReplicationOneBlocksStat;
-    this.bytesInFutureBlocksStat = bytesInFutureBlocksStat;
-    this.pendingDeletionBlocksStat = pendingDeletionBlocksStat;
+  public ReplicatedBlockStats(long lowRedundancyBlocks,
+      long corruptBlocks, long missingBlocks,
+      long missingReplicationOneBlocks, long bytesInFutureBlocks,
+      long pendingDeletionBlocks) {
+    this.lowRedundancyBlocks = lowRedundancyBlocks;
+    this.corruptBlocks = corruptBlocks;
+    this.missingBlocks = missingBlocks;
+    this.missingReplicationOneBlocks = missingReplicationOneBlocks;
+    this.bytesInFutureBlocks = bytesInFutureBlocks;
+    this.pendingDeletionBlocks = pendingDeletionBlocks;
   }
 
-  public long getLowRedundancyBlocksStat() {
-    return lowRedundancyBlocksStat;
+  public long getLowRedundancyBlocks() {
+    return lowRedundancyBlocks;
   }
 
-  public long getCorruptBlocksStat() {
-    return corruptBlocksStat;
+  public long getCorruptBlocks() {
+    return corruptBlocks;
   }
 
-  public long getMissingReplicaBlocksStat() {
-    return missingBlocksStat;
+  public long getMissingReplicaBlocks() {
+    return missingBlocks;
   }
 
-  public long getMissingReplicationOneBlocksStat() {
-    return missingReplicationOneBlocksStat;
+  public long getMissingReplicationOneBlocks() {
+    return missingReplicationOneBlocks;
   }
 
-  public long getBytesInFutureBlocksStat() {
-    return bytesInFutureBlocksStat;
+  public long getBytesInFutureBlocks() {
+    return bytesInFutureBlocks;
   }
 
-  public long getPendingDeletionBlocksStat() {
-    return pendingDeletionBlocksStat;
+  public long getPendingDeletionBlocks() {
+    return pendingDeletionBlocks;
   }
 
   @Override
   public String toString() {
     StringBuilder statsBuilder = new StringBuilder();
-    statsBuilder.append("ReplicatedBlocksStats=[")
-        .append("LowRedundancyBlocks=").append(getLowRedundancyBlocksStat())
-        .append(", CorruptBlocks=").append(getCorruptBlocksStat())
-        .append(", MissingReplicaBlocks=").append(getMissingReplicaBlocksStat())
+    statsBuilder.append("ReplicatedBlockStats=[")
+        .append("LowRedundancyBlocks=").append(getLowRedundancyBlocks())
+        .append(", CorruptBlocks=").append(getCorruptBlocks())
+        .append(", MissingReplicaBlocks=").append(getMissingReplicaBlocks())
         .append(", MissingReplicationOneBlocks=").append(
-            getMissingReplicationOneBlocksStat())
-        .append(", BytesInFutureBlocks=").append(getBytesInFutureBlocksStat())
+            getMissingReplicationOneBlocks())
+        .append(", BytesInFutureBlocks=").append(getBytesInFutureBlocks())
         .append(", PendingDeletionBlocks=").append(
-            getPendingDeletionBlocksStat())
+            getPendingDeletionBlocks())
         .append("]");
     return statsBuilder.toString();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4e50dc97/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
index 53d8804..209eee7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
@@ -120,8 +120,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFil
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileLinkInfoRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileLinkInfoResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsECBlockGroupsStatsRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsBlocksStatsRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsECBlockGroupStatsRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsReplicatedBlockStatsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsStatusRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetLinkTargetRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetLinkTargetResponseProto;
@@ -246,13 +246,13 @@ public class ClientNamenodeProtocolTranslatorPB implements
   private final static GetFsStatusRequestProto VOID_GET_FSSTATUS_REQUEST =
       GetFsStatusRequestProto.newBuilder().build();
 
-  private final static GetFsBlocksStatsRequestProto
-      VOID_GET_FS_REPLICABLOCKS_STATS_REQUEST =
-      GetFsBlocksStatsRequestProto.newBuilder().build();
+  private final static GetFsReplicatedBlockStatsRequestProto
+      VOID_GET_FS_REPLICATED_BLOCK_STATS_REQUEST =
+      GetFsReplicatedBlockStatsRequestProto.newBuilder().build();
 
-  private final static GetFsECBlockGroupsStatsRequestProto
-      VOID_GET_FS_ECBLOCKGROUPS_STATS_REQUEST =
-      GetFsECBlockGroupsStatsRequestProto.newBuilder().build();
+  private final static GetFsECBlockGroupStatsRequestProto
+      VOID_GET_FS_ECBLOCKGROUP_STATS_REQUEST =
+      GetFsECBlockGroupStatsRequestProto.newBuilder().build();
 
   private final static RollEditsRequestProto VOID_ROLLEDITS_REQUEST =
       RollEditsRequestProto.getDefaultInstance();
@@ -695,20 +695,20 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public ReplicatedBlockStats getBlocksStats() throws IOException {
+  public ReplicatedBlockStats getReplicatedBlockStats() throws IOException {
     try {
-      return PBHelperClient.convert(rpcProxy.getFsBlocksStats(null,
-          VOID_GET_FS_REPLICABLOCKS_STATS_REQUEST));
+      return PBHelperClient.convert(rpcProxy.getFsReplicatedBlockStats(null,
+          VOID_GET_FS_REPLICATED_BLOCK_STATS_REQUEST));
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
   }
 
   @Override
-  public ECBlockGroupStats getECBlockGroupsStats() throws IOException {
+  public ECBlockGroupStats getECBlockGroupStats() throws IOException {
     try {
-      return PBHelperClient.convert(rpcProxy.getFsECBlockGroupsStats(null,
-          VOID_GET_FS_ECBLOCKGROUPS_STATS_REQUEST));
+      return PBHelperClient.convert(rpcProxy.getFsECBlockGroupStats(null,
+          VOID_GET_FS_ECBLOCKGROUP_STATS_REQUEST));
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4e50dc97/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
index 684ad70..d92d91e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
@@ -122,8 +122,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Create
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DatanodeReportTypeProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DatanodeStorageReportProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsECBlockGroupsStatsResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsBlocksStatsResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsECBlockGroupStatsResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsReplicatedBlockStatsResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsStatsResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.OpenFilesBatchResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeActionProto;
@@ -1811,7 +1811,7 @@ public class PBHelperClient {
   }
 
   public static ReplicatedBlockStats convert(
-      GetFsBlocksStatsResponseProto res) {
+      GetFsReplicatedBlockStatsResponseProto res) {
     return new ReplicatedBlockStats(res.getLowRedundancy(),
         res.getCorruptBlocks(), res.getMissingBlocks(),
         res.getMissingReplOneBlocks(), res.getBlocksInFuture(),
@@ -1819,7 +1819,7 @@ public class PBHelperClient {
   }
 
   public static ECBlockGroupStats convert(
-      GetFsECBlockGroupsStatsResponseProto res) {
+      GetFsECBlockGroupStatsResponseProto res) {
     return new ECBlockGroupStats(res.getLowRedundancy(),
         res.getCorruptBlocks(), res.getMissingBlocks(),
         res.getBlocksInFuture(), res.getPendingDeletionBlocks());
@@ -2236,37 +2236,37 @@ public class PBHelperClient {
     return result.build();
   }
 
-  public static GetFsBlocksStatsResponseProto convert(
+  public static GetFsReplicatedBlockStatsResponseProto convert(
       ReplicatedBlockStats replicatedBlockStats) {
-    GetFsBlocksStatsResponseProto.Builder result =
-        GetFsBlocksStatsResponseProto.newBuilder();
+    GetFsReplicatedBlockStatsResponseProto.Builder result =
+        GetFsReplicatedBlockStatsResponseProto.newBuilder();
     result.setLowRedundancy(
-        replicatedBlockStats.getLowRedundancyBlocksStat());
+        replicatedBlockStats.getLowRedundancyBlocks());
     result.setCorruptBlocks(
-        replicatedBlockStats.getCorruptBlocksStat());
+        replicatedBlockStats.getCorruptBlocks());
     result.setMissingBlocks(
-        replicatedBlockStats.getMissingReplicaBlocksStat());
+        replicatedBlockStats.getMissingReplicaBlocks());
     result.setMissingReplOneBlocks(
-        replicatedBlockStats.getMissingReplicationOneBlocksStat());
+        replicatedBlockStats.getMissingReplicationOneBlocks());
     result.setBlocksInFuture(
-        replicatedBlockStats.getBytesInFutureBlocksStat());
+        replicatedBlockStats.getBytesInFutureBlocks());
     result.setPendingDeletionBlocks(
-        replicatedBlockStats.getPendingDeletionBlocksStat());
+        replicatedBlockStats.getPendingDeletionBlocks());
     return result.build();
   }
 
-  public static GetFsECBlockGroupsStatsResponseProto convert(
+  public static GetFsECBlockGroupStatsResponseProto convert(
       ECBlockGroupStats ecBlockGroupStats) {
-    GetFsECBlockGroupsStatsResponseProto.Builder result =
-        GetFsECBlockGroupsStatsResponseProto.newBuilder();
+    GetFsECBlockGroupStatsResponseProto.Builder result =
+        GetFsECBlockGroupStatsResponseProto.newBuilder();
     result.setLowRedundancy(
-        ecBlockGroupStats.getLowRedundancyBlockGroupsStat());
-    result.setCorruptBlocks(ecBlockGroupStats.getCorruptBlockGroupsStat());
-    result.setMissingBlocks(ecBlockGroupStats.getMissingBlockGroupsStat());
+        ecBlockGroupStats.getLowRedundancyBlockGroups());
+    result.setCorruptBlocks(ecBlockGroupStats.getCorruptBlockGroups());
+    result.setMissingBlocks(ecBlockGroupStats.getMissingBlockGroups());
     result.setBlocksInFuture(
-        ecBlockGroupStats.getBytesInFutureBlockGroupsStat());
+        ecBlockGroupStats.getBytesInFutureBlockGroups());
     result.setPendingDeletionBlocks(
-        ecBlockGroupStats.getPendingDeletionBlockGroupsStat());
+        ecBlockGroupStats.getPendingDeletionBlocks());
     return result.build();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4e50dc97/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
index 3f108fa..6db6ad0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
@@ -327,10 +327,10 @@ message GetFsStatsResponseProto {
   optional uint64 pending_deletion_blocks = 9;
 }
 
-message GetFsBlocksStatsRequestProto { // no input paramters
+message GetFsReplicatedBlockStatsRequestProto { // no input paramters
 }
 
-message GetFsBlocksStatsResponseProto {
+message GetFsReplicatedBlockStatsResponseProto {
   required uint64 low_redundancy = 1;
   required uint64 corrupt_blocks = 2;
   required uint64 missing_blocks = 3;
@@ -339,10 +339,10 @@ message GetFsBlocksStatsResponseProto {
   required uint64 pending_deletion_blocks = 6;
 }
 
-message GetFsECBlockGroupsStatsRequestProto { // no input paramters
+message GetFsECBlockGroupStatsRequestProto { // no input paramters
 }
 
-message GetFsECBlockGroupsStatsResponseProto {
+message GetFsECBlockGroupStatsResponseProto {
   required uint64 low_redundancy = 1;
   required uint64 corrupt_blocks = 2;
   required uint64 missing_blocks = 3;
@@ -831,10 +831,10 @@ service ClientNamenodeProtocol {
   rpc recoverLease(RecoverLeaseRequestProto)
       returns(RecoverLeaseResponseProto);
   rpc getFsStats(GetFsStatusRequestProto) returns(GetFsStatsResponseProto);
-  rpc getFsBlocksStats(GetFsBlocksStatsRequestProto)
-      returns (GetFsBlocksStatsResponseProto);
-  rpc getFsECBlockGroupsStats(GetFsECBlockGroupsStatsRequestProto)
-      returns (GetFsECBlockGroupsStatsResponseProto);
+  rpc getFsReplicatedBlockStats(GetFsReplicatedBlockStatsRequestProto)
+      returns (GetFsReplicatedBlockStatsResponseProto);
+  rpc getFsECBlockGroupStats(GetFsECBlockGroupStatsRequestProto)
+      returns (GetFsECBlockGroupStatsResponseProto);
   rpc getDatanodeReport(GetDatanodeReportRequestProto)
       returns(GetDatanodeReportResponseProto);
   rpc getDatanodeStorageReport(GetDatanodeStorageReportRequestProto)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4e50dc97/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
index 44d5216..a79e75f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
@@ -124,12 +124,12 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFil
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileLinkInfoRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileLinkInfoResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsECBlockGroupStatsRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsECBlockGroupStatsResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsReplicatedBlockStatsRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsReplicatedBlockStatsResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsStatsResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsBlocksStatsResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsECBlockGroupsStatsResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsStatusRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsBlocksStatsRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsECBlockGroupsStatsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetLinkTargetRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetLinkTargetResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetListingRequestProto;
@@ -763,22 +763,22 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   }
 
   @Override
-  public GetFsBlocksStatsResponseProto getFsBlocksStats(
-      RpcController controller, GetFsBlocksStatsRequestProto request)
+  public GetFsReplicatedBlockStatsResponseProto getFsReplicatedBlockStats(
+      RpcController controller, GetFsReplicatedBlockStatsRequestProto request)
       throws ServiceException {
     try {
-      return PBHelperClient.convert(server.getBlocksStats());
+      return PBHelperClient.convert(server.getReplicatedBlockStats());
     } catch (IOException e) {
       throw new ServiceException(e);
     }
   }
 
   @Override
-  public GetFsECBlockGroupsStatsResponseProto getFsECBlockGroupsStats(
-      RpcController controller, GetFsECBlockGroupsStatsRequestProto request)
+  public GetFsECBlockGroupStatsResponseProto getFsECBlockGroupStats(
+      RpcController controller, GetFsECBlockGroupStatsRequestProto request)
       throws ServiceException {
     try {
-      return PBHelperClient.convert(server.getECBlockGroupsStats());
+      return PBHelperClient.convert(server.getECBlockGroupStats());
     } catch (IOException e) {
       throw new ServiceException(e);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4e50dc97/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index e83cbc6..f4e5cb4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -233,47 +233,47 @@ public class BlockManager implements BlockStatsMXBean {
 
   /** Used by metrics. */
   public long getLowRedundancyBlocks() {
-    return neededReconstruction.getLowRedundancyBlocksStat();
+    return neededReconstruction.getLowRedundancyBlocks();
   }
 
   /** Used by metrics. */
   public long getCorruptBlocks() {
-    return corruptReplicas.getCorruptBlocksStat();
+    return corruptReplicas.getCorruptBlocks();
   }
 
   /** Used by metrics. */
   public long getMissingBlocks() {
-    return neededReconstruction.getCorruptBlocksStat();
+    return neededReconstruction.getCorruptBlocks();
   }
 
   /** Used by metrics. */
   public long getMissingReplicationOneBlocks() {
-    return neededReconstruction.getCorruptReplicationOneBlocksStat();
+    return neededReconstruction.getCorruptReplicationOneBlocks();
   }
 
   /** Used by metrics. */
   public long getPendingDeletionReplicatedBlocks() {
-    return invalidateBlocks.getBlocksStat();
+    return invalidateBlocks.getBlocks();
   }
 
   /** Used by metrics. */
   public long getLowRedundancyECBlockGroups() {
-    return neededReconstruction.getLowRedundancyECBlockGroupsStat();
+    return neededReconstruction.getLowRedundancyECBlockGroups();
   }
 
   /** Used by metrics. */
   public long getCorruptECBlockGroups() {
-    return corruptReplicas.getCorruptECBlockGroupsStat();
+    return corruptReplicas.getCorruptECBlockGroups();
   }
 
   /** Used by metrics. */
   public long getMissingECBlockGroups() {
-    return neededReconstruction.getCorruptECBlockGroupsStat();
+    return neededReconstruction.getCorruptECBlockGroups();
   }
 
   /** Used by metrics. */
-  public long getPendingDeletionECBlockGroups() {
-    return invalidateBlocks.getECBlockGroupsStat();
+  public long getPendingDeletionECBlocks() {
+    return invalidateBlocks.getECBlocks();
   }
 
   /**
@@ -748,7 +748,7 @@ public class BlockManager implements BlockStatsMXBean {
     invalidateBlocks.dump(out);
 
     //Dump corrupt blocks and their storageIDs
-    Set<Block> corruptBlocks = corruptReplicas.getCorruptBlocks();
+    Set<Block> corruptBlocks = corruptReplicas.getCorruptBlocksSet();
     out.println("Corrupt Blocks:");
     for(Block block : corruptBlocks) {
       Collection<DatanodeDescriptor> corruptNodes =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4e50dc97/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CorruptReplicasMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CorruptReplicasMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CorruptReplicasMap.java
index d158b64..7a576bb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CorruptReplicasMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CorruptReplicasMap.java
@@ -240,7 +240,7 @@ public class CorruptReplicasMap{
    * method to get the set of corrupt blocks in corruptReplicasMap.
    * @return Set of Block objects
    */
-  Set<Block> getCorruptBlocks() {
+  Set<Block> getCorruptBlocksSet() {
     Set<Block> corruptBlocks = new HashSet<Block>();
     corruptBlocks.addAll(corruptReplicasMap.keySet());
     return corruptBlocks;
@@ -267,11 +267,11 @@ public class CorruptReplicasMap{
     }
   }
 
-  long getCorruptBlocksStat() {
+  long getCorruptBlocks() {
     return totalCorruptBlocks.longValue();
   }
 
-  long getCorruptECBlockGroupsStat() {
+  long getCorruptECBlockGroups() {
     return totalCorruptECBlockGroups.longValue();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4e50dc97/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/InvalidateBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/InvalidateBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/InvalidateBlocks.java
index 7b6b8a9..75561ca 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/InvalidateBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/InvalidateBlocks.java
@@ -53,9 +53,9 @@ class InvalidateBlocks {
   private final Map<DatanodeInfo, LightWeightHashSet<Block>>
       nodeToBlocks = new HashMap<>();
   private final Map<DatanodeInfo, LightWeightHashSet<Block>>
-      nodeToECBlockGroups = new HashMap<>();
+      nodeToECBlocks = new HashMap<>();
   private final LongAdder numBlocks = new LongAdder();
-  private final LongAdder numECBlockGroups = new LongAdder();
+  private final LongAdder numECBlocks = new LongAdder();
   private final int blockInvalidateLimit;
 
   /**
@@ -87,7 +87,7 @@ class InvalidateBlocks {
    * @return The total number of blocks to be invalidated.
    */
   long numBlocks() {
-    return getECBlockGroupsStat() + getBlocksStat();
+    return getECBlocks() + getBlocks();
   }
 
   /**
@@ -95,7 +95,7 @@ class InvalidateBlocks {
    * {@link org.apache.hadoop.hdfs.protocol.BlockType#CONTIGUOUS}
    * to be invalidated.
    */
-  long getBlocksStat() {
+  long getBlocks() {
     return numBlocks.longValue();
   }
 
@@ -104,8 +104,8 @@ class InvalidateBlocks {
    * {@link org.apache.hadoop.hdfs.protocol.BlockType#STRIPED}
    * to be invalidated.
    */
-  long getECBlockGroupsStat() {
-    return numECBlockGroups.longValue();
+  long getECBlocks() {
+    return numECBlocks.longValue();
   }
 
   private LightWeightHashSet<Block> getBlocksSet(final DatanodeInfo dn) {
@@ -115,9 +115,9 @@ class InvalidateBlocks {
     return null;
   }
 
-  private LightWeightHashSet<Block> getECBlockGroupsSet(final DatanodeInfo dn) {
-    if (nodeToECBlockGroups.containsKey(dn)) {
-      return nodeToECBlockGroups.get(dn);
+  private LightWeightHashSet<Block> getECBlocksSet(final DatanodeInfo dn) {
+    if (nodeToECBlocks.containsKey(dn)) {
+      return nodeToECBlocks.get(dn);
     }
     return null;
   }
@@ -125,7 +125,7 @@ class InvalidateBlocks {
   private LightWeightHashSet<Block> getBlocksSet(final DatanodeInfo dn,
       final Block block) {
     if (BlockIdManager.isStripedBlockID(block.getBlockId())) {
-      return getECBlockGroupsSet(dn);
+      return getECBlocksSet(dn);
     } else {
       return getBlocksSet(dn);
     }
@@ -134,8 +134,8 @@ class InvalidateBlocks {
   private void putBlocksSet(final DatanodeInfo dn, final Block block,
       final LightWeightHashSet set) {
     if (BlockIdManager.isStripedBlockID(block.getBlockId())) {
-      assert getECBlockGroupsSet(dn) == null;
-      nodeToECBlockGroups.put(dn, set);
+      assert getECBlocksSet(dn) == null;
+      nodeToECBlocks.put(dn, set);
     } else {
       assert getBlocksSet(dn) == null;
       nodeToBlocks.put(dn, set);
@@ -144,7 +144,7 @@ class InvalidateBlocks {
 
   private long getBlockSetsSize(final DatanodeInfo dn) {
     LightWeightHashSet<Block> replicaBlocks = getBlocksSet(dn);
-    LightWeightHashSet<Block> stripedBlocks = getECBlockGroupsSet(dn);
+    LightWeightHashSet<Block> stripedBlocks = getECBlocksSet(dn);
     return ((replicaBlocks == null ? 0 : replicaBlocks.size()) +
         (stripedBlocks == null ? 0 : stripedBlocks.size()));
   }
@@ -179,7 +179,7 @@ class InvalidateBlocks {
     }
     if (set.add(block)) {
       if (BlockIdManager.isStripedBlockID(block.getBlockId())) {
-        numECBlockGroups.increment();
+        numECBlocks.increment();
       } else {
         numBlocks.increment();
       }
@@ -196,9 +196,9 @@ class InvalidateBlocks {
     if (replicaBlockSets != null) {
       numBlocks.add(replicaBlockSets.size() * -1);
     }
-    LightWeightHashSet<Block> blockGroupSets = nodeToECBlockGroups.remove(dn);
-    if (blockGroupSets != null) {
-      numECBlockGroups.add(blockGroupSets.size() * -1);
+    LightWeightHashSet<Block> ecBlocksSet = nodeToECBlocks.remove(dn);
+    if (ecBlocksSet != null) {
+      numECBlocks.add(ecBlocksSet.size() * -1);
     }
   }
 
@@ -207,7 +207,7 @@ class InvalidateBlocks {
     final LightWeightHashSet<Block> v = getBlocksSet(dn, block);
     if (v != null && v.remove(block)) {
       if (BlockIdManager.isStripedBlockID(block.getBlockId())) {
-        numECBlockGroups.decrement();
+        numECBlocks.decrement();
       } else {
         numBlocks.decrement();
       }
@@ -231,21 +231,21 @@ class InvalidateBlocks {
   /** Print the contents to out. */
   synchronized void dump(final PrintWriter out) {
     final int size = nodeToBlocks.values().size() +
-        nodeToECBlockGroups.values().size();
+        nodeToECBlocks.values().size();
     out.println("Metasave: Blocks " + numBlocks()
         + " waiting deletion from " + size + " datanodes.");
     if (size == 0) {
       return;
     }
     dumpBlockSet(nodeToBlocks, out);
-    dumpBlockSet(nodeToECBlockGroups, out);
+    dumpBlockSet(nodeToECBlocks, out);
   }
 
   /** @return a list of the storage IDs. */
   synchronized List<DatanodeInfo> getDatanodes() {
     HashSet<DatanodeInfo> set = new HashSet<>();
     set.addAll(nodeToBlocks.keySet());
-    set.addAll(nodeToECBlockGroups.keySet());
+    set.addAll(nodeToECBlocks.keySet());
     return new ArrayList<>(set);
   }
 
@@ -289,9 +289,9 @@ class InvalidateBlocks {
       remainingLimit = getBlocksToInvalidateByLimit(nodeToBlocks.get(dn),
           toInvalidate, numBlocks, remainingLimit);
     }
-    if ((remainingLimit > 0) && (nodeToECBlockGroups.get(dn) != null)) {
-      getBlocksToInvalidateByLimit(nodeToECBlockGroups.get(dn),
-          toInvalidate, numECBlockGroups, remainingLimit);
+    if ((remainingLimit > 0) && (nodeToECBlocks.get(dn) != null)) {
+      getBlocksToInvalidateByLimit(nodeToECBlocks.get(dn),
+          toInvalidate, numECBlocks, remainingLimit);
     }
     if (toInvalidate.size() > 0 && getBlockSetsSize(dn) == 0) {
       remove(dn);
@@ -302,8 +302,8 @@ class InvalidateBlocks {
   
   synchronized void clear() {
     nodeToBlocks.clear();
-    nodeToECBlockGroups.clear();
+    nodeToECBlocks.clear();
     numBlocks.reset();
-    numECBlockGroups.reset();
+    numECBlocks.reset();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4e50dc97/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/LowRedundancyBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/LowRedundancyBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/LowRedundancyBlocks.java
index af2cb7e..347d606 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/LowRedundancyBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/LowRedundancyBlocks.java
@@ -144,33 +144,33 @@ class LowRedundancyBlocks implements Iterable<BlockInfo> {
 
   /** Return the number of corrupt blocks with replication factor 1 */
   long getCorruptReplicationOneBlockSize() {
-    return getCorruptReplicationOneBlocksStat();
+    return getCorruptReplicationOneBlocks();
   }
 
   /**
    * Return under replicated block count excluding corrupt replicas.
    */
-  long getLowRedundancyBlocksStat() {
-    return lowRedundancyBlocks.longValue() - getCorruptBlocksStat();
+  long getLowRedundancyBlocks() {
+    return lowRedundancyBlocks.longValue() - getCorruptBlocks();
   }
 
-  long getCorruptBlocksStat() {
+  long getCorruptBlocks() {
     return corruptBlocks.longValue();
   }
 
-  long getCorruptReplicationOneBlocksStat() {
+  long getCorruptReplicationOneBlocks() {
     return corruptReplicationOneBlocks.longValue();
   }
 
   /**
    *  Return low redundancy striped blocks excluding corrupt blocks.
    */
-  long getLowRedundancyECBlockGroupsStat() {
+  long getLowRedundancyECBlockGroups() {
     return lowRedundancyECBlockGroups.longValue() -
-        getCorruptECBlockGroupsStat();
+        getCorruptECBlockGroups();
   }
 
-  long getCorruptECBlockGroupsStat() {
+  long getCorruptECBlockGroups() {
     return corruptECBlockGroups.longValue();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4e50dc97/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index aada5bf..08c8562 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -89,7 +89,6 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY;
 import static org.apache.hadoop.hdfs.server.namenode.FSDirStatAndListingOp.*;
 
-import org.apache.hadoop.crypto.key.KeyProvider.KeyVersion;
 import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
 import org.apache.hadoop.hdfs.protocol.ECBlockGroupStats;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
@@ -4081,9 +4080,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * Get statistics pertaining to blocks of type {@link BlockType#CONTIGUOUS}
    * in the filesystem.
    * <p>
-   * @see ClientProtocol#getBlocksStats()
+   * @see ClientProtocol#getReplicatedBlockStats()
    */
-  ReplicatedBlockStats getBlocksStats() {
+  ReplicatedBlockStats getReplicatedBlockStats() {
     return new ReplicatedBlockStats(getLowRedundancyReplicatedBlocks(),
         getCorruptReplicatedBlocks(), getMissingReplicatedBlocks(),
         getMissingReplicationOneBlocks(), getBytesInFutureReplicatedBlocks(),
@@ -4094,12 +4093,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * Get statistics pertaining to blocks of type {@link BlockType#STRIPED}
    * in the filesystem.
    * <p>
-   * @see ClientProtocol#getECBlockGroupsStats()
+   * @see ClientProtocol#getECBlockGroupStats()
    */
-  ECBlockGroupStats getECBlockGroupsStats() {
+  ECBlockGroupStats getECBlockGroupStats() {
     return new ECBlockGroupStats(getLowRedundancyECBlockGroups(),
         getCorruptECBlockGroups(), getMissingECBlockGroups(),
-        getBytesInFutureECBlockGroups(), getPendingDeletionECBlockGroups());
+        getBytesInFutureECBlockGroups(), getPendingDeletionECBlocks());
   }
 
   @Override // FSNamesystemMBean
@@ -4712,10 +4711,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   @Override // ECBlockGroupsMBean
-  @Metric({"PendingDeletionECBlockGroups", "Number of erasure coded block " +
-      "groups that are pending deletion"})
-  public long getPendingDeletionECBlockGroups() {
-    return blockManager.getPendingDeletionECBlockGroups();
+  @Metric({"PendingDeletionECBlocks", "Number of erasure coded blocks " +
+      "that are pending deletion"})
+  public long getPendingDeletionECBlocks() {
+    return blockManager.getPendingDeletionECBlocks();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4e50dc97/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index 7b14226..1ef3f55 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -1163,17 +1163,17 @@ public class NameNodeRpcServer implements NamenodeProtocols {
   }
 
   @Override // ClientProtocol
-  public ReplicatedBlockStats getBlocksStats() throws IOException {
+  public ReplicatedBlockStats getReplicatedBlockStats() throws IOException {
     checkNNStartup();
     namesystem.checkOperation(OperationCategory.READ);
-    return namesystem.getBlocksStats();
+    return namesystem.getReplicatedBlockStats();
   }
 
   @Override // ClientProtocol
-  public ECBlockGroupStats getECBlockGroupsStats() throws IOException {
+  public ECBlockGroupStats getECBlockGroupStats() throws IOException {
     checkNNStartup();
     namesystem.checkOperation(OperationCategory.READ);
-    return namesystem.getECBlockGroupsStats();
+    return namesystem.getECBlockGroupStats();
   }
 
   @Override // ClientProtocol

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4e50dc97/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/ECBlockGroupsMBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/ECBlockGroupsMBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/ECBlockGroupsMBean.java
index 5fa646a..474f3ed 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/ECBlockGroupsMBean.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/ECBlockGroupsMBean.java
@@ -53,7 +53,7 @@ public interface ECBlockGroupsMBean {
   long getBytesInFutureECBlockGroups();
 
   /**
-   * Return count of erasure coded block groups that are pending deletion.
+   * Return count of erasure coded blocks that are pending deletion.
    */
-  long getPendingDeletionECBlockGroups();
+  long getPendingDeletionECBlocks();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4e50dc97/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
index a2bb2c05..cc7eb1b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
@@ -534,30 +534,31 @@ public class DFSAdmin extends FsShell {
      * minutes. Use "-metaSave" to list of all such blocks and accurate 
      * counts.
      */
-    ReplicatedBlockStats replicatedBlockStats = dfs.getClient().getNamenode().getBlocksStats();
+    ReplicatedBlockStats replicatedBlockStats =
+        dfs.getClient().getNamenode().getReplicatedBlockStats();
     System.out.println("Replicated Blocks:");
     System.out.println("\tUnder replicated blocks: " +
-        replicatedBlockStats.getLowRedundancyBlocksStat());
+        replicatedBlockStats.getLowRedundancyBlocks());
     System.out.println("\tBlocks with corrupt replicas: " +
-        replicatedBlockStats.getCorruptBlocksStat());
+        replicatedBlockStats.getCorruptBlocks());
     System.out.println("\tMissing blocks: " +
-        replicatedBlockStats.getMissingReplicaBlocksStat());
+        replicatedBlockStats.getMissingReplicaBlocks());
     System.out.println("\tMissing blocks (with replication factor 1): " +
-        replicatedBlockStats.getMissingReplicationOneBlocksStat());
+        replicatedBlockStats.getMissingReplicationOneBlocks());
     System.out.println("\tPending deletion blocks: " +
-        replicatedBlockStats.getPendingDeletionBlocksStat());
+        replicatedBlockStats.getPendingDeletionBlocks());
 
     ECBlockGroupStats ecBlockGroupStats =
-        dfs.getClient().getNamenode().getECBlockGroupsStats();
+        dfs.getClient().getNamenode().getECBlockGroupStats();
     System.out.println("Erasure Coded Block Groups: ");
     System.out.println("\tLow redundancy block groups: " +
-        ecBlockGroupStats.getLowRedundancyBlockGroupsStat());
+        ecBlockGroupStats.getLowRedundancyBlockGroups());
     System.out.println("\tBlock groups with corrupt internal blocks: " +
-        ecBlockGroupStats.getCorruptBlockGroupsStat());
+        ecBlockGroupStats.getCorruptBlockGroups());
     System.out.println("\tMissing block groups: " +
-        ecBlockGroupStats.getMissingBlockGroupsStat());
-    System.out.println("\tPending deletion block groups: " +
-        ecBlockGroupStats.getPendingDeletionBlockGroupsStat());
+        ecBlockGroupStats.getMissingBlockGroups());
+    System.out.println("\tPending deletion blocks: " +
+        ecBlockGroupStats.getPendingDeletionBlocks());
 
     System.out.println();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4e50dc97/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
index 0926b44..b00eff2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
@@ -1657,8 +1657,8 @@ public class DFSTestUtil {
 
   /**
    * Verify the aggregated {@link ClientProtocol#getStats()} block counts equal
-   * the sum of {@link ClientProtocol#getBlocksStats()} and
-   * {@link ClientProtocol#getECBlockGroupsStats()}.
+   * the sum of {@link ClientProtocol#getReplicatedBlockStats()} and
+   * {@link ClientProtocol#getECBlockGroupStats()}.
    * @throws Exception
    */
   public static  void verifyClientStats(Configuration conf,
@@ -1668,35 +1668,35 @@ public class DFSTestUtil {
         ClientProtocol.class).getProxy();
     long[] aggregatedStats = cluster.getNameNode().getRpcServer().getStats();
     ReplicatedBlockStats replicatedBlockStats =
-        client.getBlocksStats();
-    ECBlockGroupStats ecBlockGroupStats = client.getECBlockGroupsStats();
+        client.getReplicatedBlockStats();
+    ECBlockGroupStats ecBlockGroupStats = client.getECBlockGroupStats();
 
     assertEquals("Under replicated stats not matching!",
         aggregatedStats[ClientProtocol.GET_STATS_LOW_REDUNDANCY_IDX],
         aggregatedStats[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX]);
     assertEquals("Low redundancy stats not matching!",
         aggregatedStats[ClientProtocol.GET_STATS_LOW_REDUNDANCY_IDX],
-        replicatedBlockStats.getLowRedundancyBlocksStat() +
-            ecBlockGroupStats.getLowRedundancyBlockGroupsStat());
+        replicatedBlockStats.getLowRedundancyBlocks() +
+            ecBlockGroupStats.getLowRedundancyBlockGroups());
     assertEquals("Corrupt blocks stats not matching!",
         aggregatedStats[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX],
-        replicatedBlockStats.getCorruptBlocksStat() +
-            ecBlockGroupStats.getCorruptBlockGroupsStat());
+        replicatedBlockStats.getCorruptBlocks() +
+            ecBlockGroupStats.getCorruptBlockGroups());
     assertEquals("Missing blocks stats not matching!",
         aggregatedStats[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX],
-        replicatedBlockStats.getMissingReplicaBlocksStat() +
-            ecBlockGroupStats.getMissingBlockGroupsStat());
+        replicatedBlockStats.getMissingReplicaBlocks() +
+            ecBlockGroupStats.getMissingBlockGroups());
     assertEquals("Missing blocks with replication factor one not matching!",
         aggregatedStats[ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX],
-        replicatedBlockStats.getMissingReplicationOneBlocksStat());
+        replicatedBlockStats.getMissingReplicationOneBlocks());
     assertEquals("Bytes in future blocks stats not matching!",
         aggregatedStats[ClientProtocol.GET_STATS_BYTES_IN_FUTURE_BLOCKS_IDX],
-        replicatedBlockStats.getBytesInFutureBlocksStat() +
-            ecBlockGroupStats.getBytesInFutureBlockGroupsStat());
+        replicatedBlockStats.getBytesInFutureBlocks() +
+            ecBlockGroupStats.getBytesInFutureBlockGroups());
     assertEquals("Pending deletion blocks stats not matching!",
         aggregatedStats[ClientProtocol.GET_STATS_PENDING_DELETION_BLOCKS_IDX],
-        replicatedBlockStats.getPendingDeletionBlocksStat() +
-            ecBlockGroupStats.getPendingDeletionBlockGroupsStat());
+        replicatedBlockStats.getPendingDeletionBlocks() +
+            ecBlockGroupStats.getPendingDeletionBlocks());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4e50dc97/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestComputeInvalidateWork.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestComputeInvalidateWork.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestComputeInvalidateWork.java
index 2413918..e7bd3d2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestComputeInvalidateWork.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestComputeInvalidateWork.java
@@ -268,9 +268,9 @@ public class TestComputeInvalidateWork {
           "Striped BlockGroups!",
           (long) expected, invalidateBlocks.numBlocks());
       assertEquals("Unexpected invalidate count for replicas!",
-          totalReplicas, invalidateBlocks.getBlocksStat());
+          totalReplicas, invalidateBlocks.getBlocks());
       assertEquals("Unexpected invalidate count for striped block groups!",
-          totalStripedDataBlocks, invalidateBlocks.getECBlockGroupsStat());
+          totalStripedDataBlocks, invalidateBlocks.getECBlocks());
     } finally {
       namesystem.writeUnlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4e50dc97/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestCorruptReplicaInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestCorruptReplicaInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestCorruptReplicaInfo.java
index 3f8a5cd..3510bc3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestCorruptReplicaInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestCorruptReplicaInfo.java
@@ -78,10 +78,10 @@ public class TestCorruptReplicaInfo {
     assertEquals("Unexpected total corrupt blocks count!",
         totalExpectedCorruptBlocks, corruptReplicasMap.size());
     assertEquals("Unexpected replica blocks count!",
-        expectedReplicaCount, corruptReplicasMap.getCorruptBlocksStat());
+        expectedReplicaCount, corruptReplicasMap.getCorruptBlocks());
     assertEquals("Unexpected striped blocks count!",
         expectedStripedBlockCount,
-        corruptReplicasMap.getCorruptECBlockGroupsStat());
+        corruptReplicasMap.getCorruptECBlockGroups());
   }
   
   @Test
@@ -93,9 +93,9 @@ public class TestCorruptReplicaInfo {
     assertEquals("Total number of corrupt blocks must initially be 0!",
         0, crm.size());
     assertEquals("Number of corrupt replicas must initially be 0!",
-        0, crm.getCorruptBlocksStat());
+        0, crm.getCorruptBlocks());
     assertEquals("Number of corrupt striped block groups must initially be 0!",
-        0, crm.getCorruptECBlockGroupsStat());
+        0, crm.getCorruptECBlockGroups());
     assertNull("Param n cannot be less than 0",
         crm.getCorruptBlockIdsForTesting(BlockType.CONTIGUOUS, -1, null));
     assertNull("Param n cannot be greater than 100",

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4e50dc97/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestLowRedundancyBlockQueues.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestLowRedundancyBlockQueues.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestLowRedundancyBlockQueues.java
index c65fc64..2b28f1e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestLowRedundancyBlockQueues.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestLowRedundancyBlockQueues.java
@@ -50,16 +50,16 @@ public class TestLowRedundancyBlockQueues {
       int corruptReplicationOneCount, int lowRedundancyStripedCount,
       int corruptStripedCount) {
     assertEquals("Low redundancy replica count incorrect!",
-        lowRedundancyReplicaCount, queues.getLowRedundancyBlocksStat());
+        lowRedundancyReplicaCount, queues.getLowRedundancyBlocks());
     assertEquals("Corrupt replica count incorrect!",
-        corruptReplicaCount, queues.getCorruptBlocksStat());
+        corruptReplicaCount, queues.getCorruptBlocks());
     assertEquals("Corrupt replica one count incorrect!",
         corruptReplicationOneCount,
-        queues.getCorruptReplicationOneBlocksStat());
+        queues.getCorruptReplicationOneBlocks());
     assertEquals("Low redundancy striped blocks count incorrect!",
-        lowRedundancyStripedCount, queues.getLowRedundancyECBlockGroupsStat());
+        lowRedundancyStripedCount, queues.getLowRedundancyECBlockGroups());
     assertEquals("Corrupt striped blocks count incorrect!",
-        corruptStripedCount, queues.getCorruptECBlockGroupsStat());
+        corruptStripedCount, queues.getCorruptECBlockGroups());
     assertEquals("Low Redundancy count incorrect!",
         lowRedundancyReplicaCount + lowRedundancyStripedCount,
         queues.getLowRedundancyBlockCount());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4e50dc97/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
index b983fd1..e046b50 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
@@ -449,7 +449,7 @@ public class TestNameNodeMetrics {
     assertEquals("Pending deletion blocks metrics not matching!",
         namesystem.getPendingDeletionBlocks(),
         namesystem.getPendingDeletionReplicatedBlocks() +
-            namesystem.getPendingDeletionECBlockGroups());
+            namesystem.getPendingDeletionECBlocks());
   }
 
   /** Corrupt a block and ensure metrics reflects it */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4e50dc97/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java
index 2d38f2f..c515df3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java
@@ -778,9 +778,9 @@ public class TestDFSAdmin {
     assertEquals(numCorruptBlocks + numCorruptECBlockGroups,
         client.getCorruptBlocksCount());
     assertEquals(numCorruptBlocks, client.getNamenode()
-        .getBlocksStats().getCorruptBlocksStat());
+        .getReplicatedBlockStats().getCorruptBlocks());
     assertEquals(numCorruptECBlockGroups, client.getNamenode()
-        .getECBlockGroupsStats().getCorruptBlockGroupsStat());
+        .getECBlockGroupStats().getCorruptBlockGroups());
   }
 
   @Test


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


[44/50] [abbrv] hadoop git commit: HADOOP-14851 LambdaTestUtils.eventually() doesn't spin on Assertion failures. Contributed by Steve Loughran

Posted by as...@apache.org.
HADOOP-14851 LambdaTestUtils.eventually() doesn't spin on Assertion failures.  Contributed by Steve Loughran


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

Branch: refs/heads/YARN-5972
Commit: 180e814b081d3707c95641171d649b547db41a04
Parents: 3fddabc
Author: Aaron Fabbri <fa...@cloudera.com>
Authored: Fri Sep 8 19:26:27 2017 -0700
Committer: Aaron Fabbri <fa...@cloudera.com>
Committed: Fri Sep 8 19:32:07 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/test/LambdaTestUtils.java |  68 +++++++---
 .../apache/hadoop/test/TestLambdaTestUtils.java | 127 +++++++++++++++++--
 2 files changed, 163 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/180e814b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/LambdaTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/LambdaTestUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/LambdaTestUtils.java
index 00cfa44..3ea9ab8 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/LambdaTestUtils.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/LambdaTestUtils.java
@@ -70,7 +70,7 @@ public final class LambdaTestUtils {
      * @throws Exception if the handler wishes to raise an exception
      * that way.
      */
-    Exception evaluate(int timeoutMillis, Exception caught) throws Exception;
+    Throwable evaluate(int timeoutMillis, Throwable caught) throws Throwable;
   }
 
   /**
@@ -116,7 +116,7 @@ public final class LambdaTestUtils {
     Preconditions.checkNotNull(timeoutHandler);
 
     long endTime = Time.now() + timeoutMillis;
-    Exception ex = null;
+    Throwable ex = null;
     boolean running = true;
     int iterations = 0;
     while (running) {
@@ -128,9 +128,11 @@ public final class LambdaTestUtils {
         // the probe failed but did not raise an exception. Reset any
         // exception raised by a previous probe failure.
         ex = null;
-      } catch (InterruptedException | FailFastException e) {
+      } catch (InterruptedException
+          | FailFastException
+          | VirtualMachineError e) {
         throw e;
-      } catch (Exception e) {
+      } catch (Throwable e) {
         LOG.debug("eventually() iteration {}", iterations, e);
         ex = e;
       }
@@ -145,15 +147,20 @@ public final class LambdaTestUtils {
       }
     }
     // timeout
-    Exception evaluate = timeoutHandler.evaluate(timeoutMillis, ex);
-    if (evaluate == null) {
-      // bad timeout handler logic; fall back to GenerateTimeout so the
-      // underlying problem isn't lost.
-      LOG.error("timeout handler {} did not throw an exception ",
-          timeoutHandler);
-      evaluate = new GenerateTimeout().evaluate(timeoutMillis, ex);
+    Throwable evaluate;
+    try {
+      evaluate = timeoutHandler.evaluate(timeoutMillis, ex);
+      if (evaluate == null) {
+        // bad timeout handler logic; fall back to GenerateTimeout so the
+        // underlying problem isn't lost.
+        LOG.error("timeout handler {} did not throw an exception ",
+            timeoutHandler);
+        evaluate = new GenerateTimeout().evaluate(timeoutMillis, ex);
+      }
+    } catch (Throwable throwable) {
+      evaluate = throwable;
     }
-    throw evaluate;
+    return raise(evaluate);
   }
 
   /**
@@ -217,6 +224,7 @@ public final class LambdaTestUtils {
    * @throws Exception the last exception thrown before timeout was triggered
    * @throws FailFastException if raised -without any retry attempt.
    * @throws InterruptedException if interrupted during the sleep operation.
+   * @throws OutOfMemoryError you've run out of memory.
    */
   public static <T> T eventually(int timeoutMillis,
       Callable<T> eval,
@@ -224,7 +232,7 @@ public final class LambdaTestUtils {
     Preconditions.checkArgument(timeoutMillis >= 0,
         "timeoutMillis must be >= 0");
     long endTime = Time.now() + timeoutMillis;
-    Exception ex;
+    Throwable ex;
     boolean running;
     int sleeptime;
     int iterations = 0;
@@ -232,10 +240,12 @@ public final class LambdaTestUtils {
       iterations++;
       try {
         return eval.call();
-      } catch (InterruptedException | FailFastException e) {
+      } catch (InterruptedException
+          | FailFastException
+          | VirtualMachineError e) {
         // these two exceptions trigger an immediate exit
         throw e;
-      } catch (Exception e) {
+      } catch (Throwable e) {
         LOG.debug("evaluate() iteration {}", iterations, e);
         ex = e;
       }
@@ -245,7 +255,26 @@ public final class LambdaTestUtils {
       }
     } while (running);
     // timeout. Throw the last exception raised
-    throw ex;
+    return raise(ex);
+  }
+
+  /**
+   * Take the throwable and raise it as an exception or an error, depending
+   * upon its type. This allows callers to declare that they only throw
+   * Exception (i.e. can be invoked by Callable) yet still rethrow a
+   * previously caught Throwable.
+   * @param throwable Throwable to rethrow
+   * @param <T> expected return type
+   * @return never
+   * @throws Exception if throwable is an Exception
+   * @throws Error if throwable is not an Exception
+   */
+  private static <T> T raise(Throwable throwable) throws Exception {
+    if (throwable instanceof Exception) {
+      throw (Exception) throwable;
+    } else {
+      throw (Error) throwable;
+    }
   }
 
   /**
@@ -365,6 +394,7 @@ public final class LambdaTestUtils {
    * @throws Exception any other exception raised
    * @throws AssertionError if the evaluation call didn't raise an exception.
    */
+  @SuppressWarnings("unchecked")
   public static <E extends Throwable> E intercept(
       Class<E> clazz,
       VoidCallable eval)
@@ -487,14 +517,14 @@ public final class LambdaTestUtils {
      * @return TimeoutException
      */
     @Override
-    public Exception evaluate(int timeoutMillis, Exception caught)
-        throws Exception {
+    public Throwable evaluate(int timeoutMillis, Throwable caught)
+        throws Throwable {
       String s = String.format("%s: after %d millis", message,
           timeoutMillis);
       String caughtText = caught != null
           ? ("; " + robustToString(caught)) : "";
 
-      return (TimeoutException) (new TimeoutException(s + caughtText)
+      return (new TimeoutException(s + caughtText)
                                      .initCause(caught));
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/180e814b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestLambdaTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestLambdaTestUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestLambdaTestUtils.java
index d3d5cb4..c790a18 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestLambdaTestUtils.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestLambdaTestUtils.java
@@ -25,6 +25,7 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.concurrent.Callable;
 import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import static org.apache.hadoop.test.LambdaTestUtils.*;
 import static org.apache.hadoop.test.GenericTestUtils.*;
@@ -123,6 +124,27 @@ public class TestLambdaTestUtils extends Assert {
         minCount <= retry.getInvocationCount());
   }
 
+  /**
+   * Raise an exception.
+   * @param e exception to raise
+   * @return never
+   * @throws Exception passed in exception
+   */
+  private boolean r(Exception e) throws Exception {
+    throw e;
+  }
+
+  /**
+   * Raise an error.
+   * @param e error to raise
+   * @return never
+   * @throws Exception never
+   * @throws Error the passed in error
+   */
+  private boolean r(Error e) throws Exception {
+    throw e;
+  }
+
   @Test
   public void testAwaitAlwaysTrue() throws Throwable {
     await(TIMEOUT,
@@ -140,7 +162,7 @@ public class TestLambdaTestUtils extends Assert {
           TIMEOUT_FAILURE_HANDLER);
       fail("should not have got here");
     } catch (TimeoutException e) {
-      assertTrue(retry.getInvocationCount() > 4);
+      assertMinRetryCount(1);
     }
   }
 
@@ -316,9 +338,7 @@ public class TestLambdaTestUtils extends Assert {
     IOException ioe = intercept(IOException.class,
         () -> await(
             TIMEOUT,
-            () -> {
-              throw new IOException("inner " + ++count);
-            },
+            () -> r(new IOException("inner " + ++count)),
             retry,
             (timeout, ex) -> ex));
     assertRetryCount(count - 1);
@@ -339,9 +359,7 @@ public class TestLambdaTestUtils extends Assert {
   public void testInterceptAwaitFailFastLambda() throws Throwable {
     intercept(FailFastException.class,
         () -> await(TIMEOUT,
-            () -> {
-              throw new FailFastException("ffe");
-            },
+            () -> r(new FailFastException("ffe")),
             retry,
             (timeout, ex) -> ex));
     assertRetryCount(0);
@@ -361,14 +379,13 @@ public class TestLambdaTestUtils extends Assert {
     assertRetryCount(0);
   }
 
+
   @Test
   public void testInterceptEventuallyLambdaFailures() throws Throwable {
     intercept(IOException.class,
         "oops",
         () -> eventually(TIMEOUT,
-            () -> {
-              throw new IOException("oops");
-            },
+            () -> r(new IOException("oops")),
             retry));
     assertMinRetryCount(1);
   }
@@ -385,11 +402,95 @@ public class TestLambdaTestUtils extends Assert {
     intercept(FailFastException.class, "oops",
         () -> eventually(
             TIMEOUT,
-            () -> {
-              throw new FailFastException("oops");
-            },
+            () -> r(new FailFastException("oops")),
+            retry));
+    assertRetryCount(0);
+  }
+
+  /**
+   * Verify that assertions trigger catch and retry.
+   * @throws Throwable if the code is broken
+   */
+  @Test
+  public void testEventuallySpinsOnAssertions() throws Throwable {
+    AtomicInteger counter = new AtomicInteger(0);
+    eventually(TIMEOUT,
+        () -> {
+          while (counter.incrementAndGet() < 5) {
+            fail("if you see this, we are in trouble");
+          }
+        },
+        retry);
+    assertMinRetryCount(4);
+  }
+
+  /**
+   * Verify that VirtualMachineError errors are immediately rethrown.
+   * @throws Throwable if the code is broken
+   */
+  @Test
+  public void testInterceptEventuallyThrowsVMErrors() throws Throwable {
+    intercept(OutOfMemoryError.class, "OOM",
+        () -> eventually(
+            TIMEOUT,
+            () -> r(new OutOfMemoryError("OOM")),
             retry));
     assertRetryCount(0);
   }
 
+  /**
+   * Verify that you can declare that an intercept will intercept Errors.
+   * @throws Throwable if the code is broken
+   */
+  @Test
+  public void testInterceptHandlesErrors() throws Throwable {
+    intercept(OutOfMemoryError.class, "OOM",
+        () -> r(new OutOfMemoryError("OOM")));
+  }
+
+  /**
+   * Verify that if an Error raised is not the one being intercepted,
+   * it gets rethrown.
+   * @throws Throwable if the code is broken
+   */
+  @Test
+  public void testInterceptRethrowsVMErrors() throws Throwable {
+    intercept(StackOverflowError.class, "",
+        () -> intercept(OutOfMemoryError.class, "",
+            () -> r(new StackOverflowError())));
+  }
+
+  @Test
+  public void testAwaitHandlesAssertions() throws Throwable {
+    // await a state which is never reached, expect a timeout exception
+    // with the text "failure" in it
+    TimeoutException ex = intercept(TimeoutException.class,
+        "failure",
+        () -> await(TIMEOUT,
+            () -> r(new AssertionError("failure")),
+            retry,
+            TIMEOUT_FAILURE_HANDLER));
+
+    // the retry handler must have been invoked
+    assertMinRetryCount(1);
+    // and the nested cause is tha raised assertion
+    if (!(ex.getCause() instanceof AssertionError)) {
+      throw ex;
+    }
+  }
+
+  @Test
+  public void testAwaitRethrowsVMErrors() throws Throwable {
+    // await a state which is never reached, expect a timeout exception
+    // with the text "failure" in it
+    intercept(StackOverflowError.class,
+        () -> await(TIMEOUT,
+            () -> r(new StackOverflowError()),
+            retry,
+            TIMEOUT_FAILURE_HANDLER));
+
+    // the retry handler must not have been invoked
+    assertMinRetryCount(0);
+  }
+
 }


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


[08/50] [abbrv] hadoop git commit: Revert "Plan/ResourceAllocation data structure enhancements required to support recurring reservations in ReservationSystem."

Posted by as...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.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/reservation/RLESparseResourceAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.java
index 100d38c..658387b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.java
@@ -18,7 +18,8 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
-import java.util.Collections;
+import java.io.IOException;
+import java.io.StringWriter;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -32,6 +33,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.Plan
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
+import com.google.gson.stream.JsonWriter;
+
 /**
  * This is a run length encoded sparse data structure that maintains resource
  * allocations over time.
@@ -41,14 +44,12 @@ public class RLESparseResourceAllocation {
   private static final int THRESHOLD = 100;
   private static final Resource ZERO_RESOURCE = Resources.none();
 
-  @SuppressWarnings("checkstyle:visibilitymodifier")
-  protected NavigableMap<Long, Resource> cumulativeCapacity =
+  private NavigableMap<Long, Resource> cumulativeCapacity =
       new TreeMap<Long, Resource>();
 
   private final ReentrantReadWriteLock readWriteLock =
       new ReentrantReadWriteLock();
-  @SuppressWarnings("checkstyle:visibilitymodifier")
-  protected final Lock readLock = readWriteLock.readLock();
+  private final Lock readLock = readWriteLock.readLock();
   private final Lock writeLock = readWriteLock.writeLock();
 
   private final ResourceCalculator resourceCalculator;
@@ -235,6 +236,34 @@ public class RLESparseResourceAllocation {
   }
 
   /**
+   * Returns the JSON string representation of the current resources allocated
+   * over time.
+   *
+   * @return the JSON string representation of the current resources allocated
+   *         over time
+   */
+  public String toMemJSONString() {
+    StringWriter json = new StringWriter();
+    JsonWriter jsonWriter = new JsonWriter(json);
+    readLock.lock();
+    try {
+      jsonWriter.beginObject();
+      // jsonWriter.name("timestamp").value("resource");
+      for (Map.Entry<Long, Resource> r : cumulativeCapacity.entrySet()) {
+        jsonWriter.name(r.getKey().toString()).value(r.getValue().toString());
+      }
+      jsonWriter.endObject();
+      jsonWriter.close();
+      return json.toString();
+    } catch (IOException e) {
+      // This should not happen
+      return "";
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  /**
    * Returns the representation of the current resources allocated over time as
    * an interval map (in the defined non-null range).
    *
@@ -275,7 +304,7 @@ public class RLESparseResourceAllocation {
   public NavigableMap<Long, Resource> getCumulative() {
     readLock.lock();
     try {
-      return Collections.unmodifiableNavigableMap(cumulativeCapacity);
+      return cumulativeCapacity;
     } finally {
       readLock.unlock();
     }
@@ -408,8 +437,8 @@ public class RLESparseResourceAllocation {
       Resource val = Resources.negate(e.getValue());
       // test for negative value and throws
       if (operator == RLEOperator.subtractTestNonNegative
-          && (Resources.fitsIn(val, ZERO_RESOURCE)
-              && !Resources.equals(val, ZERO_RESOURCE))) {
+          && (Resources.fitsIn(val, ZERO_RESOURCE) &&
+              !Resources.equals(val, ZERO_RESOURCE))) {
         throw new PlanningException(
             "RLESparseResourceAllocation: merge failed as the "
                 + "resulting RLESparseResourceAllocation would be negative");
@@ -475,29 +504,22 @@ public class RLESparseResourceAllocation {
 
   }
 
-  /**
-   * Get a {@link RLESparseResourceAllocation} view of the {@link Resource}
-   * allocations between the specified start and end times.
-   *
-   * @param start the time from which the {@link Resource} allocations are
-   *          required
-   * @param end the time upto which the {@link Resource} allocations are
-   *          required
-   * @return the overlapping allocations
-   */
   public RLESparseResourceAllocation getRangeOverlapping(long start, long end) {
     readLock.lock();
     try {
       NavigableMap<Long, Resource> a = this.getCumulative();
+
       if (a != null && !a.isEmpty()) {
         // include the portion of previous entry that overlaps start
         if (start > a.firstKey()) {
           long previous = a.floorKey(start);
           a = a.tailMap(previous, true);
         }
+
         if (end < a.lastKey()) {
           a = a.headMap(end, true);
         }
+
       }
       RLESparseResourceAllocation ret =
           new RLESparseResourceAllocation(a, resourceCalculator);
@@ -505,33 +527,7 @@ public class RLESparseResourceAllocation {
     } finally {
       readLock.unlock();
     }
-  }
 
-  /**
-   * This method shifts all the timestamp of the {@link Resource} entries by the
-   * specified "delta".
-   *
-   * @param delta the time by which to shift the {@link Resource} allocations
-   */
-  public void shift(long delta) {
-    writeLock.lock();
-    try {
-      TreeMap<Long, Resource> newCum = new TreeMap<>();
-      long start;
-      for (Map.Entry<Long, Resource> entry : cumulativeCapacity.entrySet()) {
-        if (delta > 0) {
-          start = (entry.getKey() == Long.MAX_VALUE) ? Long.MAX_VALUE
-              : entry.getKey() + delta;
-        } else {
-          start = (entry.getKey() == Long.MIN_VALUE) ? Long.MIN_VALUE
-              : entry.getKey() + delta;
-        }
-        newCum.put(start, entry.getValue());
-      }
-      cumulativeCapacity = newCum;
-    } finally {
-      writeLock.unlock();
-    }
   }
 
   /**
@@ -545,8 +541,8 @@ public class RLESparseResourceAllocation {
   /**
    * Get the maximum capacity across specified time instances. The search-space
    * is specified using the starting value, tick, and the periodic interval for
-   * search. Maximum resource allocation across tick, tick + period, tick + 2 *
-   * period,..., tick + n * period .. is returned.
+   * search. Maximum resource allocation across tick, tick + period,
+   * tick + 2 * period,..., tick + n * period .. is returned.
    *
    * @param tick the starting time instance
    * @param period interval at which capacity is evaluated
@@ -554,19 +550,14 @@ public class RLESparseResourceAllocation {
    */
   public Resource getMaximumPeriodicCapacity(long tick, long period) {
     Resource maxCapacity = ZERO_RESOURCE;
-    readLock.lock();
-    try {
-      if (!cumulativeCapacity.isEmpty()) {
-        Long lastKey = cumulativeCapacity.lastKey();
-        for (long t = tick; t <= lastKey; t = t + period) {
-          maxCapacity = Resources.componentwiseMax(maxCapacity,
-              cumulativeCapacity.floorEntry(t).getValue());
-        }
+    if (!cumulativeCapacity.isEmpty()) {
+      Long lastKey = cumulativeCapacity.lastKey();
+      for (long t = tick; t <= lastKey; t = t + period) {
+        maxCapacity = Resources.componentwiseMax(maxCapacity,
+            cumulativeCapacity.floorEntry(t).getValue());
       }
-      return maxCapacity;
-    } finally {
-      readLock.unlock();
     }
+    return maxCapacity;
   }
 
   /**
@@ -576,17 +567,17 @@ public class RLESparseResourceAllocation {
    * @return minimum resource allocation
    */
   public Resource getMinimumCapacityInInterval(ReservationInterval interval) {
-    Resource minCapacity =
-        Resource.newInstance(Integer.MAX_VALUE, Integer.MAX_VALUE);
+    Resource minCapacity = Resource.newInstance(
+        Integer.MAX_VALUE, Integer.MAX_VALUE);
     long start = interval.getStartTime();
     long end = interval.getEndTime();
     NavigableMap<Long, Resource> capacityRange =
-        getRangeOverlapping(start, end).getCumulative();
+        this.getRangeOverlapping(start, end).getCumulative();
     if (!capacityRange.isEmpty()) {
       for (Map.Entry<Long, Resource> entry : capacityRange.entrySet()) {
         if (entry.getValue() != null) {
-          minCapacity =
-              Resources.componentwiseMin(minCapacity, entry.getValue());
+          minCapacity = Resources.componentwiseMin(minCapacity,
+              entry.getValue());
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationAllocation.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/reservation/ReservationAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationAllocation.java
index bb4a7fb..0da95ac 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationAllocation.java
@@ -24,16 +24,14 @@ import org.apache.hadoop.yarn.api.records.ReservationDefinition;
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
 
-import com.google.common.annotations.VisibleForTesting;
-
 /**
  * A ReservationAllocation represents a concrete allocation of resources over
  * time that satisfy a certain {@link ReservationDefinition}. This is used
  * internally by a {@link Plan} to store information about how each of the
  * accepted {@link ReservationDefinition} have been allocated.
  */
-public interface ReservationAllocation
-    extends Comparable<ReservationAllocation> {
+public interface ReservationAllocation extends
+    Comparable<ReservationAllocation> {
 
   /**
    * Returns the unique identifier {@link ReservationId} that represents the
@@ -42,28 +40,28 @@ public interface ReservationAllocation
    * @return reservationId the unique identifier {@link ReservationId} that
    *         represents the reservation
    */
-  ReservationId getReservationId();
+  public ReservationId getReservationId();
 
   /**
    * Returns the original {@link ReservationDefinition} submitted by the client
    * 
    * @return the {@link ReservationDefinition} submitted by the client
    */
-  ReservationDefinition getReservationDefinition();
+  public ReservationDefinition getReservationDefinition();
 
   /**
    * Returns the time at which the reservation is activated.
    * 
    * @return the time at which the reservation is activated
    */
-  long getStartTime();
+  public long getStartTime();
 
   /**
    * Returns the time at which the reservation terminates.
    * 
    * @return the time at which the reservation terminates
    */
-  long getEndTime();
+  public long getEndTime();
 
   /**
    * Returns the map of resources requested against the time interval for which
@@ -72,28 +70,28 @@ public interface ReservationAllocation
    * @return the allocationRequests the map of resources requested against the
    *         time interval for which they were
    */
-  Map<ReservationInterval, Resource> getAllocationRequests();
+  public Map<ReservationInterval, Resource> getAllocationRequests();
 
   /**
    * Return a string identifying the plan to which the reservation belongs
    * 
    * @return the plan to which the reservation belongs
    */
-  String getPlanName();
+  public String getPlanName();
 
   /**
    * Returns the user who requested the reservation
    * 
    * @return the user who requested the reservation
    */
-  String getUser();
+  public String getUser();
 
   /**
    * Returns whether the reservation has gang semantics or not
    * 
    * @return true if there is a gang request, false otherwise
    */
-  boolean containsGangs();
+  public boolean containsGangs();
 
   /**
    * Sets the time at which the reservation was accepted by the system
@@ -101,14 +99,14 @@ public interface ReservationAllocation
    * @param acceptedAt the time at which the reservation was accepted by the
    *          system
    */
-  void setAcceptanceTimestamp(long acceptedAt);
+  public void setAcceptanceTimestamp(long acceptedAt);
 
   /**
    * Returns the time at which the reservation was accepted by the system
    * 
    * @return the time at which the reservation was accepted by the system
    */
-  long getAcceptanceTime();
+  public long getAcceptanceTime();
 
   /**
    * Returns the capacity represented by cumulative resources reserved by the
@@ -118,42 +116,12 @@ public interface ReservationAllocation
    *          requested
    * @return the resources reserved at the specified time
    */
-  Resource getResourcesAtTime(long tick);
-
-  /**
-   * Return a RLE representation of used resources.
-   *
-   * @return a RLE encoding of resources allocated over time.
-   */
-  RLESparseResourceAllocation getResourcesOverTime();
-
+  public Resource getResourcesAtTime(long tick);
 
   /**
    * Return a RLE representation of used resources.
-   *
-   * @param start start of the time interval.
-   * @param end end of the time interval.
    * @return a RLE encoding of resources allocated over time.
    */
-  RLESparseResourceAllocation getResourcesOverTime(long start, long end);
-
-  /**
-   * Get the periodicity of this reservation representing the time period of the
-   * periodic job. Period is represented in milliseconds for periodic jobs.
-   * Period is 0 for non-periodic jobs.
-   *
-   * @return periodicity of this reservation
-   */
-  long getPeriodicity();
-
-  /**
-   * Set the periodicity of this reservation representing the time period of the
-   * periodic job. Period is represented in milliseconds for periodic jobs.
-   * Period is 0 for non-periodic jobs.
-   *
-   * @param period periodicity of this reservation
-   */
-  @VisibleForTesting
-  void setPeriodicity(long period);
+  public RLESparseResourceAllocation getResourcesOverTime();
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.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/reservation/ReservationInputValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.java
index a66d222..027d066 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.java
@@ -44,8 +44,6 @@ public class ReservationInputValidator {
 
   /**
    * Utility class to validate reservation requests.
-   *
-   * @param clock the {@link Clock} to use
    */
   public ReservationInputValidator(Clock clock) {
     this.clock = clock;
@@ -55,21 +53,22 @@ public class ReservationInputValidator {
       ReservationId reservationId, String auditConstant) throws YarnException {
     // check if the reservation id is valid
     if (reservationId == null) {
-      String message = "Missing reservation id."
-          + " Please try again by specifying a reservation id.";
+      String message =
+          "Missing reservation id."
+              + " Please try again by specifying a reservation id.";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
     }
     String queue = reservationSystem.getQueueForReservation(reservationId);
     String nullQueueErrorMessage =
-        "The specified reservation with ID: " + reservationId
-            + " is unknown. Please try again with a valid reservation.";
+            "The specified reservation with ID: " + reservationId
+                    + " is unknown. Please try again with a valid reservation.";
     String nullPlanErrorMessage = "The specified reservation: " + reservationId
-        + " is not associated with any valid plan."
-        + " Please try again with a valid reservation.";
+                            + " is not associated with any valid plan."
+                            + " Please try again with a valid reservation.";
     return getPlanFromQueue(reservationSystem, queue, auditConstant,
-        nullQueueErrorMessage, nullPlanErrorMessage);
+            nullQueueErrorMessage, nullPlanErrorMessage);
   }
 
   private void validateReservationDefinition(ReservationId reservationId,
@@ -78,15 +77,17 @@ public class ReservationInputValidator {
     String message = "";
     // check if deadline is in the past
     if (contract == null) {
-      message = "Missing reservation definition."
-          + " Please try again by specifying a reservation definition.";
+      message =
+          "Missing reservation definition."
+              + " Please try again by specifying a reservation definition.";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input definition", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
     }
     if (contract.getDeadline() <= clock.getTime()) {
-      message = "The specified deadline: " + contract.getDeadline()
-          + " is the past. Please try again with deadline in the future.";
+      message =
+          "The specified deadline: " + contract.getDeadline()
+              + " is the past. Please try again with deadline in the future.";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input definition", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
@@ -94,16 +95,18 @@ public class ReservationInputValidator {
     // Check if at least one RR has been specified
     ReservationRequests resReqs = contract.getReservationRequests();
     if (resReqs == null) {
-      message = "No resources have been specified to reserve."
-          + "Please try again by specifying the resources to reserve.";
+      message =
+          "No resources have been specified to reserve."
+              + "Please try again by specifying the resources to reserve.";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input definition", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
     }
     List<ReservationRequest> resReq = resReqs.getReservationResources();
     if (resReq == null || resReq.isEmpty()) {
-      message = "No resources have been specified to reserve."
-          + " Please try again by specifying the resources to reserve.";
+      message =
+          "No resources have been specified to reserve."
+              + " Please try again by specifying the resources to reserve.";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input definition", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
@@ -120,18 +123,22 @@ public class ReservationInputValidator {
       } else {
         minDuration += rr.getDuration();
       }
-      maxGangSize = Resources.max(plan.getResourceCalculator(),
-          plan.getTotalCapacity(), maxGangSize,
-          Resources.multiply(rr.getCapability(), rr.getConcurrency()));
+      maxGangSize =
+          Resources.max(plan.getResourceCalculator(), plan.getTotalCapacity(),
+              maxGangSize,
+              Resources.multiply(rr.getCapability(), rr.getConcurrency()));
     }
     // verify the allocation is possible (skip for ANY)
     long duration = contract.getDeadline() - contract.getArrival();
-    if (duration < minDuration && type != ReservationRequestInterpreter.R_ANY) {
-      message = "The time difference (" + (duration) + ") between arrival ("
-          + contract.getArrival() + ") " + "and deadline ("
-          + contract.getDeadline() + ") must "
-          + " be greater or equal to the minimum resource duration ("
-          + minDuration + ")";
+    if (duration < minDuration
+        && type != ReservationRequestInterpreter.R_ANY) {
+      message =
+          "The time difference ("
+              + (duration)
+              + ") between arrival (" + contract.getArrival() + ") "
+              + "and deadline (" + contract.getDeadline() + ") must "
+              + " be greater or equal to the minimum resource duration ("
+              + minDuration + ")";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input definition", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
@@ -141,9 +148,10 @@ public class ReservationInputValidator {
     if (Resources.greaterThan(plan.getResourceCalculator(),
         plan.getTotalCapacity(), maxGangSize, plan.getTotalCapacity())
         && type != ReservationRequestInterpreter.R_ANY) {
-      message = "The size of the largest gang in the reservation definition ("
-          + maxGangSize + ") exceed the capacity available ("
-          + plan.getTotalCapacity() + " )";
+      message =
+          "The size of the largest gang in the reservation definition ("
+              + maxGangSize + ") exceed the capacity available ("
+              + plan.getTotalCapacity() + " )";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input definition", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
@@ -171,32 +179,32 @@ public class ReservationInputValidator {
     }
   }
 
-  private Plan getPlanFromQueue(ReservationSystem reservationSystem,
-      String queue, String auditConstant) throws YarnException {
+  private Plan getPlanFromQueue(ReservationSystem reservationSystem, String
+          queue, String auditConstant) throws YarnException {
     String nullQueueErrorMessage = "The queue is not specified."
-        + " Please try again with a valid reservable queue.";
+            + " Please try again with a valid reservable queue.";
     String nullPlanErrorMessage = "The specified queue: " + queue
-        + " is not managed by reservation system."
-        + " Please try again with a valid reservable queue.";
+            + " is not managed by reservation system."
+            + " Please try again with a valid reservable queue.";
     return getPlanFromQueue(reservationSystem, queue, auditConstant,
-        nullQueueErrorMessage, nullPlanErrorMessage);
+            nullQueueErrorMessage, nullPlanErrorMessage);
   }
 
-  private Plan getPlanFromQueue(ReservationSystem reservationSystem,
-      String queue, String auditConstant, String nullQueueErrorMessage,
-      String nullPlanErrorMessage) throws YarnException {
+  private Plan getPlanFromQueue(ReservationSystem reservationSystem, String
+          queue, String auditConstant, String nullQueueErrorMessage,
+          String nullPlanErrorMessage) throws YarnException {
     if (queue == null || queue.isEmpty()) {
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
-          "validate reservation input", "ClientRMService",
-          nullQueueErrorMessage);
+              "validate reservation input", "ClientRMService",
+              nullQueueErrorMessage);
       throw RPCUtil.getRemoteException(nullQueueErrorMessage);
     }
     // check if the associated plan is valid
     Plan plan = reservationSystem.getPlan(queue);
     if (plan == null) {
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
-          "validate reservation input", "ClientRMService",
-          nullPlanErrorMessage);
+              "validate reservation input", "ClientRMService",
+              nullPlanErrorMessage);
       throw RPCUtil.getRemoteException(nullPlanErrorMessage);
     }
     return plan;
@@ -214,21 +222,22 @@ public class ReservationInputValidator {
    * @param reservationId the {@link ReservationId} associated with the current
    *          request
    * @return the {@link Plan} to submit the request to
-   * @throws YarnException if validation fails
+   * @throws YarnException
    */
   public Plan validateReservationSubmissionRequest(
-      ReservationSystem reservationSystem, ReservationSubmissionRequest request,
-      ReservationId reservationId) throws YarnException {
+      ReservationSystem reservationSystem,
+      ReservationSubmissionRequest request, ReservationId reservationId)
+      throws YarnException {
     String message;
     if (reservationId == null) {
-      message = "Reservation id cannot be null. Please try again specifying "
-          + " a valid reservation id by creating a new reservation id.";
+      message = "Reservation id cannot be null. Please try again " +
+        "specifying a valid reservation id by creating a new reservation id.";
       throw RPCUtil.getRemoteException(message);
     }
     // Check if it is a managed queue
     String queue = request.getQueue();
     Plan plan = getPlanFromQueue(reservationSystem, queue,
-        AuditConstants.SUBMIT_RESERVATION_REQUEST);
+            AuditConstants.SUBMIT_RESERVATION_REQUEST);
 
     validateReservationDefinition(reservationId,
         request.getReservationDefinition(), plan,
@@ -246,14 +255,15 @@ public class ReservationInputValidator {
    * @param request the {@link ReservationUpdateRequest} defining the resources
    *          required over time for the request
    * @return the {@link Plan} to submit the request to
-   * @throws YarnException if validation fails
+   * @throws YarnException
    */
   public Plan validateReservationUpdateRequest(
       ReservationSystem reservationSystem, ReservationUpdateRequest request)
       throws YarnException {
     ReservationId reservationId = request.getReservationId();
-    Plan plan = validateReservation(reservationSystem, reservationId,
-        AuditConstants.UPDATE_RESERVATION_REQUEST);
+    Plan plan =
+        validateReservation(reservationSystem, reservationId,
+            AuditConstants.UPDATE_RESERVATION_REQUEST);
     validateReservationDefinition(reservationId,
         request.getReservationDefinition(), plan,
         AuditConstants.UPDATE_RESERVATION_REQUEST);
@@ -268,26 +278,28 @@ public class ReservationInputValidator {
    *
    * @param reservationSystem the {@link ReservationSystem} to validate against
    * @param request the {@link ReservationListRequest} defining search
-   *          parameters for reservations in the {@link ReservationSystem} that
-   *          is being validated against.
+   *                parameters for reservations in the {@link ReservationSystem}
+   *                that is being validated against.
    * @return the {@link Plan} to list reservations of.
-   * @throws YarnException if validation fails
+   * @throws YarnException
    */
   public Plan validateReservationListRequest(
-      ReservationSystem reservationSystem, ReservationListRequest request)
+      ReservationSystem reservationSystem,
+      ReservationListRequest request)
       throws YarnException {
     String queue = request.getQueue();
     if (request.getEndTime() < request.getStartTime()) {
-      String errorMessage = "The specified end time must be greater than "
-          + "the specified start time.";
+      String errorMessage = "The specified end time must be greater than " +
+              "the specified start time.";
       RMAuditLogger.logFailure("UNKNOWN",
-          AuditConstants.LIST_RESERVATION_REQUEST,
-          "validate list reservation input", "ClientRMService", errorMessage);
+              AuditConstants.LIST_RESERVATION_REQUEST,
+              "validate list reservation input", "ClientRMService",
+              errorMessage);
       throw RPCUtil.getRemoteException(errorMessage);
     }
     // Check if it is a managed queue
     return getPlanFromQueue(reservationSystem, queue,
-        AuditConstants.LIST_RESERVATION_REQUEST);
+            AuditConstants.LIST_RESERVATION_REQUEST);
   }
 
   /**
@@ -300,7 +312,7 @@ public class ReservationInputValidator {
    * @param request the {@link ReservationDeleteRequest} defining the resources
    *          required over time for the request
    * @return the {@link Plan} to submit the request to
-   * @throws YarnException if validation fails
+   * @throws YarnException
    */
   public Plan validateReservationDeleteRequest(
       ReservationSystem reservationSystem, ReservationDeleteRequest request)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystem.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/reservation/ReservationSystem.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystem.java
index a6c8fcf..8b62972 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystem.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystem.java
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
-import java.util.Map;
-
 import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
@@ -31,6 +29,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ReservationsACLsManager;
 
+import java.util.Map;
+
 /**
  * This interface is the one implemented by any system that wants to support
  * Reservations i.e. make {@code Resource} allocations in future. Implementors
@@ -57,7 +57,7 @@ public interface ReservationSystem extends Recoverable {
    * 
    * @param conf configuration
    * @param rmContext current context of the {@code ResourceManager}
-   * @throws YarnException if initialization of the configured plan fails
+   * @throws YarnException
    */
   void reinitialize(Configuration conf, RMContext rmContext)
       throws YarnException;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/SharingPolicy.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/reservation/SharingPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/SharingPolicy.java
index cbf0f38..e458055 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/SharingPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/SharingPolicy.java
@@ -38,7 +38,7 @@ public interface SharingPolicy {
    * @param planQueuePath the name of the queue for this plan
    * @param conf the system configuration
    */
-  void init(String planQueuePath, ReservationSchedulerConfiguration conf);
+  public void init(String planQueuePath, ReservationSchedulerConfiguration conf);
 
   /**
    * This method runs the policy validation logic, and return true/false on
@@ -51,7 +51,7 @@ public interface SharingPolicy {
    * @throws PlanningException if the policy is respected if we add this
    *           {@link ReservationAllocation} to the {@link Plan}
    */
-  void validate(Plan plan, ReservationAllocation newAllocation)
+  public void validate(Plan plan, ReservationAllocation newAllocation)
       throws PlanningException;
 
   /**
@@ -68,13 +68,9 @@ public interface SharingPolicy {
    * @param start the start time for the range we are querying
    * @param end the end time for the range we are querying
    * @param oldId (optional) the id of a reservation being updated
-   *
-   * @return the available resources expressed as a
-   *         {@link RLESparseResourceAllocation}
-   *
    * @throws PlanningException throws if the request is not valid
    */
-  RLESparseResourceAllocation availableResources(
+  public RLESparseResourceAllocation availableResources(
       RLESparseResourceAllocation available, Plan plan, String user,
       ReservationId oldId, long start, long end) throws PlanningException;
 
@@ -86,6 +82,7 @@ public interface SharingPolicy {
    * 
    * @return validWindow the window of validity considered by the policy.
    */
-  long getValidWindow();
+  public long getValidWindow();
+
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/Planner.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/reservation/planning/Planner.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/Planner.java
index af0e712..abac6ac 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/Planner.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/Planner.java
@@ -34,7 +34,7 @@ public interface Planner {
    *
    * @param plan the {@link Plan} to replan
    * @param contracts the list of reservation requests
-   * @throws PlanningException if operation is unsuccessful
+   * @throws PlanningException
    */
   public void plan(Plan plan, List<ReservationDefinition> contracts)
       throws PlanningException;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/PlanningAlgorithm.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/reservation/planning/PlanningAlgorithm.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/PlanningAlgorithm.java
index bbbf0d6..199bfa5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/PlanningAlgorithm.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/PlanningAlgorithm.java
@@ -50,7 +50,7 @@ public abstract class PlanningAlgorithm implements ReservationAgent {
    * @return whether the allocateUser function was successful or not
    *
    * @throws PlanningException if the session cannot be fitted into the plan
-   * @throws ContractValidationException if validation fails
+   * @throws ContractValidationException
    */
   protected boolean allocateUser(ReservationId reservationId, String user,
       Plan plan, ReservationDefinition contract,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocator.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/reservation/planning/StageAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocator.java
index 8934b0f..ec6d9c0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocator.java
@@ -50,7 +50,7 @@ public interface StageAllocator {
    *
    * @return The computed allocation (or null if the stage could not be
    *         allocated)
-   * @throws PlanningException if operation is unsuccessful
+   * @throws PlanningException
    */
   Map<ReservationInterval, Resource> computeStageAllocation(Plan plan,
       RLESparseResourceAllocation planLoads,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedy.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/reservation/planning/StageAllocatorGreedy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedy.java
index d107487..da04336 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedy.java
@@ -69,7 +69,7 @@ public class StageAllocatorGreedy implements StageAllocator {
 
     RLESparseResourceAllocation netAvailable =
         plan.getAvailableResourceOverTime(user, oldId, stageEarliestStart,
-            stageDeadline, 0);
+            stageDeadline);
 
     netAvailable =
         RLESparseResourceAllocation.merge(plan.getResourceCalculator(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedyRLE.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/reservation/planning/StageAllocatorGreedyRLE.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedyRLE.java
index ae7d91a..ec83e02 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedyRLE.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedyRLE.java
@@ -83,8 +83,9 @@ public class StageAllocatorGreedyRLE implements StageAllocator {
     int gangsToPlace = rr.getNumContainers() / rr.getConcurrency();
 
     // get available resources from plan
-    RLESparseResourceAllocation netRLERes = plan.getAvailableResourceOverTime(
-        user, oldId, stageEarliestStart, stageDeadline, 0);
+    RLESparseResourceAllocation netRLERes =
+        plan.getAvailableResourceOverTime(user, oldId, stageEarliestStart,
+            stageDeadline);
 
     // remove plan modifications
     netRLERes =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorLowCostAligned.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/reservation/planning/StageAllocatorLowCostAligned.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorLowCostAligned.java
index c014549..e45f58c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorLowCostAligned.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorLowCostAligned.java
@@ -77,8 +77,8 @@ public class StageAllocatorLowCostAligned implements StageAllocator {
     ResourceCalculator resCalc = plan.getResourceCalculator();
     Resource capacity = plan.getTotalCapacity();
 
-    RLESparseResourceAllocation netRLERes = plan.getAvailableResourceOverTime(
-        user, oldId, stageArrival, stageDeadline, 0);
+    RLESparseResourceAllocation netRLERes = plan
+        .getAvailableResourceOverTime(user, oldId, stageArrival, stageDeadline);
 
     long step = plan.getStep();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.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/reservation/ReservationSystemTestUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
index 5337e06..e99842e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
@@ -19,10 +19,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anySetOf;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.*;
 
 import java.io.FileWriter;
 import java.io.IOException;
@@ -79,8 +76,7 @@ public class ReservationSystemTestUtil {
       String reservationQ, long timeWindow, float instConstraint,
       float avgConstraint) {
 
-    ReservationSchedulerConfiguration realConf =
-        new CapacitySchedulerConfiguration();
+    ReservationSchedulerConfiguration realConf = new CapacitySchedulerConfiguration();
     ReservationSchedulerConfiguration conf = spy(realConf);
     when(conf.getReservationWindow(reservationQ)).thenReturn(timeWindow);
     when(conf.getInstantaneousMaxCapacity(reservationQ))
@@ -172,6 +168,7 @@ public class ReservationSystemTestUtil {
     scheduler.start();
     scheduler.reinitialize(conf, rmContext);
 
+
     Resource resource =
         ReservationSystemTestUtil.calculateClusterResource(numContainers);
     RMNode node1 = MockNodes.newNodeInfo(1, resource, 1, "127.0.0.1");
@@ -187,16 +184,10 @@ public class ReservationSystemTestUtil {
 
   public static ReservationDefinition createSimpleReservationDefinition(
       long arrival, long deadline, long duration, int parallelism) {
-    return createSimpleReservationDefinition(arrival, deadline, duration,
-        parallelism, null);
-  }
-
-  public static ReservationDefinition createSimpleReservationDefinition(
-      long arrival, long deadline, long duration, int parallelism,
-      String recurrenceExpression) {
     // create a request with a single atomic ask
-    ReservationRequest r = ReservationRequest.newInstance(
-        Resource.newInstance(1024, 1), parallelism, parallelism, duration);
+    ReservationRequest r =
+        ReservationRequest.newInstance(Resource.newInstance(1024, 1),
+            parallelism, parallelism, duration);
     ReservationDefinition rDef = new ReservationDefinitionPBImpl();
     ReservationRequests reqs = new ReservationRequestsPBImpl();
     reqs.setReservationResources(Collections.singletonList(r));
@@ -204,31 +195,32 @@ public class ReservationSystemTestUtil {
     rDef.setReservationRequests(reqs);
     rDef.setArrival(arrival);
     rDef.setDeadline(deadline);
-    if (recurrenceExpression != null) {
-      rDef.setRecurrenceExpression(recurrenceExpression);
-    }
     return rDef;
   }
 
   public static ReservationSubmissionRequest createSimpleReservationRequest(
       ReservationId reservationId, int numContainers, long arrival,
       long deadline, long duration) {
-    return createSimpleReservationRequest(reservationId, numContainers, arrival,
-        deadline, duration, Priority.UNDEFINED);
+    return createSimpleReservationRequest(reservationId, numContainers,
+        arrival, deadline, duration, Priority.UNDEFINED);
   }
 
   public static ReservationSubmissionRequest createSimpleReservationRequest(
       ReservationId reservationId, int numContainers, long arrival,
       long deadline, long duration, Priority priority) {
     // create a request with a single atomic ask
-    ReservationRequest r = ReservationRequest
-        .newInstance(Resource.newInstance(1024, 1), numContainers, 1, duration);
-    ReservationRequests reqs = ReservationRequests.newInstance(
-        Collections.singletonList(r), ReservationRequestInterpreter.R_ALL);
-    ReservationDefinition rDef = ReservationDefinition.newInstance(arrival,
-        deadline, reqs, "testClientRMService#reservation", "0", priority);
-    ReservationSubmissionRequest request = ReservationSubmissionRequest
-        .newInstance(rDef, reservationQ, reservationId);
+    ReservationRequest r =
+        ReservationRequest.newInstance(Resource.newInstance(1024, 1),
+            numContainers, 1, duration);
+    ReservationRequests reqs =
+        ReservationRequests.newInstance(Collections.singletonList(r),
+            ReservationRequestInterpreter.R_ALL);
+    ReservationDefinition rDef =
+        ReservationDefinition.newInstance(arrival, deadline, reqs,
+            "testClientRMService#reservation", "0", priority);
+    ReservationSubmissionRequest request =
+        ReservationSubmissionRequest.newInstance(rDef,
+            reservationQ, reservationId);
     return request;
   }
 
@@ -260,9 +252,9 @@ public class ReservationSystemTestUtil {
     return cs;
   }
 
-  @SuppressWarnings("rawtypes")
-  public static void initializeRMContext(int numContainers,
-      AbstractYarnScheduler scheduler, RMContext mockRMContext) {
+  @SuppressWarnings("rawtypes") public static void initializeRMContext(
+      int numContainers, AbstractYarnScheduler scheduler,
+      RMContext mockRMContext) {
 
     when(mockRMContext.getScheduler()).thenReturn(scheduler);
     Resource r = calculateClusterResource(numContainers);
@@ -270,25 +262,26 @@ public class ReservationSystemTestUtil {
   }
 
   public static RMContext createRMContext(Configuration conf) {
-    RMContext mockRmContext = Mockito.spy(new RMContextImpl(null, null, null,
-        null, null, null, new RMContainerTokenSecretManager(conf),
-        new NMTokenSecretManagerInRM(conf),
-        new ClientToAMTokenSecretManagerInRM(), null));
+    RMContext mockRmContext = Mockito.spy(
+        new RMContextImpl(null, null, null, null, null, null,
+            new RMContainerTokenSecretManager(conf),
+            new NMTokenSecretManagerInRM(conf),
+            new ClientToAMTokenSecretManagerInRM(), null));
 
     RMNodeLabelsManager nlm = mock(RMNodeLabelsManager.class);
     when(nlm.getQueueResource(any(String.class), anySetOf(String.class),
-        any(Resource.class))).thenAnswer(new Answer<Resource>() {
-          @Override
-          public Resource answer(InvocationOnMock invocation) throws Throwable {
-            Object[] args = invocation.getArguments();
-            return (Resource) args[2];
-          }
-        });
+            any(Resource.class))).thenAnswer(new Answer<Resource>() {
+      @Override public Resource answer(InvocationOnMock invocation)
+          throws Throwable {
+        Object[] args = invocation.getArguments();
+        return (Resource) args[2];
+      }
+    });
 
     when(nlm.getResourceByLabel(any(String.class), any(Resource.class)))
         .thenAnswer(new Answer<Resource>() {
-          @Override
-          public Resource answer(InvocationOnMock invocation) throws Throwable {
+          @Override public Resource answer(InvocationOnMock invocation)
+              throws Throwable {
             Object[] args = invocation.getArguments();
             return (Resource) args[1];
           }
@@ -311,8 +304,9 @@ public class ReservationSystemTestUtil {
     final String A = CapacitySchedulerConfiguration.ROOT + ".a";
     conf.setCapacity(A, 10);
 
-    final String dedicated = CapacitySchedulerConfiguration.ROOT
-        + CapacitySchedulerConfiguration.DOT + reservationQ;
+    final String dedicated =
+        CapacitySchedulerConfiguration.ROOT + CapacitySchedulerConfiguration.DOT
+            + reservationQ;
     conf.setCapacity(dedicated, 80);
     // Set as reservation queue
     conf.setReservable(dedicated, true);
@@ -411,55 +405,26 @@ public class ReservationSystemTestUtil {
 
   public static Map<ReservationInterval, Resource> generateAllocation(
       long startTime, long step, int[] alloc) {
-    return generateAllocation(startTime, step, alloc, null);
-  }
-
-  public static Map<ReservationInterval, Resource> generateAllocation(
-      long startTime, long step, int[] alloc, String recurrenceExpression) {
     Map<ReservationInterval, Resource> req = new TreeMap<>();
-
-    long period = 0;
-    if (recurrenceExpression != null) {
-      period = Long.parseLong(recurrenceExpression);
-    }
-
-    long rStart;
-    long rEnd;
-    for (int j = 0; j < 86400000; j += period) {
-      for (int i = 0; i < alloc.length; i++) {
-        rStart = (startTime + i * step) + j * period;
-        rEnd = (startTime + (i + 1) * step) + j * period;
-        if (period > 0) {
-          rStart = rStart % period + j * period;
-          rEnd = rEnd % period + j * period;
-          if (rStart > rEnd) {
-            // skip wrap-around entry
-            continue;
-          }
-        }
-
-        req.put(new ReservationInterval(rStart, rEnd),
-            ReservationSystemUtil.toResource(ReservationRequest
-                .newInstance(Resource.newInstance(1024, 1), alloc[i])));
-
-      }
-      // execute only once if non-periodic
-      if (period == 0) {
-        break;
-      }
+    for (int i = 0; i < alloc.length; i++) {
+      req.put(new ReservationInterval(startTime + i * step,
+          startTime + (i + 1) * step), ReservationSystemUtil.toResource(
+          ReservationRequest
+              .newInstance(Resource.newInstance(1024, 1), alloc[i])));
     }
     return req;
   }
 
-  public static RLESparseResourceAllocation generateRLESparseResourceAllocation(
-      int[] alloc, long[] timeSteps) {
+  public static RLESparseResourceAllocation
+      generateRLESparseResourceAllocation(int[] alloc, long[] timeSteps) {
     TreeMap<Long, Resource> allocationsMap = new TreeMap<>();
     for (int i = 0; i < alloc.length; i++) {
       allocationsMap.put(timeSteps[i],
           Resource.newInstance(alloc[i], alloc[i]));
     }
-    RLESparseResourceAllocation rleVector = new RLESparseResourceAllocation(
-        allocationsMap, new DefaultResourceCalculator());
+    RLESparseResourceAllocation rleVector =
+        new RLESparseResourceAllocation(allocationsMap,
+            new DefaultResourceCalculator());
     return rleVector;
   }
 


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


[33/50] [abbrv] hadoop git commit: HADOOP-13421. Switch to v2 of the S3 List Objects API in S3A. Contributed by Aaron Fabbri

Posted by as...@apache.org.
HADOOP-13421. Switch to v2 of the S3 List Objects API in S3A.
Contributed by Aaron Fabbri


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

Branch: refs/heads/YARN-5972
Commit: 5bbca80428ffbe776650652de86a3bba885edb31
Parents: ab8368d
Author: Steve Loughran <st...@apache.org>
Authored: Fri Sep 8 12:07:02 2017 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Sep 8 12:07:02 2017 +0100

----------------------------------------------------------------------
 .../src/main/resources/core-default.xml         |   9 ++
 .../org/apache/hadoop/fs/s3a/Constants.java     |   9 ++
 .../fs/s3a/InconsistentAmazonS3Client.java      | 143 ++++++++++++++++---
 .../java/org/apache/hadoop/fs/s3a/Listing.java  |  22 +--
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java |  91 ++++++++----
 .../org/apache/hadoop/fs/s3a/S3ListRequest.java |  69 +++++++++
 .../org/apache/hadoop/fs/s3a/S3ListResult.java  |  97 +++++++++++++
 .../src/site/markdown/tools/hadoop-aws/index.md |   9 ++
 .../ITestS3AContractGetFileStatusV1List.java    |  59 ++++++++
 .../fs/s3a/ITestS3GuardListConsistency.java     |  22 +--
 .../hadoop/fs/s3a/TestS3AGetFileStatus.java     |  41 +++---
 11 files changed, 492 insertions(+), 79 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5bbca804/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
index 9e2c553..23739b0 100644
--- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
+++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
@@ -1428,6 +1428,15 @@
   <description>The implementation class of the S3A AbstractFileSystem.</description>
 </property>
 
+<property>
+  <name>fs.s3a.list.version</name>
+  <value>2</value>
+  <description>
+    Select which version of the S3 SDK's List Objects API to use.  Currently
+    support 2 (default) and 1 (older API).
+  </description>
+</property>
+
 <!-- Azure file system properties -->
 <property>
   <name>fs.wasb.impl</name>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5bbca804/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
index 1a464d0..4e2af3a 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
@@ -451,4 +451,13 @@ public final class Constants {
   public static final String FAIL_INJECT_INCONSISTENCY_PROBABILITY =
       "fs.s3a.failinject.inconsistency.probability";
 
+  /**
+   * S3 API level parameters.
+   */
+  @InterfaceStability.Unstable
+  public static final String LIST_VERSION = "fs.s3a.list.version";
+
+  @InterfaceStability.Unstable
+  public static final int DEFAULT_LIST_VERSION = 2;
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5bbca804/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentAmazonS3Client.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentAmazonS3Client.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentAmazonS3Client.java
index 5e9cb3f..6476f5d 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentAmazonS3Client.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentAmazonS3Client.java
@@ -28,6 +28,8 @@ import com.amazonaws.services.s3.model.DeleteObjectRequest;
 import com.amazonaws.services.s3.model.DeleteObjectsRequest;
 import com.amazonaws.services.s3.model.DeleteObjectsResult;
 import com.amazonaws.services.s3.model.ListObjectsRequest;
+import com.amazonaws.services.s3.model.ListObjectsV2Request;
+import com.amazonaws.services.s3.model.ListObjectsV2Result;
 import com.amazonaws.services.s3.model.ObjectListing;
 import com.amazonaws.services.s3.model.PutObjectRequest;
 import com.amazonaws.services.s3.model.PutObjectResult;
@@ -109,8 +111,10 @@ public class InconsistentAmazonS3Client extends AmazonS3Client {
     }
   }
 
-  /** Map of key to delay -> time it was deleted + object summary (object
-   * summary is null for prefixes. */
+  /**
+   * Map of key to delay -> time it was deleted + object summary (object summary
+   * is null for prefixes.
+   */
   private Map<String, Delete> delayedDeletes = new HashMap<>();
 
   /** Map of key to delay -> time it was created. */
@@ -196,17 +200,29 @@ public class InconsistentAmazonS3Client extends AmazonS3Client {
     return super.putObject(putObjectRequest);
   }
 
-  /* We should only need to override this version of listObjects() */
+  /* We should only need to override these versions of listObjects() */
   @Override
   public ObjectListing listObjects(ListObjectsRequest listObjectsRequest)
       throws AmazonClientException, AmazonServiceException {
     LOG.debug("prefix {}", listObjectsRequest.getPrefix());
     ObjectListing listing = super.listObjects(listObjectsRequest);
-    listing = filterListObjects(listObjectsRequest, listing);
+    listing = filterListObjects(listing);
     listing = restoreListObjects(listObjectsRequest, listing);
     return listing;
   }
 
+  /* We should only need to override these versions of listObjects() */
+  @Override
+  public ListObjectsV2Result listObjectsV2(ListObjectsV2Request request)
+      throws AmazonClientException, AmazonServiceException {
+    LOG.debug("prefix {}", request.getPrefix());
+    ListObjectsV2Result listing = super.listObjectsV2(request);
+    listing = filterListObjectsV2(listing);
+    listing = restoreListObjectsV2(request, listing);
+    return listing;
+  }
+
+
   private void addSummaryIfNotPresent(List<S3ObjectSummary> list,
       S3ObjectSummary item) {
     // Behavior of S3ObjectSummary
@@ -282,21 +298,58 @@ public class InconsistentAmazonS3Client extends AmazonS3Client {
     // recursive list has no delimiter, returns everything that matches a
     // prefix.
     boolean recursiveObjectList = !("/".equals(request.getDelimiter()));
+    String prefix = request.getPrefix();
+
+    restoreDeleted(outputList, outputPrefixes, recursiveObjectList, prefix);
+    return new CustomObjectListing(rawListing, outputList, outputPrefixes);
+  }
+
+  /**
+   * V2 list API variant of
+   * {@link #restoreListObjects(ListObjectsRequest, ObjectListing)}.
+   * @param request original v2 list request
+   * @param result raw s3 result
+   */
+  private ListObjectsV2Result restoreListObjectsV2(ListObjectsV2Request request,
+      ListObjectsV2Result result) {
+    List<S3ObjectSummary> outputList = result.getObjectSummaries();
+    List<String> outputPrefixes = result.getCommonPrefixes();
+    // recursive list has no delimiter, returns everything that matches a
+    // prefix.
+    boolean recursiveObjectList = !("/".equals(request.getDelimiter()));
+    String prefix = request.getPrefix();
+
+    restoreDeleted(outputList, outputPrefixes, recursiveObjectList, prefix);
+    return new CustomListObjectsV2Result(result, outputList, outputPrefixes);
+  }
+
+
+  /**
+   * Main logic for
+   * {@link #restoreListObjects(ListObjectsRequest, ObjectListing)} and
+   * the v2 variant above.
+   * @param summaries object summary list to modify.
+   * @param prefixes prefix list to modify
+   * @param recursive true if recursive list request
+   * @param prefix prefix for original list request
+   */
+  private void restoreDeleted(List<S3ObjectSummary> summaries,
+      List<String> prefixes, boolean recursive, String prefix) {
 
     // Go through all deleted keys
     for (String key : new HashSet<>(delayedDeletes.keySet())) {
       Delete delete = delayedDeletes.get(key);
       if (isKeyDelayed(delete.time(), key)) {
-        if (isDescendant(request.getPrefix(), key, recursiveObjectList)) {
+        if (isDescendant(prefix, key, recursive)) {
           if (delete.summary() != null) {
-            addSummaryIfNotPresent(outputList, delete.summary());
+            addSummaryIfNotPresent(summaries, delete.summary());
           }
         }
         // Non-recursive list has delimiter: will return rolled-up prefixes for
         // all keys that are not direct children
-        if (!recursiveObjectList) {
-          if (isDescendant(request.getPrefix(), key, true)) {
-            addPrefixIfNotPresent(outputPrefixes, request.getPrefix(), key);
+        if (!recursive) {
+          if (isDescendant(prefix, key, true)) {
+            addPrefixIfNotPresent(prefixes, prefix, key);
           }
         }
       } else {
@@ -304,31 +357,52 @@ public class InconsistentAmazonS3Client extends AmazonS3Client {
         delayedDeletes.remove(key);
       }
     }
+  }
+
+  private ObjectListing filterListObjects(ObjectListing rawListing) {
+
+    // Filter object listing
+    List<S3ObjectSummary> outputList = filterSummaries(
+        rawListing.getObjectSummaries());
+
+    // Filter prefixes (directories)
+    List<String> outputPrefixes = filterPrefixes(
+        rawListing.getCommonPrefixes());
 
     return new CustomObjectListing(rawListing, outputList, outputPrefixes);
   }
 
-  private ObjectListing filterListObjects(ListObjectsRequest request,
-      ObjectListing rawListing) {
-
+  private ListObjectsV2Result filterListObjectsV2(ListObjectsV2Result raw) {
     // Filter object listing
+    List<S3ObjectSummary> outputList = filterSummaries(
+        raw.getObjectSummaries());
+
+    // Filter prefixes (directories)
+    List<String> outputPrefixes = filterPrefixes(raw.getCommonPrefixes());
+
+    return new CustomListObjectsV2Result(raw, outputList, outputPrefixes);
+  }
+
+  private List<S3ObjectSummary> filterSummaries(
+      List<S3ObjectSummary> summaries) {
     List<S3ObjectSummary> outputList = new ArrayList<>();
-    for (S3ObjectSummary s : rawListing.getObjectSummaries()) {
+    for (S3ObjectSummary s : summaries) {
       String key = s.getKey();
       if (!isKeyDelayed(delayedPutKeys.get(key), key)) {
         outputList.add(s);
       }
     }
+    return outputList;
+  }
 
-    // Filter prefixes (directories)
+  private List<String> filterPrefixes(List<String> prefixes) {
     List<String> outputPrefixes = new ArrayList<>();
-    for (String key : rawListing.getCommonPrefixes()) {
+    for (String key : prefixes) {
       if (!isKeyDelayed(delayedPutKeys.get(key), key)) {
         outputPrefixes.add(key);
       }
     }
-
-    return new CustomObjectListing(rawListing, outputList, outputPrefixes);
+    return outputPrefixes;
   }
 
   private boolean isKeyDelayed(Long enqueueTime, String key) {
@@ -342,7 +416,7 @@ public class InconsistentAmazonS3Client extends AmazonS3Client {
       delayedDeletes.remove(key);
       LOG.debug("no longer delaying {}", key);
       return false;
-    } else  {
+    } else {
       LOG.info("delaying {}", key);
       return true;
     }
@@ -431,4 +505,37 @@ public class InconsistentAmazonS3Client extends AmazonS3Client {
       return customPrefixes;
     }
   }
+
+  private static class CustomListObjectsV2Result extends ListObjectsV2Result {
+
+    private final List<S3ObjectSummary> customListing;
+    private final List<String> customPrefixes;
+
+    CustomListObjectsV2Result(ListObjectsV2Result raw,
+        List<S3ObjectSummary> customListing, List<String> customPrefixes) {
+      super();
+      this.customListing = customListing;
+      this.customPrefixes = customPrefixes;
+
+      this.setBucketName(raw.getBucketName());
+      this.setCommonPrefixes(raw.getCommonPrefixes());
+      this.setDelimiter(raw.getDelimiter());
+      this.setEncodingType(raw.getEncodingType());
+      this.setStartAfter(raw.getStartAfter());
+      this.setMaxKeys(raw.getMaxKeys());
+      this.setContinuationToken(raw.getContinuationToken());
+      this.setPrefix(raw.getPrefix());
+      this.setTruncated(raw.isTruncated());
+    }
+
+    @Override
+    public List<S3ObjectSummary> getObjectSummaries() {
+      return customListing;
+    }
+
+    @Override
+    public List<String> getCommonPrefixes() {
+      return customPrefixes;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5bbca804/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java
index 8efa218..d9f059b 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java
@@ -19,8 +19,6 @@
 package org.apache.hadoop.fs.s3a;
 
 import com.amazonaws.AmazonClientException;
-import com.amazonaws.services.s3.model.ListObjectsRequest;
-import com.amazonaws.services.s3.model.ObjectListing;
 import com.amazonaws.services.s3.model.S3ObjectSummary;
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.fs.FileStatus;
@@ -90,7 +88,7 @@ public class Listing {
    */
   FileStatusListingIterator createFileStatusListingIterator(
       Path listPath,
-      ListObjectsRequest request,
+      S3ListRequest request,
       PathFilter filter,
       Listing.FileStatusAcceptor acceptor) throws IOException {
     return createFileStatusListingIterator(listPath, request, filter, acceptor,
@@ -112,7 +110,7 @@ public class Listing {
    */
   FileStatusListingIterator createFileStatusListingIterator(
       Path listPath,
-      ListObjectsRequest request,
+      S3ListRequest request,
       PathFilter filter,
       Listing.FileStatusAcceptor acceptor,
       RemoteIterator<FileStatus> providedStatus) throws IOException {
@@ -432,7 +430,7 @@ public class Listing {
      * @param objects the next object listing
      * @return true if this added any entries after filtering
      */
-    private boolean buildNextStatusBatch(ObjectListing objects) {
+    private boolean buildNextStatusBatch(S3ListResult objects) {
       // counters for debug logs
       int added = 0, ignored = 0;
       // list to fill in with results. Initial size will be list maximum.
@@ -512,13 +510,16 @@ public class Listing {
    *
    * Thread safety: none.
    */
-  class ObjectListingIterator implements RemoteIterator<ObjectListing> {
+  class ObjectListingIterator implements RemoteIterator<S3ListResult> {
 
     /** The path listed. */
     private final Path listPath;
 
     /** The most recent listing results. */
-    private ObjectListing objects;
+    private S3ListResult objects;
+
+    /** The most recent listing request. */
+    private S3ListRequest request;
 
     /** Indicator that this is the first listing. */
     private boolean firstListing = true;
@@ -542,10 +543,11 @@ public class Listing {
      * */
     ObjectListingIterator(
         Path listPath,
-        ListObjectsRequest request) {
+        S3ListRequest request) {
       this.listPath = listPath;
       this.maxKeys = owner.getMaxKeys();
       this.objects = owner.listObjects(request);
+      this.request = request;
     }
 
     /**
@@ -569,7 +571,7 @@ public class Listing {
      * @throws NoSuchElementException if there is no more data to list.
      */
     @Override
-    public ObjectListing next() throws IOException {
+    public S3ListResult next() throws IOException {
       if (firstListing) {
         // on the first listing, don't request more data.
         // Instead just clear the firstListing flag so that it future calls
@@ -585,7 +587,7 @@ public class Listing {
           // need to request a new set of objects.
           LOG.debug("[{}], Requesting next {} objects under {}",
               listingCount, maxKeys, listPath);
-          objects = owner.continueListObjects(objects);
+          objects = owner.continueListObjects(request, objects);
           listingCount++;
           LOG.debug("New listing status: {}", this);
         } catch (AmazonClientException e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5bbca804/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
index c22383a..e76ef0b 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
@@ -53,8 +53,8 @@ import com.amazonaws.services.s3.model.DeleteObjectsRequest;
 import com.amazonaws.services.s3.model.GetObjectMetadataRequest;
 import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest;
 import com.amazonaws.services.s3.model.ListObjectsRequest;
+import com.amazonaws.services.s3.model.ListObjectsV2Request;
 import com.amazonaws.services.s3.model.MultiObjectDeleteException;
-import com.amazonaws.services.s3.model.ObjectListing;
 import com.amazonaws.services.s3.model.ObjectMetadata;
 import com.amazonaws.services.s3.model.PartETag;
 import com.amazonaws.services.s3.model.PutObjectRequest;
@@ -167,6 +167,7 @@ public class S3AFileSystem extends FileSystem {
   private String blockOutputBuffer;
   private S3ADataBlocks.BlockFactory blockFactory;
   private int blockOutputActiveBlocks;
+  private boolean useListV1;
 
   /** Add any deprecated keys. */
   @SuppressWarnings("deprecation")
@@ -261,6 +262,13 @@ public class S3AFileSystem extends FileSystem {
           BlockingThreadPoolExecutorService.newDaemonThreadFactory(
               "s3a-transfer-unbounded"));
 
+      int listVersion = conf.getInt(LIST_VERSION, DEFAULT_LIST_VERSION);
+      if (listVersion < 1 || listVersion > 2) {
+        LOG.warn("Configured fs.s3a.list.version {} is invalid, forcing " +
+            "version 2", listVersion);
+      }
+      useListV1 = (listVersion == 1);
+
       initTransferManager();
 
       initCannedAcls(conf);
@@ -1056,21 +1064,37 @@ public class S3AFileSystem extends FileSystem {
    * @param request request to initiate
    * @return the results
    */
-  protected ObjectListing listObjects(ListObjectsRequest request) {
+  protected S3ListResult listObjects(S3ListRequest request) {
     incrementStatistic(OBJECT_LIST_REQUESTS);
     incrementReadOperations();
-    return s3.listObjects(request);
+    if (useListV1) {
+      Preconditions.checkArgument(request.isV1());
+      return S3ListResult.v1(s3.listObjects(request.getV1()));
+    } else {
+      Preconditions.checkArgument(!request.isV1());
+      return S3ListResult.v2(s3.listObjectsV2(request.getV2()));
+    }
   }
 
   /**
    * List the next set of objects.
-   * @param objects paged result
+   * @param request last list objects request to continue
+   * @param prevResult last paged result to continue from
    * @return the next result object
    */
-  protected ObjectListing continueListObjects(ObjectListing objects) {
+  protected S3ListResult continueListObjects(S3ListRequest request,
+      S3ListResult prevResult) {
     incrementStatistic(OBJECT_CONTINUE_LIST_REQUESTS);
     incrementReadOperations();
-    return s3.listNextBatchOfObjects(objects);
+    if (useListV1) {
+      Preconditions.checkArgument(request.isV1());
+      return S3ListResult.v1(s3.listNextBatchOfObjects(prevResult.getV1()));
+    } else {
+      Preconditions.checkArgument(!request.isV1());
+      request.getV2().setContinuationToken(prevResult.getV2()
+          .getNextContinuationToken());
+      return S3ListResult.v2(s3.listObjectsV2(request.getV2()));
+    }
   }
 
   /**
@@ -1464,9 +1488,9 @@ public class S3AFileSystem extends FileSystem {
       } else {
         LOG.debug("Getting objects for directory prefix {} to delete", key);
 
-        ListObjectsRequest request = createListObjectsRequest(key, null);
+        S3ListRequest request = createListObjectsRequest(key, null);
 
-        ObjectListing objects = listObjects(request);
+        S3ListResult objects = listObjects(request);
         List<DeleteObjectsRequest.KeyVersion> keys =
             new ArrayList<>(objects.getObjectSummaries().size());
         while (true) {
@@ -1481,7 +1505,7 @@ public class S3AFileSystem extends FileSystem {
           }
 
           if (objects.isTruncated()) {
-            objects = continueListObjects(objects);
+            objects = continueListObjects(request, objects);
           } else {
             if (!keys.isEmpty()) {
               // TODO: HADOOP-13761 S3Guard: retries
@@ -1589,7 +1613,7 @@ public class S3AFileSystem extends FileSystem {
         return S3Guard.dirMetaToStatuses(dirMeta);
       }
 
-      ListObjectsRequest request = createListObjectsRequest(key, "/");
+      S3ListRequest request = createListObjectsRequest(key, "/");
       LOG.debug("listStatus: doing listObjects for directory {}", key);
 
       Listing.FileStatusListingIterator files =
@@ -1619,16 +1643,38 @@ public class S3AFileSystem extends FileSystem {
    * @return the request
    */
   @VisibleForTesting
-  ListObjectsRequest createListObjectsRequest(String key,
+  S3ListRequest createListObjectsRequest(String key,
       String delimiter) {
-    ListObjectsRequest request = new ListObjectsRequest();
-    request.setBucketName(bucket);
-    request.setMaxKeys(maxKeys);
-    request.setPrefix(key);
-    if (delimiter != null) {
-      request.setDelimiter(delimiter);
+    return createListObjectsRequest(key, delimiter, null);
+  }
+
+  private S3ListRequest createListObjectsRequest(String key,
+      String delimiter, Integer overrideMaxKeys) {
+    if (!useListV1) {
+      ListObjectsV2Request request =
+          new ListObjectsV2Request().withBucketName(bucket)
+              .withMaxKeys(maxKeys)
+              .withPrefix(key);
+      if (delimiter != null) {
+        request.setDelimiter(delimiter);
+      }
+      if (overrideMaxKeys != null) {
+        request.setMaxKeys(overrideMaxKeys);
+      }
+      return S3ListRequest.v2(request);
+    } else {
+      ListObjectsRequest request = new ListObjectsRequest();
+      request.setBucketName(bucket);
+      request.setMaxKeys(maxKeys);
+      request.setPrefix(key);
+      if (delimiter != null) {
+        request.setDelimiter(delimiter);
+      }
+      if (overrideMaxKeys != null) {
+        request.setMaxKeys(overrideMaxKeys);
+      }
+      return S3ListRequest.v1(request);
     }
-    return request;
   }
 
   /**
@@ -1885,13 +1931,9 @@ public class S3AFileSystem extends FileSystem {
 
     try {
       key = maybeAddTrailingSlash(key);
-      ListObjectsRequest request = new ListObjectsRequest();
-      request.setBucketName(bucket);
-      request.setPrefix(key);
-      request.setDelimiter("/");
-      request.setMaxKeys(1);
+      S3ListRequest request = createListObjectsRequest(key, "/", 1);
 
-      ObjectListing objects = listObjects(request);
+      S3ListResult objects = listObjects(request);
 
       Collection<String> prefixes = objects.getCommonPrefixes();
       Collection<S3ObjectSummary> summaries = objects.getObjectSummaries();
@@ -2441,6 +2483,7 @@ public class S3AFileSystem extends FileSystem {
     }
     sb.append(", metastore=").append(metadataStore);
     sb.append(", authoritative=").append(allowAuthoritative);
+    sb.append(", useListV1=").append(useListV1);
     sb.append(", boundedExecutor=").append(boundedThreadPool);
     sb.append(", unboundedExecutor=").append(unboundedThreadPool);
     sb.append(", statistics {")

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5bbca804/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListRequest.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListRequest.java
new file mode 100644
index 0000000..6b3bd46
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListRequest.java
@@ -0,0 +1,69 @@
+/*
+ * 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.fs.s3a;
+
+import com.amazonaws.services.s3.model.ListObjectsRequest;
+import com.amazonaws.services.s3.model.ListObjectsV2Request;
+
+/**
+ * API version-independent container for S3 List requests.
+ */
+public class S3ListRequest {
+  private ListObjectsRequest v1Request;
+  private ListObjectsV2Request v2Request;
+
+  protected S3ListRequest(ListObjectsRequest v1, ListObjectsV2Request v2) {
+    v1Request = v1;
+    v2Request = v2;
+  }
+
+  /**
+   * Restricted constructors to ensure v1 or v2, not both.
+   * @param request v1 request
+   * @return new list request container
+   */
+  public static S3ListRequest v1(ListObjectsRequest request) {
+    return new S3ListRequest(request, null);
+  }
+
+  /**
+   * Restricted constructors to ensure v1 or v2, not both.
+   * @param request v2 request
+   * @return new list request container
+   */
+  public static S3ListRequest v2(ListObjectsV2Request request) {
+    return new S3ListRequest(null, request);
+  }
+
+  /**
+   * Is this a v1 API request or v2?
+   * @return true if v1, false if v2
+   */
+  public boolean isV1() {
+    return v1Request != null;
+  }
+
+  public ListObjectsRequest getV1() {
+    return v1Request;
+  }
+
+  public ListObjectsV2Request getV2() {
+    return v2Request;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5bbca804/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListResult.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListResult.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListResult.java
new file mode 100644
index 0000000..e8aff32
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListResult.java
@@ -0,0 +1,97 @@
+/*
+ * 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.fs.s3a;
+
+import com.amazonaws.services.s3.model.ListObjectsV2Result;
+import com.amazonaws.services.s3.model.ObjectListing;
+import com.amazonaws.services.s3.model.S3ObjectSummary;
+
+import java.util.List;
+
+/**
+ * API version-independent container for S3 List responses.
+ */
+public class S3ListResult {
+  private ObjectListing v1Result;
+  private ListObjectsV2Result v2Result;
+
+  protected S3ListResult(ObjectListing v1, ListObjectsV2Result v2) {
+    v1Result = v1;
+    v2Result = v2;
+  }
+
+  /**
+   * Restricted constructors to ensure v1 or v2, not both.
+   * @param result v1 result
+   * @return new list result container
+   */
+  public static S3ListResult v1(ObjectListing result) {
+    return new S3ListResult(result, null);
+  }
+
+  /**
+   * Restricted constructors to ensure v1 or v2, not both.
+   * @param result v2 result
+   * @return new list result container
+   */
+  public static S3ListResult v2(ListObjectsV2Result result) {
+    return new S3ListResult(null, result);
+  }
+
+  /**
+   * Is this a v1 API result or v2?
+   * @return true if v1, false if v2
+   */
+  public boolean isV1() {
+    return v1Result != null;
+  }
+
+  public ObjectListing getV1() {
+    return v1Result;
+  }
+
+  public ListObjectsV2Result getV2() {
+    return v2Result;
+  }
+
+  public List<S3ObjectSummary> getObjectSummaries() {
+    if (isV1()) {
+      return v1Result.getObjectSummaries();
+    } else {
+      return v2Result.getObjectSummaries();
+    }
+  }
+
+  public boolean isTruncated() {
+    if (isV1()) {
+      return v1Result.isTruncated();
+    } else {
+      return v2Result.isTruncated();
+    }
+  }
+
+  public List<String> getCommonPrefixes() {
+    if (isV1()) {
+      return v1Result.getCommonPrefixes();
+    } else {
+      return v2Result.getCommonPrefixes();
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5bbca804/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
index b8d37c6..ffae1e9 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
@@ -895,6 +895,15 @@ from placing its declaration on the command line.
       any call to setReadahead() is made to an open stream.</description>
     </property>
 
+    <property>
+      <name>fs.s3a.list.version</name>
+      <value>2</value>
+      <description>
+        Select which version of the S3 SDK's List Objects API to use.  Currently
+        support 2 (default) and 1 (older API).
+      </description>
+    </property>
+
 ### Configuring different S3 buckets
 
 Different S3 buckets can be accessed with different S3A client configurations.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5bbca804/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AContractGetFileStatusV1List.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AContractGetFileStatusV1List.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AContractGetFileStatusV1List.java
new file mode 100644
index 0000000..5275336
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AContractGetFileStatusV1List.java
@@ -0,0 +1,59 @@
+/*
+ * 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.fs.s3a;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractGetFileStatusTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.contract.s3a.S3AContract;
+
+import static org.apache.hadoop.fs.s3a.Constants.LIST_VERSION;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.maybeEnableS3Guard;
+
+/**
+ * S3A contract tests for getFileStatus, using the v1 List Objects API.
+ */
+public class ITestS3AContractGetFileStatusV1List
+    extends AbstractContractGetFileStatusTest {
+
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new S3AContract(conf);
+  }
+
+  @Override
+  public void teardown() throws Exception {
+    getLog().info("FS details {}", getFileSystem());
+    super.teardown();
+  }
+
+  @Override
+  protected Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    disableFilesystemCaching(conf);
+    conf.setInt(Constants.MAX_PAGING_KEYS, 2);
+    maybeEnableS3Guard(conf);
+
+    // Use v1 List Objects API
+    conf.setInt(LIST_VERSION, 1);
+    return conf;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5bbca804/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java
index 6cff533..da7699e 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java
@@ -18,7 +18,7 @@
 
 package org.apache.hadoop.fs.s3a;
 
-import com.amazonaws.services.s3.model.ObjectListing;
+import com.amazonaws.services.s3.model.ListObjectsV2Result;
 import com.amazonaws.services.s3.AmazonS3;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -488,6 +488,10 @@ public class ITestS3GuardListConsistency extends AbstractS3ATestBase {
 
   @Test
   public void testInconsistentS3ClientDeletes() throws Throwable {
+    // Test only implemented for v2 S3 list API
+    Assume.assumeTrue(getConfiguration()
+        .getInt(LIST_VERSION, DEFAULT_LIST_VERSION) == 2);
+
     S3AFileSystem fs = getFileSystem();
     Path root = path("testInconsistentClient" + DEFAULT_DELAY_KEY_SUBSTRING);
     for (int i = 0; i < 3; i++) {
@@ -502,17 +506,17 @@ public class ITestS3GuardListConsistency extends AbstractS3ATestBase {
     AmazonS3 client = fs.getAmazonS3Client();
     String key = fs.pathToKey(root) + "/";
 
-    ObjectListing preDeleteDelimited = client.listObjects(
-        fs.createListObjectsRequest(key, "/"));
-    ObjectListing preDeleteUndelimited = client.listObjects(
-        fs.createListObjectsRequest(key, null));
+    ListObjectsV2Result preDeleteDelimited = client.listObjectsV2(
+        fs.createListObjectsRequest(key, "/").getV2());
+    ListObjectsV2Result preDeleteUndelimited = client.listObjectsV2(
+        fs.createListObjectsRequest(key, null).getV2());
 
     fs.delete(root, true);
 
-    ObjectListing postDeleteDelimited = client.listObjects(
-        fs.createListObjectsRequest(key, "/"));
-    ObjectListing postDeleteUndelimited = client.listObjects(
-        fs.createListObjectsRequest(key, null));
+    ListObjectsV2Result postDeleteDelimited = client.listObjectsV2(
+        fs.createListObjectsRequest(key, "/").getV2());
+    ListObjectsV2Result postDeleteUndelimited = client.listObjectsV2(
+        fs.createListObjectsRequest(key, null).getV2());
 
     assertEquals("InconsistentAmazonS3Client added back objects incorrectly " +
             "in a non-recursive listing",

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5bbca804/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AGetFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AGetFileStatus.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AGetFileStatus.java
index 58e4d30..586264d 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AGetFileStatus.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AGetFileStatus.java
@@ -25,9 +25,12 @@ import static org.mockito.Mockito.*;
 import java.io.FileNotFoundException;
 import java.util.Collections;
 import java.util.Date;
+import java.util.List;
 
 import com.amazonaws.services.s3.model.GetObjectMetadataRequest;
 import com.amazonaws.services.s3.model.ListObjectsRequest;
+import com.amazonaws.services.s3.model.ListObjectsV2Request;
+import com.amazonaws.services.s3.model.ListObjectsV2Result;
 import com.amazonaws.services.s3.model.ObjectListing;
 import com.amazonaws.services.s3.model.ObjectMetadata;
 import com.amazonaws.services.s3.model.S3ObjectSummary;
@@ -93,12 +96,7 @@ public class TestS3AGetFileStatus extends AbstractS3AMockTest {
     when(s3.getObjectMetadata(argThat(
       correctGetMetadataRequest(BUCKET, key + "/"))
     )).thenThrow(NOT_FOUND);
-    ObjectListing objects = mock(ObjectListing.class);
-    when(objects.getCommonPrefixes()).thenReturn(
-        Collections.singletonList("dir/"));
-    when(objects.getObjectSummaries()).thenReturn(
-        Collections.<S3ObjectSummary>emptyList());
-    when(s3.listObjects(any(ListObjectsRequest.class))).thenReturn(objects);
+    setupListMocks(Collections.singletonList("dir/"), Collections.emptyList());
     FileStatus stat = fs.getFileStatus(path);
     assertNotNull(stat);
     assertEquals(fs.makeQualified(path), stat.getPath());
@@ -118,12 +116,7 @@ public class TestS3AGetFileStatus extends AbstractS3AMockTest {
     when(s3.getObjectMetadata(argThat(
       correctGetMetadataRequest(BUCKET, key + "/")
     ))).thenThrow(NOT_FOUND);
-    ObjectListing objects = mock(ObjectListing.class);
-    when(objects.getCommonPrefixes()).thenReturn(
-        Collections.<String>emptyList());
-    when(objects.getObjectSummaries()).thenReturn(
-        Collections.<S3ObjectSummary>emptyList());
-    when(s3.listObjects(any(ListObjectsRequest.class))).thenReturn(objects);
+    setupListMocks(Collections.emptyList(), Collections.emptyList());
     FileStatus stat = fs.getFileStatus(path);
     assertNotNull(stat);
     assertEquals(fs.makeQualified(path), stat.getPath());
@@ -140,16 +133,28 @@ public class TestS3AGetFileStatus extends AbstractS3AMockTest {
     when(s3.getObjectMetadata(argThat(
       correctGetMetadataRequest(BUCKET, key + "/")
     ))).thenThrow(NOT_FOUND);
-    ObjectListing objects = mock(ObjectListing.class);
-    when(objects.getCommonPrefixes()).thenReturn(
-        Collections.<String>emptyList());
-    when(objects.getObjectSummaries()).thenReturn(
-        Collections.<S3ObjectSummary>emptyList());
-    when(s3.listObjects(any(ListObjectsRequest.class))).thenReturn(objects);
+    setupListMocks(Collections.emptyList(), Collections.emptyList());
     exception.expect(FileNotFoundException.class);
     fs.getFileStatus(path);
   }
 
+  private void setupListMocks(List<String> prefixes,
+      List<S3ObjectSummary> summaries) {
+
+    // V1 list API mock
+    ObjectListing objects = mock(ObjectListing.class);
+    when(objects.getCommonPrefixes()).thenReturn(prefixes);
+    when(objects.getObjectSummaries()).thenReturn(summaries);
+    when(s3.listObjects(any(ListObjectsRequest.class))).thenReturn(objects);
+
+    // V2 list API mock
+    ListObjectsV2Result v2Result = mock(ListObjectsV2Result.class);
+    when(v2Result.getCommonPrefixes()).thenReturn(prefixes);
+    when(v2Result.getObjectSummaries()).thenReturn(summaries);
+    when(s3.listObjectsV2(any(ListObjectsV2Request.class)))
+        .thenReturn(v2Result);
+  }
+
   private Matcher<GetObjectMetadataRequest> correctGetMetadataRequest(
       String bucket, String key) {
     return new BaseMatcher<GetObjectMetadataRequest>() {


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


[38/50] [abbrv] hadoop git commit: HADOOP-14849. some wrong spelling words update. Contributed by Chen Hongfei.

Posted by as...@apache.org.
HADOOP-14849. some wrong spelling words update. Contributed by Chen Hongfei.


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

Branch: refs/heads/YARN-5972
Commit: c35510a465cbda72c08239bcb5537375478bec3a
Parents: e8278b0
Author: Anu Engineer <ae...@apache.org>
Authored: Fri Sep 8 10:02:34 2017 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Fri Sep 8 10:02:34 2017 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/mapred/Task.java             | 2 +-
 .../main/java/org/apache/hadoop/mapred/lib/MultipleOutputs.java  | 4 ++--
 .../java/org/apache/hadoop/mapred/nativetask/NativeRuntime.java  | 2 +-
 .../main/java/org/apache/hadoop/examples/QuasiMonteCarlo.java    | 2 +-
 4 files changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c35510a4/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 b0347fd..542e956 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
@@ -887,7 +887,7 @@ abstract public class Task implements Writable, Configurable {
     }
     public void stopCommunicationThread() throws InterruptedException {
       if (pingThread != null) {
-        // Intent of the lock is to not send an interupt in the middle of an
+        // Intent of the lock is to not send an interrupt in the middle of an
         // umbilical.ping or umbilical.statusUpdate
         synchronized(lock) {
         //Interrupt if sleeping. Otherwise wait for the RPC call to return.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c35510a4/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/MultipleOutputs.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/MultipleOutputs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/MultipleOutputs.java
index f0f3652..3ef6601 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/MultipleOutputs.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/MultipleOutputs.java
@@ -27,7 +27,7 @@ import java.io.IOException;
 import java.util.*;
 
 /**
- * The MultipleOutputs class simplifies writting to additional outputs other
+ * The MultipleOutputs class simplifies writing to additional outputs other
  * than the job default output via the <code>OutputCollector</code> passed to
  * the <code>map()</code> and <code>reduce()</code> methods of the
  * <code>Mapper</code> and <code>Reducer</code> implementations.
@@ -36,7 +36,7 @@ import java.util.*;
  * <code>OutputFormat</code>, with its own key class and with its own value
  * class.
  * <p>
- * A named output can be a single file or a multi file. The later is refered as
+ * A named output can be a single file or a multi file. The later is referred as
  * a multi named output.
  * <p>
  * A multi named output is an unbound set of files all sharing the same

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c35510a4/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/NativeRuntime.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/NativeRuntime.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/NativeRuntime.java
index a0e88bd..cc9adba 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/NativeRuntime.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/NativeRuntime.java
@@ -103,7 +103,7 @@ public class NativeRuntime {
   }
 
   /**
-   * destroy native object We use to destory native handlers
+   * destroy native object We use to destroy native handlers
    */
   public synchronized static void releaseNativeObject(long addr) {
     assertNativeLibraryLoaded();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c35510a4/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/QuasiMonteCarlo.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/QuasiMonteCarlo.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/QuasiMonteCarlo.java
index 1a0c372..3048fd3 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/QuasiMonteCarlo.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/QuasiMonteCarlo.java
@@ -155,7 +155,7 @@ public class QuasiMonteCarlo extends Configured implements Tool {
     /** Map method.
      * @param offset samples starting from the (offset+1)th sample.
      * @param size the number of samples for this map
-     * @param context output {ture-&gt;numInside, false-&gt;numOutside}
+     * @param context output {true-&gt;numInside, false-&gt;numOutside}
      */
     public void map(LongWritable offset,
                     LongWritable size,


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


[24/50] [abbrv] hadoop git commit: HDFS-12369. Edit log corruption due to hard lease recovery of not-closed file which has snapshots.

Posted by as...@apache.org.
HDFS-12369. Edit log corruption due to hard lease recovery of not-closed file which has snapshots.


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

Branch: refs/heads/YARN-5972
Commit: 52b894db33bc68b46eec5cdf2735dfcf4030853a
Parents: b0b535d
Author: Xiao Chen <xi...@apache.org>
Authored: Thu Sep 7 15:54:52 2017 -0700
Committer: Xiao Chen <xi...@apache.org>
Committed: Thu Sep 7 16:30:12 2017 -0700

----------------------------------------------------------------------
 .../hdfs/server/namenode/FSNamesystem.java      |  1 +
 .../hdfs/server/namenode/LeaseManager.java      |  6 ++
 .../hdfs/server/namenode/TestDeleteRace.java    | 84 ++++++++++++++++++++
 3 files changed, 91 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/52b894db/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index c30999b..dedb11e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -4993,6 +4993,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   boolean isFileDeleted(INodeFile file) {
+    assert hasReadLock();
     // Not in the inodeMap or in the snapshot but marked deleted.
     if (dir.getInode(file.getId()) == null) {
       return true;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52b894db/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
index 35ec063..45699cb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
@@ -557,6 +557,12 @@ public class LeaseManager {
           if (!p.startsWith("/")) {
             throw new IOException("Invalid path in the lease " + p);
           }
+          final INodeFile lastINode = iip.getLastINode().asFile();
+          if (fsnamesystem.isFileDeleted(lastINode)) {
+            // INode referred by the lease could have been deleted.
+            removeLease(lastINode.getId());
+            continue;
+          }
           boolean completed = false;
           try {
             completed = fsnamesystem.internalReleaseLease(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52b894db/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeleteRace.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeleteRace.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeleteRace.java
index 133a18e..a13574f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeleteRace.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeleteRace.java
@@ -20,11 +20,13 @@ package org.apache.hadoop.hdfs.server.namenode;
 import java.io.FileNotFoundException;
 import java.util.AbstractMap;
 import java.util.ArrayList;
+import java.util.Comparator;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.TreeSet;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -49,16 +51,21 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.InternalDataNodeTestUtils;
+import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils.DelayAnswer;
 import org.junit.Assert;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.Timeout;
 import org.mockito.Mockito;
 import org.mockito.internal.util.reflection.Whitebox;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LEASE_RECHECK_INTERVAL_MS_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LEASE_RECHECK_INTERVAL_MS_KEY;
 
 /**
  * Test race between delete and other operations.  For now only addBlock()
@@ -71,6 +78,9 @@ public class TestDeleteRace {
   private static final Configuration conf = new HdfsConfiguration();
   private MiniDFSCluster cluster;
 
+  @Rule
+  public Timeout timeout = new Timeout(60000 * 3);
+
   @Test  
   public void testDeleteAddBlockRace() throws Exception {
     testDeleteAddBlockRace(false);
@@ -358,4 +368,78 @@ public class TestDeleteRace {
       throws Exception {
     testDeleteAndCommitBlockSynchronizationRace(true);
   }
+
+
+  /**
+   * Test the sequence of deleting a file that has snapshot,
+   * and lease manager's hard limit recovery.
+   */
+  @Test
+  public void testDeleteAndLeaseRecoveryHardLimitSnapshot() throws Exception {
+    final Path rootPath = new Path("/");
+    final Configuration config = new Configuration();
+    // Disable permissions so that another user can recover the lease.
+    config.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, false);
+    config.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
+    FSDataOutputStream stm = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(config).numDataNodes(3).build();
+      cluster.waitActive();
+
+      final DistributedFileSystem fs = cluster.getFileSystem();
+      final Path testPath = new Path("/testfile");
+      stm = fs.create(testPath);
+      LOG.info("test on " + testPath);
+
+      // write a half block
+      AppendTestUtil.write(stm, 0, BLOCK_SIZE / 2);
+      stm.hflush();
+
+      // create a snapshot, so delete does not release the file's inode.
+      SnapshotTestHelper.createSnapshot(fs, rootPath, "snap");
+
+      // delete the file without closing it.
+      fs.delete(testPath, false);
+
+      // write enough bytes to trigger an addBlock, which would fail in
+      // the streamer.
+      AppendTestUtil.write(stm, 0, BLOCK_SIZE);
+
+      // Mock a scenario that the lease reached hard limit.
+      final LeaseManager lm = (LeaseManager) Whitebox
+          .getInternalState(cluster.getNameNode().getNamesystem(),
+              "leaseManager");
+      final TreeSet<Lease> leases =
+          (TreeSet<Lease>) Whitebox.getInternalState(lm, "sortedLeases");
+      final TreeSet<Lease> spyLeases = new TreeSet<>(new Comparator<Lease>() {
+        @Override
+        public int compare(Lease o1, Lease o2) {
+          return Long.signum(o1.getLastUpdate() - o2.getLastUpdate());
+        }
+      });
+      while (!leases.isEmpty()) {
+        final Lease lease = leases.first();
+        final Lease spyLease = Mockito.spy(lease);
+        Mockito.doReturn(true).when(spyLease).expiredHardLimit();
+        spyLeases.add(spyLease);
+        leases.remove(lease);
+      }
+      Whitebox.setInternalState(lm, "sortedLeases", spyLeases);
+
+      // wait for lease manager's background 'Monitor' class to check leases.
+      Thread.sleep(2 * conf.getLong(DFS_NAMENODE_LEASE_RECHECK_INTERVAL_MS_KEY,
+          DFS_NAMENODE_LEASE_RECHECK_INTERVAL_MS_DEFAULT));
+
+      LOG.info("Now check we can restart");
+      cluster.restartNameNodes();
+      LOG.info("Restart finished");
+    } finally {
+      if (stm != null) {
+        IOUtils.closeStream(stm);
+      }
+      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


[02/50] [abbrv] hadoop git commit: HADOOP-12077. Provide a multi-URI replication Inode for ViewFs. Contributed by Gera Shegalov

Posted by as...@apache.org.
HADOOP-12077. Provide a multi-URI replication Inode for ViewFs. Contributed by Gera Shegalov


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

Branch: refs/heads/YARN-5972
Commit: 1f3bc63e6772be81bc9a6a7d93ed81d2a9e066c0
Parents: 63720ef
Author: Chris Douglas <cd...@apache.org>
Authored: Tue Sep 5 23:30:18 2017 -0700
Committer: Chris Douglas <cd...@apache.org>
Committed: Tue Sep 5 23:51:51 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/fs/viewfs/ConfigUtil.java |  27 +
 .../org/apache/hadoop/fs/viewfs/Constants.java  |   8 +-
 .../org/apache/hadoop/fs/viewfs/InodeTree.java  |  64 +-
 .../apache/hadoop/fs/viewfs/NflyFSystem.java    | 951 +++++++++++++++++++
 .../apache/hadoop/fs/viewfs/ViewFileSystem.java |  37 +-
 .../org/apache/hadoop/fs/viewfs/ViewFs.java     |  10 +-
 .../TestViewFileSystemLocalFileSystem.java      |  77 +-
 .../hadoop/fs/viewfs/TestViewFsConfig.java      |  13 +-
 .../fs/viewfs/TestViewFileSystemHdfs.java       | 151 ++-
 9 files changed, 1275 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f3bc63e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ConfigUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ConfigUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ConfigUtil.java
index 6900df2..a5fc62e 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ConfigUtil.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ConfigUtil.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.fs.viewfs;
 import java.net.URI;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.StringUtils;
 
 /**
  * Utilities for config variables of the viewFs See {@link ViewFs}
@@ -69,6 +70,32 @@ public class ConfigUtil {
   }
   
   /**
+   *
+   * @param conf
+   * @param mountTableName
+   * @param src
+   * @param settings
+   * @param targets
+   */
+  public static void addLinkNfly(Configuration conf, String mountTableName,
+      String src, String settings, final URI ... targets) {
+
+    settings = settings == null
+        ? "minReplication=2,repairOnRead=true"
+        : settings;
+
+    conf.set(getConfigViewFsPrefix(mountTableName) + "." +
+            Constants.CONFIG_VIEWFS_LINK_NFLY + "." + settings + "." + src,
+        StringUtils.uriToString(targets));
+  }
+
+  public static void addLinkNfly(final Configuration conf, final String src,
+      final URI ... targets) {
+    addLinkNfly(conf, Constants.CONFIG_VIEWFS_DEFAULT_MOUNT_TABLE, src, null,
+        targets);
+  }
+
+  /**
    * Add config variable for homedir for default mount table
    * @param conf - add to this conf
    * @param homedir - the home dir path starting with slash

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f3bc63e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java
index 9882a8e..1a07c10 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java
@@ -57,7 +57,13 @@ public interface Constants {
    * Config variable for specifying a merge link
    */
   public static final String CONFIG_VIEWFS_LINK_MERGE = "linkMerge";
-  
+
+  /**
+   * Config variable for specifying an nfly link. Nfly writes to multiple
+   * locations, and allows reads from the closest one.
+   */
+  String CONFIG_VIEWFS_LINK_NFLY = "linkNfly";
+
   /**
    * Config variable for specifying a merge of the root of the mount-table
    *  with the root of another file system. 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f3bc63e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java
index c62d5cc..665c9c9 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java
@@ -133,6 +133,12 @@ abstract class InodeTree<T> {
     }
   }
 
+  enum LinkType {
+    SINGLE,
+    MERGE,
+    NFLY
+  }
+
   /**
    * An internal class to represent a mount link.
    * A mount link can be single dir link or a merge dir link.
@@ -146,19 +152,17 @@ abstract class InodeTree<T> {
    * is changed later it is then ignored (a dir with null entries)
    */
   static class INodeLink<T> extends INode<T> {
-    final boolean isMergeLink; // true if MergeLink
     final URI[] targetDirLinkList;
     final T targetFileSystem;   // file system object created from the link.
 
     /**
-     * Construct a mergeLink.
+     * Construct a mergeLink or nfly.
      */
     INodeLink(final String pathToNode, final UserGroupInformation aUgi,
         final T targetMergeFs, final URI[] aTargetDirLinkList) {
       super(pathToNode, aUgi);
       targetFileSystem = targetMergeFs;
       targetDirLinkList = aTargetDirLinkList;
-      isMergeLink = true;
     }
 
     /**
@@ -170,7 +174,6 @@ abstract class InodeTree<T> {
       targetFileSystem = targetFs;
       targetDirLinkList = new URI[1];
       targetDirLinkList[0] = aTargetDirLink;
-      isMergeLink = false;
     }
 
     /**
@@ -188,7 +191,9 @@ abstract class InodeTree<T> {
   }
 
   private void createLink(final String src, final String target,
-      final boolean isLinkMerge, final UserGroupInformation aUgi)
+      final LinkType linkType, final String settings,
+      final UserGroupInformation aUgi,
+      final Configuration config)
       throws URISyntaxException, IOException,
       FileAlreadyExistsException, UnsupportedFileSystemException {
     // Validate that src is valid absolute path
@@ -235,18 +240,20 @@ abstract class InodeTree<T> {
     final INodeLink<T> newLink;
     final String fullPath = curInode.fullPath + (curInode == root ? "" : "/")
         + iPath;
-    if (isLinkMerge) { // Target is list of URIs
-      String[] targetsList = StringUtils.getStrings(target);
-      URI[] targetsListURI = new URI[targetsList.length];
-      int k = 0;
-      for (String itarget : targetsList) {
-        targetsListURI[k++] = new URI(itarget);
-      }
-      newLink = new INodeLink<T>(fullPath, aUgi,
-          getTargetFileSystem(targetsListURI), targetsListURI);
-    } else {
+    switch (linkType) {
+    case SINGLE:
       newLink = new INodeLink<T>(fullPath, aUgi,
           getTargetFileSystem(new URI(target)), new URI(target));
+      break;
+    case MERGE:
+    case NFLY:
+      final URI[] targetUris = StringUtils.stringToURI(
+          StringUtils.getStrings(target));
+      newLink = new INodeLink<T>(fullPath, aUgi,
+            getTargetFileSystem(settings, targetUris), targetUris);
+      break;
+    default:
+      throw new IllegalArgumentException(linkType + ": Infeasible linkType");
     }
     curInode.addLink(iPath, newLink);
     mountPoints.add(new MountPoint<T>(src, newLink));
@@ -257,14 +264,14 @@ abstract class InodeTree<T> {
    * 3 abstract methods.
    * @throws IOException
    */
-  protected abstract T getTargetFileSystem(final URI uri)
+  protected abstract T getTargetFileSystem(URI uri)
       throws UnsupportedFileSystemException, URISyntaxException, IOException;
 
-  protected abstract T getTargetFileSystem(final INodeDir<T> dir)
+  protected abstract T getTargetFileSystem(INodeDir<T> dir)
       throws URISyntaxException;
 
-  protected abstract T getTargetFileSystem(final URI[] mergeFsURIList)
-      throws UnsupportedFileSystemException, URISyntaxException;
+  protected abstract T getTargetFileSystem(String settings, URI[] mergeFsURIs)
+      throws UnsupportedFileSystemException, URISyntaxException, IOException;
 
   /**
    * Create Inode Tree from the specified mount-table specified in Config
@@ -298,8 +305,9 @@ abstract class InodeTree<T> {
       final String key = si.getKey();
       if (key.startsWith(mtPrefix)) {
         gotMountTableEntry = true;
-        boolean isMergeLink = false;
+        LinkType linkType = LinkType.SINGLE;
         String src = key.substring(mtPrefix.length());
+        String settings = null;
         if (src.startsWith(linkPrefix)) {
           src = src.substring(linkPrefix.length());
           if (src.equals(SlashPath.toString())) {
@@ -309,8 +317,20 @@ abstract class InodeTree<T> {
                 + "supported yet.");
           }
         } else if (src.startsWith(linkMergePrefix)) { // A merge link
-          isMergeLink = true;
+          linkType = LinkType.MERGE;
           src = src.substring(linkMergePrefix.length());
+        } else if (src.startsWith(Constants.CONFIG_VIEWFS_LINK_NFLY)) {
+          // prefix.settings.src
+          src = src.substring(Constants.CONFIG_VIEWFS_LINK_NFLY.length() + 1);
+          // settings.src
+          settings = src.substring(0, src.indexOf('.'));
+          // settings
+
+          // settings.src
+          src = src.substring(settings.length() + 1);
+          // src
+
+          linkType = LinkType.NFLY;
         } else if (src.startsWith(Constants.CONFIG_VIEWFS_HOMEDIR)) {
           // ignore - we set home dir from config
           continue;
@@ -319,7 +339,7 @@ abstract class InodeTree<T> {
               "Mount table in config: " + src);
         }
         final String target = si.getValue(); // link or merge link
-        createLink(src, target, isMergeLink, ugi);
+        createLink(src, target, linkType, settings, ugi, config);
       }
     }
     if (!gotMountTableEntry) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f3bc63e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/NflyFSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/NflyFSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/NflyFSystem.java
new file mode 100644
index 0000000..53966b8
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/NflyFSystem.java
@@ -0,0 +1,951 @@
+/**
+ * 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.fs.viewfs;
+
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.MultipleIOException;
+import org.apache.hadoop.net.DNSToSwitchMapping;
+import org.apache.hadoop.net.NetworkTopology;
+import org.apache.hadoop.net.Node;
+import org.apache.hadoop.net.NodeBase;
+import org.apache.hadoop.net.ScriptBasedMapping;
+import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.StringUtils;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.InetAddress;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.EnumSet;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * Nfly is a multi filesystem mount point.
+ */
+@Private
+final class NflyFSystem extends FileSystem {
+  private static final Log LOG = LogFactory.getLog(NflyFSystem.class);
+  private static final String NFLY_TMP_PREFIX = "_nfly_tmp_";
+
+  enum NflyKey {
+    // minimum replication, if local filesystem is included +1 is recommended
+    minReplication,
+
+    // forces to check all the replicas and fetch the one with the most recent
+    // time stamp
+    //
+    readMostRecent,
+
+    // create missing replica from far to near, including local?
+    repairOnRead
+  }
+
+  private static final int DEFAULT_MIN_REPLICATION = 2;
+  private static URI nflyURI = URI.create("nfly:///");
+
+  private final NflyNode[] nodes;
+  private final int minReplication;
+  private final EnumSet<NflyKey> nflyFlags;
+  private final Node myNode;
+  private final NetworkTopology topology;
+
+  /**
+   * URI's authority is used as an approximation of the distance from the
+   * client. It's sufficient for DC but not accurate because worker nodes can be
+   * closer.
+   */
+  private static class NflyNode extends NodeBase {
+    private final ChRootedFileSystem fs;
+    NflyNode(String hostName, String rackName, URI uri,
+        Configuration conf) throws IOException {
+      this(hostName, rackName, new ChRootedFileSystem(uri, conf));
+    }
+
+    NflyNode(String hostName, String rackName, ChRootedFileSystem fs) {
+      super(hostName, rackName);
+      this.fs = fs;
+    }
+
+    ChRootedFileSystem getFs() {
+      return fs;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      // satisfy findbugs
+      return super.equals(o);
+    }
+
+    @Override
+    public int hashCode() {
+      // satisfy findbugs
+      return super.hashCode();
+    }
+
+  }
+
+  private static final class MRNflyNode
+      extends NflyNode implements Comparable<MRNflyNode> {
+
+    private FileStatus status;
+
+    private MRNflyNode(NflyNode n) {
+      super(n.getName(), n.getNetworkLocation(), n.fs);
+    }
+
+    private void updateFileStatus(Path f) throws IOException {
+      final FileStatus tmpStatus = getFs().getFileStatus(f);
+      status = tmpStatus == null
+          ? notFoundStatus(f)
+          : tmpStatus;
+    }
+
+    // TODO allow configurable error margin for FileSystems with different
+    // timestamp precisions
+    @Override
+    public int compareTo(MRNflyNode other) {
+      if (status == null) {
+        return other.status == null ? 0 : 1; // move non-null towards head
+      } else if (other.status == null) {
+        return -1; // move this towards head
+      } else {
+        final long mtime = status.getModificationTime();
+        final long their = other.status.getModificationTime();
+        return Long.compare(their, mtime); // move more recent towards head
+      }
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (!(o instanceof MRNflyNode)) {
+        return false;
+      }
+      MRNflyNode other = (MRNflyNode) o;
+      return 0 == compareTo(other);
+    }
+
+    @Override
+    public int hashCode() {
+      // satisfy findbugs
+      return super.hashCode();
+    }
+
+    private FileStatus nflyStatus() throws IOException {
+      return new NflyStatus(getFs(), status);
+    }
+
+    private FileStatus cloneStatus() throws IOException {
+      return new FileStatus(status.getLen(),
+          status.isDirectory(),
+          status.getReplication(),
+          status.getBlockSize(),
+          status.getModificationTime(),
+          status.getAccessTime(),
+          null, null, null,
+          status.isSymlink() ? status.getSymlink() : null,
+          status.getPath());
+    }
+  }
+
+  private MRNflyNode[] workSet() {
+    final MRNflyNode[] res = new MRNflyNode[nodes.length];
+    for (int i = 0; i < res.length; i++) {
+      res[i] = new MRNflyNode(nodes[i]);
+    }
+    return res;
+  }
+
+
+  /**
+   * Utility to replace null with DEFAULT_RACK.
+   *
+   * @param rackString rack value, can be null
+   * @return non-null rack string
+   */
+  private static String getRack(String rackString) {
+    return rackString == null ? NetworkTopology.DEFAULT_RACK : rackString;
+  }
+
+  /**
+   * Creates a new Nfly instance.
+   *
+   * @param uris the list of uris in the mount point
+   * @param conf configuration object
+   * @param minReplication minimum copies to commit a write op
+   * @param nflyFlags modes such readMostRecent
+   * @throws IOException
+   */
+  private NflyFSystem(URI[] uris, Configuration conf, int minReplication,
+      EnumSet<NflyKey> nflyFlags) throws IOException {
+    if (uris.length < minReplication) {
+      throw new IOException(minReplication + " < " + uris.length
+          + ": Minimum replication < #destinations");
+    }
+    setConf(conf);
+    final String localHostName = InetAddress.getLocalHost().getHostName();
+
+    // build a list for topology resolution
+    final List<String> hostStrings = new ArrayList<String>(uris.length + 1);
+    for (URI uri : uris) {
+      final String uriHost = uri.getHost();
+      // assume local file system or another closest filesystem if no authority
+      hostStrings.add(uriHost == null ? localHostName : uriHost);
+    }
+    // resolve the client node
+    hostStrings.add(localHostName);
+
+    final DNSToSwitchMapping tmpDns = ReflectionUtils.newInstance(conf.getClass(
+        CommonConfigurationKeys.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
+        ScriptBasedMapping.class, DNSToSwitchMapping.class), conf);
+
+    // this is an ArrayList
+    final List<String> rackStrings = tmpDns.resolve(hostStrings);
+    nodes = new NflyNode[uris.length];
+    final Iterator<String> rackIter = rackStrings.iterator();
+    for (int i = 0; i < nodes.length; i++) {
+      nodes[i] = new NflyNode(hostStrings.get(i), rackIter.next(), uris[i],
+          conf);
+    }
+    // sort all the uri's by distance from myNode, the local file system will
+    // automatically be the the first one.
+    //
+    myNode = new NodeBase(localHostName, getRack(rackIter.next()));
+    topology = NetworkTopology.getInstance(conf);
+    topology.sortByDistance(myNode, nodes, nodes.length);
+
+    this.minReplication = minReplication;
+    this.nflyFlags = nflyFlags;
+    statistics = getStatistics(nflyURI.getScheme(), getClass());
+  }
+
+  /**
+   * Transactional output stream. When creating path /dir/file
+   * 1) create invisible /real/dir_i/_nfly_tmp_file
+   * 2) when more than min replication was written, write is committed by
+   *   renaming all successfully written files to /real/dir_i/file
+   */
+  private final class NflyOutputStream extends OutputStream {
+    // actual path
+    private final Path nflyPath;
+    // tmp path before commit
+    private final Path tmpPath;
+    // broadcast set
+    private final FSDataOutputStream[] outputStreams;
+    // status set: 1 working, 0 problem
+    private final BitSet opSet;
+    private final boolean useOverwrite;
+
+    private NflyOutputStream(Path f, FsPermission permission, boolean overwrite,
+        int bufferSize, short replication, long blockSize,
+        Progressable progress) throws IOException {
+      nflyPath = f;
+      tmpPath = getNflyTmpPath(f);
+      outputStreams = new FSDataOutputStream[nodes.length];
+      for (int i = 0; i < outputStreams.length; i++) {
+        outputStreams[i] = nodes[i].fs.create(tmpPath, permission, true,
+            bufferSize, replication, blockSize, progress);
+      }
+      opSet = new BitSet(outputStreams.length);
+      opSet.set(0, outputStreams.length);
+      useOverwrite = false;
+    }
+
+    //
+    // TODO consider how to clean up and throw an exception early when the clear
+    // bits under min replication
+    //
+
+    private void mayThrow(List<IOException> ioExceptions) throws IOException {
+      final IOException ioe = MultipleIOException
+          .createIOException(ioExceptions);
+      if (opSet.cardinality() < minReplication) {
+        throw ioe;
+      } else {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Exceptions occurred: " + ioe);
+        }
+      }
+    }
+
+
+    @Override
+    public void write(int d) throws IOException {
+      final List<IOException> ioExceptions = new ArrayList<IOException>();
+      for (int i = opSet.nextSetBit(0);
+           i >=0;
+           i = opSet.nextSetBit(i + 1)) {
+        try {
+          outputStreams[i].write(d);
+        } catch (Throwable t) {
+          osException(i, "write", t, ioExceptions);
+        }
+      }
+      mayThrow(ioExceptions);
+    }
+
+    private void osException(int i, String op, Throwable t,
+        List<IOException> ioExceptions) {
+      opSet.clear(i);
+      processThrowable(nodes[i], op, t, ioExceptions, tmpPath, nflyPath);
+    }
+
+    @Override
+    public void write(byte[] bytes, int offset, int len) throws IOException {
+      final List<IOException> ioExceptions = new ArrayList<IOException>();
+      for (int i = opSet.nextSetBit(0);
+           i >= 0;
+           i = opSet.nextSetBit(i + 1)) {
+        try {
+          outputStreams[i].write(bytes, offset, len);
+        } catch (Throwable t) {
+          osException(i, "write", t, ioExceptions);
+        }
+      }
+      mayThrow(ioExceptions);
+    }
+
+    @Override
+    public void flush() throws IOException {
+      final List<IOException> ioExceptions = new ArrayList<IOException>();
+      for (int i = opSet.nextSetBit(0);
+           i >= 0;
+           i = opSet.nextSetBit(i + 1)) {
+        try {
+          outputStreams[i].flush();
+        } catch (Throwable t) {
+          osException(i, "flush", t, ioExceptions);
+        }
+      }
+      mayThrow(ioExceptions);
+    }
+
+    @Override
+    public void close() throws IOException {
+      final List<IOException> ioExceptions = new ArrayList<IOException>();
+      for (int i = opSet.nextSetBit(0);
+           i >= 0;
+           i = opSet.nextSetBit(i + 1)) {
+        try {
+          outputStreams[i].close();
+        } catch (Throwable t) {
+          osException(i, "close", t, ioExceptions);
+        }
+      }
+      if (opSet.cardinality() < minReplication) {
+        cleanupAllTmpFiles();
+        throw new IOException("Failed to sufficiently replicate: min="
+            + minReplication + " actual=" + opSet.cardinality());
+      } else {
+        commit();
+      }
+    }
+
+    private void cleanupAllTmpFiles() throws IOException {
+      for (int i = 0; i < outputStreams.length; i++) {
+        try {
+          nodes[i].fs.delete(tmpPath);
+        } catch (Throwable t) {
+          processThrowable(nodes[i], "delete", t, null, tmpPath);
+        }
+      }
+    }
+
+    private void commit() throws IOException {
+      final List<IOException> ioExceptions = new ArrayList<IOException>();
+      for (int i = opSet.nextSetBit(0);
+           i >= 0;
+           i = opSet.nextSetBit(i + 1)) {
+        final NflyNode nflyNode = nodes[i];
+        try {
+          if (useOverwrite) {
+            nflyNode.fs.delete(nflyPath);
+          }
+          nflyNode.fs.rename(tmpPath, nflyPath);
+
+        } catch (Throwable t) {
+          osException(i, "commit", t, ioExceptions);
+        }
+      }
+
+      if (opSet.cardinality() < minReplication) {
+        // cleanup should be done outside. If rename failed, it's unlikely that
+        // delete will work either. It's the same kind of metadata-only op
+        //
+        throw MultipleIOException.createIOException(ioExceptions);
+      }
+
+      // best effort to have a consistent timestamp
+      final long commitTime = System.currentTimeMillis();
+      for (int i = opSet.nextSetBit(0);
+          i >= 0;
+          i = opSet.nextSetBit(i + 1)) {
+        try {
+          nodes[i].fs.setTimes(nflyPath, commitTime, commitTime);
+        } catch (Throwable t) {
+          LOG.info("Failed to set timestamp: " + nodes[i] + " " + nflyPath);
+        }
+      }
+    }
+  }
+
+  private Path getNflyTmpPath(Path f) {
+    return new Path(f.getParent(), NFLY_TMP_PREFIX + f.getName());
+  }
+
+  /**
+   * // TODO
+   * Some file status implementations have expensive deserialization or metadata
+   * retrieval. This probably does not go beyond RawLocalFileSystem. Wrapping
+   * the the real file status to preserve this behavior. Otherwise, calling
+   * realStatus getters in constructor defeats this design.
+   */
+  static final class NflyStatus extends FileStatus {
+    private static final long serialVersionUID = 0x21f276d8;
+
+    private final FileStatus realStatus;
+    private final String strippedRoot;
+
+    private NflyStatus(ChRootedFileSystem realFs, FileStatus realStatus)
+        throws IOException {
+      this.realStatus = realStatus;
+      this.strippedRoot = realFs.stripOutRoot(realStatus.getPath());
+    }
+
+    String stripRoot() throws IOException {
+      return strippedRoot;
+    }
+
+    @Override
+    public long getLen() {
+      return realStatus.getLen();
+    }
+
+    @Override
+    public boolean isFile() {
+      return realStatus.isFile();
+    }
+
+    @Override
+    public boolean isDirectory() {
+      return realStatus.isDirectory();
+    }
+
+    @Override
+    public boolean isSymlink() {
+      return realStatus.isSymlink();
+    }
+
+    @Override
+    public long getBlockSize() {
+      return realStatus.getBlockSize();
+    }
+
+    @Override
+    public short getReplication() {
+      return realStatus.getReplication();
+    }
+
+    @Override
+    public long getModificationTime() {
+      return realStatus.getModificationTime();
+    }
+
+    @Override
+    public long getAccessTime() {
+      return realStatus.getAccessTime();
+    }
+
+    @Override
+    public FsPermission getPermission() {
+      return realStatus.getPermission();
+    }
+
+    @Override
+    public String getOwner() {
+      return realStatus.getOwner();
+    }
+
+    @Override
+    public String getGroup() {
+      return realStatus.getGroup();
+    }
+
+    @Override
+    public Path getPath() {
+      return realStatus.getPath();
+    }
+
+    @Override
+    public void setPath(Path p) {
+      realStatus.setPath(p);
+    }
+
+    @Override
+    public Path getSymlink() throws IOException {
+      return realStatus.getSymlink();
+    }
+
+    @Override
+    public void setSymlink(Path p) {
+      realStatus.setSymlink(p);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      return realStatus.equals(o);
+    }
+
+    @Override
+    public int hashCode() {
+      return realStatus.hashCode();
+    }
+
+    @Override
+    public String toString() {
+      return realStatus.toString();
+    }
+  }
+
+  @Override
+  public URI getUri() {
+    return nflyURI;
+  }
+
+  /**
+   * Category: READ.
+   *
+   * @param f the file name to open
+   * @param bufferSize the size of the buffer to be used.
+   * @return input stream according to nfly flags (closest, most recent)
+   * @throws IOException
+   * @throws FileNotFoundException iff all destinations generate this exception
+   */
+  @Override
+  public FSDataInputStream open(Path f, int bufferSize) throws IOException {
+    // TODO proxy stream for reads
+    final List<IOException> ioExceptions =
+        new ArrayList<IOException>(nodes.length);
+    int numNotFounds = 0;
+    final MRNflyNode[] mrNodes = workSet();
+
+    // naively iterate until one can be opened
+    //
+    for (final MRNflyNode nflyNode : mrNodes) {
+      try {
+        if (nflyFlags.contains(NflyKey.repairOnRead)
+            || nflyFlags.contains(NflyKey.readMostRecent)) {
+          // calling file status to avoid pulling bytes prematurely
+          nflyNode.updateFileStatus(f);
+        } else {
+          return nflyNode.getFs().open(f, bufferSize);
+        }
+      } catch (FileNotFoundException fnfe) {
+        nflyNode.status = notFoundStatus(f);
+        numNotFounds++;
+        processThrowable(nflyNode, "open", fnfe, ioExceptions, f);
+      } catch (Throwable t) {
+        processThrowable(nflyNode, "open", t, ioExceptions, f);
+      }
+    }
+
+    if (nflyFlags.contains(NflyKey.readMostRecent)) {
+      // sort from most recent to least recent
+      Arrays.sort(mrNodes);
+    }
+
+    final FSDataInputStream fsdisAfterRepair = repairAndOpen(mrNodes, f,
+        bufferSize);
+
+    if (fsdisAfterRepair != null) {
+      return fsdisAfterRepair;
+    }
+
+    mayThrowFileNotFound(ioExceptions, numNotFounds);
+    throw MultipleIOException.createIOException(ioExceptions);
+  }
+
+  private static FileStatus notFoundStatus(Path f) {
+    return new FileStatus(-1, false, 0, 0, 0, f);
+  }
+
+  /**
+   * Iterate all available nodes in the proximity order to attempt repair of all
+   * FileNotFound nodes.
+   *
+   * @param mrNodes work set copy of nodes
+   * @param f path to repair and open
+   * @param bufferSize buffer size for read RPC
+   * @return the closest/most recent replica stream AFTER repair
+   */
+  private FSDataInputStream repairAndOpen(MRNflyNode[] mrNodes, Path f,
+      int bufferSize) {
+    long maxMtime = 0L;
+    for (final MRNflyNode srcNode : mrNodes) {
+      if (srcNode.status == null  // not available
+          || srcNode.status.getLen() < 0L) { // not found
+        continue; // not available
+      }
+      if (srcNode.status.getModificationTime() > maxMtime) {
+        maxMtime = srcNode.status.getModificationTime();
+      }
+
+      // attempt to repair all notFound nodes with srcNode
+      //
+      for (final MRNflyNode dstNode : mrNodes) {
+        if (dstNode.status == null // not available
+            || srcNode.compareTo(dstNode) == 0) { // same mtime
+          continue;
+        }
+
+        try {
+          // status is absolute from the underlying mount, making it chrooted
+          //
+          final FileStatus srcStatus = srcNode.cloneStatus();
+          srcStatus.setPath(f);
+          final Path tmpPath = getNflyTmpPath(f);
+          FileUtil.copy(srcNode.getFs(), srcStatus, dstNode.getFs(), tmpPath,
+              false,                // don't delete
+              true,                 // overwrite
+              getConf());
+          dstNode.getFs().delete(f, false);
+          if (dstNode.getFs().rename(tmpPath, f)) {
+            try {
+              dstNode.getFs().setTimes(f, srcNode.status.getModificationTime(),
+                  srcNode.status.getAccessTime());
+            } finally {
+              // save getFileStatus rpc
+              srcStatus.setPath(dstNode.getFs().makeQualified(f));
+              dstNode.status = srcStatus;
+            }
+          }
+        } catch (IOException ioe) {
+          // can blame the source by statusSet.clear(ai), however, it would
+          // cost an extra RPC, so just rely on the loop below that will attempt
+          // an open anyhow
+          //
+          LOG.info(f + " " + srcNode + "->" + dstNode + ": Failed to repair",
+                ioe);
+        }
+      }
+    }
+
+    // Since Java7, QuickSort is used instead of MergeSort.
+    // QuickSort may not be stable and thus the equal most recent nodes, may no
+    // longer appear in the NetworkTopology order.
+    //
+    if (maxMtime > 0) {
+      final List<MRNflyNode> mrList = new ArrayList<MRNflyNode>();
+      for (final MRNflyNode openNode : mrNodes) {
+        if (openNode.status != null && openNode.status.getLen() >= 0L) {
+          if (openNode.status.getModificationTime() == maxMtime) {
+            mrList.add(openNode);
+          }
+        }
+      }
+      // assert mrList.size > 0
+      final MRNflyNode[] readNodes = mrList.toArray(new MRNflyNode[0]);
+      topology.sortByDistance(myNode, readNodes, readNodes.length);
+      for (final MRNflyNode rNode : readNodes) {
+        try {
+          return rNode.getFs().open(f, bufferSize);
+        } catch (IOException e) {
+          LOG.info(f + ": Failed to open at " + rNode.getFs().getUri());
+        }
+      }
+    }
+    return null;
+  }
+
+  private void mayThrowFileNotFound(List<IOException> ioExceptions,
+      int numNotFounds) throws FileNotFoundException {
+    if (numNotFounds == nodes.length) {
+      throw (FileNotFoundException)ioExceptions.get(nodes.length - 1);
+    }
+  }
+
+  // WRITE
+  @Override
+  public FSDataOutputStream create(Path f, FsPermission permission,
+      boolean overwrite, int bufferSize, short replication, long blockSize,
+      Progressable progress) throws IOException {
+    return new FSDataOutputStream(new NflyOutputStream(f, permission, overwrite,
+        bufferSize, replication, blockSize, progress), statistics);
+  }
+
+  // WRITE
+  @Override
+  public FSDataOutputStream append(Path f, int bufferSize,
+      Progressable progress) throws IOException {
+    return null;
+  }
+
+  // WRITE
+  @Override
+  public boolean rename(Path src, Path dst) throws IOException {
+    final List<IOException> ioExceptions = new ArrayList<IOException>();
+    int numNotFounds = 0;
+    boolean succ = true;
+    for (final NflyNode nflyNode : nodes) {
+      try {
+        succ &= nflyNode.fs.rename(src, dst);
+      } catch (FileNotFoundException fnfe) {
+        numNotFounds++;
+        processThrowable(nflyNode, "rename", fnfe, ioExceptions, src, dst);
+      } catch (Throwable t) {
+        processThrowable(nflyNode, "rename", t, ioExceptions, src, dst);
+        succ = false;
+      }
+    }
+
+    mayThrowFileNotFound(ioExceptions, numNotFounds);
+
+    // if all destinations threw exceptions throw, otherwise return
+    //
+    if (ioExceptions.size() == nodes.length) {
+      throw MultipleIOException.createIOException(ioExceptions);
+    }
+
+    return succ;
+  }
+
+  // WRITE
+  @Override
+  public boolean delete(Path f, boolean recursive) throws IOException {
+    final List<IOException> ioExceptions = new ArrayList<IOException>();
+    int numNotFounds = 0;
+    boolean succ = true;
+    for (final NflyNode nflyNode : nodes) {
+      try {
+        succ &= nflyNode.fs.delete(f);
+      } catch (FileNotFoundException fnfe) {
+        numNotFounds++;
+        processThrowable(nflyNode, "delete", fnfe, ioExceptions, f);
+      } catch (Throwable t) {
+        processThrowable(nflyNode, "delete", t, ioExceptions, f);
+        succ = false;
+      }
+    }
+    mayThrowFileNotFound(ioExceptions, numNotFounds);
+
+    // if all destinations threw exceptions throw, otherwise return
+    //
+    if (ioExceptions.size() == nodes.length) {
+      throw MultipleIOException.createIOException(ioExceptions);
+    }
+
+    return succ;
+  }
+
+
+  /**
+   * Returns the closest non-failing destination's result.
+   *
+   * @param f given path
+   * @return array of file statuses according to nfly modes
+   * @throws FileNotFoundException
+   * @throws IOException
+   */
+  @Override
+  public FileStatus[] listStatus(Path f) throws FileNotFoundException,
+      IOException {
+    final List<IOException> ioExceptions =
+        new ArrayList<IOException>(nodes.length);
+
+    final MRNflyNode[] mrNodes = workSet();
+    if (nflyFlags.contains(NflyKey.readMostRecent)) {
+      int numNotFounds = 0;
+      for (final MRNflyNode nflyNode : mrNodes) {
+        try {
+          nflyNode.updateFileStatus(f);
+        } catch (FileNotFoundException fnfe) {
+          numNotFounds++;
+          processThrowable(nflyNode, "listStatus", fnfe, ioExceptions, f);
+        } catch (Throwable t) {
+          processThrowable(nflyNode, "listStatus", t, ioExceptions, f);
+        }
+      }
+      mayThrowFileNotFound(ioExceptions, numNotFounds);
+      Arrays.sort(mrNodes);
+    }
+
+    int numNotFounds = 0;
+    for (final MRNflyNode nflyNode : mrNodes) {
+      try {
+        final FileStatus[] realStats = nflyNode.getFs().listStatus(f);
+        final FileStatus[] nflyStats = new FileStatus[realStats.length];
+        for (int i = 0; i < realStats.length; i++) {
+          nflyStats[i] = new NflyStatus(nflyNode.getFs(), realStats[i]);
+        }
+        return nflyStats;
+      } catch (FileNotFoundException fnfe) {
+        numNotFounds++;
+        processThrowable(nflyNode, "listStatus", fnfe, ioExceptions, f);
+      } catch (Throwable t) {
+        processThrowable(nflyNode, "listStatus", t, ioExceptions, f);
+      }
+    }
+    mayThrowFileNotFound(ioExceptions, numNotFounds);
+    throw MultipleIOException.createIOException(ioExceptions);
+  }
+
+  @Override
+  public RemoteIterator<LocatedFileStatus> listLocatedStatus(Path f)
+      throws FileNotFoundException, IOException {
+    // TODO important for splits
+    return super.listLocatedStatus(f);
+  }
+
+  @Override
+  public void setWorkingDirectory(Path newDir) {
+    for (final NflyNode nflyNode : nodes) {
+      nflyNode.fs.setWorkingDirectory(newDir);
+    }
+  }
+
+  @Override
+  public Path getWorkingDirectory() {
+    return nodes[0].fs.getWorkingDirectory(); // 0 is as good as any
+  }
+
+  @Override
+  public boolean mkdirs(Path f, FsPermission permission) throws IOException {
+    boolean succ = true;
+    for (final NflyNode nflyNode : nodes) {
+      succ &= nflyNode.fs.mkdirs(f, permission);
+    }
+    return succ;
+  }
+
+  @Override
+  public FileStatus getFileStatus(Path f) throws IOException {
+    // TODO proxy stream for reads
+    final List<IOException> ioExceptions =
+        new ArrayList<IOException>(nodes.length);
+    int numNotFounds = 0;
+    final MRNflyNode[] mrNodes = workSet();
+
+    long maxMtime = Long.MIN_VALUE;
+    int maxMtimeIdx = Integer.MIN_VALUE;
+
+    // naively iterate until one can be returned
+    //
+    for (int i = 0; i < mrNodes.length; i++) {
+      MRNflyNode nflyNode = mrNodes[i];
+      try {
+        nflyNode.updateFileStatus(f);
+        if (nflyFlags.contains(NflyKey.readMostRecent)) {
+          final long nflyTime = nflyNode.status.getModificationTime();
+          if (nflyTime > maxMtime) {
+            maxMtime = nflyTime;
+            maxMtimeIdx = i;
+          }
+        } else {
+          return nflyNode.nflyStatus();
+        }
+      } catch (FileNotFoundException fnfe) {
+        numNotFounds++;
+        processThrowable(nflyNode, "getFileStatus", fnfe, ioExceptions, f);
+      } catch (Throwable t) {
+        processThrowable(nflyNode, "getFileStatus", t, ioExceptions, f);
+      }
+    }
+
+    if (maxMtimeIdx >= 0) {
+      return mrNodes[maxMtimeIdx].nflyStatus();
+    }
+
+    mayThrowFileNotFound(ioExceptions, numNotFounds);
+    throw MultipleIOException.createIOException(ioExceptions);
+  }
+
+  private static void processThrowable(NflyNode nflyNode, String op,
+      Throwable t, List<IOException> ioExceptions,
+      Path... f) {
+    final String errMsg = Arrays.toString(f)
+        + ": failed to " + op + " " + nflyNode.fs.getUri();
+    final IOException ioex;
+    if (t instanceof FileNotFoundException) {
+      ioex = new FileNotFoundException(errMsg);
+      ioex.initCause(t);
+    } else {
+      ioex = new IOException(errMsg, t);
+    }
+
+    if (ioExceptions != null) {
+      ioExceptions.add(ioex);
+    }
+  }
+
+  /**
+   * Initializes an nfly mountpoint in viewfs.
+   *
+   * @param uris destinations to replicate writes to
+   * @param conf file system configuration
+   * @param settings comma-separated list of k=v pairs.
+   * @return an Nfly filesystem
+   * @throws IOException
+   */
+  static FileSystem createFileSystem(URI[] uris, Configuration conf,
+      String settings) throws IOException {
+    // assert settings != null
+    int minRepl = DEFAULT_MIN_REPLICATION;
+    EnumSet<NflyKey> nflyFlags = EnumSet.noneOf(NflyKey.class);
+    final String[] kvPairs = StringUtils.split(settings);
+    for (String kv : kvPairs) {
+      final String[] kvPair = StringUtils.split(kv, '=');
+      if (kvPair.length != 2) {
+        throw new IllegalArgumentException(kv);
+      }
+      NflyKey nflyKey = NflyKey.valueOf(kvPair[0]);
+      switch (nflyKey) {
+      case minReplication:
+        minRepl = Integer.parseInt(kvPair[1]);
+        break;
+      case repairOnRead:
+      case readMostRecent:
+        if (Boolean.valueOf(kvPair[1])) {
+          nflyFlags.add(nflyKey);
+        }
+        break;
+      default:
+        throw new IllegalArgumentException(nflyKey + ": Infeasible");
+      }
+    }
+    return new NflyFSystem(uris, conf, minRepl, nflyFlags);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f3bc63e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
index 158b099..ca1380a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
@@ -54,7 +54,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.QuotaUsage;
 import org.apache.hadoop.fs.RemoteIterator;
-import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
@@ -186,25 +185,21 @@ public class ViewFileSystem extends FileSystem {
       fsState = new InodeTree<FileSystem>(conf, authority) {
 
         @Override
-        protected
-        FileSystem getTargetFileSystem(final URI uri)
+        protected FileSystem getTargetFileSystem(final URI uri)
           throws URISyntaxException, IOException {
             return new ChRootedFileSystem(uri, config);
         }
 
         @Override
-        protected
-        FileSystem getTargetFileSystem(final INodeDir<FileSystem> dir)
+        protected FileSystem getTargetFileSystem(final INodeDir<FileSystem> dir)
           throws URISyntaxException {
           return new InternalDirOfViewFs(dir, creationTime, ugi, myUri, config);
         }
 
         @Override
-        protected
-        FileSystem getTargetFileSystem(URI[] mergeFsURIList)
-            throws URISyntaxException, UnsupportedFileSystemException {
-          throw new UnsupportedFileSystemException("mergefs not implemented");
-          // return MergeFs.createMergeFs(mergeFsURIList, config);
+        protected FileSystem getTargetFileSystem(final String settings,
+            final URI[] uris) throws URISyntaxException, IOException {
+          return NflyFSystem.createFileSystem(uris, config, settings);
         }
       };
       workingDir = this.getHomeDirectory();
@@ -455,8 +450,13 @@ public class ViewFileSystem extends FileSystem {
 
   private Path getChrootedPath(InodeTree.ResolveResult<FileSystem> res,
       FileStatus status, Path f) throws IOException {
-    final String suffix = ((ChRootedFileSystem)res.targetFileSystem)
-        .stripOutRoot(status.getPath());
+    final String suffix;
+    if (res.targetFileSystem instanceof ChRootedFileSystem) {
+      suffix = ((ChRootedFileSystem)res.targetFileSystem)
+          .stripOutRoot(status.getPath());
+    } else { // nfly
+      suffix = ((NflyFSystem.NflyStatus)status).stripRoot();
+    }
     return this.makeQualified(
         suffix.length() == 0 ? f : new Path(res.resolvedPath, suffix));
   }
@@ -501,10 +501,15 @@ public class ViewFileSystem extends FileSystem {
     verifyRenameStrategy(srcUri, dstUri,
         resSrc.targetFileSystem == resDst.targetFileSystem, renameStrategy);
 
-    ChRootedFileSystem srcFS = (ChRootedFileSystem) resSrc.targetFileSystem;
-    ChRootedFileSystem dstFS = (ChRootedFileSystem) resDst.targetFileSystem;
-    return srcFS.getMyFs().rename(srcFS.fullPath(resSrc.remainingPath),
-        dstFS.fullPath(resDst.remainingPath));
+    if (resSrc.targetFileSystem instanceof ChRootedFileSystem &&
+        resDst.targetFileSystem instanceof ChRootedFileSystem) {
+      ChRootedFileSystem srcFS = (ChRootedFileSystem) resSrc.targetFileSystem;
+      ChRootedFileSystem dstFS = (ChRootedFileSystem) resDst.targetFileSystem;
+      return srcFS.getMyFs().rename(srcFS.fullPath(resSrc.remainingPath),
+          dstFS.fullPath(resDst.remainingPath));
+    } else {
+      return resSrc.targetFileSystem.rename(resSrc.remainingPath, resDst.remainingPath);
+    }
   }
 
   static void verifyRenameStrategy(URI srcUri, URI dstUri,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f3bc63e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java
index 364485f..6a89f27 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java
@@ -212,8 +212,7 @@ public class ViewFs extends AbstractFileSystem {
     fsState = new InodeTree<AbstractFileSystem>(conf, authority) {
 
       @Override
-      protected
-      AbstractFileSystem getTargetFileSystem(final URI uri)
+      protected AbstractFileSystem getTargetFileSystem(final URI uri)
         throws URISyntaxException, UnsupportedFileSystemException {
           String pathString = uri.getPath();
           if (pathString.isEmpty()) {
@@ -225,15 +224,14 @@ public class ViewFs extends AbstractFileSystem {
       }
 
       @Override
-      protected
-      AbstractFileSystem getTargetFileSystem(
+      protected AbstractFileSystem getTargetFileSystem(
           final INodeDir<AbstractFileSystem> dir) throws URISyntaxException {
         return new InternalDirOfViewFs(dir, creationTime, ugi, getUri());
       }
 
       @Override
-      protected
-      AbstractFileSystem getTargetFileSystem(URI[] mergeFsURIList)
+      protected AbstractFileSystem getTargetFileSystem(final String settings,
+          final URI[] mergeFsURIList)
           throws URISyntaxException, UnsupportedFileSystemException {
         throw new UnsupportedFileSystemException("mergefs not implemented yet");
         // return MergeFs.createMergeFs(mergeFsURIList, config);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f3bc63e/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLocalFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLocalFileSystem.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLocalFileSystem.java
index 4943792..808d8b0 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLocalFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLocalFileSystem.java
@@ -18,13 +18,25 @@
 package org.apache.hadoop.fs.viewfs;
 
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.net.URI;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FileSystemTestHelper;
+import org.apache.hadoop.fs.Path;
 
 import org.junit.After;
 import org.junit.Before;
-
+import org.junit.Test;
 
 
 /**
@@ -37,6 +49,8 @@ import org.junit.Before;
  */
 
 public class TestViewFileSystemLocalFileSystem extends ViewFileSystemBaseTest {
+  private static final Log LOG =
+      LogFactory.getLog(TestViewFileSystemLocalFileSystem.class);
 
   @Override
   @Before
@@ -47,6 +61,65 @@ public class TestViewFileSystemLocalFileSystem extends ViewFileSystemBaseTest {
     
   }
 
+  @Test
+  public void testNflyWriteSimple() throws IOException {
+    LOG.info("Starting testNflyWriteSimple");
+    final URI[] testUris = new URI[] {
+        URI.create(targetTestRoot + "/nfwd1"),
+        URI.create(targetTestRoot + "/nfwd2")
+    };
+    final String testFileName = "test.txt";
+    final Configuration testConf = new Configuration(conf);
+    final String testString = "Hello Nfly!";
+    final Path nflyRoot = new Path("/nflyroot");
+    ConfigUtil.addLinkNfly(testConf, nflyRoot.toString(), testUris);
+    final FileSystem nfly = FileSystem.get(URI.create("viewfs:///"), testConf);
+
+    final FSDataOutputStream fsDos = nfly.create(
+        new Path(nflyRoot, "test.txt"));
+    try {
+      fsDos.writeUTF(testString);
+    } finally {
+      fsDos.close();
+    }
+
+    FileStatus[] statuses = nfly.listStatus(nflyRoot);
+
+    FileSystem lfs = FileSystem.getLocal(testConf);
+    for (final URI testUri : testUris) {
+      final Path testFile = new Path(new Path(testUri), testFileName);
+      assertTrue(testFile + " should exist!",  lfs.exists(testFile));
+      final FSDataInputStream fsdis = lfs.open(testFile);
+      try {
+        assertEquals("Wrong file content", testString, fsdis.readUTF());
+      } finally {
+        fsdis.close();
+      }
+    }
+  }
+
+
+  @Test
+  public void testNflyInvalidMinReplication() throws Exception {
+    LOG.info("Starting testNflyInvalidMinReplication");
+    final URI[] testUris = new URI[] {
+        URI.create(targetTestRoot + "/nfwd1"),
+        URI.create(targetTestRoot + "/nfwd2")
+    };
+
+    final Configuration conf = new Configuration();
+    ConfigUtil.addLinkNfly(conf, "mt", "/nflyroot", "minReplication=4",
+        testUris);
+    try {
+      FileSystem.get(URI.create("viewfs://mt/"), conf);
+      fail("Expected bad minReplication exception.");
+    } catch (IOException ioe) {
+      assertTrue("No minReplication message",
+          ioe.getMessage().contains("Minimum replication"));
+    }
+  }
+
+
   @Override
   @After
   public void tearDown() throws Exception {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f3bc63e/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsConfig.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsConfig.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsConfig.java
index 895ae0c..136837f 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsConfig.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsConfig.java
@@ -24,7 +24,6 @@ import java.net.URISyntaxException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
-import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.junit.Test;
 
 public class TestViewFsConfig {
@@ -43,23 +42,21 @@ public class TestViewFsConfig {
     new InodeTree<Foo>(conf, null) {
 
       @Override
-      protected Foo getTargetFileSystem(final URI uri)
-          throws URISyntaxException, UnsupportedFileSystemException {
+      protected Foo getTargetFileSystem(final URI uri) {
         return null;
       }
 
       @Override
-      protected Foo getTargetFileSystem(
-          org.apache.hadoop.fs.viewfs.InodeTree.INodeDir<Foo> dir)
-          throws URISyntaxException {
+      protected Foo getTargetFileSystem(final INodeDir<Foo> dir) {
         return null;
       }
 
       @Override
-      protected Foo getTargetFileSystem(URI[] mergeFsURIList)
-          throws URISyntaxException, UnsupportedFileSystemException {
+      protected Foo getTargetFileSystem(final String settings,
+          final URI[] mergeFsURIList) {
         return null;
       }
+
     };
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f3bc63e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java
index b8f5379..b8bed1d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java
@@ -17,11 +17,9 @@
  */
 package org.apache.hadoop.fs.viewfs;
 
-
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY;
-
 import java.io.File;
 import java.io.IOException;
+import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.EnumSet;
 
@@ -31,6 +29,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.key.JavaKeyStoreProvider;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -45,17 +45,26 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.client.CreateEncryptionZoneFlag;
 import org.apache.hadoop.hdfs.client.HdfsAdmin;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY;
+
 import org.junit.After;
 import org.junit.AfterClass;
-import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import static org.junit.Assert.*;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class TestViewFileSystemHdfs extends ViewFileSystemBaseTest {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestViewFileSystemHdfs.class);
+
 
   private static MiniDFSCluster cluster;
   private static Path defaultWorkingDirectory;
@@ -190,12 +199,12 @@ public class TestViewFileSystemHdfs extends ViewFileSystemBaseTest {
 
     //Verify file deletion within EZ
     DFSTestUtil.verifyDelete(shell, fsTarget, encFile, true);
-    Assert.assertTrue("ViewFileSystem trash roots should include EZ file trash",
+    assertTrue("ViewFileSystem trash roots should include EZ file trash",
         (fsView.getTrashRoots(true).size() == 1));
 
     //Verify deletion of EZ
     DFSTestUtil.verifyDelete(shell, fsTarget, zone, true);
-    Assert.assertTrue("ViewFileSystem trash roots should include EZ zone trash",
+    assertTrue("ViewFileSystem trash roots should include EZ zone trash",
         (fsView.getTrashRoots(true).size() == 2));
   }
 
@@ -239,14 +248,14 @@ public class TestViewFileSystemHdfs extends ViewFileSystemBaseTest {
         viewFs.getFileChecksum(mountDataFilePath);
     FileChecksum fileChecksumViaTargetFs =
         fsTarget.getFileChecksum(fsTargetFilePath);
-    Assert.assertTrue("File checksum not matching!",
+    assertTrue("File checksum not matching!",
         fileChecksumViaViewFs.equals(fileChecksumViaTargetFs));
 
     fileChecksumViaViewFs =
         viewFs.getFileChecksum(mountDataFilePath, fileLength / 2);
     fileChecksumViaTargetFs =
         fsTarget.getFileChecksum(fsTargetFilePath, fileLength / 2);
-    Assert.assertTrue("File checksum not matching!",
+    assertTrue("File checksum not matching!",
         fileChecksumViaViewFs.equals(fileChecksumViaTargetFs));
   }
 
@@ -269,4 +278,130 @@ public class TestViewFileSystemHdfs extends ViewFileSystemBaseTest {
               e);
     }
   }
+
+  @Test
+  public void testNflyClosestRepair() throws Exception {
+    testNflyRepair(NflyFSystem.NflyKey.repairOnRead);
+  }
+
+  @Test
+  public void testNflyMostRecentRepair() throws Exception {
+    testNflyRepair(NflyFSystem.NflyKey.readMostRecent);
+  }
+
+  private void testNflyRepair(NflyFSystem.NflyKey repairKey)
+      throws Exception {
+    LOG.info("Starting testNflyWriteSimpleFailover");
+    final URI uri1 = targetTestRoot.toUri();
+    final URI uri2 = targetTestRoot2.toUri();
+    final URI[] testUris = new URI[] {
+        new URI(uri1.getScheme(), uri1.getAuthority(), "/", null, null),
+        new URI(uri2.getScheme(), uri2.getAuthority(), "/", null, null)
+    };
+
+    final Configuration testConf = new Configuration(conf);
+    testConf.setInt(IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 1);
+
+    final String testString = "Hello Nfly!";
+    final Path nflyRoot = new Path("/nflyroot");
+
+    ConfigUtil.addLinkNfly(testConf,
+        Constants.CONFIG_VIEWFS_DEFAULT_MOUNT_TABLE,
+        nflyRoot.toString(),
+        "minReplication=2," + repairKey + "=true", testUris);
+
+    final FileSystem nfly = FileSystem.get(URI.create("viewfs:///"), testConf);
+    // wd = /nflyroot/user/<user>
+    nfly.setWorkingDirectory(new Path(nflyRoot
+        + nfly.getWorkingDirectory().toUri().getPath()));
+
+    // 1. test mkdirs
+    final Path testDir = new Path("testdir1/sub1/sub3");
+    final Path testDir_tmp = new Path("testdir1/sub1/sub3_temp");
+    assertTrue(testDir + ": Failed to create!", nfly.mkdirs(testDir));
+
+    // Test renames
+    assertTrue(nfly.rename(testDir, testDir_tmp));
+    assertTrue(nfly.rename(testDir_tmp, testDir));
+
+    for (final URI testUri : testUris) {
+      final FileSystem fs = FileSystem.get(testUri, testConf);
+      assertTrue(testDir + " should exist!", fs.exists(testDir));
+    }
+
+    // 2. test write
+    final Path testFile = new Path("test.txt");
+    final FSDataOutputStream fsDos = nfly.create(testFile);
+    try {
+      fsDos.writeUTF(testString);
+    } finally {
+      fsDos.close();
+    }
+
+    for (final URI testUri : testUris) {
+      final FileSystem fs = FileSystem.get(testUri, testConf);
+      final FSDataInputStream fsdis = fs.open(testFile);
+      try {
+        assertEquals("Wrong file content", testString, fsdis.readUTF());
+      } finally {
+        fsdis.close();
+      }
+    }
+
+    // 3. test reads when one unavailable
+    //
+    // bring one NN down and read through nfly should still work
+    //
+    for (int i = 0; i < cluster.getNumNameNodes(); i++) {
+      cluster.shutdownNameNode(i);
+      FSDataInputStream fsDis = null;
+      try {
+        fsDis = nfly.open(testFile);
+        assertEquals("Wrong file content", testString, fsDis.readUTF());
+      } finally {
+        IOUtils.cleanupWithLogger(LOG, fsDis);
+        cluster.restartNameNode(i);
+      }
+    }
+
+    // both nodes are up again, test repair
+    final FileSystem fs1 = FileSystem.get(testUris[0], conf);
+    assertTrue(fs1.delete(testFile, false));
+    assertFalse(fs1.exists(testFile));
+    FSDataInputStream fsDis = null;
+    try {
+      fsDis = nfly.open(testFile);
+      assertEquals("Wrong file content", testString, fsDis.readUTF());
+      assertTrue(fs1.exists(testFile));
+    } finally {
+      IOUtils.cleanupWithLogger(LOG, fsDis);
+    }
+
+    // test most recent repair
+    if (repairKey == NflyFSystem.NflyKey.readMostRecent) {
+      final FileSystem fs2 = FileSystem.get(testUris[0], conf);
+      final long expectedMtime = fs2.getFileStatus(testFile)
+          .getModificationTime();
+
+      for (final URI testUri : testUris) {
+        final FileSystem fs = FileSystem.get(testUri, conf);
+        fs.setTimes(testFile, 1L, 1L);
+        assertEquals(testUri + "Set mtime failed!", 1L,
+            fs.getFileStatus(testFile).getModificationTime());
+        assertEquals("nfly file status wrong", expectedMtime,
+            nfly.getFileStatus(testFile).getModificationTime());
+        FSDataInputStream fsDis2 = null;
+        try {
+          fsDis2 = nfly.open(testFile);
+          assertEquals("Wrong file content", testString, fsDis2.readUTF());
+          // repair is done, now trying via normal fs
+          //
+          assertEquals("Repair most recent failed!", expectedMtime,
+              fs.getFileStatus(testFile).getModificationTime());
+        } finally {
+          IOUtils.cleanupWithLogger(LOG, fsDis2);
+        }
+      }
+    }
+  }
 }


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


[49/50] [abbrv] hadoop git commit: YARN-5292. NM Container lifecycle and state transitions to support for PAUSED container state. (Hitesh Sharma via asuresh)

Posted by as...@apache.org.
YARN-5292. NM Container lifecycle and state transitions to support for PAUSED container state. (Hitesh Sharma via asuresh)


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

Branch: refs/heads/YARN-5972
Commit: 01eb8ce4a8bb2b2e3d4e43f9cf07308c0c23e039
Parents: 5ab21df
Author: Arun Suresh <as...@apache.org>
Authored: Fri Dec 9 07:51:03 2016 -0800
Committer: Arun Suresh <as...@apache.org>
Committed: Sun Sep 10 23:28:03 2017 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/api/records/ContainerState.java |   7 +-
 .../src/main/proto/yarn_protos.proto            |   1 +
 .../server/nodemanager/ContainerExecutor.java   |  22 +++
 .../container/ContainerEventType.java           |   6 +-
 .../container/ContainerImpl.java                | 170 ++++++++++++++++++-
 .../container/ContainerPauseEvent.java          |  40 +++++
 .../container/ContainerResumeEvent.java         |  39 +++++
 .../container/ContainerState.java               |   3 +-
 .../launcher/ContainerLaunch.java               |  90 +++++++++-
 .../launcher/ContainersLauncher.java            |  32 ++++
 .../launcher/ContainersLauncherEventType.java   |   3 +
 .../scheduler/ContainerSchedulerEventType.java  |   1 +
 .../container/TestContainer.java                |  51 ++++++
 13 files changed, 454 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/01eb8ce4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerState.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerState.java
index 696fe06..45e5bd4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerState.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerState.java
@@ -33,11 +33,14 @@ public enum ContainerState {
   
   /** Running container */
   RUNNING, 
-  
+
   /** Completed container */
   COMPLETE,
 
   /** Scheduled (awaiting resources) at the NM. */
   @InterfaceStability.Unstable
-  SCHEDULED
+  SCHEDULED,
+
+  /** Paused at the NM. */
+  PAUSED
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/01eb8ce4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
index c5f485f..a39c4dd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
@@ -83,6 +83,7 @@ enum ContainerStateProto {
   C_RUNNING = 2;
   C_COMPLETE = 3;
   C_SCHEDULED = 4;
+  C_PAUSED = 5;
 }
 
 message ContainerProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/01eb8ce4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerExecutor.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/ContainerExecutor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerExecutor.java
index 072cca7..da50d7a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerExecutor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerExecutor.java
@@ -700,6 +700,28 @@ public abstract class ContainerExecutor implements Configurable {
   }
 
   /**
+   * Pause the container. The default implementation is to raise a kill event.
+   * Specific executor implementations can override this behavior.
+   * @param container
+   *          the Container
+   */
+  public void pauseContainer(Container container) {
+    LOG.warn(container.getContainerId() + " doesn't support pausing.");
+    throw new UnsupportedOperationException();
+  }
+
+  /**
+   * Resume the container from pause state. The default implementation ignores
+   * this event. Specific implementations can override this behavior.
+   * @param container
+   *          the Container
+   */
+  public void resumeContainer(Container container) {
+    LOG.warn(container.getContainerId() + " doesn't support resume.");
+    throw new UnsupportedOperationException();
+  }
+
+  /**
    * Get the process-identifier for the container.
    *
    * @param containerID the container ID

http://git-wip-us.apache.org/repos/asf/hadoop/blob/01eb8ce4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerEventType.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/container/ContainerEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerEventType.java
index afea0e6..1475435 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerEventType.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerEventType.java
@@ -27,6 +27,8 @@ public enum ContainerEventType {
   CONTAINER_DONE,
   REINITIALIZE_CONTAINER,
   ROLLBACK_REINIT,
+  PAUSE_CONTAINER,
+  RESUME_CONTAINER,
 
   // DownloadManager
   CONTAINER_INITED,
@@ -38,5 +40,7 @@ public enum ContainerEventType {
   CONTAINER_LAUNCHED,
   CONTAINER_EXITED_WITH_SUCCESS,
   CONTAINER_EXITED_WITH_FAILURE,
-  CONTAINER_KILLED_ON_REQUEST
+  CONTAINER_KILLED_ON_REQUEST,
+  CONTAINER_PAUSED,
+  CONTAINER_RESUMED
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/01eb8ce4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.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/container/ContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
index 1a48b12..7a12371 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
@@ -307,6 +307,8 @@ public class ContainerImpl implements Container {
         UPDATE_DIAGNOSTICS_TRANSITION)
     .addTransition(ContainerState.NEW, ContainerState.DONE,
         ContainerEventType.KILL_CONTAINER, new KillOnNewTransition())
+    .addTransition(ContainerState.NEW, ContainerState.DONE,
+            ContainerEventType.PAUSE_CONTAINER, new KillOnPauseTransition())
 
     // From LOCALIZING State
     .addTransition(ContainerState.LOCALIZING,
@@ -322,6 +324,8 @@ public class ContainerImpl implements Container {
     .addTransition(ContainerState.LOCALIZING, ContainerState.KILLING,
         ContainerEventType.KILL_CONTAINER,
         new KillBeforeRunningTransition())
+    .addTransition(ContainerState.LOCALIZING, ContainerState.KILLING,
+        ContainerEventType.PAUSE_CONTAINER, new KillOnPauseTransition())
 
     // From LOCALIZATION_FAILED State
     .addTransition(ContainerState.LOCALIZATION_FAILED,
@@ -335,7 +339,8 @@ public class ContainerImpl implements Container {
     // container not launched so kill is a no-op
     .addTransition(ContainerState.LOCALIZATION_FAILED,
         ContainerState.LOCALIZATION_FAILED,
-        ContainerEventType.KILL_CONTAINER)
+        EnumSet.of(ContainerEventType.KILL_CONTAINER,
+            ContainerEventType.PAUSE_CONTAINER))
     // container cleanup triggers a release of all resources
     // regardless of whether they were localized or not
     // LocalizedResource handles release event in all states
@@ -391,6 +396,76 @@ public class ContainerImpl implements Container {
         ContainerState.EXITED_WITH_FAILURE,
         ContainerEventType.CONTAINER_KILLED_ON_REQUEST,
         new KilledExternallyTransition())
+    .addTransition(ContainerState.RUNNING, ContainerState.PAUSING,
+    ContainerEventType.PAUSE_CONTAINER, new PauseContainerTransition())
+
+    // From PAUSING State
+    .addTransition(ContainerState.PAUSING, ContainerState.KILLING,
+        ContainerEventType.KILL_CONTAINER, new KillTransition())
+    .addTransition(ContainerState.PAUSING, ContainerState.PAUSING,
+        ContainerEventType.UPDATE_DIAGNOSTICS_MSG,
+        UPDATE_DIAGNOSTICS_TRANSITION)
+    .addTransition(ContainerState.PAUSING, ContainerState.PAUSED,
+        ContainerEventType.CONTAINER_PAUSED, new PausedContainerTransition())
+    // In case something goes wrong then container will exit from the
+    // PAUSING state
+    .addTransition(ContainerState.PAUSING,
+        ContainerState.EXITED_WITH_SUCCESS,
+        ContainerEventType.CONTAINER_EXITED_WITH_SUCCESS)
+    .addTransition(ContainerState.PAUSING,
+        ContainerState.EXITED_WITH_FAILURE,
+        ContainerEventType.CONTAINER_EXITED_WITH_FAILURE,
+        new ExitedWithFailureTransition(true))
+    .addTransition(ContainerState.PAUSING, ContainerState.EXITED_WITH_FAILURE,
+        ContainerEventType.CONTAINER_KILLED_ON_REQUEST,
+        new KilledExternallyTransition())
+
+    // From PAUSED State
+    .addTransition(ContainerState.PAUSED, ContainerState.KILLING,
+        ContainerEventType.KILL_CONTAINER, new KillTransition())
+    .addTransition(ContainerState.PAUSED, ContainerState.PAUSED,
+        ContainerEventType.UPDATE_DIAGNOSTICS_MSG,
+        UPDATE_DIAGNOSTICS_TRANSITION)
+    .addTransition(ContainerState.PAUSED, ContainerState.PAUSED,
+        ContainerEventType.PAUSE_CONTAINER)
+    .addTransition(ContainerState.PAUSED, ContainerState.RESUMING,
+        ContainerEventType.RESUME_CONTAINER, new ResumeContainerTransition())
+    // In case something goes wrong then container will exit from the
+    // PAUSED state
+    .addTransition(ContainerState.PAUSED,
+        ContainerState.EXITED_WITH_FAILURE,
+        ContainerEventType.CONTAINER_EXITED_WITH_FAILURE,
+        new ExitedWithFailureTransition(true))
+    .addTransition(ContainerState.PAUSED, ContainerState.EXITED_WITH_FAILURE,
+        ContainerEventType.CONTAINER_KILLED_ON_REQUEST,
+        new KilledExternallyTransition())
+    .addTransition(ContainerState.PAUSED,
+        ContainerState.EXITED_WITH_SUCCESS,
+        ContainerEventType.CONTAINER_EXITED_WITH_SUCCESS,
+        new ExitedWithSuccessTransition(true))
+
+    // From RESUMING State
+    .addTransition(ContainerState.RESUMING, ContainerState.KILLING,
+        ContainerEventType.KILL_CONTAINER, new KillTransition())
+    .addTransition(ContainerState.RESUMING, ContainerState.RUNNING,
+        ContainerEventType.CONTAINER_RESUMED)
+    .addTransition(ContainerState.RESUMING, ContainerState.RESUMING,
+        ContainerEventType.UPDATE_DIAGNOSTICS_MSG,
+        UPDATE_DIAGNOSTICS_TRANSITION)
+    // In case something goes wrong then container will exit from the
+    // RESUMING state
+    .addTransition(ContainerState.RESUMING,
+        ContainerState.EXITED_WITH_FAILURE,
+        ContainerEventType.CONTAINER_EXITED_WITH_FAILURE,
+        new ExitedWithFailureTransition(true))
+    .addTransition(ContainerState.RESUMING,
+        ContainerState.EXITED_WITH_FAILURE,
+        ContainerEventType.CONTAINER_KILLED_ON_REQUEST,
+        new KilledExternallyTransition())
+    .addTransition(ContainerState.RESUMING,
+        ContainerState.EXITED_WITH_SUCCESS,
+        ContainerEventType.CONTAINER_EXITED_WITH_SUCCESS,
+        new ExitedWithSuccessTransition(true))
 
     // From REINITIALIZING State
     .addTransition(ContainerState.REINITIALIZING,
@@ -414,6 +489,8 @@ public class ContainerImpl implements Container {
         UPDATE_DIAGNOSTICS_TRANSITION)
     .addTransition(ContainerState.REINITIALIZING, ContainerState.KILLING,
         ContainerEventType.KILL_CONTAINER, new KillTransition())
+    .addTransition(ContainerState.REINITIALIZING, ContainerState.KILLING,
+        ContainerEventType.PAUSE_CONTAINER, new KillOnPauseTransition())
     .addTransition(ContainerState.REINITIALIZING,
         ContainerState.SCHEDULED,
         ContainerEventType.CONTAINER_KILLED_ON_REQUEST,
@@ -431,6 +508,8 @@ public class ContainerImpl implements Container {
         UPDATE_DIAGNOSTICS_TRANSITION)
     .addTransition(ContainerState.RELAUNCHING, ContainerState.KILLING,
         ContainerEventType.KILL_CONTAINER, new KillTransition())
+    .addTransition(ContainerState.RELAUNCHING, ContainerState.KILLING,
+        ContainerEventType.PAUSE_CONTAINER, new KillOnPauseTransition())
 
     // From CONTAINER_EXITED_WITH_SUCCESS State
     .addTransition(ContainerState.EXITED_WITH_SUCCESS, ContainerState.DONE,
@@ -442,7 +521,8 @@ public class ContainerImpl implements Container {
         UPDATE_DIAGNOSTICS_TRANSITION)
     .addTransition(ContainerState.EXITED_WITH_SUCCESS,
         ContainerState.EXITED_WITH_SUCCESS,
-        ContainerEventType.KILL_CONTAINER)
+        EnumSet.of(ContainerEventType.KILL_CONTAINER,
+            ContainerEventType.PAUSE_CONTAINER))
 
     // From EXITED_WITH_FAILURE State
     .addTransition(ContainerState.EXITED_WITH_FAILURE, ContainerState.DONE,
@@ -454,7 +534,8 @@ public class ContainerImpl implements Container {
         UPDATE_DIAGNOSTICS_TRANSITION)
     .addTransition(ContainerState.EXITED_WITH_FAILURE,
                    ContainerState.EXITED_WITH_FAILURE,
-                   ContainerEventType.KILL_CONTAINER)
+        EnumSet.of(ContainerEventType.KILL_CONTAINER,
+            ContainerEventType.PAUSE_CONTAINER))
 
     // From KILLING State.
     .addTransition(ContainerState.KILLING,
@@ -488,7 +569,8 @@ public class ContainerImpl implements Container {
     // in the container launcher
     .addTransition(ContainerState.KILLING,
         ContainerState.KILLING,
-        ContainerEventType.CONTAINER_LAUNCHED)
+        EnumSet.of(ContainerEventType.CONTAINER_LAUNCHED,
+            ContainerEventType.PAUSE_CONTAINER))
 
     // From CONTAINER_CLEANEDUP_AFTER_KILL State.
     .addTransition(ContainerState.CONTAINER_CLEANEDUP_AFTER_KILL,
@@ -504,11 +586,13 @@ public class ContainerImpl implements Container {
         EnumSet.of(ContainerEventType.KILL_CONTAINER,
             ContainerEventType.RESOURCE_FAILED,
             ContainerEventType.CONTAINER_EXITED_WITH_SUCCESS,
-            ContainerEventType.CONTAINER_EXITED_WITH_FAILURE))
+            ContainerEventType.CONTAINER_EXITED_WITH_FAILURE,
+            ContainerEventType.PAUSE_CONTAINER))
 
     // From DONE
     .addTransition(ContainerState.DONE, ContainerState.DONE,
-        ContainerEventType.KILL_CONTAINER)
+        EnumSet.of(ContainerEventType.KILL_CONTAINER,
+            ContainerEventType.PAUSE_CONTAINER))
     .addTransition(ContainerState.DONE, ContainerState.DONE,
         ContainerEventType.INIT_CONTAINER)
     .addTransition(ContainerState.DONE, ContainerState.DONE,
@@ -534,6 +618,8 @@ public class ContainerImpl implements Container {
     case LOCALIZING:
     case LOCALIZATION_FAILED:
     case SCHEDULED:
+    case PAUSED:
+    case RESUMING:
       return org.apache.hadoop.yarn.api.records.ContainerState.SCHEDULED;
     case RUNNING:
     case RELAUNCHING:
@@ -543,6 +629,7 @@ public class ContainerImpl implements Container {
     case KILLING:
     case CONTAINER_CLEANEDUP_AFTER_KILL:
     case CONTAINER_RESOURCES_CLEANINGUP:
+    case PAUSING:
       return org.apache.hadoop.yarn.api.records.ContainerState.RUNNING;
     case DONE:
     default:
@@ -1501,6 +1588,26 @@ public class ContainerImpl implements Container {
   }
 
   /**
+   * Transitions upon receiving PAUSE_CONTAINER.
+   * - LOCALIZED -> KILLING.
+   * - REINITIALIZING -> KILLING.
+   */
+  @SuppressWarnings("unchecked") // dispatcher not typed
+  static class KillOnPauseTransition implements
+      SingleArcTransition<ContainerImpl, ContainerEvent> {
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void transition(ContainerImpl container, ContainerEvent event) {
+      // Kill the process/process-grp
+      container.setIsReInitializing(false);
+      container.dispatcher.getEventHandler().handle(
+          new ContainersLauncherEvent(container,
+              ContainersLauncherEventType.CLEANUP_CONTAINER));
+    }
+  }
+
+  /**
    * Transition from KILLING to CONTAINER_CLEANEDUP_AFTER_KILL
    * upon receiving CONTAINER_KILLED_ON_REQUEST.
    */
@@ -1690,6 +1797,57 @@ public class ContainerImpl implements Container {
     }
   }
 
+  /**
+   * Transitions upon receiving PAUSE_CONTAINER.
+   * - RUNNING -> PAUSED
+   */
+  @SuppressWarnings("unchecked") // dispatcher not typed
+  static class PauseContainerTransition implements
+      SingleArcTransition<ContainerImpl, ContainerEvent> {
+    @Override
+    public void transition(ContainerImpl container, ContainerEvent event) {
+      // Pause the process/process-grp if it is supported by the container
+      container.dispatcher.getEventHandler().handle(
+          new ContainersLauncherEvent(container,
+              ContainersLauncherEventType.PAUSE_CONTAINER));
+      ContainerPauseEvent pauseEvent = (ContainerPauseEvent) event;
+      container.addDiagnostics(pauseEvent.getDiagnostic(), "\n");
+    }
+  }
+
+  /**
+   * Transitions upon receiving PAUSED_CONTAINER.
+   */
+  @SuppressWarnings("unchecked") // dispatcher not typed
+  static class PausedContainerTransition implements
+      SingleArcTransition<ContainerImpl, ContainerEvent> {
+    @Override
+    public void transition(ContainerImpl container, ContainerEvent event) {
+      // Container was PAUSED so tell the scheduler
+      container.dispatcher.getEventHandler().handle(
+          new ContainerSchedulerEvent(container,
+              ContainerSchedulerEventType.CONTAINER_PAUSED));
+    }
+  }
+
+  /**
+   * Transitions upon receiving RESUME_CONTAINER.
+   * - PAUSED -> RUNNING
+   */
+  @SuppressWarnings("unchecked") // dispatcher not typed
+  static class ResumeContainerTransition implements
+      SingleArcTransition<ContainerImpl, ContainerEvent> {
+    @Override
+    public void transition(ContainerImpl container, ContainerEvent event) {
+      // Pause the process/process-grp if it is supported by the container
+      container.dispatcher.getEventHandler().handle(
+          new ContainersLauncherEvent(container,
+              ContainersLauncherEventType.RESUME_CONTAINER));
+      ContainerResumeEvent resumeEvent = (ContainerResumeEvent) event;
+      container.addDiagnostics(resumeEvent.getDiagnostic(), "\n");
+    }
+  }
+
   @Override
   public void handle(ContainerEvent event) {
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/01eb8ce4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerPauseEvent.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/container/ContainerPauseEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerPauseEvent.java
new file mode 100644
index 0000000..898304e
--- /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/container/ContainerPauseEvent.java
@@ -0,0 +1,40 @@
+/**
+ * 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.container;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+
+/**
+ * ContainerEvent for ContainerEventType.PAUSE_CONTAINER.
+ */
+public class ContainerPauseEvent extends ContainerEvent {
+
+  private final String diagnostic;
+
+  public ContainerPauseEvent(ContainerId cId,
+      String diagnostic) {
+    super(cId, ContainerEventType.PAUSE_CONTAINER);
+    this.diagnostic = diagnostic;
+  }
+
+  public String getDiagnostic() {
+    return this.diagnostic;
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/01eb8ce4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerResumeEvent.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/container/ContainerResumeEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerResumeEvent.java
new file mode 100644
index 0000000..d7c9e9a
--- /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/container/ContainerResumeEvent.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.yarn.server.nodemanager.containermanager.container;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+
+/**
+ * ContainerEvent for ContainerEventType.RESUME_CONTAINER.
+ */
+public class ContainerResumeEvent extends ContainerEvent {
+
+  private final String diagnostic;
+
+  public ContainerResumeEvent(ContainerId cId,
+      String diagnostic) {
+    super(cId, ContainerEventType.RESUME_CONTAINER);
+    this.diagnostic = diagnostic;
+  }
+
+  public String getDiagnostic() {
+    return this.diagnostic;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/01eb8ce4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerState.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/container/ContainerState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerState.java
index 91d1356..7c3fea8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerState.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerState.java
@@ -21,5 +21,6 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.container;
 public enum ContainerState {
   NEW, LOCALIZING, LOCALIZATION_FAILED, SCHEDULED, RUNNING, RELAUNCHING,
   REINITIALIZING, EXITED_WITH_SUCCESS, EXITED_WITH_FAILURE, KILLING,
-  CONTAINER_CLEANEDUP_AFTER_KILL, CONTAINER_RESOURCES_CLEANINGUP, DONE
+  CONTAINER_CLEANEDUP_AFTER_KILL, CONTAINER_RESOURCES_CLEANINGUP, DONE,
+  PAUSING, PAUSED, RESUMING
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/01eb8ce4/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 50443f3..bc6323a 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
@@ -75,6 +75,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Cont
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEventType;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerExitEvent;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerKillEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService;
@@ -86,6 +87,7 @@ import org.apache.hadoop.yarn.util.Apps;
 import org.apache.hadoop.yarn.util.AuxiliaryServiceHelper;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.yarn.util.ConverterUtils;
 
 public class ContainerLaunch implements Callable<Integer> {
 
@@ -106,8 +108,10 @@ public class ContainerLaunch implements Callable<Integer> {
   private final Configuration conf;
   private final Context context;
   private final ContainerManagerImpl containerManager;
-  
+
   protected AtomicBoolean containerAlreadyLaunched = new AtomicBoolean(false);
+  protected AtomicBoolean shouldPauseContainer = new AtomicBoolean(false);
+
   protected AtomicBoolean completed = new AtomicBoolean(false);
 
   private volatile boolean killedBeforeStart = false;
@@ -803,6 +807,90 @@ public class ContainerLaunch implements Callable<Integer> {
   }
 
   /**
+   * Pause the container.
+   * Cancels the launch if the container isn't launched yet. Otherwise asks the
+   * executor to pause the container.
+   * @throws IOException in case of errors.
+   */
+  @SuppressWarnings("unchecked") // dispatcher not typed
+  public void pauseContainer() throws IOException {
+    ContainerId containerId = container.getContainerId();
+    String containerIdStr = containerId.toString();
+    LOG.info("Pausing the container " + containerIdStr);
+
+    // The pause event is only handled if the container is in the running state
+    // (the container state machine), so we don't check for
+    // shouldLaunchContainer over here
+
+    if (!shouldPauseContainer.compareAndSet(false, true)) {
+      LOG.info("Container " + containerId + " not paused as "
+          + "resume already called");
+      return;
+    }
+
+    try {
+      // Pause the container
+      exec.pauseContainer(container);
+
+      // PauseContainer is a blocking call. We are here almost means the
+      // container is paused, so send out the event.
+      dispatcher.getEventHandler().handle(new ContainerEvent(
+          containerId,
+          ContainerEventType.CONTAINER_PAUSED));
+    } catch (Exception e) {
+      String message =
+          "Exception when trying to pause container " + containerIdStr
+              + ": " + StringUtils.stringifyException(e);
+      LOG.info(message);
+      container.handle(new ContainerKillEvent(container.getContainerId(),
+          ContainerExitStatus.PREEMPTED, "Container preempted as there was "
+          + " an exception in pausing it."));
+    }
+  }
+
+  /**
+   * Resume the container.
+   * Cancels the launch if the container isn't launched yet. Otherwise asks the
+   * executor to pause the container.
+   * @throws IOException in case of error.
+   */
+  @SuppressWarnings("unchecked") // dispatcher not typed
+  public void resumeContainer() throws IOException {
+    ContainerId containerId = container.getContainerId();
+    String containerIdStr = containerId.toString();
+    LOG.info("Resuming the container " + containerIdStr);
+
+    // The resume event is only handled if the container is in a paused state
+    // so we don't check for the launched flag here.
+
+    // paused flag will be set to true if process already paused
+    boolean alreadyPaused = !shouldPauseContainer.compareAndSet(false, true);
+    if (!alreadyPaused) {
+      LOG.info("Container " + containerIdStr + " not paused."
+          + " No resume necessary");
+      return;
+    }
+
+    // If the container has already started
+    try {
+        exec.resumeContainer(container);
+        // ResumeContainer is a blocking call. We are here almost means the
+        // container is resumed, so send out the event.
+        dispatcher.getEventHandler().handle(new ContainerEvent(
+            containerId,
+            ContainerEventType.CONTAINER_RESUMED));
+    } catch (Exception e) {
+      String message =
+          "Exception when trying to resume container " + containerIdStr
+              + ": " + StringUtils.stringifyException(e);
+      LOG.info(message);
+      container.handle(new ContainerKillEvent(container.getContainerId(),
+          ContainerExitStatus.PREEMPTED, "Container preempted as there was "
+          + " an exception in pausing it."));
+    }
+  }
+
+  /**
    * Loop through for a time-bounded interval waiting to
    * read the process id from a file generated by a running process.
    * @param pidFilePath File from which to read the process id

http://git-wip-us.apache.org/repos/asf/hadoop/blob/01eb8ce4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainersLauncher.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/ContainersLauncher.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/ContainersLauncher.java
index 25909b9..ca69712 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/ContainersLauncher.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/ContainersLauncher.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.concurrent.HadoopExecutors;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.event.Dispatcher;
@@ -41,6 +42,7 @@ import org.apache.hadoop.yarn.server.nodemanager.LocalDirsHandlerService;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -171,6 +173,36 @@ public class ContainersLauncher extends AbstractService
               + " with command " + signalEvent.getCommand());
         }
         break;
+      case PAUSE_CONTAINER:
+        ContainerLaunch launchedContainer = running.get(containerId);
+        if (launchedContainer == null) {
+          // Container not launched. So nothing needs to be done.
+          return;
+        }
+
+        // Pause the container
+        try {
+          launchedContainer.pauseContainer();
+        } catch (Exception e) {
+          LOG.info("Got exception while pausing container: " +
+            StringUtils.stringifyException(e));
+        }
+        break;
+      case RESUME_CONTAINER:
+        ContainerLaunch launchCont = running.get(containerId);
+        if (launchCont == null) {
+          // Container not launched. So nothing needs to be done.
+          return;
+        }
+
+        // Resume the container.
+        try {
+          launchCont.resumeContainer();
+        } catch (Exception e) {
+          LOG.info("Got exception while resuming container: " +
+            StringUtils.stringifyException(e));
+        }
+        break;
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/01eb8ce4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainersLauncherEventType.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/ContainersLauncherEventType.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/ContainersLauncherEventType.java
index 380a032..1054e06 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/ContainersLauncherEventType.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/ContainersLauncherEventType.java
@@ -25,4 +25,7 @@ public enum ContainersLauncherEventType {
   CLEANUP_CONTAINER, // The process(grp) itself.
   CLEANUP_CONTAINER_FOR_REINIT, // The process(grp) itself.
   SIGNAL_CONTAINER,
+  PAUSE_CONTAINER,
+  RESUME_CONTAINER
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/01eb8ce4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerSchedulerEventType.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/scheduler/ContainerSchedulerEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerSchedulerEventType.java
index 917eda0..a9cbf74 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerSchedulerEventType.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerSchedulerEventType.java
@@ -27,4 +27,5 @@ public enum ContainerSchedulerEventType {
   UPDATE_CONTAINER,
   // Producer: Node HB response - RM has asked to shed the queue
   SHED_QUEUED_CONTAINERS,
+  CONTAINER_PAUSED
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/01eb8ce4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.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/container/TestContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.java
index 33f4609..8909088 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.java
@@ -103,6 +103,7 @@ import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.junit.Assert;
 import org.junit.Test;
 import org.mockito.ArgumentMatcher;
+import org.mockito.Mockito;
 
 public class TestContainer {
 
@@ -207,6 +208,42 @@ public class TestContainer {
 
   @Test
   @SuppressWarnings("unchecked") // mocked generic
+  public void testContainerPauseAndResume() throws Exception {
+    WrappedContainer wc = null;
+    try {
+      wc = new WrappedContainer(13, 314159265358979L, 4344, "yak");
+      wc.initContainer();
+      wc.localizeResources();
+      int running = metrics.getRunningContainers();
+      wc.launchContainer();
+      assertEquals(running + 1, metrics.getRunningContainers());
+      reset(wc.localizerBus);
+      wc.pauseContainer();
+      assertEquals(ContainerState.PAUSED,
+          wc.c.getContainerState());
+      wc.resumeContainer();
+      assertEquals(ContainerState.RUNNING,
+          wc.c.getContainerState());
+      wc.containerKilledOnRequest();
+      assertEquals(ContainerState.EXITED_WITH_FAILURE,
+          wc.c.getContainerState());
+      assertNull(wc.c.getLocalizedResources());
+      verifyCleanupCall(wc);
+      int failed = metrics.getFailedContainers();
+      wc.containerResourcesCleanup();
+      assertEquals(ContainerState.DONE, wc.c.getContainerState());
+      assertEquals(failed + 1, metrics.getFailedContainers());
+      assertEquals(running, metrics.getRunningContainers());
+    }
+    finally {
+      if (wc != null) {
+        wc.finished();
+      }
+    }
+  }
+
+  @Test
+  @SuppressWarnings("unchecked") // mocked generic
   public void testCleanupOnFailure() throws Exception {
     WrappedContainer wc = null;
     try {
@@ -955,6 +992,8 @@ public class TestContainer {
       NodeStatusUpdater nodeStatusUpdater = mock(NodeStatusUpdater.class);
       when(context.getNodeStatusUpdater()).thenReturn(nodeStatusUpdater);
       ContainerExecutor executor = mock(ContainerExecutor.class);
+      Mockito.doNothing().when(executor).pauseContainer(any(Container.class));
+      Mockito.doNothing().when(executor).resumeContainer(any(Container.class));
       launcher =
           new ContainersLauncher(context, dispatcher, executor, null, null);
       // create a mock ExecutorService, which will not really launch
@@ -1143,6 +1182,18 @@ public class TestContainer {
       drainDispatcherEvents();
     }
 
+    public void pauseContainer() {
+      c.handle(new ContainerPauseEvent(cId,
+          "PauseRequest"));
+      drainDispatcherEvents();
+    }
+
+    public void resumeContainer() {
+      c.handle(new ContainerResumeEvent(cId,
+          "ResumeRequest"));
+      drainDispatcherEvents();
+    }
+
     public void containerKilledOnRequest() {
       int exitCode = ContainerExitStatus.KILLED_BY_RESOURCEMANAGER;
       String diagnosticMsg = "Container completed with exit code " + exitCode;


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


[42/50] [abbrv] hadoop git commit: YARN-7072. Add a new log aggregation file format controller. Contributed by Xuan Gong.

Posted by as...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fddabc2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/LogAggregationTFileController.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/LogAggregationTFileController.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/LogAggregationTFileController.java
index aaed538..989b326 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/LogAggregationTFileController.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/LogAggregationTFileController.java
@@ -27,7 +27,6 @@ import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
-import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.math3.util.Pair;
@@ -192,7 +191,7 @@ public class LogAggregationTFileController
           while (valueStream != null) {
             if (getAllContainers || (key.toString().equals(containerIdStr))) {
               if (createPrintStream) {
-                os = createPrintStream(
+                os = LogToolUtils.createPrintStream(
                     logRequest.getOutputLocalDir(),
                     thisNodeFile.getPath().getName(), key.toString());
               }
@@ -209,12 +208,7 @@ public class LogAggregationTFileController
                           Times.format(thisNodeFile.getModificationTime()),
                           valueStream, os, buf,
                           ContainerLogAggregationType.AGGREGATED);
-                      StringBuilder sb = new StringBuilder();
-                      String endOfFile = "End of LogType:" + fileType;
-                      sb.append("\n" + endOfFile + "\n");
-                      sb.append(StringUtils.repeat("*", endOfFile.length() + 50)
-                          + "\n\n");
-                      byte[] b = sb.toString().getBytes(
+                      byte[] b = aggregatedLogSuffix(fileType).getBytes(
                           Charset.forName("UTF-8"));
                       os.write(b, 0, b.length);
                       findLogs = true;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fddabc2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/TestLogAggregationIndexFileController.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/TestLogAggregationIndexFileController.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/TestLogAggregationIndexFileController.java
new file mode 100644
index 0000000..5f61710
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/TestLogAggregationIndexFileController.java
@@ -0,0 +1,316 @@
+/**
+ * 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.logaggregation.filecontroller.ifile;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.io.Writer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.logaggregation.ContainerLogMeta;
+import org.apache.hadoop.yarn.logaggregation.ContainerLogsRequest;
+import org.apache.hadoop.yarn.logaggregation.LogAggregationUtils;
+import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogKey;
+import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogValue;
+import org.apache.hadoop.yarn.logaggregation.ContainerLogFileInfo;
+import org.apache.hadoop.yarn.logaggregation.filecontroller.LogAggregationFileControllerContext;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Function test for {@link LogAggregationIndexFileController}.
+ *
+ */
+public class TestLogAggregationIndexFileController {
+
+  private final String rootLocalLogDir = "target/LocalLogs";
+  private final Path rootLocalLogDirPath = new Path(rootLocalLogDir);
+  private final String remoteLogDir = "target/remote-app";
+  private static final FsPermission LOG_FILE_UMASK = FsPermission
+      .createImmutable((short) (0777));
+  private static final UserGroupInformation USER_UGI = UserGroupInformation
+      .createRemoteUser("testUser");
+  private FileSystem fs;
+  private Configuration conf;
+  private ApplicationId appId;
+  private ContainerId containerId;
+  private NodeId nodeId;
+
+  private ByteArrayOutputStream sysOutStream;
+  private PrintStream sysOut;
+
+  private ByteArrayOutputStream sysErrStream;
+  private PrintStream sysErr;
+
+  @Before
+  public void setUp() throws IOException {
+    appId = ApplicationId.newInstance(123456, 1);
+    ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(
+        appId, 1);
+    containerId = ContainerId.newContainerId(attemptId, 1);
+    nodeId = NodeId.newInstance("localhost", 9999);
+    conf = new Configuration();
+    conf.set("yarn.log-aggregation.Indexed.remote-app-log-dir",
+        remoteLogDir);
+    conf.set("yarn.log-aggregation.Indexed.remote-app-log-dir-suffix",
+        "logs");
+    conf.set(YarnConfiguration.NM_LOG_AGG_COMPRESSION_TYPE, "gz");
+    fs = FileSystem.get(conf);
+    sysOutStream = new ByteArrayOutputStream();
+    sysOut =  new PrintStream(sysOutStream);
+    System.setOut(sysOut);
+
+    sysErrStream = new ByteArrayOutputStream();
+    sysErr = new PrintStream(sysErrStream);
+    System.setErr(sysErr);
+  }
+
+  @After
+  public void teardown() throws Exception {
+    fs.delete(rootLocalLogDirPath, true);
+    fs.delete(new Path(remoteLogDir), true);
+  }
+
+  @Test(timeout = 15000)
+  public void testLogAggregationIndexFileFormat() throws Exception {
+    if (fs.exists(rootLocalLogDirPath)) {
+      fs.delete(rootLocalLogDirPath, true);
+    }
+    assertTrue(fs.mkdirs(rootLocalLogDirPath));
+
+    Path appLogsDir = new Path(rootLocalLogDirPath, appId.toString());
+    if (fs.exists(appLogsDir)) {
+      fs.delete(appLogsDir, true);
+    }
+    assertTrue(fs.mkdirs(appLogsDir));
+
+    List<String> logTypes = new ArrayList<String>();
+    logTypes.add("syslog");
+    logTypes.add("stdout");
+    logTypes.add("stderr");
+
+    Set<File> files = new HashSet<>();
+
+    LogKey key1 = new LogKey(containerId.toString());
+
+    for(String logType : logTypes) {
+      File file = createAndWriteLocalLogFile(containerId, appLogsDir,
+          logType);
+      files.add(file);
+    }
+    LogValue value = mock(LogValue.class);
+    when(value.getPendingLogFilesToUploadForThisContainer()).thenReturn(files);
+
+    LogAggregationIndexedFileController fileFormat
+        = new LogAggregationIndexedFileController();
+    fileFormat.initialize(conf, "Indexed");
+
+    Map<ApplicationAccessType, String> appAcls = new HashMap<>();
+    Path appDir = fileFormat.getRemoteAppLogDir(appId,
+        USER_UGI.getShortUserName());
+    if (fs.exists(appDir)) {
+      fs.delete(appDir, true);
+    }
+    assertTrue(fs.mkdirs(appDir));
+
+    Path logPath = fileFormat.getRemoteNodeLogFileForApp(
+        appId, USER_UGI.getShortUserName(), nodeId);
+    LogAggregationFileControllerContext context =
+        new LogAggregationFileControllerContext(
+            logPath, logPath, true, 1000, appId, appAcls, nodeId, USER_UGI);
+    // initialize the writer
+    fileFormat.initializeWriter(context);
+
+    fileFormat.write(key1, value);
+    LogAggregationFileControllerContext record = mock(
+        LogAggregationFileControllerContext.class);
+    fileFormat.postWrite(record);
+    fileFormat.closeWriter();
+
+    ContainerLogsRequest logRequest = new ContainerLogsRequest();
+    logRequest.setAppId(appId);
+    logRequest.setNodeId(nodeId.toString());
+    logRequest.setAppOwner(USER_UGI.getShortUserName());
+    logRequest.setContainerId(containerId.toString());
+    logRequest.setBytes(Long.MAX_VALUE);
+    List<ContainerLogMeta> meta = fileFormat.readAggregatedLogsMeta(
+        logRequest);
+    Assert.assertTrue(meta.size() == 1);
+    List<String> fileNames = new ArrayList<>();
+    for (ContainerLogMeta log : meta) {
+      Assert.assertTrue(log.getContainerId().equals(containerId.toString()));
+      Assert.assertTrue(log.getNodeId().equals(nodeId.toString()));
+      Assert.assertTrue(log.getContainerLogMeta().size() == 3);
+      for (ContainerLogFileInfo file : log.getContainerLogMeta()) {
+        fileNames.add(file.getFileName());
+      }
+    }
+    fileNames.removeAll(logTypes);
+    Assert.assertTrue(fileNames.isEmpty());
+
+    boolean foundLogs = fileFormat.readAggregatedLogs(logRequest, System.out);
+    Assert.assertTrue(foundLogs);
+    for (String logType : logTypes) {
+      Assert.assertTrue(sysOutStream.toString().contains(logMessage(
+          containerId, logType)));
+    }
+    sysOutStream.reset();
+
+    // create a checksum file
+    Path checksumFile = new Path(fileFormat.getRemoteAppLogDir(
+        appId, USER_UGI.getShortUserName()),
+        LogAggregationUtils.getNodeString(nodeId)
+        + LogAggregationIndexedFileController.CHECK_SUM_FILE_SUFFIX);
+    FSDataOutputStream fInput = null;
+    try {
+      fInput = FileSystem.create(fs, checksumFile, LOG_FILE_UMASK);
+      fInput.writeLong(0);
+    } finally {
+      IOUtils.closeQuietly(fInput);
+    }
+    meta = fileFormat.readAggregatedLogsMeta(
+        logRequest);
+    Assert.assertTrue(meta.size() == 0);
+    foundLogs = fileFormat.readAggregatedLogs(logRequest, System.out);
+    Assert.assertFalse(foundLogs);
+    sysOutStream.reset();
+    fs.delete(checksumFile, false);
+    Assert.assertFalse(fs.exists(checksumFile));
+
+    List<String> newLogTypes = new ArrayList<>(logTypes);
+    files.clear();
+    newLogTypes.add("test1");
+    files.add(createAndWriteLocalLogFile(containerId, appLogsDir,
+        "test1"));
+    newLogTypes.add("test2");
+    files.add(createAndWriteLocalLogFile(containerId, appLogsDir,
+        "test2"));
+    LogValue value2 = mock(LogValue.class);
+    when(value2.getPendingLogFilesToUploadForThisContainer())
+        .thenReturn(files);
+
+    // initialize the writer
+    fileFormat.initializeWriter(context);
+    fileFormat.write(key1, value2);
+    fileFormat.closeWriter();
+
+    // We did not call postWriter which we would keep the checksum file.
+    // We can only get the logs/logmeta from the first write.
+    fileFormat.readAggregatedLogsMeta(
+        logRequest);
+    Assert.assertEquals(meta.size(), meta.size(), 1);
+    for (ContainerLogMeta log : meta) {
+      Assert.assertTrue(log.getContainerId().equals(containerId.toString()));
+      Assert.assertTrue(log.getNodeId().equals(nodeId.toString()));
+      Assert.assertTrue(log.getContainerLogMeta().size() == 3);
+      for (ContainerLogFileInfo file : log.getContainerLogMeta()) {
+        fileNames.add(file.getFileName());
+      }
+    }
+    fileNames.removeAll(logTypes);
+    Assert.assertTrue(fileNames.isEmpty());
+    foundLogs = fileFormat.readAggregatedLogs(logRequest, System.out);
+    Assert.assertTrue(foundLogs);
+    for (String logType : logTypes) {
+      Assert.assertTrue(sysOutStream.toString().contains(logMessage(
+          containerId, logType)));
+    }
+    Assert.assertFalse(sysOutStream.toString().contains(logMessage(
+        containerId, "test1")));
+    Assert.assertFalse(sysOutStream.toString().contains(logMessage(
+        containerId, "test2")));
+    sysOutStream.reset();
+
+    // Call postWrite and we should get all logs/logmetas for both
+    // first write and second write
+    fileFormat.initializeWriter(context);
+    fileFormat.write(key1, value2);
+    fileFormat.postWrite(record);
+    fileFormat.closeWriter();
+    fileFormat.readAggregatedLogsMeta(
+            logRequest);
+    Assert.assertEquals(meta.size(), meta.size(), 2);
+    for (ContainerLogMeta log : meta) {
+      Assert.assertTrue(log.getContainerId().equals(containerId.toString()));
+      Assert.assertTrue(log.getNodeId().equals(nodeId.toString()));
+      for (ContainerLogFileInfo file : log.getContainerLogMeta()) {
+        fileNames.add(file.getFileName());
+      }
+    }
+    fileNames.removeAll(newLogTypes);
+    Assert.assertTrue(fileNames.isEmpty());
+    foundLogs = fileFormat.readAggregatedLogs(logRequest, System.out);
+    Assert.assertTrue(foundLogs);
+    for (String logType : newLogTypes) {
+      Assert.assertTrue(sysOutStream.toString().contains(logMessage(
+          containerId, logType)));
+    }
+    sysOutStream.reset();
+  }
+
+  private File createAndWriteLocalLogFile(ContainerId containerId,
+      Path localLogDir, String logType) throws IOException {
+    File file = new File(localLogDir.toString(), logType);
+    if (file.exists()) {
+      file.delete();
+    }
+    file.createNewFile();
+    Writer writer = null;
+    try {
+      writer = new FileWriter(file);
+      writer.write(logMessage(containerId, logType));
+      writer.close();
+      return file;
+    } finally {
+      IOUtils.closeQuietly(writer);
+    }
+  }
+
+  private String logMessage(ContainerId containerId, String logType) {
+    StringBuilder sb = new StringBuilder();
+    sb.append("Hello " + containerId + " in " + logType + "!");
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fddabc2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java
index dc692a5..bca5d5c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java
@@ -53,7 +53,7 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.logaggregation.ContainerLogAggregationType;
-import org.apache.hadoop.yarn.logaggregation.PerContainerLogFileInfo;
+import org.apache.hadoop.yarn.logaggregation.ContainerLogFileInfo;
 import org.apache.hadoop.yarn.logaggregation.TestContainerLogsUtils;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryClientService;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryManagerOnTimelineStore;
@@ -851,7 +851,7 @@ public class TestAHSWebServices extends JerseyTestBase {
     for (ContainerLogsInfo logInfo : responseText) {
       if(logInfo.getLogType().equals(
           ContainerLogAggregationType.AGGREGATED.toString())) {
-        List<PerContainerLogFileInfo> logMeta = logInfo
+        List<ContainerLogFileInfo> logMeta = logInfo
             .getContainerLogsInfo();
         assertTrue(logMeta.size() == 1);
         assertEquals(logMeta.get(0).getFileName(), fileName);
@@ -879,7 +879,7 @@ public class TestAHSWebServices extends JerseyTestBase {
     for (ContainerLogsInfo logInfo : responseText) {
       if(logInfo.getLogType().equals(
           ContainerLogAggregationType.AGGREGATED.toString())) {
-        List<PerContainerLogFileInfo> logMeta = logInfo
+        List<ContainerLogFileInfo> logMeta = logInfo
             .getContainerLogsInfo();
         assertTrue(logMeta.size() == 1);
         assertEquals(logMeta.get(0).getFileName(), fileName);
@@ -917,7 +917,7 @@ public class TestAHSWebServices extends JerseyTestBase {
     assertTrue(responseText.size() == 1);
     assertEquals(responseText.get(0).getLogType(),
         ContainerLogAggregationType.AGGREGATED.toString());
-    List<PerContainerLogFileInfo> logMeta = responseText.get(0)
+    List<ContainerLogFileInfo> logMeta = responseText.get(0)
         .getContainerLogsInfo();
     assertTrue(logMeta.size() == 1);
     assertEquals(logMeta.get(0).getFileName(), fileName);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fddabc2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/ContainerLogsInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/ContainerLogsInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/ContainerLogsInfo.java
index bc3ab39..1bb0408 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/ContainerLogsInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/ContainerLogsInfo.java
@@ -27,14 +27,14 @@ import javax.xml.bind.annotation.XmlRootElement;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.logaggregation.ContainerLogMeta;
 import org.apache.hadoop.yarn.logaggregation.ContainerLogAggregationType;
-import org.apache.hadoop.yarn.logaggregation.PerContainerLogFileInfo;
+import org.apache.hadoop.yarn.logaggregation.ContainerLogFileInfo;
 
 /**
  * {@code ContainerLogsInfo} includes the log meta-data of containers.
  * <p>
  * The container log meta-data includes details such as:
  * <ul>
- *   <li>A list of {@link PerContainerLogFileInfo}.</li>
+ *   <li>A list of {@link ContainerLogFileInfo}.</li>
  *   <li>The container Id.</li>
  *   <li>The NodeManager Id.</li>
  *   <li>The logType: could be local or aggregated</li>
@@ -46,7 +46,7 @@ import org.apache.hadoop.yarn.logaggregation.PerContainerLogFileInfo;
 public class ContainerLogsInfo {
 
   @XmlElement(name = "containerLogInfo")
-  protected List<PerContainerLogFileInfo> containerLogsInfo;
+  protected List<ContainerLogFileInfo> containerLogsInfo;
 
   @XmlElement(name = "logAggregationType")
   protected String logType;
@@ -62,14 +62,14 @@ public class ContainerLogsInfo {
 
   public ContainerLogsInfo(ContainerLogMeta logMeta,
       ContainerLogAggregationType logType) throws YarnException {
-    this.containerLogsInfo = new ArrayList<PerContainerLogFileInfo>(
+    this.containerLogsInfo = new ArrayList<ContainerLogFileInfo>(
         logMeta.getContainerLogMeta());
     this.logType = logType.toString();
     this.containerId = logMeta.getContainerId();
     this.nodeId = logMeta.getNodeId();
   }
 
-  public List<PerContainerLogFileInfo> getContainerLogsInfo() {
+  public List<ContainerLogFileInfo> getContainerLogsInfo() {
     return this.containerLogsInfo;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fddabc2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/dao/NMContainerLogsInfo.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/webapp/dao/NMContainerLogsInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/dao/NMContainerLogsInfo.java
index 5415e04..193ec62 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/dao/NMContainerLogsInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/dao/NMContainerLogsInfo.java
@@ -27,7 +27,7 @@ import javax.xml.bind.annotation.XmlRootElement;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.logaggregation.ContainerLogAggregationType;
-import org.apache.hadoop.yarn.logaggregation.PerContainerLogFileInfo;
+import org.apache.hadoop.yarn.logaggregation.ContainerLogFileInfo;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.webapp.ContainerLogsUtils;
 import org.apache.hadoop.yarn.server.webapp.dao.ContainerLogsInfo;
@@ -55,10 +55,10 @@ public class NMContainerLogsInfo extends ContainerLogsInfo {
         containerId, remoteUser, nmContext);
   }
 
-  private static List<PerContainerLogFileInfo> getContainerLogsInfo(
+  private static List<ContainerLogFileInfo> getContainerLogsInfo(
       ContainerId id, String remoteUser, Context nmContext)
       throws YarnException {
-    List<PerContainerLogFileInfo> logFiles = new ArrayList<>();
+    List<ContainerLogFileInfo> logFiles = new ArrayList<>();
     List<File> logDirs = ContainerLogsUtils.getContainerLogDirs(
         id, remoteUser, nmContext);
     for (File containerLogsDir : logDirs) {
@@ -66,7 +66,7 @@ public class NMContainerLogsInfo extends ContainerLogsInfo {
       if (logs != null) {
         for (File log : logs) {
           if (log.isFile()) {
-            PerContainerLogFileInfo logMeta = new PerContainerLogFileInfo(
+            ContainerLogFileInfo logMeta = new ContainerLogFileInfo(
                 log.getName(), Long.toString(log.length()),
                 Times.format(log.lastModified()));
             logFiles.add(logMeta);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fddabc2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.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/webapp/TestNMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java
index 1641171..fbab34a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java
@@ -51,7 +51,7 @@ import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.logaggregation.ContainerLogAggregationType;
-import org.apache.hadoop.yarn.logaggregation.PerContainerLogFileInfo;
+import org.apache.hadoop.yarn.logaggregation.ContainerLogFileInfo;
 import org.apache.hadoop.yarn.logaggregation.TestContainerLogsUtils;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.LocalDirsHandlerService;
@@ -537,7 +537,7 @@ public class TestNMWebServices extends JerseyTestBase {
     assertTrue(responseList.size() == 1);
     assertEquals(responseList.get(0).getLogType(),
         ContainerLogAggregationType.LOCAL.toString());
-    List<PerContainerLogFileInfo> logMeta = responseList.get(0)
+    List<ContainerLogFileInfo> logMeta = responseList.get(0)
         .getContainerLogsInfo();
     assertTrue(logMeta.size() == 1);
     assertEquals(logMeta.get(0).getFileName(), filename);
@@ -564,13 +564,13 @@ public class TestNMWebServices extends JerseyTestBase {
       for (ContainerLogsInfo logInfo : responseList) {
         if(logInfo.getLogType().equals(
             ContainerLogAggregationType.AGGREGATED.toString())) {
-          List<PerContainerLogFileInfo> meta = logInfo.getContainerLogsInfo();
+          List<ContainerLogFileInfo> meta = logInfo.getContainerLogsInfo();
           assertTrue(meta.size() == 1);
           assertEquals(meta.get(0).getFileName(), aggregatedLogFile);
         } else {
           assertEquals(logInfo.getLogType(),
               ContainerLogAggregationType.LOCAL.toString());
-          List<PerContainerLogFileInfo> meta = logInfo.getContainerLogsInfo();
+          List<ContainerLogFileInfo> meta = logInfo.getContainerLogsInfo();
           assertTrue(meta.size() == 1);
           assertEquals(meta.get(0).getFileName(), filename);
         }


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


[04/50] [abbrv] hadoop git commit: YARN-7164. TestAMRMClientOnRMRestart fails sporadically with bind address in use. Contributed by Jason Lowe

Posted by as...@apache.org.
YARN-7164. TestAMRMClientOnRMRestart fails sporadically with bind address in use. Contributed by Jason Lowe


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

Branch: refs/heads/YARN-5972
Commit: 704267cb4909bf9968dd30067617f74543e07ffb
Parents: 63231a2
Author: Jason Lowe <jl...@apache.org>
Authored: Wed Sep 6 15:23:49 2017 -0500
Committer: Jason Lowe <jl...@apache.org>
Committed: Wed Sep 6 15:23:49 2017 -0500

----------------------------------------------------------------------
 .../hadoop/yarn/client/api/impl/TestAMRMClientOnRMRestart.java   | 4 +---
 1 file changed, 1 insertion(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/704267cb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientOnRMRestart.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientOnRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientOnRMRestart.java
index fa3c6af..337d7d4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientOnRMRestart.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientOnRMRestart.java
@@ -29,7 +29,6 @@ import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.net.NetworkTopology;
-import org.apache.hadoop.net.ServerSocketUtil;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
@@ -93,6 +92,7 @@ public class TestAMRMClientOnRMRestart {
         YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
     conf.setBoolean(YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, true);
     conf.setLong(YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_SCHEDULING_WAIT_MS, 0);
+    conf.set(YarnConfiguration.RM_SCHEDULER_ADDRESS, "0.0.0.0:0");
   }
 
   @AfterClass
@@ -496,8 +496,6 @@ public class TestAMRMClientOnRMRestart {
       .getMasterKey().getKeyId() == newToken.decodeIdentifier().getKeyId());
 
     // start 2nd RM
-    conf.set(YarnConfiguration.RM_SCHEDULER_ADDRESS, "0.0.0.0:"
-        + ServerSocketUtil.getPort(45020, 10));
     final MyResourceManager2 rm2 = new MyResourceManager2(conf, memStore);
     rm2.start();
     nm1.setResourceTrackerService(rm2.getResourceTrackerService());


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


[30/50] [abbrv] hadoop git commit: YARN-5330. SharingPolicy enhancements required to support recurring reservations in ReservationSystem. (Carlo Curino via Subru).

Posted by as...@apache.org.
YARN-5330. SharingPolicy enhancements required to support recurring reservations in ReservationSystem. (Carlo Curino via Subru).


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

Branch: refs/heads/YARN-5972
Commit: fa6137501c1499ae33f6e0e2adc31671a7e782dc
Parents: 56d93d2
Author: Subru Krishnan <su...@apache.org>
Authored: Thu Sep 7 19:07:17 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Thu Sep 7 19:07:17 2017 -0700

----------------------------------------------------------------------
 .../reservation/CapacityOverTimePolicy.java     |  32 +-
 .../reservation/NoOverCommitPolicy.java         |   8 +-
 .../reservation/BaseSharingPolicyTest.java      | 189 +++++++++++
 .../reservation/ReservationSystemTestUtil.java  |  28 +-
 .../reservation/TestCapacityOverTimePolicy.java | 339 +++++--------------
 .../reservation/TestNoOverCommitPolicy.java     | 185 +++-------
 6 files changed, 388 insertions(+), 393 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa613750/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacityOverTimePolicy.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/reservation/CapacityOverTimePolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacityOverTimePolicy.java
index bb1a4e8..acd5774 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacityOverTimePolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacityOverTimePolicy.java
@@ -95,26 +95,29 @@ public class CapacityOverTimePolicy extends NoOverCommitPolicy {
       throw new PlanningQuotaException(p);
     }
 
+    long checkStart = reservation.getStartTime() - validWindow;
+    long checkEnd = reservation.getEndTime() + validWindow;
+
     //---- check for integral violations of capacity --------
 
     // Gather a view of what to check (curr allocation of user, minus old
     // version of this reservation, plus new version)
     RLESparseResourceAllocation consumptionForUserOverTime =
         plan.getConsumptionForUserOverTime(reservation.getUser(),
-            reservation.getStartTime() - validWindow,
-            reservation.getEndTime() + validWindow);
+            checkStart, checkEnd);
 
     ReservationAllocation old =
         plan.getReservationById(reservation.getReservationId());
     if (old != null) {
-      consumptionForUserOverTime = RLESparseResourceAllocation
-          .merge(plan.getResourceCalculator(), plan.getTotalCapacity(),
-              consumptionForUserOverTime, old.getResourcesOverTime(),
-              RLEOperator.add, reservation.getStartTime() - validWindow,
-              reservation.getEndTime() + validWindow);
+      consumptionForUserOverTime =
+          RLESparseResourceAllocation.merge(plan.getResourceCalculator(),
+              plan.getTotalCapacity(), consumptionForUserOverTime,
+              old.getResourcesOverTime(checkStart, checkEnd), RLEOperator.add,
+              checkStart, checkEnd);
     }
 
-    RLESparseResourceAllocation resRLE = reservation.getResourcesOverTime();
+    RLESparseResourceAllocation resRLE =
+        reservation.getResourcesOverTime(checkStart, checkEnd);
 
     RLESparseResourceAllocation toCheck = RLESparseResourceAllocation
         .merge(plan.getResourceCalculator(), plan.getTotalCapacity(),
@@ -191,11 +194,11 @@ public class CapacityOverTimePolicy extends NoOverCommitPolicy {
 
     // compare using merge() limit with integral
     try {
-      RLESparseResourceAllocation
-          .merge(plan.getResourceCalculator(), plan.getTotalCapacity(),
-              targetLimit, integral, RLEOperator.subtractTestNonNegative,
-              reservation.getStartTime() - validWindow,
-              reservation.getEndTime() + validWindow);
+
+      RLESparseResourceAllocation.merge(plan.getResourceCalculator(),
+          plan.getTotalCapacity(), targetLimit, integral,
+          RLEOperator.subtractTestNonNegative, checkStart, checkEnd);
+
     } catch (PlanningException p) {
       throw new PlanningQuotaException(
           "Integral (avg over time) quota capacity " + maxAvg
@@ -240,7 +243,8 @@ public class CapacityOverTimePolicy extends NoOverCommitPolicy {
     if (old != null) {
       used = RLESparseResourceAllocation.merge(plan.getResourceCalculator(),
           Resources.clone(plan.getTotalCapacity()), used,
-          old.getResourcesOverTime(), RLEOperator.subtract, start, end);
+          old.getResourcesOverTime(start, end), RLEOperator.subtract, start,
+          end);
     }
 
     instRLEQuota = RLESparseResourceAllocation

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa613750/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/NoOverCommitPolicy.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/reservation/NoOverCommitPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/NoOverCommitPolicy.java
index 49d4702..98ef582 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/NoOverCommitPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/NoOverCommitPolicy.java
@@ -40,13 +40,17 @@ public class NoOverCommitPolicy implements SharingPolicy {
 
     RLESparseResourceAllocation available = plan.getAvailableResourceOverTime(
         reservation.getUser(), reservation.getReservationId(),
-        reservation.getStartTime(), reservation.getEndTime(), 0);
+        reservation.getStartTime(), reservation.getEndTime(),
+        reservation.getPeriodicity());
 
     // test the reservation does not exceed what is available
     try {
+
+      RLESparseResourceAllocation ask = reservation.getResourcesOverTime(
+              reservation.getStartTime(), reservation.getEndTime());
       RLESparseResourceAllocation
           .merge(plan.getResourceCalculator(), plan.getTotalCapacity(),
-              available, reservation.getResourcesOverTime(),
+              available, ask,
               RLESparseResourceAllocation.RLEOperator.subtractTestNonNegative,
               reservation.getStartTime(), reservation.getEndTime());
     } catch (PlanningException p) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa613750/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/BaseSharingPolicyTest.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/reservation/BaseSharingPolicyTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/BaseSharingPolicyTest.java
new file mode 100644
index 0000000..294564a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/BaseSharingPolicyTest.java
@@ -0,0 +1,189 @@
+/*******************************************************************************
+ *   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.reservation;
+
+import static junit.framework.TestCase.fail;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.hadoop.yarn.api.records.ReservationDefinition;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.planning.ReservationAgent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
+import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.Resources;
+import org.junit.Before;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import net.jcip.annotations.NotThreadSafe;
+
+/**
+ * This class is a base test for {@code SharingPolicy} implementors.
+ */
+@RunWith(value = Parameterized.class)
+@NotThreadSafe
+@SuppressWarnings("VisibilityModifier")
+public abstract class BaseSharingPolicyTest {
+
+  @Parameterized.Parameter(value = 0)
+  public long duration;
+
+  @Parameterized.Parameter(value = 1)
+  public double height;
+
+  @Parameterized.Parameter(value = 2)
+  public int numSubmissions;
+
+  @Parameterized.Parameter(value = 3)
+  public String recurrenceExpression;
+
+  @Parameterized.Parameter(value = 4)
+  public Class expectedError;
+
+  private long step;
+  private long initTime;
+
+  private InMemoryPlan plan;
+  private ReservationAgent mAgent;
+  private Resource minAlloc;
+  private ResourceCalculator res;
+  private Resource maxAlloc;
+
+  private int totCont = 1000;
+
+  protected ReservationSchedulerConfiguration conf;
+
+  @Before
+  public void setup() {
+    // 1 sec step
+    step = 1000L;
+    initTime = System.currentTimeMillis();
+
+    minAlloc = Resource.newInstance(1024, 1);
+    res = new DefaultResourceCalculator();
+    maxAlloc = Resource.newInstance(1024 * 8, 8);
+
+    mAgent = mock(ReservationAgent.class);
+
+    QueueMetrics rootQueueMetrics = mock(QueueMetrics.class);
+    Resource clusterResource =
+        ReservationSystemTestUtil.calculateClusterResource(totCont);
+
+    // invoke implementors initialization of policy
+    SharingPolicy policy = getInitializedPolicy();
+
+    RMContext context = ReservationSystemTestUtil.createMockRMContext();
+
+    plan = new InMemoryPlan(rootQueueMetrics, policy, mAgent, clusterResource,
+        step, res, minAlloc, maxAlloc, "dedicated", null, true, context);
+  }
+
+  public void runTest() throws IOException, PlanningException {
+
+    long period = 1;
+    if (recurrenceExpression != null) {
+      period = Long.parseLong(recurrenceExpression);
+    }
+
+    try {
+      RLESparseResourceAllocation rle = generateRLEAlloc(period);
+
+      // Generate the intervalMap (trimming out-of-period entries)
+      Map<ReservationInterval, Resource> reservationIntervalResourceMap;
+      if (period > 1) {
+        rle = new PeriodicRLESparseResourceAllocation(rle, period);
+        reservationIntervalResourceMap =
+            ReservationSystemTestUtil.toAllocation(rle, 0, period);
+      } else {
+        reservationIntervalResourceMap = ReservationSystemTestUtil
+            .toAllocation(rle, Long.MIN_VALUE, Long.MAX_VALUE);
+      }
+
+      ReservationDefinition rDef =
+          ReservationSystemTestUtil.createSimpleReservationDefinition(
+              initTime % period, initTime % period + duration + 1, duration, 1,
+              recurrenceExpression);
+
+      // perform multiple submissions where required
+      for (int i = 0; i < numSubmissions; i++) {
+
+        long rstart = rle.getEarliestStartTime();
+        long rend = rle.getLatestNonNullTime();
+
+        InMemoryReservationAllocation resAlloc =
+            new InMemoryReservationAllocation(
+                ReservationSystemTestUtil.getNewReservationId(), rDef, "u1",
+                "dedicated", rstart, rend, reservationIntervalResourceMap, res,
+                minAlloc);
+
+        assertTrue(plan.toString(), plan.addReservation(resAlloc, false));
+      }
+      // fail if error was expected
+      if (expectedError != null) {
+        System.out.println(plan.toString());
+        fail();
+      }
+    } catch (Exception e) {
+      if (expectedError == null || !e.getClass().getCanonicalName()
+          .equals(expectedError.getCanonicalName())) {
+        // fail on unexpected errors
+        throw e;
+      }
+    }
+  }
+
+  private RLESparseResourceAllocation generateRLEAlloc(long period) {
+    RLESparseResourceAllocation rle =
+        new RLESparseResourceAllocation(new DefaultResourceCalculator());
+
+    Resource alloc = Resources.multiply(minAlloc, height * totCont);
+
+    // loop in case the periodicity of the reservation is smaller than LCM
+    long rStart = initTime % period;
+    long rEnd = initTime % period + duration;
+
+
+    // handle wrap-around
+    if (period > 1 && rEnd > period) {
+      long diff = rEnd - period;
+      rEnd = period;
+
+      // handle multiple wrap-arounds (e.g., 5h duration on a 2h periodicity)
+      if(duration > period) {
+        rle.addInterval(new ReservationInterval(0, period),
+            Resources.multiply(alloc, duration / period - 1));
+        rle.addInterval(new ReservationInterval(0, diff % period), alloc);
+      } else {
+        rle.addInterval(new ReservationInterval(0, diff), alloc);
+      }
+    }
+
+
+    rle.addInterval(new ReservationInterval(rStart, rEnd), alloc);
+    return rle;
+  }
+
+  public abstract SharingPolicy getInitializedPolicy();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa613750/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.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/reservation/ReservationSystemTestUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
index 5337e06..eef86a4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
@@ -1,4 +1,4 @@
-/*******************************************************************************
+/******************************************************************************
  * 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
@@ -14,7 +14,7 @@
  * 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.reservation;
 
 import static org.mockito.Matchers.any;
@@ -466,4 +466,28 @@ public class ReservationSystemTestUtil {
   public static Resource calculateClusterResource(int numContainers) {
     return Resource.newInstance(numContainers * 1024, numContainers);
   }
+
+
+  public static Map<ReservationInterval, Resource> toAllocation(
+      RLESparseResourceAllocation rle, long start, long end) {
+    Map<ReservationInterval, Resource> resAlloc = new TreeMap<>();
+
+    for (Map.Entry<Long, Resource> e : rle.getCumulative().entrySet()) {
+      Long nextKey = rle.getCumulative().higherKey(e.getKey());
+      if (nextKey == null) {
+        break;
+      } else {
+        if (e.getKey() >= start && e.getKey() <= end && nextKey >= start
+            && nextKey <= end) {
+          resAlloc.put(new ReservationInterval(e.getKey(), nextKey),
+              e.getValue());
+        }
+      }
+    }
+
+    return resAlloc;
+  }
+
+
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa613750/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacityOverTimePolicy.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/reservation/TestCapacityOverTimePolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacityOverTimePolicy.java
index 2dee60c..d054d3a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacityOverTimePolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacityOverTimePolicy.java
@@ -6,9 +6,9 @@
  *   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.
@@ -17,269 +17,118 @@
  *******************************************************************************/
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.mock;
-
 import java.io.IOException;
-import java.util.Map;
-import java.util.TreeMap;
+import java.util.Arrays;
+import java.util.Collection;
 
-import org.apache.hadoop.yarn.api.records.ReservationDefinition;
-import org.apache.hadoop.yarn.api.records.ReservationRequest;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import net.jcip.annotations.NotThreadSafe;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningQuotaException;
-import org.apache.hadoop.yarn.server.resourcemanager.reservation.planning.ReservationAgent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
-import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
-import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
-import org.junit.Assert;
-import org.junit.Before;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+/**
+ * This class tests the {@code CapacityOvertimePolicy} sharing policy.
+ */
+@RunWith(value = Parameterized.class)
+@NotThreadSafe
+@SuppressWarnings("VisibilityModifier")
+public class TestCapacityOverTimePolicy extends BaseSharingPolicyTest {
+
+  final static long ONEDAY = 86400 * 1000;
+  final static long ONEHOUR = 3600 * 1000;
+  final static long ONEMINUTE = 60 * 1000;
+  final static String TWODAYPERIOD = "7200000";
+  final static String ONEDAYPERIOD = "86400000";
+
+  @Parameterized.Parameters(name = "Duration {0}, height {1}," +
+          " submission {2}, periodic {3})")
+  public static Collection<Object[]> data() {
+    return Arrays.asList(new Object[][] {
+
+        // easy fit
+        {ONEHOUR, 0.25, 1, null, null },
+        {ONEHOUR, 0.25, 1, TWODAYPERIOD, null },
+        {ONEHOUR, 0.25, 1, ONEDAYPERIOD, null },
+
+        // instantaneous high, but fit integral and inst limits
+        {ONEMINUTE, 0.74, 1, null, null },
+        {ONEMINUTE, 0.74, 1, TWODAYPERIOD, null },
+        {ONEMINUTE, 0.74, 1, ONEDAYPERIOD, null },
+
+        // barely fit
+        {ONEHOUR, 0.76, 1, null, PlanningQuotaException.class },
+        {ONEHOUR, 0.76, 1, TWODAYPERIOD, PlanningQuotaException.class },
+        {ONEHOUR, 0.76, 1, ONEDAYPERIOD, PlanningQuotaException.class },
+
+        // overcommit with single reservation
+        {ONEHOUR, 1.1, 1, null, PlanningQuotaException.class },
+        {ONEHOUR, 1.1, 1, TWODAYPERIOD, PlanningQuotaException.class },
+        {ONEHOUR, 1.1, 1, ONEDAYPERIOD, PlanningQuotaException.class },
+
+        // barely fit with multiple reservations (instantaneously, lowering to
+        // 1min to fit integral)
+        {ONEMINUTE, 0.25, 3, null, null },
+        {ONEMINUTE, 0.25, 3, TWODAYPERIOD, null },
+        {ONEMINUTE, 0.25, 3, ONEDAYPERIOD, null },
+
+        // overcommit with multiple reservations (instantaneously)
+        {ONEMINUTE, 0.25, 4, null, PlanningQuotaException.class },
+        {ONEMINUTE, 0.25, 4, TWODAYPERIOD, PlanningQuotaException.class },
+        {ONEMINUTE, 0.25, 4, ONEDAYPERIOD, PlanningQuotaException.class },
+
+        // (non-periodic) reservation longer than window
+        {25 * ONEHOUR, 0.25, 1, null, PlanningQuotaException.class },
+        {25 * ONEHOUR, 0.25, 1, TWODAYPERIOD, PlanningQuotaException.class },
+        {25 * ONEHOUR, 0.25, 1, ONEDAYPERIOD, PlanningQuotaException.class },
+
+        // (non-periodic) reservation longer than window
+        {25 * ONEHOUR, 0.05, 5, null, PlanningQuotaException.class },
+        {25 * ONEHOUR, 0.05, 5, TWODAYPERIOD, PlanningQuotaException.class },
+        {25 * ONEHOUR, 0.05, 5, ONEDAYPERIOD, PlanningQuotaException.class },
+
+        // overcommit integral
+        {ONEDAY, 0.26, 1, null, PlanningQuotaException.class },
+        {2 * ONEHOUR, 0.26, 1, TWODAYPERIOD, PlanningQuotaException.class },
+        {2 * ONEDAY, 0.26, 1, ONEDAYPERIOD, PlanningQuotaException.class },
+
+        // overcommit integral
+        {ONEDAY / 2, 0.51, 1, null, PlanningQuotaException.class },
+        {2 * ONEHOUR / 2, 0.51, 1, TWODAYPERIOD,
+            PlanningQuotaException.class },
+        {2 * ONEDAY / 2, 0.51, 1, ONEDAYPERIOD, PlanningQuotaException.class }
+
+    });
+  }
 
-public class TestCapacityOverTimePolicy {
-
-  long timeWindow;
-  long step;
-  float avgConstraint;
-  float instConstraint;
-  long initTime;
-
-  InMemoryPlan plan;
-  ReservationAgent mAgent;
-  Resource minAlloc;
-  ResourceCalculator res;
-  Resource maxAlloc;
-
-  int totCont = 1000000;
-
-  @Before
-  public void setup() throws Exception {
+  @Override
+  public SharingPolicy getInitializedPolicy() {
 
     // 24h window
-    timeWindow = 86400000L;
+    long timeWindow = 86400000L;
+
     // 1 sec step
-    step = 1000L;
+    long step = 1000L;
 
     // 25% avg cap on capacity
-    avgConstraint = 25;
+    float avgConstraint = 25;
 
     // 70% instantaneous cap on capacity
-    instConstraint = 70;
+    float instConstraint = 75;
 
-    initTime = System.currentTimeMillis();
-    minAlloc = Resource.newInstance(1024, 1);
-    res = new DefaultResourceCalculator();
-    maxAlloc = Resource.newInstance(1024 * 8, 8);
-
-    mAgent = mock(ReservationAgent.class);
-    QueueMetrics rootQueueMetrics = mock(QueueMetrics.class);
     String reservationQ =
         ReservationSystemTestUtil.getFullReservationQueueName();
-    Resource clusterResource =
-        ReservationSystemTestUtil.calculateClusterResource(totCont);
-    ReservationSchedulerConfiguration conf =
-        ReservationSystemTestUtil.createConf(reservationQ, timeWindow,
-            instConstraint, avgConstraint);
+    conf = ReservationSystemTestUtil.createConf(reservationQ, timeWindow,
+        instConstraint, avgConstraint);
     CapacityOverTimePolicy policy = new CapacityOverTimePolicy();
     policy.init(reservationQ, conf);
-    RMContext context = ReservationSystemTestUtil.createMockRMContext();
-
-    plan =
-        new InMemoryPlan(rootQueueMetrics, policy, mAgent,
-            clusterResource, step, res, minAlloc, maxAlloc,
-            "dedicated", null, true, context);
-  }
-
-  public int[] generateData(int length, int val) {
-    int[] data = new int[length];
-    for (int i = 0; i < length; i++) {
-      data[i] = val;
-    }
-    return data;
-  }
-
-  @Test
-  public void testSimplePass() throws IOException, PlanningException {
-    // generate allocation that simply fit within all constraints
-    int[] f = generateData(3600, (int) Math.ceil(0.2 * totCont));
-
-    ReservationDefinition rDef =
-        ReservationSystemTestUtil.createSimpleReservationDefinition(
-            initTime, initTime + f.length + 1, f.length);
-    assertTrue(plan.toString(),
-        plan.addReservation(new InMemoryReservationAllocation(
-            ReservationSystemTestUtil.getNewReservationId(), rDef, "u1",
-            "dedicated", initTime, initTime + f.length,
-            ReservationSystemTestUtil.generateAllocation(initTime, step, f),
-            res, minAlloc), false));
-  }
-
-  @Test(expected = PlanningException.class)
-  public void testAllocationLargerThanValidWindow() throws IOException,
-      PlanningException {
-    // generate allocation that exceed the validWindow
-    int[] f = generateData(25*3600, (int) Math.ceil(0.69 * totCont));
-
-    ReservationDefinition rDef =
-        ReservationSystemTestUtil.createSimpleReservationDefinition(
-            initTime, initTime + f.length + 1, f.length);
-    assertTrue(plan.toString(),
-        plan.addReservation(new InMemoryReservationAllocation(
-            ReservationSystemTestUtil.getNewReservationId(), rDef, "u1",
-            "dedicated", initTime, initTime + f.length,
-            ReservationSystemTestUtil.generateAllocation(initTime, step, f),
-            res, minAlloc), false));
+    return policy;
   }
 
   @Test
-  public void testSimplePass2() throws IOException, PlanningException {
-    // generate allocation from single tenant that exceed avg momentarily but
-    // fit within
-    // max instantanesou
-    int[] f = generateData(3600, (int) Math.ceil(0.69 * totCont));
-    ReservationDefinition rDef =
-        ReservationSystemTestUtil.createSimpleReservationDefinition(
-            initTime, initTime + f.length + 1, f.length);
-    assertTrue(plan.toString(),
-        plan.addReservation(new InMemoryReservationAllocation(
-            ReservationSystemTestUtil.getNewReservationId(), rDef, "u1",
-            "dedicated", initTime, initTime + f.length,
-            ReservationSystemTestUtil.generateAllocation(initTime, step, f),
-            res, minAlloc), false));
-  }
-
-  @Test
-  public void testMultiTenantPass() throws IOException, PlanningException {
-    // generate allocation from multiple tenants that barely fit in tot capacity
-    int[] f = generateData(3600, (int) Math.ceil(0.25 * totCont));
-    ReservationDefinition rDef =
-        ReservationSystemTestUtil.createSimpleReservationDefinition(
-            initTime, initTime + f.length + 1, f.length);
-    for (int i = 0; i < 4; i++) {
-      assertTrue(plan.toString(),
-          plan.addReservation(new InMemoryReservationAllocation(
-              ReservationSystemTestUtil.getNewReservationId(), rDef, "u" + i,
-              "dedicated", initTime, initTime + f.length,
-              ReservationSystemTestUtil.generateAllocation(initTime, step, f),
-              res, minAlloc), false));
-    }
-  }
-
-  @Test(expected = PlanningQuotaException.class)
-  public void testMultiTenantFail() throws IOException, PlanningException {
-    // generate allocation from multiple tenants that exceed tot capacity
-    int[] f = generateData(3600, (int) Math.ceil(0.25 * totCont));
-    ReservationDefinition rDef =
-        ReservationSystemTestUtil.createSimpleReservationDefinition(
-            initTime, initTime + f.length + 1, f.length);
-    for (int i = 0; i < 5; i++) {
-      assertTrue(plan.toString(),
-          plan.addReservation(new InMemoryReservationAllocation(
-              ReservationSystemTestUtil.getNewReservationId(), rDef, "u" + i,
-              "dedicated", initTime, initTime + f.length,
-              ReservationSystemTestUtil.generateAllocation(initTime, step, f),
-              res, minAlloc), false));
-    }
-  }
-
-  @Test(expected = PlanningQuotaException.class)
-  public void testInstFail() throws IOException, PlanningException {
-    // generate allocation that exceed the instantaneous cap single-show
-    int[] f = generateData(3600, (int) Math.ceil(0.71 * totCont));
-    ReservationDefinition rDef =
-        ReservationSystemTestUtil.createSimpleReservationDefinition(
-            initTime, initTime + f.length + 1, f.length);
-    assertTrue(plan.toString(),
-        plan.addReservation(new InMemoryReservationAllocation(
-            ReservationSystemTestUtil.getNewReservationId(), rDef, "u1",
-            "dedicated", initTime, initTime + f.length,
-            ReservationSystemTestUtil.generateAllocation(initTime, step, f),
-            res, minAlloc), false));
-    Assert.fail("should not have accepted this");
-  }
-
-  @Test
-  public void testInstFailBySum() throws IOException, PlanningException {
-    // generate allocation that exceed the instantaneous cap by sum
-    int[] f = generateData(3600, (int) Math.ceil(0.3 * totCont));
-    ReservationDefinition rDef =
-        ReservationSystemTestUtil.createSimpleReservationDefinition(
-            initTime, initTime + f.length + 1, f.length);
-    assertTrue(plan.toString(),
-        plan.addReservation(new InMemoryReservationAllocation(
-            ReservationSystemTestUtil.getNewReservationId(), rDef, "u1",
-            "dedicated", initTime, initTime + f.length,
-            ReservationSystemTestUtil.generateAllocation(initTime, step, f),
-            res, minAlloc), false));
-    assertTrue(plan.toString(),
-        plan.addReservation(new InMemoryReservationAllocation(
-            ReservationSystemTestUtil.getNewReservationId(), rDef, "u1",
-            "dedicated", initTime, initTime + f.length,
-            ReservationSystemTestUtil.generateAllocation(initTime, step, f),
-            res, minAlloc), false));
-    try {
-      assertTrue(plan.toString(),
-          plan.addReservation(new InMemoryReservationAllocation(
-              ReservationSystemTestUtil.getNewReservationId(), rDef, "u1",
-              "dedicated", initTime, initTime + f.length,
-              ReservationSystemTestUtil.generateAllocation(initTime, step, f),
-              res, minAlloc), false));
-      Assert.fail();
-    } catch (PlanningQuotaException p) {
-      // expected
-    }
-  }
-
-  @Test(expected = PlanningQuotaException.class)
-  public void testFailAvg() throws IOException, PlanningException {
-    // generate an allocation which violates the 25% average single-shot
-    Map<ReservationInterval, Resource> req =
-        new TreeMap<ReservationInterval, Resource>();
-    long win = timeWindow / 2 + 100;
-    int cont = (int) Math.ceil(0.5 * totCont);
-    req.put(new ReservationInterval(initTime, initTime + win),
-        ReservationSystemUtil.toResource(
-            ReservationRequest.newInstance(Resource.newInstance(1024, 1),
-                cont)));
-    ReservationDefinition rDef =
-        ReservationSystemTestUtil.createSimpleReservationDefinition(
-            initTime, initTime + win, win);
-    assertTrue(plan.toString(),
-        plan.addReservation(new InMemoryReservationAllocation(
-            ReservationSystemTestUtil.getNewReservationId(), rDef, "u1",
-            "dedicated", initTime, initTime + win, req, res, minAlloc), false));
-  }
-
-  @Test
-  public void testFailAvgBySum() throws IOException, PlanningException {
-    // generate an allocation which violates the 25% average by sum
-    Map<ReservationInterval, Resource> req =
-        new TreeMap<ReservationInterval, Resource>();
-    long win = 86400000 / 4 + 1;
-    int cont = (int) Math.ceil(0.5 * totCont);
-    req.put(new ReservationInterval(initTime, initTime + win),
-        ReservationSystemUtil.toResource(ReservationRequest.newInstance(Resource
-            .newInstance(1024, 1), cont)));
-    ReservationDefinition rDef =
-        ReservationSystemTestUtil.createSimpleReservationDefinition(
-            initTime, initTime + win, win);
-    assertTrue(plan.toString(),
-        plan.addReservation(new InMemoryReservationAllocation(
-            ReservationSystemTestUtil.getNewReservationId(), rDef, "u1",
-            "dedicated", initTime, initTime + win, req, res, minAlloc), false));
-
-    try {
-      assertTrue(plan.toString(),
-          plan.addReservation(new InMemoryReservationAllocation(
-              ReservationSystemTestUtil.getNewReservationId(), null, "u1",
-              "dedicated", initTime, initTime + win, req, res, minAlloc), false));
-
-      Assert.fail("should not have accepted this");
-    } catch (PlanningQuotaException e) {
-      // expected
-    }
+  public void testAllocation() throws IOException, PlanningException {
+    runTest();
   }
 
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa613750/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestNoOverCommitPolicy.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/reservation/TestNoOverCommitPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestNoOverCommitPolicy.java
index c5edaf0..accdf24 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestNoOverCommitPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestNoOverCommitPolicy.java
@@ -6,9 +6,9 @@
  *   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.
@@ -17,145 +17,70 @@
  *******************************************************************************/
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.mock;
-
 import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
 
-import org.apache.hadoop.yarn.api.records.ReservationDefinition;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import net.jcip.annotations.NotThreadSafe;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.ResourceOverCommitException;
-import org.apache.hadoop.yarn.server.resourcemanager.reservation.planning.ReservationAgent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
-import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
-import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
-import org.junit.Before;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+/**
+ * This clas tests {@code NoOverCommitPolicy} sharing policy.
+ */
+@RunWith(value = Parameterized.class)
+@NotThreadSafe
+@SuppressWarnings("VisibilityModifier")
+public class TestNoOverCommitPolicy extends BaseSharingPolicyTest {
+
+  final static long ONEHOUR = 3600 * 1000;
+  final static String TWOHOURPERIOD = "7200000";
+
+  @Parameterized.Parameters(name = "Duration {0}, height {1}," +
+          " submissions {2}, periodic {3})")
+  public static Collection<Object[]> data() {
+    return Arrays.asList(new Object[][] {
+
+        // easy fit
+        {ONEHOUR, 0.25, 1, null, null },
+        {ONEHOUR, 0.25, 1, TWOHOURPERIOD, null },
+
+        // barely fit
+        {ONEHOUR, 1, 1, null, null },
+        {ONEHOUR, 1, 1, TWOHOURPERIOD, null },
+
+        // overcommit with single reservation
+        {ONEHOUR, 1.1, 1, null, ResourceOverCommitException.class },
+        {ONEHOUR, 1.1, 1, TWOHOURPERIOD, ResourceOverCommitException.class },
+
+        // barely fit with multiple reservations
+        {ONEHOUR, 0.25, 4, null, null },
+        {ONEHOUR, 0.25, 4, TWOHOURPERIOD, null },
+
+        // overcommit with multiple reservations
+        {ONEHOUR, 0.25, 5, null, ResourceOverCommitException.class },
+        {ONEHOUR, 0.25, 5, TWOHOURPERIOD, ResourceOverCommitException.class }
+
+    });
+  }
 
-public class TestNoOverCommitPolicy {
-
-  long step;
-  long initTime;
-
-  InMemoryPlan plan;
-  ReservationAgent mAgent;
-  Resource minAlloc;
-  ResourceCalculator res;
-  Resource maxAlloc;
-
-  int totCont = 1000000;
-
-  @Before
-  public void setup() throws Exception {
-
-    // 1 sec step
-    step = 1000L;
-
-    initTime = System.currentTimeMillis();
-    minAlloc = Resource.newInstance(1024, 1);
-    res = new DefaultResourceCalculator();
-    maxAlloc = Resource.newInstance(1024 * 8, 8);
-
-    mAgent = mock(ReservationAgent.class);
+  @Override
+  public SharingPolicy getInitializedPolicy() {
     String reservationQ =
         ReservationSystemTestUtil.getFullReservationQueueName();
-    QueueMetrics rootQueueMetrics = mock(QueueMetrics.class);
-    Resource clusterResource =
-        ReservationSystemTestUtil.calculateClusterResource(totCont);
-    ReservationSchedulerConfiguration conf = mock
-        (ReservationSchedulerConfiguration.class);
-    NoOverCommitPolicy policy = new NoOverCommitPolicy();
+    conf = new CapacitySchedulerConfiguration();
+    SharingPolicy policy = new NoOverCommitPolicy();
     policy.init(reservationQ, conf);
-    RMContext context = ReservationSystemTestUtil.createMockRMContext();
-
-    plan =
-        new InMemoryPlan(rootQueueMetrics, policy, mAgent,
-            clusterResource, step, res, minAlloc, maxAlloc,
-            "dedicated", null, true, context);
-  }
-
-  public int[] generateData(int length, int val) {
-    int[] data = new int[length];
-    for (int i = 0; i < length; i++) {
-      data[i] = val;
-    }
-    return data;
-  }
-
-  @Test
-  public void testSingleUserEasyFitPass() throws IOException, PlanningException {
-    // generate allocation that easily fit within resource constraints
-    int[] f = generateData(3600, (int) Math.ceil(0.2 * totCont));
-    ReservationDefinition rDef =
-        ReservationSystemTestUtil.createSimpleReservationDefinition(
-            initTime, initTime + f.length + 1, f.length);
-    assertTrue(plan.toString(),
-        plan.addReservation(new InMemoryReservationAllocation(
-            ReservationSystemTestUtil.getNewReservationId(), rDef, "u1",
-            "dedicated", initTime, initTime + f.length,
-            ReservationSystemTestUtil.generateAllocation(initTime, step, f),
-            res, minAlloc), false));
-  }
-
-  @Test
-  public void testSingleUserBarelyFitPass() throws IOException,
-      PlanningException {
-    // generate allocation from single tenant that barely fit
-    int[] f = generateData(3600, totCont);
-    ReservationDefinition rDef =
-        ReservationSystemTestUtil.createSimpleReservationDefinition(
-            initTime, initTime + f.length + 1, f.length);
-    assertTrue(plan.toString(),
-        plan.addReservation(new InMemoryReservationAllocation(
-            ReservationSystemTestUtil.getNewReservationId(), rDef, "u1",
-            "dedicated", initTime, initTime + f.length,
-            ReservationSystemTestUtil.generateAllocation(initTime, step, f),
-            res, minAlloc), false));
-  }
-
-  @Test(expected = ResourceOverCommitException.class)
-  public void testSingleFail() throws IOException, PlanningException {
-    // generate allocation from single tenant that exceed capacity
-    int[] f = generateData(3600, (int) (1.1 * totCont));
-    plan.addReservation(new InMemoryReservationAllocation(
-        ReservationSystemTestUtil.getNewReservationId(), null, "u1",
-        "dedicated", initTime, initTime + f.length, ReservationSystemTestUtil
-            .generateAllocation(initTime, step, f), res, minAlloc), false);
+    return policy;
   }
 
   @Test
-  public void testMultiTenantPass() throws IOException, PlanningException {
-    // generate allocation from multiple tenants that barely fit in tot capacity
-    int[] f = generateData(3600, (int) Math.ceil(0.25 * totCont));
-    ReservationDefinition rDef =
-        ReservationSystemTestUtil.createSimpleReservationDefinition(
-            initTime, initTime + f.length + 1, f.length);
-    for (int i = 0; i < 4; i++) {
-      assertTrue(plan.toString(),
-          plan.addReservation(new InMemoryReservationAllocation(
-              ReservationSystemTestUtil.getNewReservationId(), rDef, "u" + i,
-              "dedicated", initTime, initTime + f.length,
-              ReservationSystemTestUtil.generateAllocation(initTime, step, f),
-              res, minAlloc), false));
-    }
+  public void testAllocation() throws IOException, PlanningException {
+    runTest();
   }
 
-  @Test(expected = ResourceOverCommitException.class)
-  public void testMultiTenantFail() throws IOException, PlanningException {
-    // generate allocation from multiple tenants that exceed tot capacity
-    int[] f = generateData(3600, (int) Math.ceil(0.25 * totCont));
-    ReservationDefinition rDef =
-        ReservationSystemTestUtil.createSimpleReservationDefinition(
-            initTime, initTime + f.length + 1, f.length);
-    for (int i = 0; i < 5; i++) {
-      assertTrue(plan.toString(),
-          plan.addReservation(new InMemoryReservationAllocation(
-              ReservationSystemTestUtil.getNewReservationId(), rDef, "u" + i,
-              "dedicated", initTime, initTime + f.length,
-              ReservationSystemTestUtil.generateAllocation(initTime, step, f),
-              res, minAlloc), false));
-    }
-  }
-}
+}
\ 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


[20/50] [abbrv] hadoop git commit: HADOOP-14774. S3A case "testRandomReadOverBuffer" failed due to improper range parameter. Contributed by Yonger.

Posted by as...@apache.org.
HADOOP-14774. S3A case "testRandomReadOverBuffer" failed due to improper range parameter.
Contributed by Yonger.


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

Branch: refs/heads/YARN-5972
Commit: 83449abb40633a36b22de538cf82fca240af067e
Parents: c41118a
Author: Steve Loughran <st...@apache.org>
Authored: Thu Sep 7 19:55:56 2017 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Thu Sep 7 19:55:56 2017 +0100

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/83449abb/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
index b88b7c1..94d7701 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
@@ -152,7 +152,7 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead {
     streamStatistics.streamOpened();
     try {
       GetObjectRequest request = new GetObjectRequest(bucket, key)
-          .withRange(targetPos, contentRangeFinish);
+          .withRange(targetPos, contentRangeFinish - 1);
       if (S3AEncryptionMethods.SSE_C.equals(serverSideEncryptionAlgorithm) &&
           StringUtils.isNotBlank(serverSideEncryptionKey)){
         request.setSSECustomerKey(new SSECustomerKey(serverSideEncryptionKey));


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


[14/50] [abbrv] hadoop git commit: HDFS-12402. Refactor ErasureCodingPolicyManager and related codes. Contributed by Sammi Chen

Posted by as...@apache.org.
HDFS-12402. Refactor ErasureCodingPolicyManager and related codes. Contributed by Sammi Chen


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

Branch: refs/heads/YARN-5972
Commit: 2adf8bed712e6d770a0d53eea198d8911ae1a258
Parents: 6f101e7
Author: Kai Zheng <ka...@intel.com>
Authored: Thu Sep 7 20:38:23 2017 +0800
Committer: Kai Zheng <ka...@intel.com>
Committed: Thu Sep 7 20:38:23 2017 +0800

----------------------------------------------------------------------
 .../hdfs/protocol/AddECPolicyResponse.java      |  4 +-
 .../hdfs/protocol/IllegalECPolicyException.java | 34 -----------------
 .../namenode/ErasureCodingPolicyManager.java    | 39 ++++++++++----------
 .../server/namenode/FSDirErasureCodingOp.java   |  3 +-
 .../hdfs/server/namenode/FSNamesystem.java      |  3 +-
 .../hadoop/hdfs/TestDistributedFileSystem.java  |  4 +-
 6 files changed, 27 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2adf8bed/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AddECPolicyResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AddECPolicyResponse.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AddECPolicyResponse.java
index e7a8435..4e55680 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AddECPolicyResponse.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AddECPolicyResponse.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdfs.protocol;
 
+import org.apache.hadoop.HadoopIllegalArgumentException;
+
 /**
  * A response of add an ErasureCoding policy.
  */
@@ -38,7 +40,7 @@ public class AddECPolicyResponse {
   }
 
   public AddECPolicyResponse(ErasureCodingPolicy policy,
-      IllegalECPolicyException e) {
+      HadoopIllegalArgumentException e) {
     this(policy, e.getMessage());
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2adf8bed/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/IllegalECPolicyException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/IllegalECPolicyException.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/IllegalECPolicyException.java
deleted file mode 100644
index 03ce2a5..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/IllegalECPolicyException.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/**
- * 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.protocol;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * An Exception indicates the error when adding an ErasureCoding policy.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class IllegalECPolicyException extends Exception {
-  static final long serialVersionUID = 1L;
-
-  public IllegalECPolicyException(String msg) {
-    super(msg);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2adf8bed/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
index 74b5ebf..4c75709 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
@@ -19,11 +19,11 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.lang.ArrayUtils;
+import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyState;
-import org.apache.hadoop.hdfs.protocol.IllegalECPolicyException;
 import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -144,7 +144,7 @@ public final class ErasureCodingPolicyManager {
             policyName,
             DFSConfigKeys.DFS_NAMENODE_EC_POLICIES_ENABLED_KEY,
             names);
-        throw new IllegalArgumentException(msg);
+        throw new HadoopIllegalArgumentException(msg);
       }
       enabledPoliciesByName.put(ecPolicy.getName(), ecPolicy);
     }
@@ -230,33 +230,34 @@ public final class ErasureCodingPolicyManager {
    * Add an erasure coding policy.
    * @return the added policy
    */
-  public synchronized ErasureCodingPolicy addPolicy(ErasureCodingPolicy policy)
-      throws IllegalECPolicyException {
+  public synchronized ErasureCodingPolicy addPolicy(
+      ErasureCodingPolicy policy) {
     // Set policy state into DISABLED when adding into Hadoop.
     policy.setState(ErasureCodingPolicyState.DISABLED);
 
     if (!CodecUtil.hasCodec(policy.getCodecName())) {
-      throw new IllegalECPolicyException("Codec name "
+      throw new HadoopIllegalArgumentException("Codec name "
           + policy.getCodecName() + " is not supported");
     }
 
     if (policy.getCellSize() > maxCellSize) {
-      throw new IllegalECPolicyException("Cell size " + policy.getCellSize()
-          + " should not exceed maximum " + maxCellSize + " byte");
+      throw new HadoopIllegalArgumentException("Cell size " +
+          policy.getCellSize() + " should not exceed maximum " +
+          maxCellSize + " bytes");
     }
 
     String assignedNewName = ErasureCodingPolicy.composePolicyName(
         policy.getSchema(), policy.getCellSize());
     for (ErasureCodingPolicy p : getPolicies()) {
       if (p.getName().equals(assignedNewName)) {
-        throw new IllegalECPolicyException("The policy name " + assignedNewName
-            + " already exists");
+        throw new HadoopIllegalArgumentException("The policy name " +
+            assignedNewName + " already exists");
       }
       if (p.getSchema().equals(policy.getSchema()) &&
           p.getCellSize() == policy.getCellSize()) {
-        throw new IllegalECPolicyException("A policy with same schema "
+        throw new HadoopIllegalArgumentException("A policy with same schema "
             + policy.getSchema().toString() + " and cell size "
-            + p.getCellSize() + " is already exists");
+            + p.getCellSize() + " already exists");
       }
     }
     policy.setName(assignedNewName);
@@ -281,12 +282,12 @@ public final class ErasureCodingPolicyManager {
   public synchronized void removePolicy(String name) {
     ErasureCodingPolicy ecPolicy = policiesByName.get(name);
     if (ecPolicy == null) {
-      throw new IllegalArgumentException("The policy name " +
-          name + " does not exists");
+      throw new HadoopIllegalArgumentException("The policy name " +
+          name + " does not exist");
     }
 
     if (ecPolicy.isSystemPolicy()) {
-      throw new IllegalArgumentException("System erasure coding policy " +
+      throw new HadoopIllegalArgumentException("System erasure coding policy " +
           name + " cannot be removed");
     }
 
@@ -317,8 +318,8 @@ public final class ErasureCodingPolicyManager {
   public synchronized void disablePolicy(String name) {
     ErasureCodingPolicy ecPolicy = policiesByName.get(name);
     if (ecPolicy == null) {
-      throw new IllegalArgumentException("The policy name " +
-          name + " does not exists");
+      throw new HadoopIllegalArgumentException("The policy name " +
+          name + " does not exist");
     }
 
     if (enabledPoliciesByName.containsKey(name)) {
@@ -336,8 +337,8 @@ public final class ErasureCodingPolicyManager {
   public synchronized void enablePolicy(String name) {
     ErasureCodingPolicy ecPolicy = policiesByName.get(name);
     if (ecPolicy == null) {
-      throw new IllegalArgumentException("The policy name " +
-          name + " does not exists");
+      throw new HadoopIllegalArgumentException("The policy name " +
+          name + " does not exist");
     }
 
     enabledPoliciesByName.put(name, ecPolicy);
@@ -346,4 +347,4 @@ public final class ErasureCodingPolicyManager {
         enabledPoliciesByName.values().toArray(new ErasureCodingPolicy[0]);
     LOG.info("Enable the erasure coding policy " + name);
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2adf8bed/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
index 4f4befe..deb03af 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
@@ -27,7 +27,6 @@ import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
-import org.apache.hadoop.hdfs.protocol.IllegalECPolicyException;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.WritableUtils;
@@ -212,7 +211,7 @@ final class FSDirErasureCodingOp {
   }
 
   static ErasureCodingPolicy addErasureCodePolicy(final FSNamesystem fsn,
-      ErasureCodingPolicy policy) throws IllegalECPolicyException {
+      ErasureCodingPolicy policy) {
     Preconditions.checkNotNull(policy);
     return fsn.getErasureCodingPolicyManager().addPolicy(policy);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2adf8bed/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index e5604c4..c30999b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -201,7 +201,6 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.ReencryptAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
-import org.apache.hadoop.hdfs.protocol.IllegalECPolicyException;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@@ -7207,7 +7206,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
               FSDirErasureCodingOp.addErasureCodePolicy(this, policy);
           addECPolicyName = newPolicy.getName();
           responses.add(new AddECPolicyResponse(newPolicy));
-        } catch (IllegalECPolicyException e) {
+        } catch (HadoopIllegalArgumentException e) {
           responses.add(new AddECPolicyResponse(policy, e));
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2adf8bed/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 bf2b002..8e54e5f 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
@@ -1617,7 +1617,7 @@ public class TestDistributedFileSystem {
         fs.enableErasureCodingPolicy("notExistECName");
         Assert.fail("enable the policy that doesn't exist should fail");
       } catch (Exception e) {
-        GenericTestUtils.assertExceptionContains("does not exists", e);
+        GenericTestUtils.assertExceptionContains("does not exist", e);
         // pass
       }
 
@@ -1626,7 +1626,7 @@ public class TestDistributedFileSystem {
         fs.disableErasureCodingPolicy("notExistECName");
         Assert.fail("disable the policy that doesn't exist should fail");
       } catch (Exception e) {
-        GenericTestUtils.assertExceptionContains("does not exists", e);
+        GenericTestUtils.assertExceptionContains("does not exist", e);
         // pass
       }
 


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


[34/50] [abbrv] hadoop git commit: YARN-7130. ATSv2 documentation changes post merge. Contributed by Varun Saxena.

Posted by as...@apache.org.
YARN-7130. ATSv2 documentation changes post merge. Contributed by Varun Saxena.


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

Branch: refs/heads/YARN-5972
Commit: 0bfb3a256612aa54437ee9c8c928c3c41d69fc52
Parents: 5bbca80
Author: Rohith Sharma K S <ro...@apache.org>
Authored: Fri Sep 8 20:21:50 2017 +0530
Committer: Rohith Sharma K S <ro...@apache.org>
Committed: Fri Sep 8 20:21:50 2017 +0530

----------------------------------------------------------------------
 .../hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md      | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bfb3a25/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md
index 2de305d..7c51ce0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md
@@ -415,13 +415,13 @@ Service v.2. Note that currently you need to be on the cluster to be able to wri
 Service. For example, an application master or code in the container can write to the Timeline
 Service, while an off-cluster MapReduce job submitter cannot.
 
-After creating the timeline v2 client, user also needs to set the timeline collector address for the application. If `AMRMClient` is used then by registering the timeline client by calling `AMRMClient#registerTimelineV2Client` is sufficient.
+After creating the timeline v2 client, user also needs to set the timeline collector info which contains the collector address and collector token(only in secure mode) for the application. If `AMRMClient` is used then by registering the timeline client by calling `AMRMClient#registerTimelineV2Client` is sufficient.
 
     amRMClient.registerTimelineV2Client(timelineClient);
 
 Else address needs to be retrieved from the AM allocate response and need to be set in timeline client explicitly.
 
-    timelineClient.setTimelineServiceAddress(response.getCollectorAddr());
+    timelineClient.setTimelineCollectorInfo(response.getCollectorInfo());
 
 You can create and publish your own entities, events, and metrics as with previous versions.
 


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


[22/50] [abbrv] hadoop git commit: YARN-7033. Add support for NM Recovery of assigned resources (e.g. GPU's, NUMA, FPGA's) to container. (Devaraj K and Wangda Tan)

Posted by as...@apache.org.
YARN-7033. Add support for NM Recovery of assigned resources (e.g. GPU's, NUMA, FPGA's) to container. (Devaraj K and Wangda Tan)

Change-Id: Iffd18bb95debe1c8cc55e30abc1d8f663e9d0e30


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

Branch: refs/heads/YARN-5972
Commit: f155ab7cfa64e822171708040cb49889338961ce
Parents: a4cd101
Author: Wangda Tan <wa...@apache.org>
Authored: Thu Sep 7 14:13:37 2017 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Thu Sep 7 14:13:37 2017 -0700

----------------------------------------------------------------------
 .../containermanager/container/Container.java   |   7 +
 .../container/ContainerImpl.java                |  13 ++
 .../container/ResourceMappings.java             | 124 ++++++++++++++
 .../recovery/NMLeveldbStateStoreService.java    |  42 +++++
 .../recovery/NMNullStateStoreService.java       |   7 +
 .../recovery/NMStateStoreService.java           |  23 +++
 .../TestContainerManagerRecovery.java           | 160 +++++++++++++------
 .../recovery/NMMemoryStateStoreService.java     |  14 ++
 .../TestNMLeveldbStateStoreService.java         | 121 +++++++++-----
 .../nodemanager/webapp/MockContainer.java       |   6 +
 10 files changed, 431 insertions(+), 86 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f155ab7c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.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/container/Container.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java
index ac9fbb7..ef5d72c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java
@@ -96,4 +96,11 @@ public interface Container extends EventHandler<ContainerEvent> {
   void sendKillEvent(int exitStatus, String description);
 
   boolean isRecovering();
+
+  /**
+   * Get assigned resource mappings to the container.
+   *
+   * @return Resource Mappings of the container
+   */
+  ResourceMappings getResourceMappings();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f155ab7c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.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/container/ContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
index 772b6e7..a768d18 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
@@ -185,6 +185,7 @@ public class ContainerImpl implements Container {
   private boolean recoveredAsKilled = false;
   private Context context;
   private ResourceSet resourceSet;
+  private ResourceMappings resourceMappings;
 
   public ContainerImpl(Configuration conf, Dispatcher dispatcher,
       ContainerLaunchContext launchContext, Credentials creds,
@@ -242,6 +243,7 @@ public class ContainerImpl implements Container {
     stateMachine = stateMachineFactory.make(this);
     this.context = context;
     this.resourceSet = new ResourceSet();
+    this.resourceMappings = new ResourceMappings();
   }
 
   private static ContainerRetryContext configureRetryContext(
@@ -282,6 +284,7 @@ public class ContainerImpl implements Container {
     this.remainingRetryAttempts = rcs.getRemainingRetryAttempts();
     this.workDir = rcs.getWorkDir();
     this.logDir = rcs.getLogDir();
+    this.resourceMappings = rcs.getResourceMappings();
   }
 
   private static final ContainerDiagnosticsUpdateTransition UPDATE_DIAGNOSTICS_TRANSITION =
@@ -1789,4 +1792,14 @@ public class ContainerImpl implements Container {
         getContainerState() == ContainerState.NEW);
     return isRecovering;
   }
+
+  /**
+   * Get assigned resource mappings to the container.
+   *
+   * @return Resource Mappings of the container
+   */
+  @Override
+  public ResourceMappings getResourceMappings() {
+    return resourceMappings;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f155ab7c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ResourceMappings.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/container/ResourceMappings.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ResourceMappings.java
new file mode 100644
index 0000000..d673341
--- /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/container/ResourceMappings.java
@@ -0,0 +1,124 @@
+/**
+ * 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.container;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.io.IOUtils;
+
+/**
+ * This class is used to store assigned resource to a single container by
+ * resource types.
+ *
+ * Assigned resource could be list of String
+ *
+ * For example, we can assign container to:
+ * "numa": ["numa0"]
+ * "gpu": ["0", "1", "2", "3"]
+ * "fpga": ["1", "3"]
+ *
+ * This will be used for NM restart container recovery.
+ */
+public class ResourceMappings {
+
+  private Map<String, AssignedResources> assignedResourcesMap = new HashMap<>();
+
+  /**
+   * Get all resource mappings.
+   * @param resourceType resourceType
+   * @return map of resource mapping
+   */
+  public List<Serializable> getAssignedResources(String resourceType) {
+    AssignedResources ar = assignedResourcesMap.get(resourceType);
+    if (null == ar) {
+      return Collections.emptyList();
+    }
+    return ar.getAssignedResources();
+  }
+
+  /**
+   * Adds the resources for a given resource type.
+   *
+   * @param resourceType Resource Type
+   * @param assigned Assigned resources to add
+   */
+  public void addAssignedResources(String resourceType,
+      AssignedResources assigned) {
+    assignedResourcesMap.put(resourceType, assigned);
+  }
+
+  /**
+   * Stores resources assigned to a container for a given resource type.
+   */
+  public static class AssignedResources implements Serializable {
+    private static final long serialVersionUID = -1059491941955757926L;
+    private List<Serializable> resources = Collections.emptyList();
+
+    public List<Serializable> getAssignedResources() {
+      return Collections.unmodifiableList(resources);
+    }
+
+    public void updateAssignedResources(List<Serializable> list) {
+      this.resources = new ArrayList<>(list);
+    }
+
+    @SuppressWarnings("unchecked")
+    public static AssignedResources fromBytes(byte[] bytes)
+        throws IOException {
+      ObjectInputStream ois = null;
+      List<Serializable> resources;
+      try {
+        ByteArrayInputStream bis = new ByteArrayInputStream(bytes);
+        ois = new ObjectInputStream(bis);
+        resources = (List<Serializable>) ois.readObject();
+      } catch (ClassNotFoundException e) {
+        throw new IOException(e);
+      } finally {
+        IOUtils.closeQuietly(ois);
+      }
+      AssignedResources ar = new AssignedResources();
+      ar.updateAssignedResources(resources);
+      return ar;
+    }
+
+    public byte[] toBytes() throws IOException {
+      ObjectOutputStream oos = null;
+      byte[] bytes;
+      try {
+        ByteArrayOutputStream bos = new ByteArrayOutputStream();
+        oos = new ObjectOutputStream(bos);
+        oos.writeObject(resources);
+        bytes = bos.toByteArray();
+      } finally {
+        IOUtils.closeQuietly(oos);
+      }
+      return bytes;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f155ab7c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.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/recovery/NMLeveldbStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java
index a31756e..db931f8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java
@@ -24,6 +24,7 @@ import org.slf4j.LoggerFactory;
 
 import java.io.File;
 import java.io.IOException;
+import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
@@ -39,6 +40,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainerRequestPBImpl;
@@ -60,6 +62,7 @@ import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.LogDelet
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainerRequestProto;
 import org.apache.hadoop.yarn.server.api.records.MasterKey;
 import org.apache.hadoop.yarn.server.api.records.impl.pb.MasterKeyPBImpl;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ResourceMappings;
 import org.apache.hadoop.yarn.server.records.Version;
 import org.apache.hadoop.yarn.server.records.impl.pb.VersionPBImpl;
 import org.apache.hadoop.yarn.server.utils.LeveldbIterator;
@@ -144,6 +147,9 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
 
   private static final String AMRMPROXY_KEY_PREFIX = "AMRMProxy/";
 
+  private static final String CONTAINER_ASSIGNED_RESOURCES_KEY_SUFFIX =
+      "/assignedResources_";
+
   private static final byte[] EMPTY_VALUE = new byte[0];
 
   private DB db;
@@ -286,6 +292,13 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
         rcs.setWorkDir(asString(entry.getValue()));
       } else if (suffix.equals(CONTAINER_LOG_DIR_KEY_SUFFIX)) {
         rcs.setLogDir(asString(entry.getValue()));
+      } else if (suffix.startsWith(CONTAINER_ASSIGNED_RESOURCES_KEY_SUFFIX)) {
+        String resourceType = suffix.substring(
+            CONTAINER_ASSIGNED_RESOURCES_KEY_SUFFIX.length());
+        ResourceMappings.AssignedResources assignedResources =
+            ResourceMappings.AssignedResources.fromBytes(entry.getValue());
+        rcs.getResourceMappings().addAssignedResources(resourceType,
+            assignedResources);
       } else {
         LOG.warn("the container " + containerId
             + " will be killed because of the unknown key " + key
@@ -1091,6 +1104,35 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
     }
   }
 
+  @Override
+  public void storeAssignedResources(ContainerId containerId,
+      String resourceType, List<Serializable> assignedResources)
+      throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("storeAssignedResources: containerId=" + containerId
+          + ", assignedResources=" + StringUtils.join(",", assignedResources));
+    }
+
+    String keyResChng = CONTAINERS_KEY_PREFIX + containerId.toString()
+        + CONTAINER_ASSIGNED_RESOURCES_KEY_SUFFIX + resourceType;
+    try {
+      WriteBatch batch = db.createWriteBatch();
+      try {
+        ResourceMappings.AssignedResources res =
+            new ResourceMappings.AssignedResources();
+        res.updateAssignedResources(assignedResources);
+
+        // New value will overwrite old values for the same key
+        batch.put(bytes(keyResChng), res.toBytes());
+        db.write(batch);
+      } finally {
+        batch.close();
+      }
+    } catch (DBException e) {
+      throw new IOException(e);
+    }
+  }
+
   @SuppressWarnings("deprecation")
   private void cleanupDeprecatedFinishedApps() {
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f155ab7c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMNullStateStoreService.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/recovery/NMNullStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMNullStateStoreService.java
index 86dc99f..dc1cece 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMNullStateStoreService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMNullStateStoreService.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.server.nodemanager.recovery;
 
 import java.io.IOException;
+import java.io.Serializable;
 import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
@@ -258,6 +259,12 @@ public class NMNullStateStoreService extends NMStateStoreService {
   }
 
   @Override
+  public void storeAssignedResources(ContainerId containerId,
+      String resourceType, List<Serializable> assignedResources)
+      throws IOException {
+  }
+
+  @Override
   protected void initStorage(Configuration conf) throws IOException {
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f155ab7c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMStateStoreService.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/recovery/NMStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMStateStoreService.java
index ec534bf..62a2b9f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMStateStoreService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMStateStoreService.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.server.nodemanager.recovery;
 
 import java.io.IOException;
+import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -42,6 +43,7 @@ import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.Deletion
 import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.LocalizedResourceProto;
 import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.LogDeleterProto;
 import org.apache.hadoop.yarn.server.api.records.MasterKey;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ResourceMappings;
 
 @Private
 @Unstable
@@ -88,6 +90,7 @@ public abstract class NMStateStoreService extends AbstractService {
     private RecoveredContainerType recoveryType =
         RecoveredContainerType.RECOVER;
     private long startTime;
+    private ResourceMappings resMappings = new ResourceMappings();
 
     public RecoveredContainerStatus getStatus() {
       return status;
@@ -172,6 +175,14 @@ public abstract class NMStateStoreService extends AbstractService {
     public void setRecoveryType(RecoveredContainerType recoveryType) {
       this.recoveryType = recoveryType;
     }
+
+    public ResourceMappings getResourceMappings() {
+      return resMappings;
+    }
+
+    public void setResourceMappings(ResourceMappings mappings) {
+      this.resMappings = mappings;
+    }
   }
 
   public static class LocalResourceTrackerState {
@@ -699,6 +710,18 @@ public abstract class NMStateStoreService extends AbstractService {
   public abstract void removeAMRMProxyAppContext(ApplicationAttemptId attempt)
       throws IOException;
 
+  /**
+   * Store the assigned resources to a container.
+   *
+   * @param containerId Container Id
+   * @param resourceType Resource Type
+   * @param assignedResources Assigned resources
+   * @throws IOException if fails
+   */
+  public abstract void storeAssignedResources(ContainerId containerId,
+      String resourceType, List<Serializable> assignedResources)
+      throws IOException;
+
   protected abstract void initStorage(Configuration conf) throws IOException;
 
   protected abstract void startStorage() throws IOException;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f155ab7c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.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/TestContainerManagerRecovery.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
index 224e99c..5ec0ae6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
@@ -31,6 +31,7 @@ import static org.mockito.Mockito.verify;
 import java.io.File;
 import java.io.IOException;
 import java.io.PrintWriter;
+import java.io.Serializable;
 import java.nio.ByteBuffer;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
@@ -90,6 +91,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Ap
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationImpl;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationState;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ResourceMappings;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainersLauncher;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainersLauncherEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService;
@@ -108,6 +110,7 @@ import org.apache.hadoop.yarn.server.nodemanager.security.NMTokenSecretManagerIn
 import org.apache.hadoop.yarn.server.nodemanager.timelineservice.NMTimelinePublisher;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -400,9 +403,8 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest {
     NMStateStoreService stateStore = new NMMemoryStateStoreService();
     stateStore.init(conf);
     stateStore.start();
-    Context context = createContext(conf, stateStore);
+    context = createContext(conf, stateStore);
     ContainerManagerImpl cm = createContainerManager(context, delSrvc);
-    cm.dispatcher.disableExitOnDispatchException();
     cm.init(conf);
     cm.start();
     // add an application by starting a container
@@ -410,55 +412,12 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest {
     ApplicationAttemptId attemptId =
         ApplicationAttemptId.newInstance(appId, 1);
     ContainerId cid = ContainerId.newContainerId(attemptId, 1);
-    Map<String, String> containerEnv = new HashMap<>();
-    setFlowContext(containerEnv, "app_name1", appId);
-    Map<String, ByteBuffer> serviceData = Collections.emptyMap();
-    Credentials containerCreds = new Credentials();
-    DataOutputBuffer dob = new DataOutputBuffer();
-    containerCreds.writeTokenStorageToStream(dob);
-    ByteBuffer containerTokens = ByteBuffer.wrap(dob.getData(), 0,
-        dob.getLength());
-    Map<ApplicationAccessType, String> acls = Collections.emptyMap();
-    File tmpDir = new File("target",
-        this.getClass().getSimpleName() + "-tmpDir");
-    File scriptFile = Shell.appendScriptExtension(tmpDir, "scriptFile");
-    PrintWriter fileWriter = new PrintWriter(scriptFile);
-    if (Shell.WINDOWS) {
-      fileWriter.println("@ping -n 100 127.0.0.1 >nul");
-    } else {
-      fileWriter.write("\numask 0");
-      fileWriter.write("\nexec sleep 100");
-    }
-    fileWriter.close();
-    FileContext localFS = FileContext.getLocalFSFileContext();
-    URL resource_alpha =
-        URL.fromPath(localFS
-            .makeQualified(new Path(scriptFile.getAbsolutePath())));
-    LocalResource rsrc_alpha = RecordFactoryProvider
-        .getRecordFactory(null).newRecordInstance(LocalResource.class);
-    rsrc_alpha.setResource(resource_alpha);
-    rsrc_alpha.setSize(-1);
-    rsrc_alpha.setVisibility(LocalResourceVisibility.APPLICATION);
-    rsrc_alpha.setType(LocalResourceType.FILE);
-    rsrc_alpha.setTimestamp(scriptFile.lastModified());
-    String destinationFile = "dest_file";
-    Map<String, LocalResource> localResources = new HashMap<>();
-    localResources.put(destinationFile, rsrc_alpha);
-    List<String> commands =
-        Arrays.asList(Shell.getRunScriptCommand(scriptFile));
-    ContainerLaunchContext clc = ContainerLaunchContext.newInstance(
-        localResources, containerEnv, commands, serviceData,
-        containerTokens, acls);
-    StartContainersResponse startResponse = startContainer(
-        context, cm, cid, clc, null);
-    assertTrue(startResponse.getFailedRequests().isEmpty());
-    assertEquals(1, context.getApplications().size());
+
+    commonLaunchContainer(appId, cid, cm);
+
     Application app = context.getApplications().get(appId);
     assertNotNull(app);
-    // make sure the container reaches RUNNING state
-    waitForNMContainerState(cm, cid,
-        org.apache.hadoop.yarn.server.nodemanager
-            .containermanager.container.ContainerState.RUNNING);
+
     Resource targetResource = Resource.newInstance(2048, 2);
     ContainerUpdateResponse updateResponse =
         updateContainers(context, cm, cid, targetResource);
@@ -481,6 +440,58 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest {
   }
 
   @Test
+  public void testResourceMappingRecoveryForContainer() throws Exception {
+    conf.setBoolean(YarnConfiguration.NM_RECOVERY_ENABLED, true);
+    conf.setBoolean(YarnConfiguration.NM_RECOVERY_SUPERVISED, true);
+    NMStateStoreService stateStore = new NMMemoryStateStoreService();
+    stateStore.init(conf);
+    stateStore.start();
+    context = createContext(conf, stateStore);
+    ContainerManagerImpl cm = createContainerManager(context, delSrvc);
+    cm.init(conf);
+    cm.start();
+
+    // add an application by starting a container
+    ApplicationId appId = ApplicationId.newInstance(0, 1);
+    ApplicationAttemptId attemptId =
+        ApplicationAttemptId.newInstance(appId, 1);
+    ContainerId cid = ContainerId.newContainerId(attemptId, 1);
+
+    commonLaunchContainer(appId, cid, cm);
+
+    Application app = context.getApplications().get(appId);
+    assertNotNull(app);
+
+    // store resource mapping of the container
+    List<Serializable> gpuResources = Arrays.asList("1", "2", "3");
+    stateStore.storeAssignedResources(cid, "gpu", gpuResources);
+    List<Serializable> numaResources = Arrays.asList("numa1");
+    stateStore.storeAssignedResources(cid, "numa", numaResources);
+    List<Serializable> fpgaResources = Arrays.asList("fpga1", "fpga2");
+    stateStore.storeAssignedResources(cid, "fpga", fpgaResources);
+
+    cm.stop();
+    context = createContext(conf, stateStore);
+    cm = createContainerManager(context);
+    cm.init(conf);
+    cm.start();
+    assertEquals(1, context.getApplications().size());
+    app = context.getApplications().get(appId);
+    assertNotNull(app);
+
+    Container nmContainer = context.getContainers().get(cid);
+    Assert.assertNotNull(nmContainer);
+    ResourceMappings resourceMappings = nmContainer.getResourceMappings();
+    List<Serializable> assignedResource = resourceMappings
+        .getAssignedResources("gpu");
+    Assert.assertTrue(assignedResource.equals(gpuResources));
+    Assert.assertTrue(
+        resourceMappings.getAssignedResources("numa").equals(numaResources));
+    Assert.assertTrue(
+        resourceMappings.getAssignedResources("fpga").equals(fpgaResources));
+  }
+
+  @Test
   public void testContainerCleanupOnShutdown() throws Exception {
     ApplicationId appId = ApplicationId.newInstance(0, 1);
     ApplicationAttemptId attemptId =
@@ -552,6 +563,57 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest {
     verify(cm, never()).handle(isA(CMgrCompletedAppsEvent.class));
   }
 
+  private void commonLaunchContainer(ApplicationId appId, ContainerId cid,
+      ContainerManagerImpl cm) throws Exception {
+    Map<String, String> containerEnv = new HashMap<>();
+    setFlowContext(containerEnv, "app_name1", appId);
+    Map<String, ByteBuffer> serviceData = Collections.emptyMap();
+    Credentials containerCreds = new Credentials();
+    DataOutputBuffer dob = new DataOutputBuffer();
+    containerCreds.writeTokenStorageToStream(dob);
+    ByteBuffer containerTokens = ByteBuffer.wrap(dob.getData(), 0,
+        dob.getLength());
+    Map<ApplicationAccessType, String> acls = Collections.emptyMap();
+    File tmpDir = new File("target",
+        this.getClass().getSimpleName() + "-tmpDir");
+    File scriptFile = Shell.appendScriptExtension(tmpDir, "scriptFile");
+    PrintWriter fileWriter = new PrintWriter(scriptFile);
+    if (Shell.WINDOWS) {
+      fileWriter.println("@ping -n 100 127.0.0.1 >nul");
+    } else {
+      fileWriter.write("\numask 0");
+      fileWriter.write("\nexec sleep 100");
+    }
+    fileWriter.close();
+    FileContext localFS = FileContext.getLocalFSFileContext();
+    URL resource_alpha =
+        URL.fromPath(localFS
+            .makeQualified(new Path(scriptFile.getAbsolutePath())));
+    LocalResource rsrc_alpha = RecordFactoryProvider
+        .getRecordFactory(null).newRecordInstance(LocalResource.class);
+    rsrc_alpha.setResource(resource_alpha);
+    rsrc_alpha.setSize(-1);
+    rsrc_alpha.setVisibility(LocalResourceVisibility.APPLICATION);
+    rsrc_alpha.setType(LocalResourceType.FILE);
+    rsrc_alpha.setTimestamp(scriptFile.lastModified());
+    String destinationFile = "dest_file";
+    Map<String, LocalResource> localResources = new HashMap<>();
+    localResources.put(destinationFile, rsrc_alpha);
+    List<String> commands =
+        Arrays.asList(Shell.getRunScriptCommand(scriptFile));
+    ContainerLaunchContext clc = ContainerLaunchContext.newInstance(
+        localResources, containerEnv, commands, serviceData,
+        containerTokens, acls);
+    StartContainersResponse startResponse = startContainer(
+        context, cm, cid, clc, null);
+    assertTrue(startResponse.getFailedRequests().isEmpty());
+    assertEquals(1, context.getApplications().size());
+    // make sure the container reaches RUNNING state
+    waitForNMContainerState(cm, cid,
+        org.apache.hadoop.yarn.server.nodemanager
+            .containermanager.container.ContainerState.RUNNING);
+  }
+
   private ContainerManagerImpl createContainerManager(Context context,
       DeletionService delSrvc) {
     return new ContainerManagerImpl(context, exec, delSrvc,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f155ab7c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMMemoryStateStoreService.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/recovery/NMMemoryStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMMemoryStateStoreService.java
index c1638df..6d6875d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMMemoryStateStoreService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMMemoryStateStoreService.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.server.nodemanager.recovery;
 
 import java.io.IOException;
+import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
@@ -40,6 +41,7 @@ import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.Localize
 import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.LogDeleterProto;
 import org.apache.hadoop.yarn.server.api.records.MasterKey;
 import org.apache.hadoop.yarn.server.api.records.impl.pb.MasterKeyPBImpl;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ResourceMappings;
 
 public class NMMemoryStateStoreService extends NMStateStoreService {
   private Map<ApplicationId, ContainerManagerApplicationProto> apps;
@@ -119,6 +121,7 @@ public class NMMemoryStateStoreService extends NMStateStoreService {
       rcsCopy.setRemainingRetryAttempts(rcs.getRemainingRetryAttempts());
       rcsCopy.setWorkDir(rcs.getWorkDir());
       rcsCopy.setLogDir(rcs.getLogDir());
+      rcsCopy.setResourceMappings(rcs.getResourceMappings());
       result.add(rcsCopy);
     }
     return result;
@@ -480,6 +483,17 @@ public class NMMemoryStateStoreService extends NMStateStoreService {
     amrmProxyState.getAppContexts().remove(attempt);
   }
 
+  @Override
+  public void storeAssignedResources(ContainerId containerId,
+      String resourceType, List<Serializable> assignedResources)
+      throws IOException {
+    ResourceMappings.AssignedResources ar =
+        new ResourceMappings.AssignedResources();
+    ar.updateAssignedResources(assignedResources);
+    containerStates.get(containerId).getResourceMappings()
+        .addAssignedResources(resourceType, ar);
+  }
+
   private static class TrackerState {
     Map<Path, LocalResourceProto> inProgressMap =
         new HashMap<Path, LocalResourceProto>();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f155ab7c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.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/recovery/TestNMLeveldbStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java
index b0a9bc9..b76f1ff 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java
@@ -32,6 +32,7 @@ import static org.mockito.Mockito.verify;
 
 import java.io.File;
 import java.io.IOException;
+import java.io.Serializable;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -961,46 +962,12 @@ public class TestNMLeveldbStateStoreService {
         .loadContainersState();
     assertTrue(recoveredContainers.isEmpty());
 
-    // create a container request
     ApplicationId appId = ApplicationId.newInstance(1234, 3);
     ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(appId,
         4);
     ContainerId containerId = ContainerId.newContainerId(appAttemptId, 5);
-    LocalResource lrsrc = LocalResource.newInstance(
-        URL.newInstance("hdfs", "somehost", 12345, "/some/path/to/rsrc"),
-        LocalResourceType.FILE, LocalResourceVisibility.APPLICATION, 123L,
-        1234567890L);
-    Map<String, LocalResource> localResources =
-        new HashMap<String, LocalResource>();
-    localResources.put("rsrc", lrsrc);
-    Map<String, String> env = new HashMap<String, String>();
-    env.put("somevar", "someval");
-    List<String> containerCmds = new ArrayList<String>();
-    containerCmds.add("somecmd");
-    containerCmds.add("somearg");
-    Map<String, ByteBuffer> serviceData = new HashMap<String, ByteBuffer>();
-    serviceData.put("someservice",
-        ByteBuffer.wrap(new byte[] { 0x1, 0x2, 0x3 }));
-    ByteBuffer containerTokens = ByteBuffer
-        .wrap(new byte[] { 0x7, 0x8, 0x9, 0xa });
-    Map<ApplicationAccessType, String> acls =
-        new HashMap<ApplicationAccessType, String>();
-    acls.put(ApplicationAccessType.VIEW_APP, "viewuser");
-    acls.put(ApplicationAccessType.MODIFY_APP, "moduser");
-    ContainerLaunchContext clc = ContainerLaunchContext.newInstance(
-        localResources, env, containerCmds,
-        serviceData, containerTokens, acls);
-    Resource containerRsrc = Resource.newInstance(1357, 3);
-    ContainerTokenIdentifier containerTokenId = new ContainerTokenIdentifier(
-        containerId, "host", "user", containerRsrc, 9876543210L, 42, 2468,
-        Priority.newInstance(7), 13579);
-    Token containerToken = Token.newInstance(containerTokenId.getBytes(),
-        ContainerTokenIdentifier.KIND.toString(), "password".getBytes(),
-        "tokenservice");
-    StartContainerRequest containerReq = StartContainerRequest.newInstance(clc,
-        containerToken);
-
-    stateStore.storeContainer(containerId, 0, 0, containerReq);
+    StartContainerRequest startContainerRequest = storeMockContainer(
+        containerId);
 
     // add a invalid key
     byte[] invalidKey = ("ContainerManager/containers/"
@@ -1013,7 +980,7 @@ public class TestNMLeveldbStateStoreService {
     assertEquals(RecoveredContainerStatus.REQUESTED, rcs.getStatus());
     assertEquals(ContainerExitStatus.INVALID, rcs.getExitCode());
     assertEquals(false, rcs.getKilled());
-    assertEquals(containerReq, rcs.getStartRequest());
+    assertEquals(startContainerRequest, rcs.getStartRequest());
     assertTrue(rcs.getDiagnostics().isEmpty());
     assertEquals(RecoveredContainerType.KILL, rcs.getRecoveryType());
     // assert unknown keys are cleaned up finally
@@ -1121,6 +1088,86 @@ public class TestNMLeveldbStateStoreService {
     }
   }
 
+  @Test
+  public void testStateStoreForResourceMapping() throws IOException {
+    // test empty when no state
+    List<RecoveredContainerState> recoveredContainers = stateStore
+        .loadContainersState();
+    assertTrue(recoveredContainers.isEmpty());
+
+    ApplicationId appId = ApplicationId.newInstance(1234, 3);
+    ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(appId,
+        4);
+    ContainerId containerId = ContainerId.newContainerId(appAttemptId, 5);
+    storeMockContainer(containerId);
+
+    // Store ResourceMapping
+    stateStore.storeAssignedResources(containerId, "gpu",
+        Arrays.asList("1", "2", "3"));
+    // This will overwrite above
+    List<Serializable> gpuRes1 = Arrays.asList("1", "2", "4");
+    stateStore.storeAssignedResources(containerId, "gpu", gpuRes1);
+    List<Serializable> fpgaRes = Arrays.asList("3", "4", "5", "6");
+    stateStore.storeAssignedResources(containerId, "fpga", fpgaRes);
+    List<Serializable> numaRes = Arrays.asList("numa1");
+    stateStore.storeAssignedResources(containerId, "numa", numaRes);
+
+    // add a invalid key
+    restartStateStore();
+    recoveredContainers = stateStore.loadContainersState();
+    assertEquals(1, recoveredContainers.size());
+    RecoveredContainerState rcs = recoveredContainers.get(0);
+    List<Serializable> res = rcs.getResourceMappings()
+        .getAssignedResources("gpu");
+    Assert.assertTrue(res.equals(gpuRes1));
+
+    res = rcs.getResourceMappings().getAssignedResources("fpga");
+    Assert.assertTrue(res.equals(fpgaRes));
+
+    res = rcs.getResourceMappings().getAssignedResources("numa");
+    Assert.assertTrue(res.equals(numaRes));
+  }
+
+  private StartContainerRequest storeMockContainer(ContainerId containerId)
+      throws IOException {
+    // create a container request
+    LocalResource lrsrc = LocalResource.newInstance(
+        URL.newInstance("hdfs", "somehost", 12345, "/some/path/to/rsrc"),
+        LocalResourceType.FILE, LocalResourceVisibility.APPLICATION, 123L,
+        1234567890L);
+    Map<String, LocalResource> localResources =
+        new HashMap<String, LocalResource>();
+    localResources.put("rsrc", lrsrc);
+    Map<String, String> env = new HashMap<String, String>();
+    env.put("somevar", "someval");
+    List<String> containerCmds = new ArrayList<String>();
+    containerCmds.add("somecmd");
+    containerCmds.add("somearg");
+    Map<String, ByteBuffer> serviceData = new HashMap<String, ByteBuffer>();
+    serviceData.put("someservice",
+        ByteBuffer.wrap(new byte[] { 0x1, 0x2, 0x3 }));
+    ByteBuffer containerTokens = ByteBuffer
+        .wrap(new byte[] { 0x7, 0x8, 0x9, 0xa });
+    Map<ApplicationAccessType, String> acls =
+        new HashMap<ApplicationAccessType, String>();
+    acls.put(ApplicationAccessType.VIEW_APP, "viewuser");
+    acls.put(ApplicationAccessType.MODIFY_APP, "moduser");
+    ContainerLaunchContext clc = ContainerLaunchContext.newInstance(
+        localResources, env, containerCmds,
+        serviceData, containerTokens, acls);
+    Resource containerRsrc = Resource.newInstance(1357, 3);
+    ContainerTokenIdentifier containerTokenId = new ContainerTokenIdentifier(
+        containerId, "host", "user", containerRsrc, 9876543210L, 42, 2468,
+        Priority.newInstance(7), 13579);
+    Token containerToken = Token.newInstance(containerTokenId.getBytes(),
+        ContainerTokenIdentifier.KIND.toString(), "password".getBytes(),
+        "tokenservice");
+    StartContainerRequest containerReq = StartContainerRequest.newInstance(clc,
+        containerToken);
+    stateStore.storeContainer(containerId, 0, 0, containerReq);
+    return containerReq;
+  }
+
   private static class NMTokenSecretManagerForTest extends
       BaseNMTokenSecretManager {
     public MasterKey generateKey() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f155ab7c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.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/webapp/MockContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java
index 57bee8c..d435ba0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ResourceMappings;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceSet;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 
@@ -239,4 +240,9 @@ public class MockContainer implements Container {
   public long getContainerStartTime() {
     return 0;
   }
+
+  @Override
+  public ResourceMappings getResourceMappings() {
+    return null;
+  }
 }


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


[11/50] [abbrv] hadoop git commit: YARN-5328. Plan/ResourceAllocation data structure enhancements required to support recurring reservations in ReservationSystem.

Posted by as...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.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/reservation/RLESparseResourceAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.java
index 658387b..3062f3d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.java
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
-import java.io.IOException;
-import java.io.StringWriter;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -33,8 +31,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.Plan
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
-import com.google.gson.stream.JsonWriter;
-
 /**
  * This is a run length encoded sparse data structure that maintains resource
  * allocations over time.
@@ -44,12 +40,14 @@ public class RLESparseResourceAllocation {
   private static final int THRESHOLD = 100;
   private static final Resource ZERO_RESOURCE = Resources.none();
 
-  private NavigableMap<Long, Resource> cumulativeCapacity =
+  @SuppressWarnings("checkstyle:visibilitymodifier")
+  protected NavigableMap<Long, Resource> cumulativeCapacity =
       new TreeMap<Long, Resource>();
 
   private final ReentrantReadWriteLock readWriteLock =
       new ReentrantReadWriteLock();
-  private final Lock readLock = readWriteLock.readLock();
+  @SuppressWarnings("checkstyle:visibilitymodifier")
+  protected final Lock readLock = readWriteLock.readLock();
   private final Lock writeLock = readWriteLock.writeLock();
 
   private final ResourceCalculator resourceCalculator;
@@ -236,34 +234,6 @@ public class RLESparseResourceAllocation {
   }
 
   /**
-   * Returns the JSON string representation of the current resources allocated
-   * over time.
-   *
-   * @return the JSON string representation of the current resources allocated
-   *         over time
-   */
-  public String toMemJSONString() {
-    StringWriter json = new StringWriter();
-    JsonWriter jsonWriter = new JsonWriter(json);
-    readLock.lock();
-    try {
-      jsonWriter.beginObject();
-      // jsonWriter.name("timestamp").value("resource");
-      for (Map.Entry<Long, Resource> r : cumulativeCapacity.entrySet()) {
-        jsonWriter.name(r.getKey().toString()).value(r.getValue().toString());
-      }
-      jsonWriter.endObject();
-      jsonWriter.close();
-      return json.toString();
-    } catch (IOException e) {
-      // This should not happen
-      return "";
-    } finally {
-      readLock.unlock();
-    }
-  }
-
-  /**
    * Returns the representation of the current resources allocated over time as
    * an interval map (in the defined non-null range).
    *
@@ -437,8 +407,8 @@ public class RLESparseResourceAllocation {
       Resource val = Resources.negate(e.getValue());
       // test for negative value and throws
       if (operator == RLEOperator.subtractTestNonNegative
-          && (Resources.fitsIn(val, ZERO_RESOURCE) &&
-              !Resources.equals(val, ZERO_RESOURCE))) {
+          && (Resources.fitsIn(val, ZERO_RESOURCE)
+              && !Resources.equals(val, ZERO_RESOURCE))) {
         throw new PlanningException(
             "RLESparseResourceAllocation: merge failed as the "
                 + "resulting RLESparseResourceAllocation would be negative");
@@ -504,22 +474,29 @@ public class RLESparseResourceAllocation {
 
   }
 
+  /**
+   * Get a {@link RLESparseResourceAllocation} view of the {@link Resource}
+   * allocations between the specified start and end times.
+   *
+   * @param start the time from which the {@link Resource} allocations are
+   *          required
+   * @param end the time upto which the {@link Resource} allocations are
+   *          required
+   * @return the overlapping allocations
+   */
   public RLESparseResourceAllocation getRangeOverlapping(long start, long end) {
     readLock.lock();
     try {
       NavigableMap<Long, Resource> a = this.getCumulative();
-
       if (a != null && !a.isEmpty()) {
         // include the portion of previous entry that overlaps start
         if (start > a.firstKey()) {
           long previous = a.floorKey(start);
           a = a.tailMap(previous, true);
         }
-
         if (end < a.lastKey()) {
           a = a.headMap(end, true);
         }
-
       }
       RLESparseResourceAllocation ret =
           new RLESparseResourceAllocation(a, resourceCalculator);
@@ -527,7 +504,33 @@ public class RLESparseResourceAllocation {
     } finally {
       readLock.unlock();
     }
+  }
 
+  /**
+   * This method shifts all the timestamp of the {@link Resource} entries by the
+   * specified "delta".
+   *
+   * @param delta the time by which to shift the {@link Resource} allocations
+   */
+  public void shift(long delta) {
+    writeLock.lock();
+    try {
+      TreeMap<Long, Resource> newCum = new TreeMap<>();
+      long start;
+      for (Map.Entry<Long, Resource> entry : cumulativeCapacity.entrySet()) {
+        if (delta > 0) {
+          start = (entry.getKey() == Long.MAX_VALUE) ? Long.MAX_VALUE
+              : entry.getKey() + delta;
+        } else {
+          start = (entry.getKey() == Long.MIN_VALUE) ? Long.MIN_VALUE
+              : entry.getKey() + delta;
+        }
+        newCum.put(start, entry.getValue());
+      }
+      cumulativeCapacity = newCum;
+    } finally {
+      writeLock.unlock();
+    }
   }
 
   /**
@@ -541,8 +544,8 @@ public class RLESparseResourceAllocation {
   /**
    * Get the maximum capacity across specified time instances. The search-space
    * is specified using the starting value, tick, and the periodic interval for
-   * search. Maximum resource allocation across tick, tick + period,
-   * tick + 2 * period,..., tick + n * period .. is returned.
+   * search. Maximum resource allocation across tick, tick + period, tick + 2 *
+   * period,..., tick + n * period .. is returned.
    *
    * @param tick the starting time instance
    * @param period interval at which capacity is evaluated
@@ -550,14 +553,19 @@ public class RLESparseResourceAllocation {
    */
   public Resource getMaximumPeriodicCapacity(long tick, long period) {
     Resource maxCapacity = ZERO_RESOURCE;
-    if (!cumulativeCapacity.isEmpty()) {
-      Long lastKey = cumulativeCapacity.lastKey();
-      for (long t = tick; t <= lastKey; t = t + period) {
-        maxCapacity = Resources.componentwiseMax(maxCapacity,
-            cumulativeCapacity.floorEntry(t).getValue());
+    readLock.lock();
+    try {
+      if (!cumulativeCapacity.isEmpty()) {
+        Long lastKey = cumulativeCapacity.lastKey();
+        for (long t = tick; t <= lastKey; t = t + period) {
+          maxCapacity = Resources.componentwiseMax(maxCapacity,
+              cumulativeCapacity.floorEntry(t).getValue());
+        }
       }
+      return maxCapacity;
+    } finally {
+      readLock.unlock();
     }
-    return maxCapacity;
   }
 
   /**
@@ -567,17 +575,17 @@ public class RLESparseResourceAllocation {
    * @return minimum resource allocation
    */
   public Resource getMinimumCapacityInInterval(ReservationInterval interval) {
-    Resource minCapacity = Resource.newInstance(
-        Integer.MAX_VALUE, Integer.MAX_VALUE);
+    Resource minCapacity =
+        Resource.newInstance(Integer.MAX_VALUE, Integer.MAX_VALUE);
     long start = interval.getStartTime();
     long end = interval.getEndTime();
     NavigableMap<Long, Resource> capacityRange =
-        this.getRangeOverlapping(start, end).getCumulative();
+        getRangeOverlapping(start, end).getCumulative();
     if (!capacityRange.isEmpty()) {
       for (Map.Entry<Long, Resource> entry : capacityRange.entrySet()) {
         if (entry.getValue() != null) {
-          minCapacity = Resources.componentwiseMin(minCapacity,
-              entry.getValue());
+          minCapacity =
+              Resources.componentwiseMin(minCapacity, entry.getValue());
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationAllocation.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/reservation/ReservationAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationAllocation.java
index 0da95ac..bb4a7fb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationAllocation.java
@@ -24,14 +24,16 @@ import org.apache.hadoop.yarn.api.records.ReservationDefinition;
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * A ReservationAllocation represents a concrete allocation of resources over
  * time that satisfy a certain {@link ReservationDefinition}. This is used
  * internally by a {@link Plan} to store information about how each of the
  * accepted {@link ReservationDefinition} have been allocated.
  */
-public interface ReservationAllocation extends
-    Comparable<ReservationAllocation> {
+public interface ReservationAllocation
+    extends Comparable<ReservationAllocation> {
 
   /**
    * Returns the unique identifier {@link ReservationId} that represents the
@@ -40,28 +42,28 @@ public interface ReservationAllocation extends
    * @return reservationId the unique identifier {@link ReservationId} that
    *         represents the reservation
    */
-  public ReservationId getReservationId();
+  ReservationId getReservationId();
 
   /**
    * Returns the original {@link ReservationDefinition} submitted by the client
    * 
    * @return the {@link ReservationDefinition} submitted by the client
    */
-  public ReservationDefinition getReservationDefinition();
+  ReservationDefinition getReservationDefinition();
 
   /**
    * Returns the time at which the reservation is activated.
    * 
    * @return the time at which the reservation is activated
    */
-  public long getStartTime();
+  long getStartTime();
 
   /**
    * Returns the time at which the reservation terminates.
    * 
    * @return the time at which the reservation terminates
    */
-  public long getEndTime();
+  long getEndTime();
 
   /**
    * Returns the map of resources requested against the time interval for which
@@ -70,28 +72,28 @@ public interface ReservationAllocation extends
    * @return the allocationRequests the map of resources requested against the
    *         time interval for which they were
    */
-  public Map<ReservationInterval, Resource> getAllocationRequests();
+  Map<ReservationInterval, Resource> getAllocationRequests();
 
   /**
    * Return a string identifying the plan to which the reservation belongs
    * 
    * @return the plan to which the reservation belongs
    */
-  public String getPlanName();
+  String getPlanName();
 
   /**
    * Returns the user who requested the reservation
    * 
    * @return the user who requested the reservation
    */
-  public String getUser();
+  String getUser();
 
   /**
    * Returns whether the reservation has gang semantics or not
    * 
    * @return true if there is a gang request, false otherwise
    */
-  public boolean containsGangs();
+  boolean containsGangs();
 
   /**
    * Sets the time at which the reservation was accepted by the system
@@ -99,14 +101,14 @@ public interface ReservationAllocation extends
    * @param acceptedAt the time at which the reservation was accepted by the
    *          system
    */
-  public void setAcceptanceTimestamp(long acceptedAt);
+  void setAcceptanceTimestamp(long acceptedAt);
 
   /**
    * Returns the time at which the reservation was accepted by the system
    * 
    * @return the time at which the reservation was accepted by the system
    */
-  public long getAcceptanceTime();
+  long getAcceptanceTime();
 
   /**
    * Returns the capacity represented by cumulative resources reserved by the
@@ -116,12 +118,42 @@ public interface ReservationAllocation extends
    *          requested
    * @return the resources reserved at the specified time
    */
-  public Resource getResourcesAtTime(long tick);
+  Resource getResourcesAtTime(long tick);
+
+  /**
+   * Return a RLE representation of used resources.
+   *
+   * @return a RLE encoding of resources allocated over time.
+   */
+  RLESparseResourceAllocation getResourcesOverTime();
+
 
   /**
    * Return a RLE representation of used resources.
+   *
+   * @param start start of the time interval.
+   * @param end end of the time interval.
    * @return a RLE encoding of resources allocated over time.
    */
-  public RLESparseResourceAllocation getResourcesOverTime();
+  RLESparseResourceAllocation getResourcesOverTime(long start, long end);
+
+  /**
+   * Get the periodicity of this reservation representing the time period of the
+   * periodic job. Period is represented in milliseconds for periodic jobs.
+   * Period is 0 for non-periodic jobs.
+   *
+   * @return periodicity of this reservation
+   */
+  long getPeriodicity();
+
+  /**
+   * Set the periodicity of this reservation representing the time period of the
+   * periodic job. Period is represented in milliseconds for periodic jobs.
+   * Period is 0 for non-periodic jobs.
+   *
+   * @param period periodicity of this reservation
+   */
+  @VisibleForTesting
+  void setPeriodicity(long period);
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.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/reservation/ReservationInputValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.java
index 027d066..a66d222 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.java
@@ -44,6 +44,8 @@ public class ReservationInputValidator {
 
   /**
    * Utility class to validate reservation requests.
+   *
+   * @param clock the {@link Clock} to use
    */
   public ReservationInputValidator(Clock clock) {
     this.clock = clock;
@@ -53,22 +55,21 @@ public class ReservationInputValidator {
       ReservationId reservationId, String auditConstant) throws YarnException {
     // check if the reservation id is valid
     if (reservationId == null) {
-      String message =
-          "Missing reservation id."
-              + " Please try again by specifying a reservation id.";
+      String message = "Missing reservation id."
+          + " Please try again by specifying a reservation id.";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
     }
     String queue = reservationSystem.getQueueForReservation(reservationId);
     String nullQueueErrorMessage =
-            "The specified reservation with ID: " + reservationId
-                    + " is unknown. Please try again with a valid reservation.";
+        "The specified reservation with ID: " + reservationId
+            + " is unknown. Please try again with a valid reservation.";
     String nullPlanErrorMessage = "The specified reservation: " + reservationId
-                            + " is not associated with any valid plan."
-                            + " Please try again with a valid reservation.";
+        + " is not associated with any valid plan."
+        + " Please try again with a valid reservation.";
     return getPlanFromQueue(reservationSystem, queue, auditConstant,
-            nullQueueErrorMessage, nullPlanErrorMessage);
+        nullQueueErrorMessage, nullPlanErrorMessage);
   }
 
   private void validateReservationDefinition(ReservationId reservationId,
@@ -77,17 +78,15 @@ public class ReservationInputValidator {
     String message = "";
     // check if deadline is in the past
     if (contract == null) {
-      message =
-          "Missing reservation definition."
-              + " Please try again by specifying a reservation definition.";
+      message = "Missing reservation definition."
+          + " Please try again by specifying a reservation definition.";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input definition", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
     }
     if (contract.getDeadline() <= clock.getTime()) {
-      message =
-          "The specified deadline: " + contract.getDeadline()
-              + " is the past. Please try again with deadline in the future.";
+      message = "The specified deadline: " + contract.getDeadline()
+          + " is the past. Please try again with deadline in the future.";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input definition", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
@@ -95,18 +94,16 @@ public class ReservationInputValidator {
     // Check if at least one RR has been specified
     ReservationRequests resReqs = contract.getReservationRequests();
     if (resReqs == null) {
-      message =
-          "No resources have been specified to reserve."
-              + "Please try again by specifying the resources to reserve.";
+      message = "No resources have been specified to reserve."
+          + "Please try again by specifying the resources to reserve.";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input definition", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
     }
     List<ReservationRequest> resReq = resReqs.getReservationResources();
     if (resReq == null || resReq.isEmpty()) {
-      message =
-          "No resources have been specified to reserve."
-              + " Please try again by specifying the resources to reserve.";
+      message = "No resources have been specified to reserve."
+          + " Please try again by specifying the resources to reserve.";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input definition", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
@@ -123,22 +120,18 @@ public class ReservationInputValidator {
       } else {
         minDuration += rr.getDuration();
       }
-      maxGangSize =
-          Resources.max(plan.getResourceCalculator(), plan.getTotalCapacity(),
-              maxGangSize,
-              Resources.multiply(rr.getCapability(), rr.getConcurrency()));
+      maxGangSize = Resources.max(plan.getResourceCalculator(),
+          plan.getTotalCapacity(), maxGangSize,
+          Resources.multiply(rr.getCapability(), rr.getConcurrency()));
     }
     // verify the allocation is possible (skip for ANY)
     long duration = contract.getDeadline() - contract.getArrival();
-    if (duration < minDuration
-        && type != ReservationRequestInterpreter.R_ANY) {
-      message =
-          "The time difference ("
-              + (duration)
-              + ") between arrival (" + contract.getArrival() + ") "
-              + "and deadline (" + contract.getDeadline() + ") must "
-              + " be greater or equal to the minimum resource duration ("
-              + minDuration + ")";
+    if (duration < minDuration && type != ReservationRequestInterpreter.R_ANY) {
+      message = "The time difference (" + (duration) + ") between arrival ("
+          + contract.getArrival() + ") " + "and deadline ("
+          + contract.getDeadline() + ") must "
+          + " be greater or equal to the minimum resource duration ("
+          + minDuration + ")";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input definition", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
@@ -148,10 +141,9 @@ public class ReservationInputValidator {
     if (Resources.greaterThan(plan.getResourceCalculator(),
         plan.getTotalCapacity(), maxGangSize, plan.getTotalCapacity())
         && type != ReservationRequestInterpreter.R_ANY) {
-      message =
-          "The size of the largest gang in the reservation definition ("
-              + maxGangSize + ") exceed the capacity available ("
-              + plan.getTotalCapacity() + " )";
+      message = "The size of the largest gang in the reservation definition ("
+          + maxGangSize + ") exceed the capacity available ("
+          + plan.getTotalCapacity() + " )";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input definition", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
@@ -179,32 +171,32 @@ public class ReservationInputValidator {
     }
   }
 
-  private Plan getPlanFromQueue(ReservationSystem reservationSystem, String
-          queue, String auditConstant) throws YarnException {
+  private Plan getPlanFromQueue(ReservationSystem reservationSystem,
+      String queue, String auditConstant) throws YarnException {
     String nullQueueErrorMessage = "The queue is not specified."
-            + " Please try again with a valid reservable queue.";
+        + " Please try again with a valid reservable queue.";
     String nullPlanErrorMessage = "The specified queue: " + queue
-            + " is not managed by reservation system."
-            + " Please try again with a valid reservable queue.";
+        + " is not managed by reservation system."
+        + " Please try again with a valid reservable queue.";
     return getPlanFromQueue(reservationSystem, queue, auditConstant,
-            nullQueueErrorMessage, nullPlanErrorMessage);
+        nullQueueErrorMessage, nullPlanErrorMessage);
   }
 
-  private Plan getPlanFromQueue(ReservationSystem reservationSystem, String
-          queue, String auditConstant, String nullQueueErrorMessage,
-          String nullPlanErrorMessage) throws YarnException {
+  private Plan getPlanFromQueue(ReservationSystem reservationSystem,
+      String queue, String auditConstant, String nullQueueErrorMessage,
+      String nullPlanErrorMessage) throws YarnException {
     if (queue == null || queue.isEmpty()) {
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
-              "validate reservation input", "ClientRMService",
-              nullQueueErrorMessage);
+          "validate reservation input", "ClientRMService",
+          nullQueueErrorMessage);
       throw RPCUtil.getRemoteException(nullQueueErrorMessage);
     }
     // check if the associated plan is valid
     Plan plan = reservationSystem.getPlan(queue);
     if (plan == null) {
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
-              "validate reservation input", "ClientRMService",
-              nullPlanErrorMessage);
+          "validate reservation input", "ClientRMService",
+          nullPlanErrorMessage);
       throw RPCUtil.getRemoteException(nullPlanErrorMessage);
     }
     return plan;
@@ -222,22 +214,21 @@ public class ReservationInputValidator {
    * @param reservationId the {@link ReservationId} associated with the current
    *          request
    * @return the {@link Plan} to submit the request to
-   * @throws YarnException
+   * @throws YarnException if validation fails
    */
   public Plan validateReservationSubmissionRequest(
-      ReservationSystem reservationSystem,
-      ReservationSubmissionRequest request, ReservationId reservationId)
-      throws YarnException {
+      ReservationSystem reservationSystem, ReservationSubmissionRequest request,
+      ReservationId reservationId) throws YarnException {
     String message;
     if (reservationId == null) {
-      message = "Reservation id cannot be null. Please try again " +
-        "specifying a valid reservation id by creating a new reservation id.";
+      message = "Reservation id cannot be null. Please try again specifying "
+          + " a valid reservation id by creating a new reservation id.";
       throw RPCUtil.getRemoteException(message);
     }
     // Check if it is a managed queue
     String queue = request.getQueue();
     Plan plan = getPlanFromQueue(reservationSystem, queue,
-            AuditConstants.SUBMIT_RESERVATION_REQUEST);
+        AuditConstants.SUBMIT_RESERVATION_REQUEST);
 
     validateReservationDefinition(reservationId,
         request.getReservationDefinition(), plan,
@@ -255,15 +246,14 @@ public class ReservationInputValidator {
    * @param request the {@link ReservationUpdateRequest} defining the resources
    *          required over time for the request
    * @return the {@link Plan} to submit the request to
-   * @throws YarnException
+   * @throws YarnException if validation fails
    */
   public Plan validateReservationUpdateRequest(
       ReservationSystem reservationSystem, ReservationUpdateRequest request)
       throws YarnException {
     ReservationId reservationId = request.getReservationId();
-    Plan plan =
-        validateReservation(reservationSystem, reservationId,
-            AuditConstants.UPDATE_RESERVATION_REQUEST);
+    Plan plan = validateReservation(reservationSystem, reservationId,
+        AuditConstants.UPDATE_RESERVATION_REQUEST);
     validateReservationDefinition(reservationId,
         request.getReservationDefinition(), plan,
         AuditConstants.UPDATE_RESERVATION_REQUEST);
@@ -278,28 +268,26 @@ public class ReservationInputValidator {
    *
    * @param reservationSystem the {@link ReservationSystem} to validate against
    * @param request the {@link ReservationListRequest} defining search
-   *                parameters for reservations in the {@link ReservationSystem}
-   *                that is being validated against.
+   *          parameters for reservations in the {@link ReservationSystem} that
+   *          is being validated against.
    * @return the {@link Plan} to list reservations of.
-   * @throws YarnException
+   * @throws YarnException if validation fails
    */
   public Plan validateReservationListRequest(
-      ReservationSystem reservationSystem,
-      ReservationListRequest request)
+      ReservationSystem reservationSystem, ReservationListRequest request)
       throws YarnException {
     String queue = request.getQueue();
     if (request.getEndTime() < request.getStartTime()) {
-      String errorMessage = "The specified end time must be greater than " +
-              "the specified start time.";
+      String errorMessage = "The specified end time must be greater than "
+          + "the specified start time.";
       RMAuditLogger.logFailure("UNKNOWN",
-              AuditConstants.LIST_RESERVATION_REQUEST,
-              "validate list reservation input", "ClientRMService",
-              errorMessage);
+          AuditConstants.LIST_RESERVATION_REQUEST,
+          "validate list reservation input", "ClientRMService", errorMessage);
       throw RPCUtil.getRemoteException(errorMessage);
     }
     // Check if it is a managed queue
     return getPlanFromQueue(reservationSystem, queue,
-            AuditConstants.LIST_RESERVATION_REQUEST);
+        AuditConstants.LIST_RESERVATION_REQUEST);
   }
 
   /**
@@ -312,7 +300,7 @@ public class ReservationInputValidator {
    * @param request the {@link ReservationDeleteRequest} defining the resources
    *          required over time for the request
    * @return the {@link Plan} to submit the request to
-   * @throws YarnException
+   * @throws YarnException if validation fails
    */
   public Plan validateReservationDeleteRequest(
       ReservationSystem reservationSystem, ReservationDeleteRequest request)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystem.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/reservation/ReservationSystem.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystem.java
index 8b62972..a6c8fcf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystem.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystem.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
+import java.util.Map;
+
 import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
@@ -29,8 +31,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ReservationsACLsManager;
 
-import java.util.Map;
-
 /**
  * This interface is the one implemented by any system that wants to support
  * Reservations i.e. make {@code Resource} allocations in future. Implementors
@@ -57,7 +57,7 @@ public interface ReservationSystem extends Recoverable {
    * 
    * @param conf configuration
    * @param rmContext current context of the {@code ResourceManager}
-   * @throws YarnException
+   * @throws YarnException if initialization of the configured plan fails
    */
   void reinitialize(Configuration conf, RMContext rmContext)
       throws YarnException;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/SharingPolicy.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/reservation/SharingPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/SharingPolicy.java
index e458055..cbf0f38 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/SharingPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/SharingPolicy.java
@@ -38,7 +38,7 @@ public interface SharingPolicy {
    * @param planQueuePath the name of the queue for this plan
    * @param conf the system configuration
    */
-  public void init(String planQueuePath, ReservationSchedulerConfiguration conf);
+  void init(String planQueuePath, ReservationSchedulerConfiguration conf);
 
   /**
    * This method runs the policy validation logic, and return true/false on
@@ -51,7 +51,7 @@ public interface SharingPolicy {
    * @throws PlanningException if the policy is respected if we add this
    *           {@link ReservationAllocation} to the {@link Plan}
    */
-  public void validate(Plan plan, ReservationAllocation newAllocation)
+  void validate(Plan plan, ReservationAllocation newAllocation)
       throws PlanningException;
 
   /**
@@ -68,9 +68,13 @@ public interface SharingPolicy {
    * @param start the start time for the range we are querying
    * @param end the end time for the range we are querying
    * @param oldId (optional) the id of a reservation being updated
+   *
+   * @return the available resources expressed as a
+   *         {@link RLESparseResourceAllocation}
+   *
    * @throws PlanningException throws if the request is not valid
    */
-  public RLESparseResourceAllocation availableResources(
+  RLESparseResourceAllocation availableResources(
       RLESparseResourceAllocation available, Plan plan, String user,
       ReservationId oldId, long start, long end) throws PlanningException;
 
@@ -82,7 +86,6 @@ public interface SharingPolicy {
    * 
    * @return validWindow the window of validity considered by the policy.
    */
-  public long getValidWindow();
-
+  long getValidWindow();
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/Planner.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/reservation/planning/Planner.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/Planner.java
index abac6ac..af0e712 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/Planner.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/Planner.java
@@ -34,7 +34,7 @@ public interface Planner {
    *
    * @param plan the {@link Plan} to replan
    * @param contracts the list of reservation requests
-   * @throws PlanningException
+   * @throws PlanningException if operation is unsuccessful
    */
   public void plan(Plan plan, List<ReservationDefinition> contracts)
       throws PlanningException;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/PlanningAlgorithm.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/reservation/planning/PlanningAlgorithm.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/PlanningAlgorithm.java
index 199bfa5..bbbf0d6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/PlanningAlgorithm.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/PlanningAlgorithm.java
@@ -50,7 +50,7 @@ public abstract class PlanningAlgorithm implements ReservationAgent {
    * @return whether the allocateUser function was successful or not
    *
    * @throws PlanningException if the session cannot be fitted into the plan
-   * @throws ContractValidationException
+   * @throws ContractValidationException if validation fails
    */
   protected boolean allocateUser(ReservationId reservationId, String user,
       Plan plan, ReservationDefinition contract,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocator.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/reservation/planning/StageAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocator.java
index ec6d9c0..8934b0f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocator.java
@@ -50,7 +50,7 @@ public interface StageAllocator {
    *
    * @return The computed allocation (or null if the stage could not be
    *         allocated)
-   * @throws PlanningException
+   * @throws PlanningException if operation is unsuccessful
    */
   Map<ReservationInterval, Resource> computeStageAllocation(Plan plan,
       RLESparseResourceAllocation planLoads,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedy.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/reservation/planning/StageAllocatorGreedy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedy.java
index da04336..d107487 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedy.java
@@ -69,7 +69,7 @@ public class StageAllocatorGreedy implements StageAllocator {
 
     RLESparseResourceAllocation netAvailable =
         plan.getAvailableResourceOverTime(user, oldId, stageEarliestStart,
-            stageDeadline);
+            stageDeadline, 0);
 
     netAvailable =
         RLESparseResourceAllocation.merge(plan.getResourceCalculator(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedyRLE.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/reservation/planning/StageAllocatorGreedyRLE.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedyRLE.java
index ec83e02..ae7d91a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedyRLE.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedyRLE.java
@@ -83,9 +83,8 @@ public class StageAllocatorGreedyRLE implements StageAllocator {
     int gangsToPlace = rr.getNumContainers() / rr.getConcurrency();
 
     // get available resources from plan
-    RLESparseResourceAllocation netRLERes =
-        plan.getAvailableResourceOverTime(user, oldId, stageEarliestStart,
-            stageDeadline);
+    RLESparseResourceAllocation netRLERes = plan.getAvailableResourceOverTime(
+        user, oldId, stageEarliestStart, stageDeadline, 0);
 
     // remove plan modifications
     netRLERes =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorLowCostAligned.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/reservation/planning/StageAllocatorLowCostAligned.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorLowCostAligned.java
index e45f58c..c014549 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorLowCostAligned.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorLowCostAligned.java
@@ -77,8 +77,8 @@ public class StageAllocatorLowCostAligned implements StageAllocator {
     ResourceCalculator resCalc = plan.getResourceCalculator();
     Resource capacity = plan.getTotalCapacity();
 
-    RLESparseResourceAllocation netRLERes = plan
-        .getAvailableResourceOverTime(user, oldId, stageArrival, stageDeadline);
+    RLESparseResourceAllocation netRLERes = plan.getAvailableResourceOverTime(
+        user, oldId, stageArrival, stageDeadline, 0);
 
     long step = plan.getStep();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6e7d136/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.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/reservation/ReservationSystemTestUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
index e99842e..5337e06 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
@@ -19,7 +19,10 @@ package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anySetOf;
-import static org.mockito.Mockito.*;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.when;
 
 import java.io.FileWriter;
 import java.io.IOException;
@@ -76,7 +79,8 @@ public class ReservationSystemTestUtil {
       String reservationQ, long timeWindow, float instConstraint,
       float avgConstraint) {
 
-    ReservationSchedulerConfiguration realConf = new CapacitySchedulerConfiguration();
+    ReservationSchedulerConfiguration realConf =
+        new CapacitySchedulerConfiguration();
     ReservationSchedulerConfiguration conf = spy(realConf);
     when(conf.getReservationWindow(reservationQ)).thenReturn(timeWindow);
     when(conf.getInstantaneousMaxCapacity(reservationQ))
@@ -168,7 +172,6 @@ public class ReservationSystemTestUtil {
     scheduler.start();
     scheduler.reinitialize(conf, rmContext);
 
-
     Resource resource =
         ReservationSystemTestUtil.calculateClusterResource(numContainers);
     RMNode node1 = MockNodes.newNodeInfo(1, resource, 1, "127.0.0.1");
@@ -184,10 +187,16 @@ public class ReservationSystemTestUtil {
 
   public static ReservationDefinition createSimpleReservationDefinition(
       long arrival, long deadline, long duration, int parallelism) {
+    return createSimpleReservationDefinition(arrival, deadline, duration,
+        parallelism, null);
+  }
+
+  public static ReservationDefinition createSimpleReservationDefinition(
+      long arrival, long deadline, long duration, int parallelism,
+      String recurrenceExpression) {
     // create a request with a single atomic ask
-    ReservationRequest r =
-        ReservationRequest.newInstance(Resource.newInstance(1024, 1),
-            parallelism, parallelism, duration);
+    ReservationRequest r = ReservationRequest.newInstance(
+        Resource.newInstance(1024, 1), parallelism, parallelism, duration);
     ReservationDefinition rDef = new ReservationDefinitionPBImpl();
     ReservationRequests reqs = new ReservationRequestsPBImpl();
     reqs.setReservationResources(Collections.singletonList(r));
@@ -195,32 +204,31 @@ public class ReservationSystemTestUtil {
     rDef.setReservationRequests(reqs);
     rDef.setArrival(arrival);
     rDef.setDeadline(deadline);
+    if (recurrenceExpression != null) {
+      rDef.setRecurrenceExpression(recurrenceExpression);
+    }
     return rDef;
   }
 
   public static ReservationSubmissionRequest createSimpleReservationRequest(
       ReservationId reservationId, int numContainers, long arrival,
       long deadline, long duration) {
-    return createSimpleReservationRequest(reservationId, numContainers,
-        arrival, deadline, duration, Priority.UNDEFINED);
+    return createSimpleReservationRequest(reservationId, numContainers, arrival,
+        deadline, duration, Priority.UNDEFINED);
   }
 
   public static ReservationSubmissionRequest createSimpleReservationRequest(
       ReservationId reservationId, int numContainers, long arrival,
       long deadline, long duration, Priority priority) {
     // create a request with a single atomic ask
-    ReservationRequest r =
-        ReservationRequest.newInstance(Resource.newInstance(1024, 1),
-            numContainers, 1, duration);
-    ReservationRequests reqs =
-        ReservationRequests.newInstance(Collections.singletonList(r),
-            ReservationRequestInterpreter.R_ALL);
-    ReservationDefinition rDef =
-        ReservationDefinition.newInstance(arrival, deadline, reqs,
-            "testClientRMService#reservation", "0", priority);
-    ReservationSubmissionRequest request =
-        ReservationSubmissionRequest.newInstance(rDef,
-            reservationQ, reservationId);
+    ReservationRequest r = ReservationRequest
+        .newInstance(Resource.newInstance(1024, 1), numContainers, 1, duration);
+    ReservationRequests reqs = ReservationRequests.newInstance(
+        Collections.singletonList(r), ReservationRequestInterpreter.R_ALL);
+    ReservationDefinition rDef = ReservationDefinition.newInstance(arrival,
+        deadline, reqs, "testClientRMService#reservation", "0", priority);
+    ReservationSubmissionRequest request = ReservationSubmissionRequest
+        .newInstance(rDef, reservationQ, reservationId);
     return request;
   }
 
@@ -252,9 +260,9 @@ public class ReservationSystemTestUtil {
     return cs;
   }
 
-  @SuppressWarnings("rawtypes") public static void initializeRMContext(
-      int numContainers, AbstractYarnScheduler scheduler,
-      RMContext mockRMContext) {
+  @SuppressWarnings("rawtypes")
+  public static void initializeRMContext(int numContainers,
+      AbstractYarnScheduler scheduler, RMContext mockRMContext) {
 
     when(mockRMContext.getScheduler()).thenReturn(scheduler);
     Resource r = calculateClusterResource(numContainers);
@@ -262,26 +270,25 @@ public class ReservationSystemTestUtil {
   }
 
   public static RMContext createRMContext(Configuration conf) {
-    RMContext mockRmContext = Mockito.spy(
-        new RMContextImpl(null, null, null, null, null, null,
-            new RMContainerTokenSecretManager(conf),
-            new NMTokenSecretManagerInRM(conf),
-            new ClientToAMTokenSecretManagerInRM(), null));
+    RMContext mockRmContext = Mockito.spy(new RMContextImpl(null, null, null,
+        null, null, null, new RMContainerTokenSecretManager(conf),
+        new NMTokenSecretManagerInRM(conf),
+        new ClientToAMTokenSecretManagerInRM(), null));
 
     RMNodeLabelsManager nlm = mock(RMNodeLabelsManager.class);
     when(nlm.getQueueResource(any(String.class), anySetOf(String.class),
-            any(Resource.class))).thenAnswer(new Answer<Resource>() {
-      @Override public Resource answer(InvocationOnMock invocation)
-          throws Throwable {
-        Object[] args = invocation.getArguments();
-        return (Resource) args[2];
-      }
-    });
+        any(Resource.class))).thenAnswer(new Answer<Resource>() {
+          @Override
+          public Resource answer(InvocationOnMock invocation) throws Throwable {
+            Object[] args = invocation.getArguments();
+            return (Resource) args[2];
+          }
+        });
 
     when(nlm.getResourceByLabel(any(String.class), any(Resource.class)))
         .thenAnswer(new Answer<Resource>() {
-          @Override public Resource answer(InvocationOnMock invocation)
-              throws Throwable {
+          @Override
+          public Resource answer(InvocationOnMock invocation) throws Throwable {
             Object[] args = invocation.getArguments();
             return (Resource) args[1];
           }
@@ -304,9 +311,8 @@ public class ReservationSystemTestUtil {
     final String A = CapacitySchedulerConfiguration.ROOT + ".a";
     conf.setCapacity(A, 10);
 
-    final String dedicated =
-        CapacitySchedulerConfiguration.ROOT + CapacitySchedulerConfiguration.DOT
-            + reservationQ;
+    final String dedicated = CapacitySchedulerConfiguration.ROOT
+        + CapacitySchedulerConfiguration.DOT + reservationQ;
     conf.setCapacity(dedicated, 80);
     // Set as reservation queue
     conf.setReservable(dedicated, true);
@@ -405,26 +411,55 @@ public class ReservationSystemTestUtil {
 
   public static Map<ReservationInterval, Resource> generateAllocation(
       long startTime, long step, int[] alloc) {
+    return generateAllocation(startTime, step, alloc, null);
+  }
+
+  public static Map<ReservationInterval, Resource> generateAllocation(
+      long startTime, long step, int[] alloc, String recurrenceExpression) {
     Map<ReservationInterval, Resource> req = new TreeMap<>();
-    for (int i = 0; i < alloc.length; i++) {
-      req.put(new ReservationInterval(startTime + i * step,
-          startTime + (i + 1) * step), ReservationSystemUtil.toResource(
-          ReservationRequest
-              .newInstance(Resource.newInstance(1024, 1), alloc[i])));
+
+    long period = 0;
+    if (recurrenceExpression != null) {
+      period = Long.parseLong(recurrenceExpression);
+    }
+
+    long rStart;
+    long rEnd;
+    for (int j = 0; j < 86400000; j += period) {
+      for (int i = 0; i < alloc.length; i++) {
+        rStart = (startTime + i * step) + j * period;
+        rEnd = (startTime + (i + 1) * step) + j * period;
+        if (period > 0) {
+          rStart = rStart % period + j * period;
+          rEnd = rEnd % period + j * period;
+          if (rStart > rEnd) {
+            // skip wrap-around entry
+            continue;
+          }
+        }
+
+        req.put(new ReservationInterval(rStart, rEnd),
+            ReservationSystemUtil.toResource(ReservationRequest
+                .newInstance(Resource.newInstance(1024, 1), alloc[i])));
+
+      }
+      // execute only once if non-periodic
+      if (period == 0) {
+        break;
+      }
     }
     return req;
   }
 
-  public static RLESparseResourceAllocation
-      generateRLESparseResourceAllocation(int[] alloc, long[] timeSteps) {
+  public static RLESparseResourceAllocation generateRLESparseResourceAllocation(
+      int[] alloc, long[] timeSteps) {
     TreeMap<Long, Resource> allocationsMap = new TreeMap<>();
     for (int i = 0; i < alloc.length; i++) {
       allocationsMap.put(timeSteps[i],
           Resource.newInstance(alloc[i], alloc[i]));
     }
-    RLESparseResourceAllocation rleVector =
-        new RLESparseResourceAllocation(allocationsMap,
-            new DefaultResourceCalculator());
+    RLESparseResourceAllocation rleVector = new RLESparseResourceAllocation(
+        allocationsMap, new DefaultResourceCalculator());
     return rleVector;
   }
 


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


[15/50] [abbrv] hadoop git commit: HDFS-12376. Enable JournalNode Sync by default. Contributed by Hanisha Koneru.

Posted by as...@apache.org.
HDFS-12376. Enable JournalNode Sync by default. Contributed by Hanisha Koneru.


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

Branch: refs/heads/YARN-5972
Commit: 5ff74167ddd378a738898425c4462f3432f9c31d
Parents: 2adf8be
Author: Arpit Agarwal <ar...@apache.org>
Authored: Thu Sep 7 08:18:28 2017 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Sep 7 08:18:28 2017 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java        | 2 +-
 .../hadoop-hdfs/src/main/resources/hdfs-default.xml                | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5ff74167/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 17cabad..bc7b716 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
@@ -924,7 +924,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_JOURNALNODE_KERBEROS_INTERNAL_SPNEGO_PRINCIPAL_KEY = "dfs.journalnode.kerberos.internal.spnego.principal";
   public static final String DFS_JOURNALNODE_ENABLE_SYNC_KEY =
       "dfs.journalnode.enable.sync";
-  public static final boolean DFS_JOURNALNODE_ENABLE_SYNC_DEFAULT = false;
+  public static final boolean DFS_JOURNALNODE_ENABLE_SYNC_DEFAULT = true;
   public static final String DFS_JOURNALNODE_SYNC_INTERVAL_KEY =
       "dfs.journalnode.sync.interval";
   public static final long DFS_JOURNALNODE_SYNC_INTERVAL_DEFAULT = 2*60*1000L;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5ff74167/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 bbe2eca..d58e54e 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
@@ -3933,7 +3933,7 @@
 
 <property>
   <name>dfs.journalnode.enable.sync</name>
-  <value>false</value>
+  <value>true</value>
   <description>
     If true, the journal nodes wil sync with each other. The journal nodes
     will periodically gossip with other journal nodes to compare edit log


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


[31/50] [abbrv] hadoop git commit: HDFS-12400. Provide a way for NN to drain the local key cache before re-encryption.

Posted by as...@apache.org.
HDFS-12400. Provide a way for NN to drain the local key cache before re-encryption.


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

Branch: refs/heads/YARN-5972
Commit: b3a4d7d2a01051e166c06ee78e8c6e8df1341948
Parents: fa61375
Author: Xiao Chen <xi...@apache.org>
Authored: Thu Sep 7 20:50:03 2017 -0700
Committer: Xiao Chen <xi...@apache.org>
Committed: Thu Sep 7 20:51:02 2017 -0700

----------------------------------------------------------------------
 .../crypto/key/KeyProviderCryptoExtension.java  |  10 ++
 .../server/namenode/FSDirEncryptionZoneOp.java  |   5 +-
 .../hdfs/server/namenode/FSNamesystem.java      |   2 +
 .../hdfs/server/namenode/TestReencryption.java  | 103 +++++++------------
 .../namenode/TestReencryptionWithKMS.java       |   5 +
 5 files changed, 56 insertions(+), 69 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b3a4d7d2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java
index 693c785..3ee3bd7 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java
@@ -555,6 +555,16 @@ public class KeyProviderCryptoExtension extends
   }
 
   /**
+   * Calls {@link CryptoExtension#drain(String)} for the given key name on the
+   * underlying {@link CryptoExtension}.
+   *
+   * @param keyName
+   */
+  public void drain(String keyName) {
+    getExtension().drain(keyName);
+  }
+
+  /**
    * Batched version of {@link #reencryptEncryptedKey(EncryptedKeyVersion)}.
    * <p>
    * For each encrypted key version, re-encrypts an encrypted key version,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b3a4d7d2/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 ee2b0f4..e284b15 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
@@ -32,7 +32,6 @@ import org.apache.hadoop.crypto.CipherSuite;
 import org.apache.hadoop.crypto.CryptoProtocolVersion;
 import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
-import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.CryptoExtension;
 import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion;
 import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.fs.FileStatus;
@@ -698,9 +697,7 @@ final class FSDirEncryptionZoneOp {
     // drain the local cache of the key provider.
     // Do not invalidateCache on the server, since that's the responsibility
     // when rolling the key version.
-    if (dir.getProvider() instanceof CryptoExtension) {
-      ((CryptoExtension) dir.getProvider()).drain(keyName);
-    }
+    dir.getProvider().drain(keyName);
     final EncryptedKeyVersion edek;
     try {
       edek = dir.getProvider().generateEncryptedKey(keyName);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b3a4d7d2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 08c8562..d9f3c0e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -7111,6 +7111,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       if (keyVersionName == null) {
         throw new IOException("Failed to get key version name for " + zone);
       }
+      LOG.info("Re-encryption using key version " + keyVersionName
+          + " for zone " + zone);
     }
     writeLock();
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b3a4d7d2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryption.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryption.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryption.java
index 5612d65..33c52bf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryption.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryption.java
@@ -91,7 +91,7 @@ public class TestReencryption {
   private FileSystemTestHelper fsHelper;
 
   private MiniDFSCluster cluster;
-  private HdfsAdmin dfsAdmin;
+  protected HdfsAdmin dfsAdmin;
   private DistributedFileSystem fs;
   private FSNamesystem fsn;
   private File testRootDir;
@@ -199,8 +199,7 @@ public class TestReencryption {
     verifyZoneStatus(zone, null, 0);
 
     // test re-encrypt after keyroll
-    fsn.getProvider().rollNewVersion(TEST_KEY);
-    fsn.getProvider().flush();
+    rollKey(TEST_KEY);
     dfsAdmin.reencryptEncryptionZone(zone, ReencryptAction.START);
     waitForReencryptedZones(2);
     FileEncryptionInfo fei1 = getFileEncryptionInfo(encFile1);
@@ -316,8 +315,7 @@ public class TestReencryption {
     final Path notReencrypted = new Path(zone, "f0");
     final FileEncryptionInfo fei = getFileEncryptionInfo(lastReencryptedFile);
     final FileEncryptionInfo feiLast = getFileEncryptionInfo(notReencrypted);
-    fsn.getProvider().rollNewVersion(TEST_KEY);
-    fsn.getProvider().flush();
+    rollKey(TEST_KEY);
     // mark pause after first checkpoint (5 files)
     getEzManager().pauseForTestingAfterNthSubmission(1);
     dfsAdmin.reencryptEncryptionZone(zone, ReencryptAction.START);
@@ -363,8 +361,7 @@ public class TestReencryption {
               0xFEED);
     }
 
-    fsn.getProvider().rollNewVersion(TEST_KEY);
-    fsn.getProvider().flush();
+    rollKey(TEST_KEY);
     // test zone deleted during re-encrypt's checkpointing
     getEzManager().pauseForTestingAfterNthSubmission(1);
     getEzManager().resetMetricsForTesting();
@@ -409,8 +406,7 @@ public class TestReencryption {
     final Path encFile9 = new Path(zone, "9");
     final FileEncryptionInfo fei0 = getFileEncryptionInfo(encFile0);
     final FileEncryptionInfo fei9 = getFileEncryptionInfo(encFile9);
-    fsn.getProvider().rollNewVersion(TEST_KEY);
-    fsn.getProvider().flush();
+    rollKey(TEST_KEY);
     dfsAdmin.reencryptEncryptionZone(zone, ReencryptAction.START);
     waitForReencryptedZones(1);
 
@@ -443,8 +439,7 @@ public class TestReencryption {
     fsWrapper.rename(new Path(zone, "f"), new Path(zone, "f1"));
 
     // re-encrypt
-    fsn.getProvider().rollNewVersion(TEST_KEY);
-    fsn.getProvider().flush();
+    rollKey(TEST_KEY);
     dfsAdmin.reencryptEncryptionZone(zone, ReencryptAction.START);
     waitForReencryptedZones(1);
 
@@ -495,8 +490,7 @@ public class TestReencryption {
     final Path encFile9 = new Path(subdir, "9");
     final FileEncryptionInfo fei0 = getFileEncryptionInfo(encFile0);
     final FileEncryptionInfo fei9 = getFileEncryptionInfo(encFile9);
-    fsn.getProvider().rollNewVersion(TEST_KEY);
-    fsn.getProvider().flush();
+    rollKey(TEST_KEY);
     // mark pause after first checkpoint (5 files)
     getEzManager().pauseForTestingAfterNthSubmission(1);
     dfsAdmin.reencryptEncryptionZone(zone, ReencryptAction.START);
@@ -540,8 +534,7 @@ public class TestReencryption {
     final Path encFile9 = new Path(zone, "9");
     final FileEncryptionInfo fei0 = getFileEncryptionInfo(encFile0);
     final FileEncryptionInfo fei9 = getFileEncryptionInfo(encFile9);
-    fsn.getProvider().rollNewVersion(TEST_KEY);
-    fsn.getProvider().flush();
+    rollKey(TEST_KEY);
     dfsAdmin.reencryptEncryptionZone(zone, ReencryptAction.START);
     waitForReencryptedZones(1);
 
@@ -585,8 +578,7 @@ public class TestReencryption {
     final Path encFile9 = new Path(zone, "9");
     final FileEncryptionInfo fei0 = getFileEncryptionInfo(encFile0);
     final FileEncryptionInfo fei9 = getFileEncryptionInfo(encFile9);
-    fsn.getProvider().rollNewVersion(TEST_KEY);
-    fsn.getProvider().flush();
+    rollKey(TEST_KEY);
     // disable re-encrypt for testing, and issue a command
     getEzManager().pauseReencryptForTesting();
     dfsAdmin.reencryptEncryptionZone(zone, ReencryptAction.START);
@@ -645,8 +637,7 @@ public class TestReencryption {
     final Path encFile9 = new Path(zone, "9");
     final FileEncryptionInfo fei0 = getFileEncryptionInfo(encFile0);
     final FileEncryptionInfo fei9 = getFileEncryptionInfo(encFile9);
-    fsn.getProvider().rollNewVersion(TEST_KEY);
-    fsn.getProvider().flush();
+    rollKey(TEST_KEY);
     // disable re-encrypt for testing, and issue a command
     getEzManager().pauseReencryptForTesting();
     dfsAdmin.reencryptEncryptionZone(zone, ReencryptAction.START);
@@ -770,8 +761,7 @@ public class TestReencryption {
               0xFEED);
     }
 
-    fsn.getProvider().rollNewVersion(TEST_KEY);
-    fsn.getProvider().flush();
+    rollKey(TEST_KEY);
     // Disable re-encrypt, send re-encrypt on '/', verify queue
     getEzManager().pauseReencryptForTesting();
     dfsAdmin.reencryptEncryptionZone(zoneRoot, ReencryptAction.START);
@@ -816,8 +806,7 @@ public class TestReencryption {
           .createFile(fs, new Path(zone, "file" + i), len, (short) 1, 0xFEED);
     }
 
-    fsn.getProvider().rollNewVersion(TEST_KEY);
-    fsn.getProvider().flush();
+    rollKey(TEST_KEY);
     // Issue the command re-encrypt and pause it
     getEzManager().pauseReencryptForTesting();
     dfsAdmin.reencryptEncryptionZone(zone, ReencryptAction.START);
@@ -883,8 +872,7 @@ public class TestReencryption {
           .createFile(fs, new Path(subdir, "file" + i), len, (short) 1, 0xFEED);
     }
 
-    fsn.getProvider().rollNewVersion(TEST_KEY);
-    fsn.getProvider().flush();
+    rollKey(TEST_KEY);
     // Issue the command re-encrypt and pause it
     getEzManager().pauseReencryptForTesting();
     dfsAdmin.reencryptEncryptionZone(zone, ReencryptAction.START);
@@ -930,8 +918,7 @@ public class TestReencryption {
           .createFile(fs, new Path(subdir, "file" + i), len, (short) 1, 0xFEED);
     }
 
-    fsn.getProvider().rollNewVersion(TEST_KEY);
-    fsn.getProvider().flush();
+    rollKey(TEST_KEY);
     // Issue the command re-encrypt and pause it
     getEzManager().pauseReencryptForTesting();
     dfsAdmin.reencryptEncryptionZone(zone, ReencryptAction.START);
@@ -984,8 +971,7 @@ public class TestReencryption {
           0xFEED);
     }
 
-    fsn.getProvider().rollNewVersion(TEST_KEY);
-    fsn.getProvider().flush();
+    rollKey(TEST_KEY);
     // Issue the command re-encrypt and pause it
     getEzManager().pauseReencryptForTesting();
     dfsAdmin.reencryptEncryptionZone(zone, ReencryptAction.START);
@@ -1029,8 +1015,7 @@ public class TestReencryption {
           0xFEED);
     }
 
-    fsn.getProvider().rollNewVersion(TEST_KEY);
-    fsn.getProvider().flush();
+    rollKey(TEST_KEY);
     // Issue the command re-encrypt and pause it
     getEzManager().pauseReencryptForTesting();
     dfsAdmin.reencryptEncryptionZone(zone, ReencryptAction.START);
@@ -1071,8 +1056,7 @@ public class TestReencryption {
           .createFile(fs, new Path(zone, "file" + i), len, (short) 1, 0xFEED);
     }
 
-    fsn.getProvider().rollNewVersion(TEST_KEY);
-    fsn.getProvider().flush();
+    rollKey(TEST_KEY);
     // Issue the command re-encrypt and pause it
     getEzManager().pauseReencryptForTesting();
     dfsAdmin.reencryptEncryptionZone(zone, ReencryptAction.START);
@@ -1122,8 +1106,7 @@ public class TestReencryption {
           .createFile(fs, new Path(zone, "file" + i), len, (short) 1, 0xFEED);
     }
 
-    fsn.getProvider().rollNewVersion(TEST_KEY);
-    fsn.getProvider().flush();
+    rollKey(TEST_KEY);
     // Issue the command re-encrypt and pause it
     getEzManager().pauseReencryptForTesting();
     dfsAdmin.reencryptEncryptionZone(zone, ReencryptAction.START);
@@ -1162,8 +1145,7 @@ public class TestReencryption {
           .createFile(fs, new Path(zone, "file" + i), len, (short) 1, 0xFEED);
     }
 
-    fsn.getProvider().rollNewVersion(TEST_KEY);
-    fsn.getProvider().flush();
+    rollKey(TEST_KEY);
     // Issue the command re-encrypt and pause it
     getEzManager().pauseReencryptForTesting();
     dfsAdmin.reencryptEncryptionZone(zone, ReencryptAction.START);
@@ -1220,8 +1202,7 @@ public class TestReencryption {
           .createFile(fs, new Path(subdir, "file" + i), len, (short) 1, 0xFEED);
     }
 
-    fsn.getProvider().rollNewVersion(TEST_KEY);
-    fsn.getProvider().flush();
+    rollKey(TEST_KEY);
     // Issue the command re-encrypt and pause it
     getEzManager().pauseReencryptForTesting();
     dfsAdmin.reencryptEncryptionZone(zone, ReencryptAction.START);
@@ -1283,8 +1264,7 @@ public class TestReencryption {
     // test re-encrypt on snapshot dir
     final Path encFile1 = new Path(zone, "0");
     final FileEncryptionInfo fei0 = getFileEncryptionInfo(encFile1);
-    fsn.getProvider().rollNewVersion(TEST_KEY);
-    fsn.getProvider().flush();
+    rollKey(TEST_KEY);
     try {
       dfsAdmin.reencryptEncryptionZone(zoneSnap, ReencryptAction.START);
       fail("Reencrypt command on snapshot path should fail.");
@@ -1423,8 +1403,7 @@ public class TestReencryption {
     fsWrapper.mkdir(subdir, FsPermission.getDirDefault(), true);
     DFSTestUtil.createFile(fs, new Path(subdir, "f"), len, (short) 1, 0xFEED);
 
-    fsn.getProvider().rollNewVersion(TEST_KEY);
-    fsn.getProvider().flush();
+    rollKey(TEST_KEY);
     // disable, test basic
     getEzManager().pauseReencryptForTesting();
     dfsAdmin.reencryptEncryptionZone(zone, ReencryptAction.START);
@@ -1442,8 +1421,7 @@ public class TestReencryption {
       assertExceptionContains("not under re-encryption", expected);
     }
 
-    fsn.getProvider().rollNewVersion(TEST_KEY);
-    fsn.getProvider().flush();
+    rollKey(TEST_KEY);
     // test cancelling half-way
     getEzManager().pauseForTestingAfterNthSubmission(1);
     getEzManager().resumeReencryptForTesting();
@@ -1537,8 +1515,7 @@ public class TestReencryption {
 
     // re-encrypt 10 files, so 2 callables. Hang 1, pause the updater so the
     // callable is taken from the executor but not processed.
-    fsn.getProvider().rollNewVersion(TEST_KEY);
-    fsn.getProvider().flush();
+    rollKey(TEST_KEY);
     getEzManager().pauseReencryptForTesting();
     dfsAdmin.reencryptEncryptionZone(zone, ReencryptAction.START);
     waitForQueuedZones(1);
@@ -1593,8 +1570,7 @@ public class TestReencryption {
     fsWrapper.mkdir(subdir, FsPermission.getDirDefault(), true);
     DFSTestUtil.createFile(fs, new Path(subdir, "f"), len, (short) 1, 0xFEED);
 
-    fsn.getProvider().rollNewVersion(TEST_KEY);
-    fsn.getProvider().flush();
+    rollKey(TEST_KEY);
     // disable, test basic
     getEzManager().pauseReencryptUpdaterForTesting();
     dfsAdmin.reencryptEncryptionZone(zone, ReencryptAction.START);
@@ -1625,8 +1601,7 @@ public class TestReencryption {
     }
 
     // re-encrypt the zone
-    fsn.getProvider().rollNewVersion(TEST_KEY);
-    fsn.getProvider().flush();
+    rollKey(TEST_KEY);
     dfsAdmin.reencryptEncryptionZone(zone, ReencryptAction.START);
     waitForReencryptedZones(1);
 
@@ -1678,8 +1653,7 @@ public class TestReencryption {
               0xFEED);
     }
 
-    fsn.getProvider().rollNewVersion(TEST_KEY);
-    fsn.getProvider().flush();
+    rollKey(TEST_KEY);
     // mark pause after first checkpoint (5 files)
     getEzManager().pauseForTestingAfterNthSubmission(1);
     dfsAdmin.reencryptEncryptionZone(zone, ReencryptAction.START);
@@ -1736,9 +1710,7 @@ public class TestReencryption {
     }
 
     // re-encrypt the zone
-    fsn.getProvider().rollNewVersion(TEST_KEY);
-    fsn.getProvider().flush();
-
+    rollKey(TEST_KEY);
     dfsAdmin.reencryptEncryptionZone(zone, ReencryptAction.START);
     waitForReencryptedZones(1);
     assertEquals(0, injector.exceptionCount);
@@ -1790,9 +1762,7 @@ public class TestReencryption {
     }
 
     // re-encrypt the zone
-    fsn.getProvider().rollNewVersion(TEST_KEY);
-    fsn.getProvider().flush();
-
+    rollKey(TEST_KEY);
     dfsAdmin.reencryptEncryptionZone(zone, ReencryptAction.START);
     waitForReencryptedZones(1);
     assertEquals(0, injector.exceptionCount);
@@ -1845,9 +1815,7 @@ public class TestReencryption {
     }
 
     // re-encrypt the zone
-    fsn.getProvider().rollNewVersion(TEST_KEY);
-    fsn.getProvider().flush();
-
+    rollKey(TEST_KEY);
     dfsAdmin.reencryptEncryptionZone(zone, ReencryptAction.START);
     waitForReencryptedZones(1);
     assertEquals(0, injector.exceptionCount);
@@ -1899,9 +1867,7 @@ public class TestReencryption {
     }
 
     // re-encrypt the zone
-    fsn.getProvider().rollNewVersion(TEST_KEY);
-    fsn.getProvider().flush();
-
+    rollKey(TEST_KEY);
     Whitebox.setInternalState(getUpdater(), "faultRetryInterval", 50);
     dfsAdmin.reencryptEncryptionZone(zone, ReencryptAction.START);
     waitForReencryptedZones(1);
@@ -1929,4 +1895,11 @@ public class TestReencryption {
     return (ReencryptionUpdater) Whitebox
         .getInternalState(getHandler(), "reencryptionUpdater");
   }
+
+  protected void rollKey(final String keyName) throws Exception {
+    dfsAdmin.getKeyProvider().rollNewVersion(keyName);
+    // need to flush for jceks provider to make the key version it returned
+    // after NN  restart consistent.
+    dfsAdmin.getKeyProvider().flush();
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b3a4d7d2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryptionWithKMS.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryptionWithKMS.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryptionWithKMS.java
index af9c381..642d5e5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryptionWithKMS.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryptionWithKMS.java
@@ -88,4 +88,9 @@ public class TestReencryptionWithKMS extends TestReencryption{
     KMSWebApp.getACLs().run();
     testReencryptionBasic();
   }
+
+  @Override
+  protected void rollKey(final String keyName) throws Exception {
+    dfsAdmin.getKeyProvider().rollNewVersion(keyName);
+  }
 }


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


[05/50] [abbrv] hadoop git commit: HADOOP-14827. Allow StopWatch to accept a Timer parameter for tests. Contributed by Erik Krogen

Posted by as...@apache.org.
HADOOP-14827. Allow StopWatch to accept a Timer parameter for tests. Contributed by Erik Krogen


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

Branch: refs/heads/YARN-5972
Commit: dd814946f68d52a9b1627ac4dd61f9ab093423ae
Parents: 704267c
Author: Jason Lowe <jl...@apache.org>
Authored: Wed Sep 6 16:04:30 2017 -0500
Committer: Jason Lowe <jl...@apache.org>
Committed: Wed Sep 6 16:04:30 2017 -0500

----------------------------------------------------------------------
 .../java/org/apache/hadoop/util/StopWatch.java     | 17 ++++++++++++++---
 1 file changed, 14 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd814946/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StopWatch.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StopWatch.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StopWatch.java
index b9d0d0b..c0eedf6 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StopWatch.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StopWatch.java
@@ -25,11 +25,22 @@ import java.util.concurrent.TimeUnit;
  * A simplified StopWatch implementation which can measure times in nanoseconds.
  */
 public class StopWatch implements Closeable {
+  private final Timer timer;
   private boolean isStarted;
   private long startNanos;
   private long currentElapsedNanos;
 
   public StopWatch() {
+    this(new Timer());
+  }
+
+  /**
+   * Used for tests to be able to create a StopWatch which does not follow real
+   * time.
+   * @param timer The timer to base this StopWatch's timekeeping off of.
+   */
+  public StopWatch(Timer timer) {
+    this.timer = timer;
   }
 
   /**
@@ -49,7 +60,7 @@ public class StopWatch implements Closeable {
       throw new IllegalStateException("StopWatch is already running");
     }
     isStarted = true;
-    startNanos = System.nanoTime();
+    startNanos = timer.monotonicNowNanos();
     return this;
   }
 
@@ -61,7 +72,7 @@ public class StopWatch implements Closeable {
     if (!isStarted) {
       throw new IllegalStateException("StopWatch is already stopped");
     }
-    long now = System.nanoTime();
+    long now = timer.monotonicNowNanos();
     isStarted = false;
     currentElapsedNanos += now - startNanos;
     return this;
@@ -90,7 +101,7 @@ public class StopWatch implements Closeable {
    */
   public long now() {
     return isStarted ?
-        System.nanoTime() - startNanos + currentElapsedNanos :
+        timer.monotonicNowNanos() - startNanos + currentElapsedNanos :
         currentElapsedNanos;
   }
 


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


[16/50] [abbrv] hadoop git commit: HDFS-12357. Let NameNode to bypass external attribute provider for configured users. Contributed by Yongjun Zhang, Arun Suresh.

Posted by as...@apache.org.
HDFS-12357. Let NameNode to bypass external attribute provider for configured users. Contributed by Yongjun Zhang, Arun Suresh.


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

Branch: refs/heads/YARN-5972
Commit: d77ed238a911fc85d6f4bbce606cac7ec44f557f
Parents: 5ff7416
Author: Yongjun Zhang <yz...@cloudera.com>
Authored: Thu Sep 7 09:45:45 2017 -0700
Committer: Yongjun Zhang <yz...@cloudera.com>
Committed: Thu Sep 7 09:50:36 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   3 +
 .../hdfs/server/namenode/FSDirectory.java       |  58 +++++++++-
 .../src/main/resources/hdfs-default.xml         |  12 +++
 .../namenode/TestINodeAttributeProvider.java    | 105 +++++++++++++++++--
 4 files changed, 167 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d77ed238/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 bc7b716..d06e378 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
@@ -641,6 +641,9 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_DATANODE_MIN_SUPPORTED_NAMENODE_VERSION_KEY = "dfs.datanode.min.supported.namenode.version";
   public static final String  DFS_DATANODE_MIN_SUPPORTED_NAMENODE_VERSION_DEFAULT = "2.1.0-beta";
   public static final String  DFS_NAMENODE_INODE_ATTRIBUTES_PROVIDER_KEY = "dfs.namenode.inode.attributes.provider.class";
+  public static final String  DFS_NAMENODE_INODE_ATTRIBUTES_PROVIDER_BYPASS_USERS_KEY = "dfs.namenode.inode.attributes.provider.bypass.users";
+  public static final String  DFS_NAMENODE_INODE_ATTRIBUTES_PROVIDER_BYPASS_USERS_DEFAULT = "";
+
   public static final String  DFS_DATANODE_BP_READY_TIMEOUT_KEY = "dfs.datanode.bp-ready.timeout";
   public static final long    DFS_DATANODE_BP_READY_TIMEOUT_DEFAULT = 20;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d77ed238/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
index e6aa533..6604b5a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
@@ -74,6 +74,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.EnumSet;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.SortedSet;
@@ -202,6 +203,10 @@ public class FSDirectory implements Closeable {
 
   private INodeAttributeProvider attributeProvider;
 
+  // A HashSet of principals of users for whom the external attribute provider
+  // will be bypassed
+  private HashSet<String> usersToBypassExtAttrProvider = null;
+
   public void setINodeAttributeProvider(INodeAttributeProvider provider) {
     attributeProvider = provider;
   }
@@ -357,6 +362,49 @@ public class FSDirectory implements Closeable {
     this.quotaInitThreads = conf.getInt(
         DFSConfigKeys.DFS_NAMENODE_QUOTA_INIT_THREADS_KEY,
         DFSConfigKeys.DFS_NAMENODE_QUOTA_INIT_THREADS_DEFAULT);
+
+    initUsersToBypassExtProvider(conf);
+  }
+
+  private void initUsersToBypassExtProvider(Configuration conf) {
+    String[] bypassUsers = conf.getTrimmedStrings(
+        DFSConfigKeys.DFS_NAMENODE_INODE_ATTRIBUTES_PROVIDER_BYPASS_USERS_KEY,
+        DFSConfigKeys.DFS_NAMENODE_INODE_ATTRIBUTES_PROVIDER_BYPASS_USERS_DEFAULT);
+    for(int i = 0; i < bypassUsers.length; i++) {
+      String tmp = bypassUsers[i].trim();
+      if (!tmp.isEmpty()) {
+        if (usersToBypassExtAttrProvider == null) {
+          usersToBypassExtAttrProvider = new HashSet<String>();
+        }
+        LOG.info("Add user " + tmp + " to the list that will bypass external"
+            + " attribute provider.");
+        usersToBypassExtAttrProvider.add(tmp);
+      }
+    }
+  }
+
+  /**
+   * Check if a given user is configured to bypass external attribute provider.
+   * @param user user principal
+   * @return true if the user is to bypass external attribute provider
+   */
+  private boolean isUserBypassingExtAttrProvider(final String user) {
+    return (usersToBypassExtAttrProvider != null) &&
+          usersToBypassExtAttrProvider.contains(user);
+  }
+
+  /**
+   * Return attributeProvider or null if ugi is to bypass attributeProvider.
+   * @param ugi
+   * @return configured attributeProvider or null
+   */
+  private INodeAttributeProvider getUserFilteredAttributeProvider(
+      UserGroupInformation ugi) {
+    if (attributeProvider == null ||
+        (ugi != null && isUserBypassingExtAttrProvider(ugi.getUserName()))) {
+      return null;
+    }
+    return attributeProvider;
   }
 
   /**
@@ -1711,7 +1759,7 @@ public class FSDirectory implements Closeable {
   FSPermissionChecker getPermissionChecker(String fsOwner, String superGroup,
       UserGroupInformation ugi) throws AccessControlException {
     return new FSPermissionChecker(
-        fsOwner, superGroup, ugi, attributeProvider);
+        fsOwner, superGroup, ugi, getUserFilteredAttributeProvider(ugi));
   }
 
   void checkOwner(FSPermissionChecker pc, INodesInPath iip)
@@ -1896,18 +1944,20 @@ public class FSDirectory implements Closeable {
   }
 
   INodeAttributes getAttributes(INodesInPath iip)
-      throws FileNotFoundException {
+      throws IOException {
     INode node = FSDirectory.resolveLastINode(iip);
     int snapshot = iip.getPathSnapshotId();
     INodeAttributes nodeAttrs = node.getSnapshotINode(snapshot);
-    if (attributeProvider != null) {
+    UserGroupInformation ugi = NameNode.getRemoteUser();
+    INodeAttributeProvider ap = this.getUserFilteredAttributeProvider(ugi);
+    if (ap != null) {
       // permission checking sends the full components array including the
       // first empty component for the root.  however file status
       // related calls are expected to strip out the root component according
       // to TestINodeAttributeProvider.
       byte[][] components = iip.getPathComponents();
       components = Arrays.copyOfRange(components, 1, components.length);
-      nodeAttrs = attributeProvider.getAttributes(components, nodeAttrs);
+      nodeAttrs = ap.getAttributes(components, nodeAttrs);
     }
     return nodeAttrs;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d77ed238/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 d58e54e..36c74f6 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
@@ -4160,6 +4160,18 @@
 </property>
 
 <property>
+  <name>dfs.namenode.authorization.provider.bypass.users</name>
+  <value></value>
+  <description>
+    A list of user principals (in secure cluster) or user names (in insecure
+    cluster) for whom the external attribute provider will be bypassed for all
+    operations. This means file attributes stored in HDFS instead of the
+    external provider will be used for permission checking and be returned when
+    requested.
+  </description>
+</property>
+
+<property>
   <name>dfs.namenode.max-num-blocks-to-log</name>
   <value>1000</value>
   <description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d77ed238/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeAttributeProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeAttributeProvider.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeAttributeProvider.java
index ffdc535..bbc5fa0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeAttributeProvider.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeAttributeProvider.java
@@ -24,6 +24,7 @@ import java.util.Map;
 import java.util.Set;
 
 import com.google.common.collect.ImmutableList;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -33,19 +34,25 @@ import org.apache.hadoop.fs.permission.*;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.server.namenode.INodeAttributeProvider.AccessControlEnforcer;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.Lists;
 
 public class TestINodeAttributeProvider {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestINodeAttributeProvider.class);
+
   private MiniDFSCluster miniDFS;
   private static final Set<String> CALLED = new HashSet<String>();
+  private static final short HDFS_PERMISSION = 0777;
+  private static final short PROVIDER_PERMISSION = 0770;
 
   public static class MyAuthorizationProvider extends INodeAttributeProvider {
 
@@ -112,7 +119,8 @@ public class TestINodeAttributeProvider {
 
         @Override
         public long getPermissionLong() {
-          return (useDefault) ? inode.getPermissionLong() : 0770;
+          return (useDefault) ? inode.getPermissionLong() :
+            (long)PROVIDER_PERMISSION;
         }
 
         @Override
@@ -177,6 +185,9 @@ public class TestINodeAttributeProvider {
     conf.set(DFSConfigKeys.DFS_NAMENODE_INODE_ATTRIBUTES_PROVIDER_KEY,
         MyAuthorizationProvider.class.getName());
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY, true);
+    conf.set(
+        DFSConfigKeys.DFS_NAMENODE_INODE_ATTRIBUTES_PROVIDER_BYPASS_USERS_KEY,
+        " u2,, ,u3, ");
     EditLogFileOutputStream.setShouldSkipFsyncForTesting(true);
     miniDFS = new MiniDFSCluster.Builder(conf).build();
   }
@@ -195,8 +206,11 @@ public class TestINodeAttributeProvider {
   public void testDelegationToProvider() throws Exception {
     Assert.assertTrue(CALLED.contains("start"));
     FileSystem fs = FileSystem.get(miniDFS.getConfiguration(0));
-    fs.mkdirs(new Path("/tmp"));
-    fs.setPermission(new Path("/tmp"), new FsPermission((short) 0777));
+    final Path tmpPath = new Path("/tmp");
+    final Path fooPath = new Path("/tmp/foo");
+
+    fs.mkdirs(tmpPath);
+    fs.setPermission(tmpPath, new FsPermission(HDFS_PERMISSION));
     UserGroupInformation ugi = UserGroupInformation.createUserForTesting("u1",
         new String[]{"g1"});
     ugi.doAs(new PrivilegedExceptionAction<Void>() {
@@ -204,17 +218,19 @@ public class TestINodeAttributeProvider {
       public Void run() throws Exception {
         FileSystem fs = FileSystem.get(miniDFS.getConfiguration(0));
         CALLED.clear();
-        fs.mkdirs(new Path("/tmp/foo"));
+        fs.mkdirs(fooPath);
         Assert.assertTrue(CALLED.contains("getAttributes"));
         Assert.assertTrue(CALLED.contains("checkPermission|null|null|null"));
         Assert.assertTrue(CALLED.contains("checkPermission|WRITE|null|null"));
+
         CALLED.clear();
-        fs.listStatus(new Path("/tmp/foo"));
+        fs.listStatus(fooPath);
         Assert.assertTrue(CALLED.contains("getAttributes"));
         Assert.assertTrue(
             CALLED.contains("checkPermission|null|null|READ_EXECUTE"));
+
         CALLED.clear();
-        fs.getAclStatus(new Path("/tmp/foo"));
+        fs.getAclStatus(fooPath);
         Assert.assertTrue(CALLED.contains("getAttributes"));
         Assert.assertTrue(CALLED.contains("checkPermission|null|null|null"));
         return null;
@@ -222,6 +238,81 @@ public class TestINodeAttributeProvider {
     });
   }
 
+  private class AssertHelper {
+    private boolean bypass = true;
+    AssertHelper(boolean bp) {
+      bypass = bp;
+    }
+    public void doAssert(boolean x) {
+      if (bypass) {
+        Assert.assertFalse(x);
+      } else {
+        Assert.assertTrue(x);
+      }
+    }
+  }
+
+  private void testBypassProviderHelper(final String[] users,
+      final short expectedPermission, final boolean bypass) throws Exception {
+    final AssertHelper asserter = new AssertHelper(bypass);
+
+    Assert.assertTrue(CALLED.contains("start"));
+
+    FileSystem fs = FileSystem.get(miniDFS.getConfiguration(0));
+    final Path userPath = new Path("/user");
+    final Path authz = new Path("/user/authz");
+    final Path authzChild = new Path("/user/authz/child2");
+
+    fs.mkdirs(userPath);
+    fs.setPermission(userPath, new FsPermission(HDFS_PERMISSION));
+    fs.mkdirs(authz);
+    fs.setPermission(authz, new FsPermission(HDFS_PERMISSION));
+    fs.mkdirs(authzChild);
+    fs.setPermission(authzChild, new FsPermission(HDFS_PERMISSION));
+    for(String user : users) {
+      UserGroupInformation ugiBypass =
+          UserGroupInformation.createUserForTesting(user,
+              new String[]{"g1"});
+      ugiBypass.doAs(new PrivilegedExceptionAction<Void>() {
+        @Override
+        public Void run() throws Exception {
+          FileSystem fs = FileSystem.get(miniDFS.getConfiguration(0));
+          Assert.assertEquals(expectedPermission,
+              fs.getFileStatus(authzChild).getPermission().toShort());
+          asserter.doAssert(CALLED.contains("getAttributes"));
+          asserter.doAssert(CALLED.contains("checkPermission|null|null|null"));
+
+          CALLED.clear();
+          Assert.assertEquals(expectedPermission,
+              fs.listStatus(userPath)[0].getPermission().toShort());
+          asserter.doAssert(CALLED.contains("getAttributes"));
+          asserter.doAssert(
+              CALLED.contains("checkPermission|null|null|READ_EXECUTE"));
+
+          CALLED.clear();
+          fs.getAclStatus(authzChild);
+          asserter.doAssert(CALLED.contains("getAttributes"));
+          asserter.doAssert(CALLED.contains("checkPermission|null|null|null"));
+          return null;
+        }
+      });
+    }
+  }
+
+  @Test
+  public void testAuthzDelegationToProvider() throws Exception {
+    LOG.info("Test not bypassing provider");
+    String[] users = {"u1"};
+    testBypassProviderHelper(users, PROVIDER_PERMISSION, false);
+  }
+
+  @Test
+  public void testAuthzBypassingProvider() throws Exception {
+    LOG.info("Test bypassing provider");
+    String[] users = {"u2", "u3"};
+    testBypassProviderHelper(users, HDFS_PERMISSION, true);
+  }
+
   @Test
   public void testCustomProvider() throws Exception {
     FileSystem fs = FileSystem.get(miniDFS.getConfiguration(0));


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


[07/50] [abbrv] hadoop git commit: Revert "Plan/ResourceAllocation data structure enhancements required to support recurring reservations in ReservationSystem."

Posted by as...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryPlan.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/reservation/TestInMemoryPlan.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryPlan.java
index 03569d4..bc98e2f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryPlan.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryPlan.java
@@ -40,7 +40,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.reservation.planning.Planne
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.planning.ReservationAgent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.util.Clock;
-import org.apache.hadoop.yarn.util.UTCClock;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.junit.After;
@@ -48,10 +47,6 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
-/**
- * Testing the class {@link InMemoryPlan}.
- */
-@SuppressWarnings("checkstyle:nowhitespaceafter")
 public class TestInMemoryPlan {
 
   private String user = "yarn";
@@ -67,7 +62,6 @@ public class TestInMemoryPlan {
   private ReservationAgent agent;
   private Planner replanner;
   private RMContext context;
-  private long maxPeriodicity;
 
   @Before
   public void setUp() throws PlanningException {
@@ -78,7 +72,7 @@ public class TestInMemoryPlan {
 
     clock = mock(Clock.class);
     queueMetrics = mock(QueueMetrics.class);
-    policy = new NoOverCommitPolicy();
+    policy = mock(SharingPolicy.class);
     replanner = mock(Planner.class);
 
     when(clock.getTime()).thenReturn(1L);
@@ -101,41 +95,15 @@ public class TestInMemoryPlan {
 
   @Test
   public void testAddReservation() {
-    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan =
+        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-        ReservationSystemTestUtil.getNewReservationId();
+            ReservationSystemTestUtil.getNewReservationId();
     int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    ReservationAllocation rAllocation =
-        createReservationAllocation(reservationID, start, alloc);
-    Assert.assertNull(plan.getReservationById(reservationID));
-    try {
-      plan.addReservation(rAllocation, false);
-    } catch (PlanningException e) {
-      Assert.fail(e.getMessage());
-    }
-    doAssertions(plan, rAllocation);
-    checkAllocation(plan, alloc, start, 0);
-  }
-
-  @Test
-  public void testAddPeriodicReservation() throws PlanningException {
-
-    maxPeriodicity = 100;
-    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-        resCalc, minAlloc, maxAlloc, planName, replanner, true, maxPeriodicity,
-        context, new UTCClock());
-
-    ReservationId reservationID =
-        ReservationSystemTestUtil.getNewReservationId();
-    int[] alloc = { 10, 50 };
-    int start = 10;
-    long period = 20;
-    ReservationAllocation rAllocation = createReservationAllocation(
-        reservationID, start, alloc, String.valueOf(period));
-    // use periodicity of 1hr
-    rAllocation.setPeriodicity(period);
+    ReservationAllocation rAllocation = createReservationAllocation
+            (reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -143,54 +111,32 @@ public class TestInMemoryPlan {
       Assert.fail(e.getMessage());
     }
     doAssertions(plan, rAllocation);
-    checkAllocation(plan, alloc, start, period);
-
-    RLESparseResourceAllocation available =
-        plan.getAvailableResourceOverTime(user, reservationID, 150, 330, 50);
-    System.out.println(available);
+    checkAllocation(plan, alloc, start);
   }
 
-  private void checkAllocation(Plan plan, int[] alloc, int start,
-      long periodicity) {
-    long end = start + alloc.length;
-    if (periodicity > 0) {
-      end = end + maxPeriodicity;
-    }
+  private void checkAllocation(Plan plan, int[] alloc, int start) {
     RLESparseResourceAllocation userCons =
-        plan.getConsumptionForUserOverTime(user, start, end * 3);
+        plan.getConsumptionForUserOverTime(user, start, start + alloc.length);
 
     for (int i = 0; i < alloc.length; i++) {
-      // only one instance for non-periodic reservation
-      if (periodicity <= 0) {
-        Assert.assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
-            plan.getTotalCommittedResources(start + i));
-        Assert.assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
-            userCons.getCapacityAtTime(start + i));
-      } else {
-        // periodic reservations should repeat
-        long y = 0;
-        Resource res = Resource.newInstance(1024 * (alloc[i]), (alloc[i]));
-        while (y <= end * 2) {
-          Assert.assertEquals("At time: " + start + i + y, res,
-              plan.getTotalCommittedResources(start + i + y));
-          Assert.assertEquals(" At time: " + (start + i + y), res,
-              userCons.getCapacityAtTime(start + i + y));
-          y = y + periodicity;
-        }
-      }
+      Assert.assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
+          plan.getTotalCommittedResources(start + i));
+      Assert.assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
+          userCons.getCapacityAtTime(start + i));
     }
   }
 
   @Test
   public void testAddEmptyReservation() {
-    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan =
+        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-        ReservationSystemTestUtil.getNewReservationId();
+            ReservationSystemTestUtil.getNewReservationId();
     int[] alloc = {};
     int start = 100;
-    ReservationAllocation rAllocation =
-        createReservationAllocation(reservationID, start, alloc);
+    ReservationAllocation rAllocation = createReservationAllocation
+            (reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -202,14 +148,15 @@ public class TestInMemoryPlan {
   @Test
   public void testAddReservationAlreadyExists() {
     // First add a reservation
-    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan =
+        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-        ReservationSystemTestUtil.getNewReservationId();
+            ReservationSystemTestUtil.getNewReservationId();
     int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    ReservationAllocation rAllocation =
-        createReservationAllocation(reservationID, start, alloc);
+    ReservationAllocation rAllocation = createReservationAllocation
+            (reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -217,7 +164,7 @@ public class TestInMemoryPlan {
       Assert.fail(e.getMessage());
     }
     doAssertions(plan, rAllocation);
-    checkAllocation(plan, alloc, start, 0);
+    checkAllocation(plan, alloc, start);
 
     // Try to add it again
     try {
@@ -233,15 +180,16 @@ public class TestInMemoryPlan {
 
   @Test
   public void testUpdateReservation() {
-    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan =
+        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
         ReservationSystemTestUtil.getNewReservationId();
     // First add a reservation
     int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    ReservationAllocation rAllocation =
-        createReservationAllocation(reservationID, start, alloc);
+    ReservationAllocation rAllocation = createReservationAllocation
+            (reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -262,8 +210,8 @@ public class TestInMemoryPlan {
     // Now update it
     start = 110;
     int[] updatedAlloc = { 0, 5, 10, 10, 5, 0 };
-    rAllocation =
-        createReservationAllocation(reservationID, start, updatedAlloc, true);
+    rAllocation = createReservationAllocation(reservationID, start,
+            updatedAlloc, true);
     try {
       plan.updateReservation(rAllocation);
     } catch (PlanningException e) {
@@ -271,71 +219,32 @@ public class TestInMemoryPlan {
     }
     doAssertions(plan, rAllocation);
 
-    userCons = plan.getConsumptionForUserOverTime(user, start,
-        start + updatedAlloc.length);
+    userCons =
+        plan.getConsumptionForUserOverTime(user, start, start
+            + updatedAlloc.length);
 
     for (int i = 0; i < updatedAlloc.length; i++) {
-      Assert.assertEquals(Resource.newInstance(1024 * (updatedAlloc[i] + i),
-          updatedAlloc[i] + i), plan.getTotalCommittedResources(start + i));
-      Assert.assertEquals(Resource.newInstance(1024 * (updatedAlloc[i] + i),
-          updatedAlloc[i] + i), userCons.getCapacityAtTime(start + i));
-    }
-  }
-
-  @Test
-  public void testUpdatePeriodicReservation() {
-    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
-    // First add a reservation
-    ReservationId reservationID =
-        ReservationSystemTestUtil.getNewReservationId();
-    int[] alloc = { 10, 20 };
-    int start = 100;
-    ReservationAllocation rAllocation =
-        createReservationAllocation(reservationID, start, alloc);
-    // use periodicity of 1hr
-    long period = 3600000;
-    rAllocation.getReservationDefinition()
-        .setRecurrenceExpression(String.valueOf(period));
-    rAllocation.setPeriodicity(period);
-    Assert.assertNull(plan.getReservationById(reservationID));
-    try {
-      plan.addReservation(rAllocation, false);
-    } catch (PlanningException e) {
-      Assert.fail(e.getMessage());
-    }
-    System.out.println(plan.toString());
-    doAssertions(plan, rAllocation);
-    checkAllocation(plan, alloc, start, period);
-
-    // Now update it
-    start = 110;
-    int[] updatedAlloc = { 30, 40, 50 };
-    rAllocation =
-        createReservationAllocation(reservationID, start, updatedAlloc);
-    rAllocation.getReservationDefinition()
-        .setRecurrenceExpression(String.valueOf(period));
-    rAllocation.setPeriodicity(period);
-    try {
-      plan.updateReservation(rAllocation);
-    } catch (PlanningException e) {
-      Assert.fail(e.getMessage());
+      Assert.assertEquals(
+     Resource.newInstance(1024 * (updatedAlloc[i] + i), updatedAlloc[i]
+              + i), plan.getTotalCommittedResources(start + i));
+      Assert.assertEquals(
+          Resource.newInstance(1024 * (updatedAlloc[i] + i), updatedAlloc[i]
+              + i), userCons.getCapacityAtTime(start + i));
     }
-    doAssertions(plan, rAllocation);
-    checkAllocation(plan, updatedAlloc, start, period);
   }
 
   @Test
   public void testUpdateNonExistingReservation() {
-    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan =
+        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
         ReservationSystemTestUtil.getNewReservationId();
     // Try to update a reservation without adding
     int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
     ReservationAllocation rAllocation =
-        createReservationAllocation(reservationID, start, alloc);
+            createReservationAllocation(reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.updateReservation(rAllocation);
@@ -351,14 +260,15 @@ public class TestInMemoryPlan {
   @Test
   public void testDeleteReservation() {
     // First add a reservation
-    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan =
+        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
         ReservationSystemTestUtil.getNewReservationId();
     int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
     ReservationAllocation rAllocation =
-        createReservationAllocation(reservationID, start, alloc, true);
+            createReservationAllocation(reservationID, start, alloc, true);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -397,46 +307,10 @@ public class TestInMemoryPlan {
   }
 
   @Test
-  public void testDeletePeriodicReservation() {
-    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
-    // First add a reservation
-    ReservationId reservationID =
-        ReservationSystemTestUtil.getNewReservationId();
-    int[] alloc = { 10, 20 };
-    int start = 100;
-    ReservationAllocation rAllocation =
-        createReservationAllocation(reservationID, start, alloc);
-    // use periodicity of 1hr
-    long period = 3600000;
-    rAllocation.getReservationDefinition()
-        .setRecurrenceExpression(String.valueOf(period));
-    rAllocation.setPeriodicity(period);
-    Assert.assertNull(plan.getReservationById(reservationID));
-    try {
-      plan.addReservation(rAllocation, false);
-    } catch (PlanningException e) {
-      Assert.fail(e.getMessage());
-    }
-    System.out.println(plan.toString());
-    doAssertions(plan, rAllocation);
-    checkAllocation(plan, alloc, start, period);
-
-    // Now delete it
-    try {
-      plan.deleteReservation(reservationID);
-    } catch (PlanningException e) {
-      Assert.fail(e.getMessage());
-    }
-    Assert.assertNull(plan.getReservationById(reservationID));
-    System.out.print(plan);
-    checkAllocation(plan, new int[] { 0, 0 }, start, period);
-  }
-
-  @Test
   public void testDeleteNonExistingReservation() {
-    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan =
+        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
         ReservationSystemTestUtil.getNewReservationId();
     // Try to delete a reservation without adding
@@ -454,9 +328,8 @@ public class TestInMemoryPlan {
 
   @Test
   public void testArchiveCompletedReservations() {
-    SharingPolicy sharingPolicy = mock(SharingPolicy.class);
     Plan plan =
-        new InMemoryPlan(queueMetrics, sharingPolicy, agent, totalCapacity, 1L,
+        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
             resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID1 =
         ReservationSystemTestUtil.getNewReservationId();
@@ -464,7 +337,7 @@ public class TestInMemoryPlan {
     int[] alloc1 = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
     ReservationAllocation rAllocation =
-        createReservationAllocation(reservationID1, start, alloc1);
+            createReservationAllocation(reservationID1, start, alloc1);
     Assert.assertNull(plan.getReservationById(reservationID1));
     try {
       plan.addReservation(rAllocation, false);
@@ -472,14 +345,15 @@ public class TestInMemoryPlan {
       Assert.fail(e.getMessage());
     }
     doAssertions(plan, rAllocation);
-    checkAllocation(plan, alloc1, start, 0);
+    checkAllocation(plan, alloc1, start);
+
 
     // Now add another one
     ReservationId reservationID2 =
         ReservationSystemTestUtil.getNewReservationId();
     int[] alloc2 = { 0, 5, 10, 5, 0 };
     rAllocation =
-        createReservationAllocation(reservationID2, start, alloc2, true);
+            createReservationAllocation(reservationID2, start, alloc2, true);
     Assert.assertNull(plan.getReservationById(reservationID2));
     try {
       plan.addReservation(rAllocation, false);
@@ -493,18 +367,16 @@ public class TestInMemoryPlan {
 
     for (int i = 0; i < alloc2.length; i++) {
       Assert.assertEquals(
-          Resource.newInstance(1024 * (alloc1[i] + alloc2[i] + i),
-              alloc1[i] + alloc2[i] + i),
-          plan.getTotalCommittedResources(start + i));
+          Resource.newInstance(1024 * (alloc1[i] + alloc2[i] + i), alloc1[i]
+              + alloc2[i] + i), plan.getTotalCommittedResources(start + i));
       Assert.assertEquals(
-          Resource.newInstance(1024 * (alloc1[i] + alloc2[i] + i),
-              alloc1[i] + alloc2[i] + i),
-          userCons.getCapacityAtTime(start + i));
+          Resource.newInstance(1024 * (alloc1[i] + alloc2[i] + i), alloc1[i]
+              + alloc2[i] + i), userCons.getCapacityAtTime(start + i));
     }
 
     // Now archive completed reservations
     when(clock.getTime()).thenReturn(106L);
-    when(sharingPolicy.getValidWindow()).thenReturn(1L);
+    when(policy.getValidWindow()).thenReturn(1L);
     try {
       // will only remove 2nd reservation as only that has fallen out of the
       // archival window
@@ -514,7 +386,7 @@ public class TestInMemoryPlan {
     }
     Assert.assertNotNull(plan.getReservationById(reservationID1));
     Assert.assertNull(plan.getReservationById(reservationID2));
-    checkAllocation(plan, alloc1, start, 0);
+    checkAllocation(plan, alloc1, start);
 
     when(clock.getTime()).thenReturn(107L);
     try {
@@ -539,14 +411,15 @@ public class TestInMemoryPlan {
 
   @Test
   public void testGetReservationsById() {
-    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan =
+            new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+                    resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-        ReservationSystemTestUtil.getNewReservationId();
-    int[] alloc = { 10, 10, 10, 10, 10, 10 };
+            ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = {10, 10, 10, 10, 10, 10};
     int start = 100;
-    ReservationAllocation rAllocation =
-        createReservationAllocation(reservationID, start, alloc);
+    ReservationAllocation rAllocation = createReservationAllocation
+            (reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -556,30 +429,31 @@ public class TestInMemoryPlan {
 
     // Verify that get by reservation id works.
     Set<ReservationAllocation> rAllocations =
-        plan.getReservations(reservationID, null, "");
+            plan.getReservations(reservationID, null, "");
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation
-        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation.compareTo(
+            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
 
     // Verify that get by reservation id works even when time range
     // and user is invalid.
     ReservationInterval interval = new ReservationInterval(0, 0);
     rAllocations = plan.getReservations(reservationID, interval, "invalid");
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation
-        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation.compareTo(
+            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
   }
 
   @Test
   public void testGetReservationsByInvalidId() {
-    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan =
+            new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+                    resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-        ReservationSystemTestUtil.getNewReservationId();
-    int[] alloc = { 10, 10, 10, 10, 10, 10 };
+            ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = {10, 10, 10, 10, 10, 10};
     int start = 100;
-    ReservationAllocation rAllocation =
-        createReservationAllocation(reservationID, start, alloc);
+    ReservationAllocation rAllocation = createReservationAllocation
+            (reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -589,22 +463,23 @@ public class TestInMemoryPlan {
 
     // If reservationId is null, then nothing is returned.
     ReservationId invalidReservationID =
-        ReservationSystemTestUtil.getNewReservationId();
+            ReservationSystemTestUtil.getNewReservationId();
     Set<ReservationAllocation> rAllocations =
-        plan.getReservations(invalidReservationID, null, "");
+            plan.getReservations(invalidReservationID, null, "");
     Assert.assertTrue(rAllocations.size() == 0);
   }
 
   @Test
   public void testGetReservationsByTimeInterval() {
-    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan =
+            new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+                    resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-        ReservationSystemTestUtil.getNewReservationId();
-    int[] alloc = { 10, 10, 10, 10, 10, 10 };
+            ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = {10, 10, 10, 10, 10, 10};
     int start = 100;
-    ReservationAllocation rAllocation =
-        createReservationAllocation(reservationID, start, alloc);
+    ReservationAllocation rAllocation = createReservationAllocation
+            (reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -614,24 +489,23 @@ public class TestInMemoryPlan {
 
     // Verify that get by time interval works if the selection interval
     // completely overlaps with the allocation.
-    ReservationInterval interval = new ReservationInterval(
-        rAllocation.getStartTime(), rAllocation.getEndTime());
+    ReservationInterval interval = new ReservationInterval(rAllocation
+            .getStartTime(), rAllocation.getEndTime());
     Set<ReservationAllocation> rAllocations =
-        plan.getReservations(null, interval, "");
+            plan.getReservations(null, interval, "");
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation
-        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation.compareTo(
+            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
 
     // Verify that get by time interval works if the selection interval
     // falls within the allocation
     long duration = rAllocation.getEndTime() - rAllocation.getStartTime();
-    interval = new ReservationInterval(
-        rAllocation.getStartTime() + duration * (long) 0.3,
-        rAllocation.getEndTime() - duration * (long) 0.3);
+    interval = new ReservationInterval(rAllocation.getStartTime() + duration
+            * (long)0.3, rAllocation.getEndTime() - duration * (long)0.3);
     rAllocations = plan.getReservations(null, interval, "");
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation
-        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation.compareTo(
+            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
 
     // Verify that get by time interval selects 1 allocation if the end
     // time of the selection interval falls right at the start of the
@@ -639,13 +513,13 @@ public class TestInMemoryPlan {
     interval = new ReservationInterval(0, rAllocation.getStartTime());
     rAllocations = plan.getReservations(null, interval, "");
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation
-        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation.compareTo(
+            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
 
     // Verify that get by time interval selects no reservations if the start
     // time of the selection interval falls right at the end of the allocation.
-    interval =
-        new ReservationInterval(rAllocation.getEndTime(), Long.MAX_VALUE);
+    interval = new ReservationInterval(rAllocation
+            .getEndTime(), Long.MAX_VALUE);
     rAllocations = plan.getReservations(null, interval, "");
     Assert.assertTrue(rAllocations.size() == 0);
 
@@ -658,14 +532,15 @@ public class TestInMemoryPlan {
 
   @Test
   public void testGetReservationsAtTime() {
-    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan =
+            new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+                    resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-        ReservationSystemTestUtil.getNewReservationId();
-    int[] alloc = { 10, 10, 10, 10, 10, 10 };
+            ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = {10, 10, 10, 10, 10, 10};
     int start = 100;
-    ReservationAllocation rAllocation =
-        createReservationAllocation(reservationID, start, alloc);
+    ReservationAllocation rAllocation = createReservationAllocation
+            (reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -674,22 +549,23 @@ public class TestInMemoryPlan {
     }
 
     Set<ReservationAllocation> rAllocations =
-        plan.getReservationsAtTime(rAllocation.getStartTime());
+            plan.getReservationsAtTime(rAllocation.getStartTime());
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation
-        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation.compareTo(
+            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
   }
 
   @Test
   public void testGetReservationsWithNoInput() {
-    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan =
+            new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+                    resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-        ReservationSystemTestUtil.getNewReservationId();
-    int[] alloc = { 10, 10, 10, 10, 10, 10 };
+            ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = {10, 10, 10, 10, 10, 10};
     int start = 100;
-    ReservationAllocation rAllocation =
-        createReservationAllocation(reservationID, start, alloc);
+    ReservationAllocation rAllocation = createReservationAllocation
+            (reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -700,21 +576,22 @@ public class TestInMemoryPlan {
     // Verify that getReservations defaults to getting all reservations if no
     // reservationID, time interval, and user is provided,
     Set<ReservationAllocation> rAllocations =
-        plan.getReservations(null, null, "");
+            plan.getReservations(null, null, "");
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation
-        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation.compareTo(
+            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
   }
 
   @Test
   public void testGetReservationsWithNoReservation() {
-    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan =
+            new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+                    resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     // Verify that get reservation returns no entries if no queries are made.
 
     ReservationInterval interval = new ReservationInterval(0, Long.MAX_VALUE);
     Set<ReservationAllocation> rAllocations =
-        plan.getReservations(null, interval, "");
+            plan.getReservations(null, interval, "");
     Assert.assertTrue(rAllocations.size() == 0);
   }
 
@@ -723,9 +600,7 @@ public class TestInMemoryPlan {
     Assert.assertNotNull(plan.getReservationById(reservationID));
     Assert.assertEquals(rAllocation, plan.getReservationById(reservationID));
     Assert.assertTrue(((InMemoryPlan) plan).getAllReservations().size() == 1);
-    if (rAllocation.getPeriodicity() <= 0) {
-      Assert.assertEquals(rAllocation.getEndTime(), plan.getLastEndTime());
-    }
+    Assert.assertEquals(rAllocation.getEndTime(), plan.getLastEndTime());
     Assert.assertEquals(totalCapacity, plan.getTotalCapacity());
     Assert.assertEquals(minAlloc, plan.getMinimumAllocation());
     Assert.assertEquals(maxAlloc, plan.getMaximumAllocation());
@@ -735,8 +610,7 @@ public class TestInMemoryPlan {
   }
 
   private ReservationDefinition createSimpleReservationDefinition(long arrival,
-      long deadline, long duration, Collection<ReservationRequest> resources,
-      String recurrenceExpression) {
+      long deadline, long duration, Collection<ReservationRequest> resources) {
     // create a request with a single atomic ask
     ReservationDefinition rDef = new ReservationDefinitionPBImpl();
     ReservationRequests reqs = new ReservationRequestsPBImpl();
@@ -745,7 +619,6 @@ public class TestInMemoryPlan {
     rDef.setReservationRequests(reqs);
     rDef.setArrival(arrival);
     rDef.setDeadline(deadline);
-    rDef.setRecurrenceExpression(recurrenceExpression);
     return rDef;
   }
 
@@ -760,43 +633,31 @@ public class TestInMemoryPlan {
       } else {
         numContainers = alloc[i];
       }
-      ReservationRequest rr = ReservationRequest
-          .newInstance(Resource.newInstance(1024, 1), (numContainers));
+      ReservationRequest rr =
+          ReservationRequest.newInstance(Resource.newInstance(1024, 1),
+              (numContainers));
       req.put(new ReservationInterval(startTime + i, startTime + i + 1), rr);
     }
     return req;
   }
 
-  private ReservationAllocation createReservationAllocation(
-      ReservationId reservationID, int start, int[] alloc) {
-    return createReservationAllocation(reservationID, start, alloc, false, "0");
-  }
-
-  private ReservationAllocation createReservationAllocation(
-      ReservationId reservationID, int start, int[] alloc, boolean isStep) {
-    return createReservationAllocation(reservationID, start, alloc, isStep,
-        "0");
-  }
-
-  private ReservationAllocation createReservationAllocation(
-      ReservationId reservationID, int start, int[] alloc,
-      String recurrenceExp) {
-    return createReservationAllocation(reservationID, start, alloc, false,
-        recurrenceExp);
+  private ReservationAllocation createReservationAllocation(ReservationId
+            reservationID, int start, int[] alloc) {
+    return createReservationAllocation(reservationID, start, alloc, false);
   }
 
-  private ReservationAllocation createReservationAllocation(
-      ReservationId reservationID, int start, int[] alloc, boolean isStep,
-      String recurrenceExp) {
+  private ReservationAllocation createReservationAllocation(ReservationId
+            reservationID, int start, int[] alloc, boolean isStep) {
     Map<ReservationInterval, ReservationRequest> allocations =
-        generateAllocation(start, alloc, isStep);
+            generateAllocation(start, alloc, isStep);
     ReservationDefinition rDef =
-        createSimpleReservationDefinition(start, start + alloc.length,
-            alloc.length, allocations.values(), recurrenceExp);
+            createSimpleReservationDefinition(start, start + alloc.length,
+                    alloc.length, allocations.values());
     Map<ReservationInterval, Resource> allocs =
-        ReservationSystemUtil.toResources(allocations);
+            ReservationSystemUtil.toResources(allocations);
     return new InMemoryReservationAllocation(reservationID, rDef, user,
-        planName, start, start + alloc.length, allocs, resCalc, minAlloc);
+            planName,
+                    start, start + alloc.length, allocs, resCalc, minAlloc);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestPeriodicRLESparseResourceAllocation.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/reservation/TestPeriodicRLESparseResourceAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestPeriodicRLESparseResourceAllocation.java
index 457e2ee..554eb58 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestPeriodicRLESparseResourceAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestPeriodicRLESparseResourceAllocation.java
@@ -19,27 +19,26 @@
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
 import org.junit.Assert;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * Testing the class {@link PeriodicRLESparseResourceAllocation}.
+ * Testing the class PeriodicRLESparseResourceAllocation.
  */
-@SuppressWarnings("checkstyle:nowhitespaceafter")
 public class TestPeriodicRLESparseResourceAllocation {
 
-  private static final Logger LOG =
-      LoggerFactory.getLogger(TestPeriodicRLESparseResourceAllocation.class);
+  private static final Logger LOG = LoggerFactory
+      .getLogger(TestPeriodicRLESparseResourceAllocation.class);
 
   @Test
   public void testPeriodicCapacity() {
-    int[] alloc = { 10, 7, 5, 2, 0 };
-    long[] timeSteps = { 0L, 5L, 10L, 15L, 19L };
-    RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
-        .generateRLESparseResourceAllocation(alloc, timeSteps);
+    int[] alloc = {10, 7, 5, 2, 0};
+    long[] timeSteps = {0L, 5L, 10L, 15L, 19L};
+    RLESparseResourceAllocation rleSparseVector =
+        ReservationSystemTestUtil.generateRLESparseResourceAllocation(
+            alloc, timeSteps);
     PeriodicRLESparseResourceAllocation periodicVector =
         new PeriodicRLESparseResourceAllocation(rleSparseVector, 20L);
     LOG.info(periodicVector.toString());
@@ -55,74 +54,43 @@ public class TestPeriodicRLESparseResourceAllocation {
 
   @Test
   public void testMaxPeriodicCapacity() {
-    int[] alloc = { 2, 5, 7, 10, 3, 4, 6, 8 };
-    long[] timeSteps = { 0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L };
-    RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
-        .generateRLESparseResourceAllocation(alloc, timeSteps);
+    int[] alloc = {2, 5, 7, 10, 3, 4, 6, 8};
+    long[] timeSteps = {0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L};
+    RLESparseResourceAllocation rleSparseVector =
+        ReservationSystemTestUtil.generateRLESparseResourceAllocation(
+            alloc, timeSteps);
     PeriodicRLESparseResourceAllocation periodicVector =
         new PeriodicRLESparseResourceAllocation(rleSparseVector, 8L);
     LOG.info(periodicVector.toString());
-    Assert.assertEquals(periodicVector.getMaximumPeriodicCapacity(0, 1),
+    Assert.assertEquals(
+        periodicVector.getMaximumPeriodicCapacity(0, 1),
         Resource.newInstance(10, 10));
-    Assert.assertEquals(periodicVector.getMaximumPeriodicCapacity(8, 2),
+    Assert.assertEquals(
+        periodicVector.getMaximumPeriodicCapacity(8, 2),
         Resource.newInstance(7, 7));
-    Assert.assertEquals(periodicVector.getMaximumPeriodicCapacity(16, 3),
+    Assert.assertEquals(
+        periodicVector.getMaximumPeriodicCapacity(16, 3),
         Resource.newInstance(10, 10));
-    Assert.assertEquals(periodicVector.getMaximumPeriodicCapacity(17, 4),
+    Assert.assertEquals(
+        periodicVector.getMaximumPeriodicCapacity(17, 4),
         Resource.newInstance(5, 5));
-    Assert.assertEquals(periodicVector.getMaximumPeriodicCapacity(32, 5),
+    Assert.assertEquals(
+        periodicVector.getMaximumPeriodicCapacity(32, 5),
         Resource.newInstance(4, 4));
   }
 
   @Test
-  public void testMixPeriodicAndNonPeriodic() throws PlanningException {
-    int[] alloc = { 2, 5, 0 };
-    long[] timeSteps = { 1L, 2L, 3L };
-    RLESparseResourceAllocation tempPeriodic = ReservationSystemTestUtil
-        .generateRLESparseResourceAllocation(alloc, timeSteps);
-    PeriodicRLESparseResourceAllocation periodic =
-        new PeriodicRLESparseResourceAllocation(tempPeriodic, 10L);
-
-    int[] alloc2 = { 10, 10, 0 };
-    long[] timeSteps2 = { 12L, 13L, 14L };
-    RLESparseResourceAllocation nonPeriodic = ReservationSystemTestUtil
-        .generateRLESparseResourceAllocation(alloc2, timeSteps2);
-
-    RLESparseResourceAllocation merged =
-        RLESparseResourceAllocation.merge(nonPeriodic.getResourceCalculator(),
-            Resource.newInstance(100 * 1024, 100), periodic, nonPeriodic,
-            RLESparseResourceAllocation.RLEOperator.add, 2, 25);
-
-    Assert.assertEquals(Resource.newInstance(5, 5),
-        merged.getCapacityAtTime(2L));
-    Assert.assertEquals(Resource.newInstance(0, 0),
-        merged.getCapacityAtTime(3L));
-    Assert.assertEquals(Resource.newInstance(2, 2),
-        merged.getCapacityAtTime(11L));
-    Assert.assertEquals(Resource.newInstance(15, 15),
-        merged.getCapacityAtTime(12L));
-    Assert.assertEquals(Resource.newInstance(10, 10),
-        merged.getCapacityAtTime(13L));
-    Assert.assertEquals(Resource.newInstance(0, 0),
-        merged.getCapacityAtTime(14L));
-    Assert.assertEquals(Resource.newInstance(2, 2),
-        merged.getCapacityAtTime(21L));
-    Assert.assertEquals(Resource.newInstance(5, 5),
-        merged.getCapacityAtTime(22L));
-    Assert.assertEquals(Resource.newInstance(0, 0),
-        merged.getCapacityAtTime(23L));
-  }
-
-  @Test
   public void testSetCapacityInInterval() {
-    int[] alloc = { 2, 5, 0 };
-    long[] timeSteps = { 1L, 2L, 3L };
-    RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
-        .generateRLESparseResourceAllocation(alloc, timeSteps);
+    int[] alloc = {2, 5, 0};
+    long[] timeSteps = {1L, 2L, 3L};
+    RLESparseResourceAllocation rleSparseVector =
+        ReservationSystemTestUtil.generateRLESparseResourceAllocation(
+            alloc, timeSteps);
     PeriodicRLESparseResourceAllocation periodicVector =
         new PeriodicRLESparseResourceAllocation(rleSparseVector, 10L);
     ReservationInterval interval = new ReservationInterval(5L, 10L);
-    periodicVector.addInterval(interval, Resource.newInstance(8, 8));
+    periodicVector.addInterval(
+        interval, Resource.newInstance(8, 8));
     Assert.assertEquals(Resource.newInstance(8, 8),
         periodicVector.getCapacityAtTime(5L));
     Assert.assertEquals(Resource.newInstance(8, 8),
@@ -131,20 +99,21 @@ public class TestPeriodicRLESparseResourceAllocation {
         periodicVector.getCapacityAtTime(10L));
     Assert.assertEquals(Resource.newInstance(0, 0),
         periodicVector.getCapacityAtTime(0L));
-    // Assert.assertFalse(periodicVector.addInterval(
-    // new ReservationInterval(7L, 12L), Resource.newInstance(8, 8)));
+    Assert.assertFalse(periodicVector.addInterval(
+        new ReservationInterval(7L, 12L), Resource.newInstance(8, 8)));
   }
 
   public void testRemoveInterval() {
-    int[] alloc = { 2, 5, 3, 4, 0 };
-    long[] timeSteps = { 1L, 3L, 5L, 7L, 9L };
-    RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
-        .generateRLESparseResourceAllocation(alloc, timeSteps);
+    int[] alloc = {2, 5, 3, 4, 0};
+    long[] timeSteps = {1L, 3L, 5L, 7L, 9L};
+    RLESparseResourceAllocation rleSparseVector =
+        ReservationSystemTestUtil.generateRLESparseResourceAllocation(
+            alloc, timeSteps);
     PeriodicRLESparseResourceAllocation periodicVector =
         new PeriodicRLESparseResourceAllocation(rleSparseVector, 10L);
     ReservationInterval interval = new ReservationInterval(3L, 7L);
-    Assert.assertTrue(
-        periodicVector.removeInterval(interval, Resource.newInstance(3, 3)));
+    Assert.assertTrue(periodicVector.removeInterval(
+        interval, Resource.newInstance(3, 3)));
     Assert.assertEquals(Resource.newInstance(2, 2),
         periodicVector.getCapacityAtTime(1L));
     Assert.assertEquals(Resource.newInstance(2, 2),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.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/reservation/TestRLESparseResourceAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.java
index 0027ceb..bfe46e1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.java
@@ -40,14 +40,10 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-/**
- * Testing the class {@link RLESparseResourceAllocation}.
- */
-@SuppressWarnings("checkstyle:nowhitespaceafter")
 public class TestRLESparseResourceAllocation {
 
-  private static final Logger LOG =
-      LoggerFactory.getLogger(TestRLESparseResourceAllocation.class);
+  private static final Logger LOG = LoggerFactory
+      .getLogger(TestRLESparseResourceAllocation.class);
 
   @Test
   public void testMergeAdd() throws PlanningException {
@@ -200,8 +196,7 @@ public class TestRLESparseResourceAllocation {
       // Expected!
     }
 
-    // Testing that the subtractTestNonNegative detects problems even if only
-    // one
+    // Testing that the subtractTestNonNegative detects problems even if only one
     // of the resource dimensions is "<0"
     a.put(10L, Resource.newInstance(10, 5));
     b.put(11L, Resource.newInstance(5, 6));
@@ -291,8 +286,9 @@ public class TestRLESparseResourceAllocation {
   public void testRangeOverlapping() {
     ResourceCalculator resCalc = new DefaultResourceCalculator();
 
-    RLESparseResourceAllocation r = new RLESparseResourceAllocation(resCalc);
-    int[] alloc = { 10, 10, 10, 10, 10, 10 };
+    RLESparseResourceAllocation r =
+        new RLESparseResourceAllocation(resCalc);
+    int[] alloc = {10, 10, 10, 10, 10, 10};
     int start = 100;
     Set<Entry<ReservationInterval, Resource>> inputs =
         generateAllocation(start, alloc, false).entrySet();
@@ -303,9 +299,9 @@ public class TestRLESparseResourceAllocation {
     long d = r.getLatestNonNullTime();
 
     // tries to trigger "out-of-range" bug
-    r = r.getRangeOverlapping(s, d);
-    r = r.getRangeOverlapping(s - 1, d - 1);
-    r = r.getRangeOverlapping(s + 1, d + 1);
+    r =  r.getRangeOverlapping(s, d);
+    r = r.getRangeOverlapping(s-1, d-1);
+    r = r.getRangeOverlapping(s+1, d+1);
   }
 
   @Test
@@ -374,29 +370,25 @@ public class TestRLESparseResourceAllocation {
     // Current bug prevents this to pass. The RLESparseResourceAllocation
     // does not handle removal of "partial"
     // allocations correctly.
-    Assert.assertEquals(102400,
-        rleSparseVector.getCapacityAtTime(10).getMemorySize());
-    Assert.assertEquals(0,
-        rleSparseVector.getCapacityAtTime(13).getMemorySize());
-    Assert.assertEquals(0,
-        rleSparseVector.getCapacityAtTime(19).getMemorySize());
-    Assert.assertEquals(102400,
-        rleSparseVector.getCapacityAtTime(21).getMemorySize());
-    Assert.assertEquals(2 * 102400,
-        rleSparseVector.getCapacityAtTime(26).getMemorySize());
+    Assert.assertEquals(102400, rleSparseVector.getCapacityAtTime(10)
+        .getMemorySize());
+    Assert.assertEquals(0, rleSparseVector.getCapacityAtTime(13).getMemorySize());
+    Assert.assertEquals(0, rleSparseVector.getCapacityAtTime(19).getMemorySize());
+    Assert.assertEquals(102400, rleSparseVector.getCapacityAtTime(21)
+        .getMemorySize());
+    Assert.assertEquals(2 * 102400, rleSparseVector.getCapacityAtTime(26)
+        .getMemorySize());
 
     ReservationInterval riRemove2 = new ReservationInterval(9, 13);
     rleSparseVector.removeInterval(riRemove2, rr);
     LOG.info(rleSparseVector.toString());
 
-    Assert.assertEquals(0,
-        rleSparseVector.getCapacityAtTime(11).getMemorySize());
-    Assert.assertEquals(-102400,
-        rleSparseVector.getCapacityAtTime(9).getMemorySize());
-    Assert.assertEquals(0,
-        rleSparseVector.getCapacityAtTime(13).getMemorySize());
-    Assert.assertEquals(102400,
-        rleSparseVector.getCapacityAtTime(20).getMemorySize());
+    Assert.assertEquals(0, rleSparseVector.getCapacityAtTime(11).getMemorySize());
+    Assert.assertEquals(-102400, rleSparseVector.getCapacityAtTime(9)
+        .getMemorySize());
+    Assert.assertEquals(0, rleSparseVector.getCapacityAtTime(13).getMemorySize());
+    Assert.assertEquals(102400, rleSparseVector.getCapacityAtTime(20)
+        .getMemorySize());
 
   }
 
@@ -508,8 +500,7 @@ public class TestRLESparseResourceAllocation {
     }
     mapAllocations = rleSparseVector.toIntervalMap();
     Assert.assertTrue(mapAllocations.size() == 5);
-    for (Entry<ReservationInterval, Resource> entry : mapAllocations
-        .entrySet()) {
+    for (Entry<ReservationInterval, Resource> entry : mapAllocations.entrySet()) {
       ReservationInterval interval = entry.getKey();
       Resource resource = entry.getValue();
       if (interval.getStartTime() == 101L) {
@@ -535,46 +526,59 @@ public class TestRLESparseResourceAllocation {
 
   @Test
   public void testMaxPeriodicCapacity() {
-    long[] timeSteps = { 0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L };
-    int[] alloc = { 2, 5, 7, 10, 3, 4, 6, 8 };
-    RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
-        .generateRLESparseResourceAllocation(alloc, timeSteps);
+    long[] timeSteps = {0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L};
+    int[] alloc = {2, 5, 7, 10, 3, 4, 6, 8};
+    RLESparseResourceAllocation rleSparseVector =
+        ReservationSystemTestUtil.generateRLESparseResourceAllocation(
+            alloc, timeSteps);
     LOG.info(rleSparseVector.toString());
-    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 1),
+    Assert.assertEquals(
+        rleSparseVector.getMaximumPeriodicCapacity(0, 1),
         Resource.newInstance(10, 10));
-    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 2),
+    Assert.assertEquals(
+        rleSparseVector.getMaximumPeriodicCapacity(0, 2),
         Resource.newInstance(7, 7));
-    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 3),
+    Assert.assertEquals(
+        rleSparseVector.getMaximumPeriodicCapacity(0, 3),
         Resource.newInstance(10, 10));
-    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 4),
+    Assert.assertEquals(
+        rleSparseVector.getMaximumPeriodicCapacity(0, 4),
         Resource.newInstance(3, 3));
-    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 5),
+    Assert.assertEquals(
+        rleSparseVector.getMaximumPeriodicCapacity(0, 5),
         Resource.newInstance(4, 4));
-    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 5),
+    Assert.assertEquals(
+        rleSparseVector.getMaximumPeriodicCapacity(0, 5),
         Resource.newInstance(4, 4));
-    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(7, 5),
+    Assert.assertEquals(
+        rleSparseVector.getMaximumPeriodicCapacity(7, 5),
         Resource.newInstance(8, 8));
-    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(10, 3),
+    Assert.assertEquals(
+        rleSparseVector.getMaximumPeriodicCapacity(10, 3),
         Resource.newInstance(0, 0));
   }
 
   @Test
   public void testGetMinimumCapacityInInterval() {
-    long[] timeSteps = { 0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L };
-    int[] alloc = { 2, 5, 7, 10, 3, 4, 0, 8 };
-    RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
-        .generateRLESparseResourceAllocation(alloc, timeSteps);
+    long[] timeSteps = {0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L};
+    int[] alloc = {2, 5, 7, 10, 3, 4, 0, 8};
+    RLESparseResourceAllocation rleSparseVector =
+        ReservationSystemTestUtil.generateRLESparseResourceAllocation(
+            alloc, timeSteps);
     LOG.info(rleSparseVector.toString());
-    Assert.assertEquals(rleSparseVector.getMinimumCapacityInInterval(
-        new ReservationInterval(1L, 3L)), Resource.newInstance(5, 5));
-    Assert.assertEquals(rleSparseVector.getMinimumCapacityInInterval(
-        new ReservationInterval(2L, 5L)), Resource.newInstance(3, 3));
-    Assert.assertEquals(rleSparseVector.getMinimumCapacityInInterval(
-        new ReservationInterval(1L, 7L)), Resource.newInstance(0, 0));
+    Assert.assertEquals(
+        rleSparseVector.getMinimumCapacityInInterval(
+            new ReservationInterval(1L, 3L)), Resource.newInstance(5, 5));
+    Assert.assertEquals(
+        rleSparseVector.getMinimumCapacityInInterval(
+            new ReservationInterval(2L, 5L)), Resource.newInstance(3, 3));
+    Assert.assertEquals(
+        rleSparseVector.getMinimumCapacityInInterval(
+            new ReservationInterval(1L, 7L)), Resource.newInstance(0, 0));
   }
 
-  private void setupArrays(TreeMap<Long, Resource> a,
-      TreeMap<Long, Resource> b) {
+  private void setupArrays(
+      TreeMap<Long, Resource> a, TreeMap<Long, Resource> b) {
     a.put(10L, Resource.newInstance(5, 5));
     a.put(20L, Resource.newInstance(10, 10));
     a.put(30L, Resource.newInstance(15, 15));
@@ -616,8 +620,8 @@ public class TestRLESparseResourceAllocation {
         numContainers = alloc[i];
       }
       req.put(new ReservationInterval(startTime + i, startTime + i + 1),
-          ReservationSystemUtil.toResource(ReservationRequest
-              .newInstance(Resource.newInstance(1024, 1), (numContainers))));
+          ReservationSystemUtil.toResource(ReservationRequest.newInstance(
+              Resource.newInstance(1024, 1), (numContainers))));
     }
     return req;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3345e98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestSimpleCapacityReplanner.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/reservation/planning/TestSimpleCapacityReplanner.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestSimpleCapacityReplanner.java
index ddd290d..c4f94c2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestSimpleCapacityReplanner.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestSimpleCapacityReplanner.java
@@ -31,7 +31,6 @@ import org.apache.hadoop.yarn.api.records.ReservationDefinition;
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.ReservationRequest;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.InMemoryPlan;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.InMemoryReservationAllocation;
@@ -79,10 +78,9 @@ public class TestSimpleCapacityReplanner {
     enf.init("blah", conf);
 
     // Initialize the plan with more resources
-    InMemoryPlan plan = new InMemoryPlan(queueMetrics, policy, agent,
-        clusterCapacity, step, res, minAlloc, maxAlloc, "dedicated", enf, true,
-        YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY,
-        context, clock);
+    InMemoryPlan plan =
+        new InMemoryPlan(queueMetrics, policy, agent, clusterCapacity, step,
+            res, minAlloc, maxAlloc, "dedicated", enf, true, context, clock);
 
     // add reservation filling the plan (separating them 1ms, so we are sure
     // s2 follows s1 on acceptance


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


[36/50] [abbrv] hadoop git commit: YARN-6849. NMContainerStatus should have the Container ExecutionType. (Kartheek Muthyala via asuresh)

Posted by as...@apache.org.
YARN-6849. NMContainerStatus should have the Container ExecutionType. (Kartheek Muthyala via asuresh)


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

Branch: refs/heads/YARN-5972
Commit: 1f53ae79728065417c6a99eb6fcc8d3a080ab4cc
Parents: 4a83170
Author: Arun Suresh <as...@apache.org>
Authored: Fri Sep 8 09:24:05 2017 -0700
Committer: Arun Suresh <as...@apache.org>
Committed: Fri Sep 8 09:24:05 2017 -0700

----------------------------------------------------------------------
 .../api/protocolrecords/NMContainerStatus.java  |  17 ++-
 .../impl/pb/NMContainerStatusPBImpl.java        |  30 ++++++
 .../yarn_server_common_service_protos.proto     |   1 +
 .../container/ContainerImpl.java                |   3 +-
 .../resourcemanager/rmnode/RMNodeImpl.java      |   3 +-
 .../scheduler/AbstractYarnScheduler.java        |   1 +
 .../scheduler/AppSchedulingInfo.java            |   4 +
 .../scheduler/SchedulerApplicationAttempt.java  |   8 +-
 .../scheduler/capacity/LeafQueue.java           |   4 +-
 .../scheduler/capacity/ParentQueue.java         |   4 +
 .../server/resourcemanager/TestRMRestart.java   |   4 +-
 .../TestResourceTrackerService.java             | 108 +++++++++++++++++++
 12 files changed, 178 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f53ae79/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NMContainerStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NMContainerStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NMContainerStatus.java
index ed950ce..180add8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NMContainerStatus.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NMContainerStatus.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.api.protocolrecords;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
@@ -40,13 +41,14 @@ public abstract class NMContainerStatus {
       long creationTime) {
     return newInstance(containerId, version, containerState, allocatedResource,
         diagnostics, containerExitStatus, priority, creationTime,
-        CommonNodeLabelsManager.NO_LABEL);
+        CommonNodeLabelsManager.NO_LABEL, ExecutionType.GUARANTEED);
   }
 
   public static NMContainerStatus newInstance(ContainerId containerId,
       int version, ContainerState containerState, Resource allocatedResource,
       String diagnostics, int containerExitStatus, Priority priority,
-      long creationTime, String nodeLabelExpression) {
+      long creationTime, String nodeLabelExpression,
+      ExecutionType executionType) {
     NMContainerStatus status =
         Records.newRecord(NMContainerStatus.class);
     status.setContainerId(containerId);
@@ -58,6 +60,7 @@ public abstract class NMContainerStatus {
     status.setPriority(priority);
     status.setCreationTime(creationTime);
     status.setNodeLabelExpression(nodeLabelExpression);
+    status.setExecutionType(executionType);
     return status;
   }
 
@@ -134,4 +137,14 @@ public abstract class NMContainerStatus {
   public void setVersion(int version) {
 
   }
+
+  /**
+   * Get the <code>ExecutionType</code> of the container.
+   * @return <code>ExecutionType</code> of the container
+   */
+  public ExecutionType getExecutionType() {
+    return ExecutionType.GUARANTEED;
+  }
+
+  public void setExecutionType(ExecutionType executionType) { }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f53ae79/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NMContainerStatusPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NMContainerStatusPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NMContainerStatusPBImpl.java
index 2380391..38df5f6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NMContainerStatusPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NMContainerStatusPBImpl.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb;
 
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
@@ -27,6 +28,7 @@ import org.apache.hadoop.yarn.api.records.impl.pb.PriorityPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
 import org.apache.hadoop.yarn.api.records.impl.pb.ResourcePBImpl;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
+import org.apache.hadoop.yarn.proto.YarnProtos;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStateProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.PriorityProto;
@@ -249,6 +251,25 @@ public class NMContainerStatusPBImpl extends NMContainerStatus {
     builder.setNodeLabelExpression(nodeLabelExpression);
   }
 
+  @Override
+  public synchronized ExecutionType getExecutionType() {
+    NMContainerStatusProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasExecutionType()) {
+      return ExecutionType.GUARANTEED;
+    }
+    return convertFromProtoFormat(p.getExecutionType());
+  }
+
+  @Override
+  public synchronized void setExecutionType(ExecutionType executionType) {
+    maybeInitBuilder();
+    if (executionType == null) {
+      builder.clearExecutionType();
+      return;
+    }
+    builder.setExecutionType(convertToProtoFormat(executionType));
+  }
+
   private void mergeLocalToBuilder() {
     if (this.containerId != null
         && !((ContainerIdPBImpl) containerId).getProto().equals(
@@ -313,4 +334,13 @@ public class NMContainerStatusPBImpl extends NMContainerStatus {
   private PriorityProto convertToProtoFormat(Priority t) {
     return ((PriorityPBImpl)t).getProto();
   }
+
+  private ExecutionType convertFromProtoFormat(
+      YarnProtos.ExecutionTypeProto e) {
+    return ProtoUtils.convertFromProtoFormat(e);
+  }
+
+  private YarnProtos.ExecutionTypeProto convertToProtoFormat(ExecutionType e) {
+    return ProtoUtils.convertToProtoFormat(e);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f53ae79/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
index c2ba677..e889cde 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
@@ -174,6 +174,7 @@ message NMContainerStatusProto {
   optional int64 creation_time = 7;
   optional string nodeLabelExpression = 8;
   optional int32 version = 9;
+  optional ExecutionTypeProto executionType = 10 [default = GUARANTEED];
 }
 
 message SCMUploaderNotifyRequestProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f53ae79/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.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/container/ContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
index a768d18..1a48b12 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
@@ -632,7 +632,8 @@ public class ContainerImpl implements Container {
           getCurrentState(), getResource(), diagnostics.toString(), exitCode,
           containerTokenIdentifier.getPriority(),
           containerTokenIdentifier.getCreationTime(),
-          containerTokenIdentifier.getNodeLabelExpression());
+          containerTokenIdentifier.getNodeLabelExpression(),
+          containerTokenIdentifier.getExecutionType());
     } finally {
       this.readLock.unlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f53ae79/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 1bdaa98..d270aa3 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
@@ -847,7 +847,8 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
         containers = startEvent.getNMContainerStatuses();
         if (containers != null && !containers.isEmpty()) {
           for (NMContainerStatus container : containers) {
-            if (container.getContainerState() == ContainerState.RUNNING) {
+            if (container.getContainerState() == ContainerState.RUNNING ||
+                container.getContainerState() == ContainerState.SCHEDULED) {
               rmNode.launchedContainers.add(container.getContainerId());
             }
           }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f53ae79/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 0c07b3e..fab02a2 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
@@ -529,6 +529,7 @@ public abstract class AbstractYarnScheduler
           node.getHttpAddress(), status.getAllocatedResource(),
           status.getPriority(), null);
     container.setVersion(status.getVersion());
+    container.setExecutionType(status.getExecutionType());
     ApplicationAttemptId attemptId =
         container.getId().getApplicationAttemptId();
     RMContainer rmContainer = new RMContainerImpl(container,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f53ae79/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.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/AppSchedulingInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
index 8acf7d5..082ec14 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 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.ExecutionType;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
@@ -525,6 +526,9 @@ public class AppSchedulingInfo {
   }
 
   public void recoverContainer(RMContainer rmContainer, String partition) {
+    if (rmContainer.getExecutionType() != ExecutionType.GUARANTEED) {
+      return;
+    }
     try {
       this.writeLock.lock();
       QueueMetrics metrics = queue.getMetrics();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f53ae79/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.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/SchedulerApplicationAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
index f9a7219..6a44cae 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
@@ -1131,9 +1131,11 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
       }
       LOG.info("SchedulerAttempt " + getApplicationAttemptId()
           + " is recovering container " + rmContainer.getContainerId());
-      liveContainers.put(rmContainer.getContainerId(), rmContainer);
-      attemptResourceUsage.incUsed(node.getPartition(),
-          rmContainer.getContainer().getResource());
+      addRMContainer(rmContainer.getContainerId(), rmContainer);
+      if (rmContainer.getExecutionType() == ExecutionType.GUARANTEED) {
+        attemptResourceUsage.incUsed(node.getPartition(),
+            rmContainer.getContainer().getResource());
+      }
 
       // resourceLimit: updated when LeafQueue#recoverContainer#allocateResource
       // is called.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f53ae79/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.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/LeafQueue.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/LeafQueue.java
index ffe0c0f..f24e30a 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/LeafQueue.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/LeafQueue.java
@@ -1802,7 +1802,9 @@ public class LeafQueue extends AbstractCSQueue {
     if (rmContainer.getState().equals(RMContainerState.COMPLETED)) {
       return;
     }
-
+    if (rmContainer.getExecutionType() != ExecutionType.GUARANTEED) {
+      return;
+    }
     // Careful! Locking order is important!
     try {
       writeLock.lock();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f53ae79/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.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/ParentQueue.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/ParentQueue.java
index 2e48000..6800b74 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/ParentQueue.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/ParentQueue.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.QueueState;
@@ -863,6 +864,9 @@ public class ParentQueue extends AbstractCSQueue {
     if (rmContainer.getState().equals(RMContainerState.COMPLETED)) {
       return;
     }
+    if (rmContainer.getExecutionType() != ExecutionType.GUARANTEED) {
+      return;
+    }
 
     // Careful! Locking order is important!
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f53ae79/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.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/TestRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
index 5cbcdbc..c9dcaef 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
@@ -77,6 +77,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
@@ -2097,7 +2098,8 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     NMContainerStatus containerReport =
         NMContainerStatus.newInstance(containerId, 0, containerState,
             Resource.newInstance(1024, 1), "recover container", 0,
-            Priority.newInstance(0), 0, nodeLabelExpression);
+            Priority.newInstance(0), 0, nodeLabelExpression,
+            ExecutionType.GUARANTEED);
     return containerReport;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f53ae79/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 5ed3278..41078d0 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
@@ -29,8 +29,11 @@ import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -44,15 +47,19 @@ import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeLabel;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.event.Dispatcher;
+import org.apache.hadoop.yarn.event.DrainDispatcher;
 import org.apache.hadoop.yarn.event.Event;
 import org.apache.hadoop.yarn.event.EventDispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
@@ -74,10 +81,14 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeStatusEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
 import org.apache.hadoop.yarn.server.timelineservice.collector.PerNodeTimelineCollectorsAuxService;
 import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineWriterImpl;
@@ -2026,6 +2037,103 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
     }
   }
 
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testHandleOpportunisticContainerStatus() throws Exception{
+    final DrainDispatcher dispatcher = new DrainDispatcher();
+    YarnConfiguration conf = new YarnConfiguration();
+    conf.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
+    conf.setBoolean(YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED,
+        true);
+    rm = new MockRM(conf){
+      @Override
+      protected Dispatcher createDispatcher() {
+        return dispatcher;
+      }
+    };
+
+    rm.start();
+    RMApp app = rm.submitApp(1024, true);
+    ApplicationAttemptId appAttemptId = app.getCurrentAppAttempt()
+        .getAppAttemptId();
+
+    ResourceTrackerService resourceTrackerService =
+        rm.getResourceTrackerService();
+    SchedulerApplicationAttempt applicationAttempt = null;
+    while (applicationAttempt == null) {
+      applicationAttempt =
+          ((AbstractYarnScheduler)rm.getRMContext().getScheduler())
+          .getApplicationAttempt(appAttemptId);
+      Thread.sleep(100);
+    }
+
+    Resource currentConsumption = applicationAttempt.getCurrentConsumption();
+    Assert.assertEquals(Resource.newInstance(0, 0), currentConsumption);
+    Resource allocResources =
+        applicationAttempt.getQueue().getMetrics().getAllocatedResources();
+    Assert.assertEquals(Resource.newInstance(0, 0), allocResources);
+
+    RegisterNodeManagerRequest req = Records.newRecord(
+        RegisterNodeManagerRequest.class);
+    NodeId nodeId = NodeId.newInstance("host2", 1234);
+    Resource capability = BuilderUtils.newResource(1024, 1);
+    req.setResource(capability);
+    req.setNodeId(nodeId);
+    req.setHttpPort(1234);
+    req.setNMVersion(YarnVersionInfo.getVersion());
+    ContainerId c1 = ContainerId.newContainerId(appAttemptId, 1);
+    ContainerId c2 = ContainerId.newContainerId(appAttemptId, 2);
+    ContainerId c3 = ContainerId.newContainerId(appAttemptId, 3);
+    NMContainerStatus queuedOpp =
+        NMContainerStatus.newInstance(c1, 1, ContainerState.SCHEDULED,
+            Resource.newInstance(1024, 1), "Dummy Queued OC",
+            ContainerExitStatus.INVALID, Priority.newInstance(5), 1234, "",
+            ExecutionType.OPPORTUNISTIC);
+    NMContainerStatus runningOpp =
+        NMContainerStatus.newInstance(c2, 1, ContainerState.RUNNING,
+            Resource.newInstance(2048, 1), "Dummy Running OC",
+            ContainerExitStatus.INVALID, Priority.newInstance(6), 1234, "",
+            ExecutionType.OPPORTUNISTIC);
+    NMContainerStatus runningGuar =
+        NMContainerStatus.newInstance(c3, 1, ContainerState.RUNNING,
+            Resource.newInstance(2048, 1), "Dummy Running GC",
+            ContainerExitStatus.INVALID, Priority.newInstance(6), 1234, "",
+            ExecutionType.GUARANTEED);
+    req.setContainerStatuses(Arrays.asList(queuedOpp, runningOpp, runningGuar));
+    // trying to register a invalid node.
+    RegisterNodeManagerResponse response =
+        resourceTrackerService.registerNodeManager(req);
+    dispatcher.await();
+    Thread.sleep(2000);
+    dispatcher.await();
+    Assert.assertEquals(NodeAction.NORMAL, response.getNodeAction());
+
+    Collection<RMContainer> liveContainers = applicationAttempt
+        .getLiveContainers();
+    Assert.assertEquals(3, liveContainers.size());
+    Iterator<RMContainer> iter = liveContainers.iterator();
+    while (iter.hasNext()) {
+      RMContainer rc = iter.next();
+      Assert.assertEquals(
+          rc.getContainerId().equals(c3) ?
+              ExecutionType.GUARANTEED : ExecutionType.OPPORTUNISTIC,
+          rc.getExecutionType());
+    }
+
+    // Should only include GUARANTEED resources
+    currentConsumption = applicationAttempt.getCurrentConsumption();
+    Assert.assertEquals(Resource.newInstance(2048, 1), currentConsumption);
+    allocResources =
+        applicationAttempt.getQueue().getMetrics().getAllocatedResources();
+    Assert.assertEquals(Resource.newInstance(2048, 1), allocResources);
+
+    SchedulerNode schedulerNode =
+        rm.getRMContext().getScheduler().getSchedulerNode(nodeId);
+    Assert.assertNotNull(schedulerNode);
+    Resource nodeResources = schedulerNode.getAllocatedResource();
+    Assert.assertEquals(Resource.newInstance(2048, 1), nodeResources);
+  }
+
   @Test(timeout = 60000)
   public void testNodeHeartBeatResponseForUnknownContainerCleanUp()
       throws Exception {


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


[03/50] [abbrv] hadoop git commit: YARN-7148. TestLogsCLI fails in trunk and branch-2 and javadoc error. Contributed by Xuan Gong.

Posted by as...@apache.org.
YARN-7148. TestLogsCLI fails in trunk and branch-2 and javadoc error. Contributed by Xuan Gong.


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

Branch: refs/heads/YARN-5972
Commit: 63231a2a3008794c5ebcbc9d3855638fc5f28216
Parents: 1f3bc63
Author: Junping Du <ju...@apache.org>
Authored: Wed Sep 6 13:19:34 2017 -0700
Committer: Junping Du <ju...@apache.org>
Committed: Wed Sep 6 13:19:34 2017 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/client/cli/TestLogsCLI.java     | 16 ++--
 .../logaggregation/AggregatedLogFormat.java     | 78 ++++++++++----------
 .../logaggregation/LogAggregationUtils.java     | 32 ++++----
 .../yarn/logaggregation/LogCLIHelpers.java      | 41 ++++++++--
 .../LogAggregationFileController.java           | 11 +--
 .../tfile/LogAggregationTFileController.java    |  1 -
 6 files changed, 103 insertions(+), 76 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/63231a2a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
index 509a790..d58732e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
@@ -285,7 +285,7 @@ public class TestLogsCLI {
   @Test (timeout = 15000)
   public void testFetchFinishedApplictionLogs() throws Exception {
     String remoteLogRootDir = "target/logs/";
-    Configuration configuration = new Configuration();
+    Configuration configuration = new YarnConfiguration();
     configuration.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true);
     configuration
       .set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogRootDir);
@@ -646,7 +646,7 @@ public class TestLogsCLI {
         any(ContainerId.class));
 
     // create local logs
-    Configuration configuration = new Configuration();
+    Configuration configuration = new YarnConfiguration();
     FileSystem fs = FileSystem.get(configuration);
     String rootLogDir = "target/LocalLogs";
     Path rootLogDirPath = new Path(rootLogDir);
@@ -790,7 +790,7 @@ public class TestLogsCLI {
     UserGroupInformation testUgi = UserGroupInformation
         .createRemoteUser(testUser);
 
-    Configuration configuration = new Configuration();
+    Configuration configuration = new YarnConfiguration();
     configuration.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true);
     configuration
         .set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogRootDir);
@@ -921,7 +921,7 @@ public class TestLogsCLI {
   public void testLogsCLIWithInvalidArgs() throws Exception {
     String localDir = "target/SaveLogs";
     Path localPath = new Path(localDir);
-    Configuration configuration = new Configuration();
+    Configuration configuration = new YarnConfiguration();
     FileSystem fs = FileSystem.get(configuration);
     ApplicationId appId = ApplicationId.newInstance(0, 1);
     YarnClient mockYarnClient =
@@ -992,7 +992,7 @@ public class TestLogsCLI {
     String localDir = "target/SaveLogs";
     Path localPath = new Path(localDir);
 
-    Configuration configuration = new Configuration();
+    Configuration configuration = new YarnConfiguration();
     configuration.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true);
     configuration
         .set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogRootDir);
@@ -1083,7 +1083,7 @@ public class TestLogsCLI {
   @Test (timeout = 15000)
   public void testPrintContainerLogMetadata() throws Exception {
     String remoteLogRootDir = "target/logs/";
-    Configuration configuration = new Configuration();
+    Configuration configuration = new YarnConfiguration();
     configuration.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true);
     configuration
       .set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogRootDir);
@@ -1188,7 +1188,7 @@ public class TestLogsCLI {
   @Test (timeout = 15000)
   public void testListNodeInfo() throws Exception {
     String remoteLogRootDir = "target/logs/";
-    Configuration configuration = new Configuration();
+    Configuration configuration = new YarnConfiguration();
     configuration.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true);
     configuration
       .set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogRootDir);
@@ -1239,7 +1239,7 @@ public class TestLogsCLI {
   @Test (timeout = 15000)
   public void testFetchApplictionLogsHar() throws Exception {
     String remoteLogRootDir = "target/logs/";
-    Configuration configuration = new Configuration();
+    Configuration configuration = new YarnConfiguration();
     configuration.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true);
     configuration
         .set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogRootDir);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63231a2a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
index 3c1dcdc..9bec147 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
@@ -583,9 +583,9 @@ public class AggregatedLogFormat {
 
     /**
      * Returns the owner of the application.
-     * 
+     *
      * @return the application owner.
-     * @throws IOException
+     * @throws IOException if we can not get the application owner.
      */
     public String getApplicationOwner() throws IOException {
       TFile.Reader.Scanner ownerScanner = null;
@@ -610,9 +610,9 @@ public class AggregatedLogFormat {
     /**
      * Returns ACLs for the application. An empty map is returned if no ACLs are
      * found.
-     * 
+     *
      * @return a map of the Application ACLs.
-     * @throws IOException
+     * @throws IOException if we can not get the application acls.
      */
     public Map<ApplicationAccessType, String> getApplicationAcls()
         throws IOException {
@@ -656,9 +656,10 @@ public class AggregatedLogFormat {
     /**
      * Read the next key and return the value-stream.
      * 
-     * @param key
-     * @return the valueStream if there are more keys or null otherwise.
-     * @throws IOException
+     * @param key the log key
+     * @return the valueStream if there are more keys or null otherwise
+     * @throws IOException if we can not get the dataInputStream
+     * for the next key
      */
     public DataInputStream next(LogKey key) throws IOException {
       if (!this.atBeginning) {
@@ -683,10 +684,10 @@ public class AggregatedLogFormat {
      * Get a ContainerLogsReader to read the logs for
      * the specified container.
      *
-     * @param containerId
+     * @param containerId the containerId
      * @return object to read the container's logs or null if the
      *         logs could not be found
-     * @throws IOException
+     * @throws IOException if we can not get the container log reader.
      */
     @Private
     public ContainerLogsReader getContainerLogsReader(
@@ -752,10 +753,10 @@ public class AggregatedLogFormat {
 
     /**
      * Writes all logs for a single container to the provided writer.
-     * @param valueStream
-     * @param writer
-     * @param logUploadedTime
-     * @throws IOException
+     * @param valueStream the valueStream
+     * @param writer the log writer
+     * @param logUploadedTime the time stamp
+     * @throws IOException if we can not read the container logs.
      */
     public static void readAcontainerLogs(DataInputStream valueStream,
         Writer writer, long logUploadedTime) throws IOException {
@@ -780,9 +781,9 @@ public class AggregatedLogFormat {
 
     /**
      * Writes all logs for a single container to the provided writer.
-     * @param valueStream
-     * @param writer
-     * @throws IOException
+     * @param valueStream the value stream
+     * @param writer the log writer
+     * @throws IOException if we can not read the container logs.
      */
     public static void readAcontainerLogs(DataInputStream valueStream,
         Writer writer) throws IOException {
@@ -849,10 +850,11 @@ public class AggregatedLogFormat {
      * Keep calling this till you get a {@link EOFException} for getting logs of
      * all types for a single container.
      * 
-     * @param valueStream
-     * @param out
-     * @param logUploadedTime
-     * @throws IOException
+     * @param valueStream the value stream
+     * @param out the print stream
+     * @param logUploadedTime the time stamp
+     * @throws IOException if we can not read the container log by specifying
+     * the container log type.
      */
     public static void readAContainerLogsForALogType(
         DataInputStream valueStream, PrintStream out, long logUploadedTime)
@@ -864,11 +866,11 @@ public class AggregatedLogFormat {
      * Keep calling this till you get a {@link EOFException} for getting logs of
      * all types for a single container for the specific bytes.
      *
-     * @param valueStream
-     * @param out
-     * @param logUploadedTime
-     * @param bytes
-     * @throws IOException
+     * @param valueStream the value stream
+     * @param out the output print stream
+     * @param logUploadedTime the log upload time stamp
+     * @param bytes the output size of the log
+     * @throws IOException if we can not read the container log
      */
     public static void readAContainerLogsForALogType(
         DataInputStream valueStream, PrintStream out, long logUploadedTime,
@@ -880,9 +882,9 @@ public class AggregatedLogFormat {
      * Keep calling this till you get a {@link EOFException} for getting logs of
      * all types for a single container.
      * 
-     * @param valueStream
-     * @param out
-     * @throws IOException
+     * @param valueStream the value stream
+     * @param out the output print stream
+     * @throws IOException if we can not read the container log
      */
     public static void readAContainerLogsForALogType(
         DataInputStream valueStream, PrintStream out)
@@ -893,11 +895,11 @@ public class AggregatedLogFormat {
     /**
      * Keep calling this till you get a {@link EOFException} for getting logs of
      * the specific types for a single container.
-     * @param valueStream
-     * @param out
-     * @param logUploadedTime
-     * @param logType
-     * @throws IOException
+     * @param valueStream the value stream
+     * @param out the output print stream
+     * @param logUploadedTime the log uploaded time stamp
+     * @param logType the given log type
+     * @throws IOException if we can not read the container logs
      */
     public static int readContainerLogsForALogType(
         DataInputStream valueStream, PrintStream out, long logUploadedTime,
@@ -909,11 +911,11 @@ public class AggregatedLogFormat {
     /**
      * Keep calling this till you get a {@link EOFException} for getting logs of
      * the specific types for a single container.
-     * @param valueStream
-     * @param out
-     * @param logUploadedTime
-     * @param logType
-     * @throws IOException
+     * @param valueStream the value stream
+     * @param out the output print stream
+     * @param logUploadedTime the log uploaded time stamp
+     * @param logType the given log type
+     * @throws IOException if we can not read the container logs
      */
     public static int readContainerLogsForALogType(
         DataInputStream valueStream, PrintStream out, long logUploadedTime,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63231a2a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogAggregationUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogAggregationUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogAggregationUtils.java
index e8a28de..6d04c29 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogAggregationUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogAggregationUtils.java
@@ -38,11 +38,11 @@ public class LogAggregationUtils {
 
   /**
    * Constructs the full filename for an application's log file per node.
-   * @param remoteRootLogDir
-   * @param appId
-   * @param user
-   * @param nodeId
-   * @param suffix
+   * @param remoteRootLogDir the aggregated remote root log dir
+   * @param appId the application Id
+   * @param user the application owner
+   * @param nodeId the node id
+   * @param suffix the log dir suffix
    * @return the remote log file.
    */
   public static Path getRemoteNodeLogFileForApp(Path remoteRootLogDir,
@@ -53,10 +53,10 @@ public class LogAggregationUtils {
 
   /**
    * Gets the remote app log dir.
-   * @param remoteRootLogDir
-   * @param appId
-   * @param user
-   * @param suffix
+   * @param remoteRootLogDir the aggregated log remote root log dir
+   * @param appId the application id
+   * @param user the application owner
+   * @param suffix the log directory suffix
    * @return the remote application specific log dir.
    */
   public static Path getRemoteAppLogDir(Path remoteRootLogDir,
@@ -67,9 +67,9 @@ public class LogAggregationUtils {
 
   /**
    * Gets the remote suffixed log dir for the user.
-   * @param remoteRootLogDir
-   * @param user
-   * @param suffix
+   * @param remoteRootLogDir the aggregated log remote root log dir
+   * @param user the application owner
+   * @param suffix the log dir suffix
    * @return the remote suffixed log dir.
    */
   public static Path getRemoteLogSuffixedDir(Path remoteRootLogDir,
@@ -83,8 +83,8 @@ public class LogAggregationUtils {
 
   /**
    * Gets the remote log user dir.
-   * @param remoteRootLogDir
-   * @param user
+   * @param remoteRootLogDir the aggregated log remote root log dir
+   * @param user the application owner
    * @return the remote per user log dir.
    */
   public static Path getRemoteLogUserDir(Path remoteRootLogDir, String user) {
@@ -93,7 +93,7 @@ public class LogAggregationUtils {
 
   /**
    * Returns the suffix component of the log dir.
-   * @param conf
+   * @param conf the configuration
    * @return the suffix which will be appended to the user log dir.
    */
   public static String getRemoteNodeLogDirSuffix(Configuration conf) {
@@ -104,7 +104,7 @@ public class LogAggregationUtils {
   
   /**
    * Converts a nodeId to a form used in the app log file name.
-   * @param nodeId
+   * @param nodeId the nodeId
    * @return the node string to be used to construct the file name.
    */
   @VisibleForTesting

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63231a2a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java
index 03acb33..0068eae 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java
@@ -128,8 +128,17 @@ public class LogCLIHelpers implements Configurable {
   @VisibleForTesting
   public int dumpAContainerLogsForLogType(ContainerLogsRequest options,
       boolean outputFailure) throws IOException {
-    boolean foundAnyLogs = this.getFileController(options.getAppId(),
-        options.getAppOwner()).readAggregatedLogs(options, null);
+    LogAggregationFileController fc = null;
+    try {
+      fc = this.getFileController(
+          options.getAppId(), options.getAppOwner());
+    } catch (IOException ex) {
+      System.err.println(ex);
+    }
+    boolean foundAnyLogs = false;
+    if (fc != null) {
+      foundAnyLogs = fc.readAggregatedLogs(options, null);
+    }
     if (!foundAnyLogs) {
       if (outputFailure) {
         containerLogNotFound(options.getContainerId());
@@ -142,9 +151,17 @@ public class LogCLIHelpers implements Configurable {
   @Private
   public int dumpAContainerLogsForLogTypeWithoutNodeId(
       ContainerLogsRequest options) throws IOException {
-    boolean foundAnyLogs = getFileController(options.getAppId(),
-        options.getAppOwner()).readAggregatedLogs(
-        options, null);
+    LogAggregationFileController fc = null;
+    try {
+      fc = this.getFileController(
+          options.getAppId(), options.getAppOwner());
+    } catch (IOException ex) {
+      System.err.println(ex);
+    }
+    boolean foundAnyLogs = false;
+    if (fc != null) {
+      foundAnyLogs = fc.readAggregatedLogs(options, null);
+    }
     if (!foundAnyLogs) {
       containerLogNotFound(options.getContainerId());
       return -1;
@@ -155,9 +172,17 @@ public class LogCLIHelpers implements Configurable {
   @Private
   public int dumpAllContainersLogs(ContainerLogsRequest options)
       throws IOException {
-    boolean foundAnyLogs = getFileController(options.getAppId(),
-        options.getAppOwner()).readAggregatedLogs(
-        options, null);
+    LogAggregationFileController fc = null;
+    try {
+      fc = this.getFileController(
+          options.getAppId(), options.getAppOwner());
+    } catch (IOException ex) {
+      System.err.println(ex);
+    }
+    boolean foundAnyLogs = false;
+    if (fc != null) {
+      foundAnyLogs = fc.readAggregatedLogs(options, null);
+    }
     if (!foundAnyLogs) {
       emptyLogDir(LogAggregationUtils.getRemoteAppLogDir(
           conf, options.getAppId(), options.getAppOwner())

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63231a2a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationFileController.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationFileController.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationFileController.java
index 87344a7..aafdb66 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationFileController.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationFileController.java
@@ -201,6 +201,7 @@ public abstract class LogAggregationFileController {
    * Output container log.
    * @param logRequest {@link ContainerLogsRequest}
    * @param os the output stream
+   * @return true if we can read the aggregated logs successfully
    * @throws IOException if we can not access the log file.
    */
   public abstract boolean readAggregatedLogs(ContainerLogsRequest logRequest,
@@ -228,9 +229,9 @@ public abstract class LogAggregationFileController {
   /**
    * Returns the owner of the application.
    *
-   * @param the aggregatedLog path.
-   * @return the application owner.
-   * @throws IOException
+   * @param aggregatedLogPath the aggregatedLog path
+   * @return the application owner
+   * @throws IOException if we can not get the application owner
    */
   public abstract String getApplicationOwner(Path aggregatedLogPath)
       throws IOException;
@@ -239,9 +240,9 @@ public abstract class LogAggregationFileController {
    * Returns ACLs for the application. An empty map is returned if no ACLs are
    * found.
    *
-   * @param the aggregatedLog path.
+   * @param aggregatedLogPath the aggregatedLog path.
    * @return a map of the Application ACLs.
-   * @throws IOException
+   * @throws IOException if we can not get the application acls
    */
   public abstract Map<ApplicationAccessType, String> getApplicationAcls(
       Path aggregatedLogPath) throws IOException;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63231a2a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/LogAggregationTFileController.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/LogAggregationTFileController.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/LogAggregationTFileController.java
index d2038e2..aaed538 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/LogAggregationTFileController.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/LogAggregationTFileController.java
@@ -172,7 +172,6 @@ public class LogAggregationTFileController
     byte[] buf = new byte[65535];
     while (nodeFiles != null && nodeFiles.hasNext()) {
       final FileStatus thisNodeFile = nodeFiles.next();
-      LOG.error(thisNodeFile.getPath().toString());
       String nodeName = thisNodeFile.getPath().getName();
       if (nodeName.equals(appId + ".har")) {
         Path p = new Path("har:///"


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


[35/50] [abbrv] hadoop git commit: YARN-7140. CollectorInfo should have Public visibility. Contributed by Varun Saxena.

Posted by as...@apache.org.
YARN-7140. CollectorInfo should have Public visibility. Contributed by Varun Saxena.


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

Branch: refs/heads/YARN-5972
Commit: 4a83170be477e6f39f134207f74084888174e96b
Parents: 0bfb3a2
Author: Rohith Sharma K S <ro...@apache.org>
Authored: Fri Sep 8 20:23:16 2017 +0530
Committer: Rohith Sharma K S <ro...@apache.org>
Committed: Fri Sep 8 20:23:16 2017 +0530

----------------------------------------------------------------------
 .../org/apache/hadoop/yarn/api/records/CollectorInfo.java    | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a83170b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/CollectorInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/CollectorInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/CollectorInfo.java
index d22b9fb..30450d6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/CollectorInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/CollectorInfo.java
@@ -18,16 +18,16 @@
 
 package org.apache.hadoop.yarn.api.records;
 
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
  * Collector info containing collector address and collector token passed from
  * RM to AM in Allocate Response.
  */
-@Private
-@InterfaceStability.Unstable
+@Public
+@Evolving
 public abstract class CollectorInfo {
 
   protected static final long DEFAULT_TIMESTAMP_VALUE = -1;


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


[32/50] [abbrv] hadoop git commit: YARN-7104. Improve Nodes Heatmap in new YARN UI with better color coding. Contributed by Da Ding.

Posted by as...@apache.org.
YARN-7104. Improve Nodes Heatmap in new YARN UI with better color coding. Contributed by Da Ding.


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

Branch: refs/heads/YARN-5972
Commit: ab8368d2e0f3931615e6ef4550ad75d9e56047d2
Parents: b3a4d7d
Author: Sunil G <su...@apache.org>
Authored: Fri Sep 8 16:16:01 2017 +0530
Committer: Sunil G <su...@apache.org>
Committed: Fri Sep 8 16:16:01 2017 +0530

----------------------------------------------------------------------
 .../app/components/base-chart-component.js      |  2 +-
 .../main/webapp/app/components/nodes-heatmap.js | 30 +++++++++++++-------
 .../src/main/webapp/app/styles/app.css          | 19 +++++++------
 3 files changed, 31 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab8368d2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/base-chart-component.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/base-chart-component.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/base-chart-component.js
index aa41893..26aa2b0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/base-chart-component.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/base-chart-component.js
@@ -108,7 +108,7 @@ export default Ember.Component.extend({
           data = d.data;
         }
 
-        this.tooltip.style("opacity", 0.9);
+        this.tooltip.style("opacity", 0.7);
         var value = data.value;
         if (this.get("type") === "memory") {
           value = Converter.memoryToSimpliedUnit(value);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab8368d2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/nodes-heatmap.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/nodes-heatmap.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/nodes-heatmap.js
index a1df480..84ff59e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/nodes-heatmap.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/nodes-heatmap.js
@@ -33,8 +33,9 @@ export default BaseChartComponent.extend({
   totalContainers: 0,
 
   bindTP: function(element, cell) {
+    var currentToolTip = this.tooltip;
     element.on("mouseover", function() {
-      this.tooltip
+      currentToolTip
         .style("left", (d3.event.pageX) + "px")
         .style("top", (d3.event.pageY - 28) + "px");
       cell.style("opacity", 1.0);
@@ -42,14 +43,20 @@ export default BaseChartComponent.extend({
       .on("mousemove", function() {
         // Handle pie chart case
         var text = cell.attr("tooltiptext");
-
-        this.tooltip.style("opacity", 0.9);
-        this.tooltip.html(text)
-          .style("left", (d3.event.pageX) + "px")
-          .style("top", (d3.event.pageY - 28) + "px");
-      }.bind(this))
+        currentToolTip
+            .style("background", "black")
+            .style("opacity", 0.7);
+        currentToolTip
+            .html(text)
+            .style('font-size', '12px')
+            .style('color', 'white')
+            .style('font-weight', '400');
+        currentToolTip
+            .style("left", (d3.event.pageX) + "px")
+            .style("top", (d3.event.pageY - 28) + "px");
+  }.bind(this))
       .on("mouseout", function() {
-        this.tooltip.style("opacity", 0);
+        currentToolTip.style("opacity", 0);
         cell.style("opacity", 0.8);
       }.bind(this));
   },
@@ -115,7 +122,10 @@ export default BaseChartComponent.extend({
     var xOffset = layout.margin;
     var yOffset = layout.margin * 3;
 
-    var colorFunc = d3.interpolate(d3.rgb("#bdddf5"), d3.rgb("#0f3957"));
+    var gradientStartColor = "#2ca02c";
+    var gradientEndColor = "#ffb014";
+
+    var colorFunc = d3.interpolateRgb(d3.rgb(gradientStartColor), d3.rgb(gradientEndColor));
 
     var sampleXOffset = (layout.x2 - layout.x1) / 2 - 2.5 * this.SAMPLE_CELL_WIDTH -
       2 * this.CELL_MARGIN;
@@ -128,7 +138,7 @@ export default BaseChartComponent.extend({
       var rect = g.append("rect")
         .attr("x", sampleXOffset)
         .attr("y", sampleYOffset)
-        .attr("fill", this.selectedCategory === i ? "#2ca02c" : colorFunc(ratio))
+        .attr("fill", this.selectedCategory === i ? "#2c7bb6" : colorFunc(ratio))
         .attr("width", this.SAMPLE_CELL_WIDTH)
         .attr("height", this.SAMPLE_HEIGHT)
         .attr("class", "hyperlink");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab8368d2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/app.css
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/app.css b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/app.css
index 38e25e4..f48c186 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/app.css
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/app.css
@@ -58,8 +58,7 @@ text.queue {
 }
 
 text.heatmap-cell {
-  font: 14px sans-serif;
-  font-weight: bold;
+  font-size: 14px;
   text-anchor: middle;
   fill: Azure;
   text-align: center;
@@ -83,11 +82,13 @@ text.heatmap-cell-notselected {
   text-anchor: middle;
   fill: Silver;
   text-align: center;
+
 }
 
 text.heatmap-rack {
-  font: 20px sans-serif;
-  fill: DimGray;
+  font-size: 18px;
+  font-weight: 400;
+  fill: #4b5052;
 }
 
 path.queue {
@@ -111,9 +112,8 @@ line.chart {
  */
 text.chart-title {
   font-size: 30px;
-  font-family: sans-serif;
   text-anchor: middle;
-  fill: Gray;
+  fill: #4b5052;
 }
 
 text.donut-highlight-text, text.donut-highlight-sub {
@@ -143,9 +143,10 @@ text.bar-chart-text {
 div.tooltip {
   position: absolute;
   text-align: center;
-  padding: 2px;
-  font: 24px sans-serif;
-  background: lightsteelblue;
+  padding: 10px;
+  font-size: 16px;
+  background: black;
+  color: white;
   border: 0px;
   border-radius: 8px;
   pointer-events: none;


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


[47/50] [abbrv] hadoop git commit: YARN-7163. RMContext need not to be injected to webapp and other Always Running services. Contributed by Rohith Sharma K S.

Posted by as...@apache.org.
YARN-7163. RMContext need not to be injected to webapp and other Always Running services. 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/722ee841
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/722ee841
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/722ee841

Branch: refs/heads/YARN-5972
Commit: 722ee841948db1f246f0056acec9a1ac464fe1f9
Parents: aa4b6fb
Author: Sunil G <su...@apache.org>
Authored: Mon Sep 11 10:28:10 2017 +0530
Committer: Sunil G <su...@apache.org>
Committed: Mon Sep 11 10:28:10 2017 +0530

----------------------------------------------------------------------
 .../yarn/server/webapp/AppAttemptBlock.java     | 26 ++++++--
 .../hadoop/yarn/server/webapp/AppBlock.java     | 34 ++++++++---
 .../hadoop/yarn/server/webapp/AppsBlock.java    | 11 +++-
 .../yarn/server/webapp/ContainerBlock.java      | 14 +++--
 .../hadoop/yarn/server/webapp/WebServices.java  | 64 ++++++++++++++------
 .../RMDelegationTokenSecretManager.java         | 16 ++---
 .../resourcemanager/webapp/ContainerPage.java   |  3 +-
 .../webapp/RMAppAttemptBlock.java               | 20 +++++-
 .../resourcemanager/webapp/RMAppBlock.java      | 35 ++++++++++-
 .../resourcemanager/webapp/RMAppsBlock.java     | 17 ++++--
 .../webapp/RMContainerBlock.java                | 47 ++++++++++++++
 .../server/resourcemanager/webapp/RMWebApp.java |  2 -
 .../resourcemanager/webapp/RMWebServices.java   | 52 +++++++++++++++-
 .../resourcemanager/TestClientRMTokens.java     |  3 +
 .../TestTokenClientRMService.java               |  3 +
 .../resourcemanager/webapp/TestAppPage.java     |  5 +-
 .../resourcemanager/webapp/TestRMWebApp.java    |  3 -
 .../webapp/TestRMWebAppFairScheduler.java       |  6 --
 .../webapp/TestRedirectionErrorPage.java        |  3 -
 19 files changed, 293 insertions(+), 71 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/722ee841/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppAttemptBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppAttemptBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppAttemptBlock.java
index 87c554d..cad14b6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppAttemptBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppAttemptBlock.java
@@ -19,8 +19,11 @@ package org.apache.hadoop.yarn.server.webapp;
 
 import static org.apache.hadoop.yarn.util.StringHelper.join;
 import static org.apache.hadoop.yarn.webapp.YarnWebParams.APPLICATION_ATTEMPT_ID;
+
+import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
 import java.util.Collection;
+import java.util.List;
 
 import org.apache.commons.lang.StringEscapeUtils;
 import org.apache.commons.logging.Log;
@@ -34,6 +37,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerReport;
 import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo;
 import org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo;
 import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
@@ -77,15 +81,13 @@ public class AppAttemptBlock extends HtmlBlock {
           GetApplicationAttemptReportRequest.newInstance(appAttemptId);
       if (callerUGI == null) {
         appAttemptReport =
-            appBaseProt.getApplicationAttemptReport(request)
-              .getApplicationAttemptReport();
+            getApplicationAttemptReport(request);
       } else {
         appAttemptReport = callerUGI.doAs(
             new PrivilegedExceptionAction<ApplicationAttemptReport> () {
           @Override
           public ApplicationAttemptReport run() throws Exception {
-            return appBaseProt.getApplicationAttemptReport(request)
-                .getApplicationAttemptReport();
+            return getApplicationAttemptReport(request);
           }
         });
       }
@@ -108,13 +110,13 @@ public class AppAttemptBlock extends HtmlBlock {
       final GetContainersRequest request =
           GetContainersRequest.newInstance(appAttemptId);
       if (callerUGI == null) {
-        containers = appBaseProt.getContainers(request).getContainerList();
+        containers = getContainers(request);
       } else {
         containers = callerUGI.doAs(
             new PrivilegedExceptionAction<Collection<ContainerReport>> () {
           @Override
           public Collection<ContainerReport> run() throws Exception {
-            return  appBaseProt.getContainers(request).getContainerList();
+            return  getContainers(request);
           }
         });
       }
@@ -190,6 +192,18 @@ public class AppAttemptBlock extends HtmlBlock {
     tbody.__().__();
   }
 
+  protected List<ContainerReport> getContainers(
+      final GetContainersRequest request) throws YarnException, IOException {
+    return appBaseProt.getContainers(request).getContainerList();
+  }
+
+  protected ApplicationAttemptReport getApplicationAttemptReport(
+      final GetApplicationAttemptReportRequest request)
+      throws YarnException, IOException {
+    return appBaseProt.getApplicationAttemptReport(request)
+        .getApplicationAttemptReport();
+  }
+
   protected void generateOverview(ApplicationAttemptReport appAttemptReport,
       Collection<ContainerReport> containers, AppAttemptInfo appAttempt,
       String node) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/722ee841/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
index 95bc0aa..08e75ac 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
@@ -22,8 +22,10 @@ import static org.apache.hadoop.yarn.util.StringHelper.join;
 import static org.apache.hadoop.yarn.webapp.YarnWebParams.APPLICATION_ID;
 import static org.apache.hadoop.yarn.webapp.YarnWebParams.WEB_UI_TYPE;
 
+import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
 import java.util.Collection;
+import java.util.List;
 import java.util.Map;
 
 import org.apache.commons.lang.StringEscapeUtils;
@@ -49,6 +51,7 @@ import org.apache.hadoop.yarn.api.records.LogAggregationStatus;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.ContainerNotFoundException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo;
 import org.apache.hadoop.yarn.server.webapp.dao.AppInfo;
 import org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo;
@@ -114,8 +117,7 @@ public class AppBlock extends HtmlBlock {
             new PrivilegedExceptionAction<ApplicationReport> () {
           @Override
           public ApplicationReport run() throws Exception {
-            return appBaseProt.getApplicationReport(request)
-                .getApplicationReport();
+            return getApplicationReport(request);
           }
         });
       }
@@ -190,8 +192,7 @@ public class AppBlock extends HtmlBlock {
               ApplicationAttemptReport>>() {
             @Override
             public Collection<ApplicationAttemptReport> run() throws Exception {
-              return appBaseProt.getApplicationAttempts(request)
-                  .getApplicationAttemptList();
+              return getApplicationAttemptsReport(request);
             }
           });
     } catch (Exception e) {
@@ -301,7 +302,7 @@ public class AppBlock extends HtmlBlock {
                       appAttemptReport.getAMContainerId());
         if (callerUGI == null) {
           containerReport =
-              appBaseProt.getContainerReport(request).getContainerReport();
+              getContainerReport(request);
         } else {
           containerReport = callerUGI.doAs(
               new PrivilegedExceptionAction<ContainerReport>() {
@@ -310,8 +311,7 @@ public class AppBlock extends HtmlBlock {
               ContainerReport report = null;
               if (request.getContainerId() != null) {
                   try {
-                    report = appBaseProt.getContainerReport(request)
-                        .getContainerReport();
+                    report = getContainerReport(request);
                   } catch (ContainerNotFoundException ex) {
                     LOG.warn(ex.getMessage());
                   }
@@ -364,6 +364,26 @@ public class AppBlock extends HtmlBlock {
     tbody.__().__();
   }
 
+  protected ContainerReport getContainerReport(
+      final GetContainerReportRequest request)
+      throws YarnException, IOException {
+    return appBaseProt.getContainerReport(request).getContainerReport();
+  }
+
+  protected List<ApplicationAttemptReport> getApplicationAttemptsReport(
+      final GetApplicationAttemptsRequest request)
+      throws YarnException, IOException {
+    return appBaseProt.getApplicationAttempts(request)
+        .getApplicationAttemptList();
+  }
+
+  protected ApplicationReport getApplicationReport(
+      final GetApplicationReportRequest request)
+      throws YarnException, IOException {
+    return appBaseProt.getApplicationReport(request).getApplicationReport();
+  }
+
+
   private String clarifyAppState(YarnApplicationState state) {
     String ret = state.toString();
     switch (state) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/722ee841/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppsBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppsBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppsBlock.java
index d836e64..e3a47de 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppsBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppsBlock.java
@@ -30,6 +30,7 @@ import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
 import java.util.Collection;
 import java.util.EnumSet;
+import java.util.List;
 
 import org.apache.commons.lang.StringEscapeUtils;
 import org.apache.commons.lang.math.LongRange;
@@ -110,20 +111,24 @@ public class AppsBlock extends HtmlBlock {
         new LongRange(appStartedTimeBegain, appStartedTimeEnd));
 
     if (callerUGI == null) {
-      appReports = appBaseProt.getApplications(request).getApplicationList();
+      appReports = getApplicationReport(request);
     } else {
       appReports =
           callerUGI
             .doAs(new PrivilegedExceptionAction<Collection<ApplicationReport>>() {
               @Override
               public Collection<ApplicationReport> run() throws Exception {
-                return appBaseProt.getApplications(request)
-                  .getApplicationList();
+                return getApplicationReport(request);
               }
             });
     }
   }
 
+  protected List<ApplicationReport> getApplicationReport(
+      final GetApplicationsRequest request) throws YarnException, IOException {
+    return appBaseProt.getApplications(request).getApplicationList();
+  }
+
   @Override
   public void render(Block html) {
     setTitle("Applications");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/722ee841/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/ContainerBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/ContainerBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/ContainerBlock.java
index fa35a3d..b8c0bed 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/ContainerBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/ContainerBlock.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.webapp;
 import static org.apache.hadoop.yarn.util.StringHelper.join;
 import static org.apache.hadoop.yarn.webapp.YarnWebParams.CONTAINER_ID;
 
+import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
 
 import org.apache.commons.logging.Log;
@@ -30,6 +31,7 @@ import org.apache.hadoop.yarn.api.ApplicationBaseProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportRequest;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerReport;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo;
 import org.apache.hadoop.yarn.util.Times;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
@@ -70,15 +72,13 @@ public class ContainerBlock extends HtmlBlock {
       final GetContainerReportRequest request =
           GetContainerReportRequest.newInstance(containerId);
       if (callerUGI == null) {
-        containerReport = appBaseProt.getContainerReport(request)
-            .getContainerReport();
+        containerReport = getContainerReport(request);
       } else {
         containerReport = callerUGI.doAs(
             new PrivilegedExceptionAction<ContainerReport> () {
           @Override
           public ContainerReport run() throws Exception {
-            return appBaseProt.getContainerReport(request)
-                .getContainerReport();
+            return getContainerReport(request);
           }
         });
       }
@@ -126,4 +126,10 @@ public class ContainerBlock extends HtmlBlock {
 
     html.__(InfoBlock.class);
   }
+
+  protected ContainerReport getContainerReport(
+      final GetContainerReportRequest request)
+      throws YarnException, IOException {
+    return appBaseProt.getContainerReport(request).getContainerReport();
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/722ee841/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebServices.java
index 904c511..6bb6c98 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebServices.java
@@ -18,11 +18,13 @@
 
 package org.apache.hadoop.yarn.server.webapp;
 
+import java.io.IOException;
 import java.lang.reflect.UndeclaredThrowableException;
 import java.security.PrivilegedExceptionAction;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Set;
 
 import javax.servlet.http.HttpServletRequest;
@@ -51,13 +53,13 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetContainersRequest;
 import org.apache.hadoop.yarn.exceptions.ApplicationAttemptNotFoundException;
 import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException;
 import org.apache.hadoop.yarn.exceptions.ContainerNotFoundException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo;
 import org.apache.hadoop.yarn.server.webapp.dao.AppAttemptsInfo;
 import org.apache.hadoop.yarn.server.webapp.dao.AppInfo;
 import org.apache.hadoop.yarn.server.webapp.dao.AppsInfo;
 import org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo;
 import org.apache.hadoop.yarn.server.webapp.dao.ContainersInfo;
-import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.webapp.BadRequestException;
 import org.apache.hadoop.yarn.webapp.ForbiddenException;
 import org.apache.hadoop.yarn.webapp.NotFoundException;
@@ -154,13 +156,13 @@ public class WebServices {
       if (callerUGI == null) {
         // TODO: the request should take the params like what RMWebServices does
         // in YARN-1819.
-        appReports = appBaseProt.getApplications(request).getApplicationList();
+        appReports = getApplicationsReport(request);
       } else {
         appReports = callerUGI.doAs(
             new PrivilegedExceptionAction<Collection<ApplicationReport>> () {
           @Override
           public Collection<ApplicationReport> run() throws Exception {
-            return appBaseProt.getApplications(request).getApplicationList();
+            return getApplicationsReport(request);
           }
         });
       }
@@ -220,7 +222,7 @@ public class WebServices {
       if (callerUGI == null) {
         GetApplicationReportRequest request =
             GetApplicationReportRequest.newInstance(id);
-        app = appBaseProt.getApplicationReport(request).getApplicationReport();
+        app = getApplicationReport(request);
       } else {
         app = callerUGI.doAs(
             new PrivilegedExceptionAction<ApplicationReport> () {
@@ -228,7 +230,7 @@ public class WebServices {
           public ApplicationReport run() throws Exception {
             GetApplicationReportRequest request =
                 GetApplicationReportRequest.newInstance(id);
-            return appBaseProt.getApplicationReport(request).getApplicationReport();
+            return getApplicationReport(request);
           }
         });
       }
@@ -251,8 +253,7 @@ public class WebServices {
         GetApplicationAttemptsRequest request =
             GetApplicationAttemptsRequest.newInstance(id);
         appAttemptReports =
-            appBaseProt.getApplicationAttempts(request)
-              .getApplicationAttemptList();
+            getApplicationAttemptsReport(request);
       } else {
         appAttemptReports = callerUGI.doAs(
             new PrivilegedExceptionAction<Collection<ApplicationAttemptReport>> () {
@@ -260,8 +261,7 @@ public class WebServices {
           public Collection<ApplicationAttemptReport> run() throws Exception {
             GetApplicationAttemptsRequest request =
                 GetApplicationAttemptsRequest.newInstance(id);
-            return appBaseProt.getApplicationAttempts(request)
-                  .getApplicationAttemptList();
+            return getApplicationAttemptsReport(request);
           }
         });
       }
@@ -292,8 +292,7 @@ public class WebServices {
         GetApplicationAttemptReportRequest request =
             GetApplicationAttemptReportRequest.newInstance(aaid);
         appAttempt =
-            appBaseProt.getApplicationAttemptReport(request)
-              .getApplicationAttemptReport();
+            getApplicationAttemptReport(request);
       } else {
         appAttempt = callerUGI.doAs(
             new PrivilegedExceptionAction<ApplicationAttemptReport> () {
@@ -301,8 +300,7 @@ public class WebServices {
           public ApplicationAttemptReport run() throws Exception {
             GetApplicationAttemptReportRequest request =
                 GetApplicationAttemptReportRequest.newInstance(aaid);
-            return appBaseProt.getApplicationAttemptReport(request)
-                  .getApplicationAttemptReport();
+            return getApplicationAttemptReport(request);
           }
         });
       }
@@ -327,14 +325,14 @@ public class WebServices {
       if (callerUGI == null) {
         GetContainersRequest request = GetContainersRequest.newInstance(aaid);
         containerReports =
-            appBaseProt.getContainers(request).getContainerList();
+            getContainersReport(request);
       } else {
         containerReports = callerUGI.doAs(
             new PrivilegedExceptionAction<Collection<ContainerReport>> () {
           @Override
           public Collection<ContainerReport> run() throws Exception {
             GetContainersRequest request = GetContainersRequest.newInstance(aaid);
-            return appBaseProt.getContainers(request).getContainerList();
+            return getContainersReport(request);
           }
         });
       }
@@ -366,7 +364,7 @@ public class WebServices {
         GetContainerReportRequest request =
             GetContainerReportRequest.newInstance(cid);
         container =
-            appBaseProt.getContainerReport(request).getContainerReport();
+            getContainerReport(request);
       } else {
         container = callerUGI.doAs(
             new PrivilegedExceptionAction<ContainerReport> () {
@@ -374,7 +372,7 @@ public class WebServices {
           public ContainerReport run() throws Exception {
             GetContainerReportRequest request =
                 GetContainerReportRequest.newInstance(cid);
-            return appBaseProt.getContainerReport(request).getContainerReport();
+            return getContainerReport(request);
           }
         });
       }
@@ -516,4 +514,36 @@ public class WebServices {
     }
   }
 
+  protected ApplicationReport getApplicationReport(
+      GetApplicationReportRequest request) throws YarnException, IOException {
+    return appBaseProt.getApplicationReport(request).getApplicationReport();
+  }
+
+  protected List<ApplicationReport> getApplicationsReport(
+      final GetApplicationsRequest request) throws YarnException, IOException {
+    return appBaseProt.getApplications(request).getApplicationList();
+  }
+
+  protected ApplicationAttemptReport getApplicationAttemptReport(
+      GetApplicationAttemptReportRequest request)
+      throws YarnException, IOException {
+    return appBaseProt.getApplicationAttemptReport(request)
+        .getApplicationAttemptReport();
+  }
+
+  protected List<ApplicationAttemptReport> getApplicationAttemptsReport(
+      GetApplicationAttemptsRequest request) throws YarnException, IOException {
+    return appBaseProt.getApplicationAttempts(request)
+        .getApplicationAttemptList();
+  }
+
+  protected ContainerReport getContainerReport(
+      GetContainerReportRequest request) throws YarnException, IOException {
+    return appBaseProt.getContainerReport(request).getContainerReport();
+  }
+
+  protected List<ContainerReport> getContainersReport(
+      GetContainersRequest request) throws YarnException, IOException {
+    return appBaseProt.getContainers(request).getContainerList();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/722ee841/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMDelegationTokenSecretManager.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/security/RMDelegationTokenSecretManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMDelegationTokenSecretManager.java
index 631ca9d..53cc471 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMDelegationTokenSecretManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMDelegationTokenSecretManager.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.security.token.delegation.DelegationKey;
 import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.Recoverable;
 
@@ -52,7 +53,7 @@ public class RMDelegationTokenSecretManager extends
   private static final Log LOG = LogFactory
       .getLog(RMDelegationTokenSecretManager.class);
 
-  protected final RMContext rmContext;
+  private final ResourceManager rm;
 
   /**
    * Create a secret manager
@@ -73,7 +74,7 @@ public class RMDelegationTokenSecretManager extends
                                       RMContext rmContext) {
     super(delegationKeyUpdateInterval, delegationTokenMaxLifetime,
           delegationTokenRenewInterval, delegationTokenRemoverScanInterval);
-    this.rmContext = rmContext;
+    this.rm = rmContext.getResourceManager();
   }
 
   @Override
@@ -85,7 +86,7 @@ public class RMDelegationTokenSecretManager extends
   protected void storeNewMasterKey(DelegationKey newKey) {
     try {
       LOG.info("storing master key with keyID " + newKey.getKeyId());
-      rmContext.getStateStore().storeRMDTMasterKey(newKey);
+      rm.getRMContext().getStateStore().storeRMDTMasterKey(newKey);
     } catch (Exception e) {
       LOG.error("Error in storing master key with KeyID: " + newKey.getKeyId());
       ExitUtil.terminate(1, e);
@@ -96,7 +97,7 @@ public class RMDelegationTokenSecretManager extends
   protected void removeStoredMasterKey(DelegationKey key) {
     try {
       LOG.info("removing master key with keyID " + key.getKeyId());
-      rmContext.getStateStore().removeRMDTMasterKey(key);
+      rm.getRMContext().getStateStore().removeRMDTMasterKey(key);
     } catch (Exception e) {
       LOG.error("Error in removing master key with KeyID: " + key.getKeyId());
       ExitUtil.terminate(1, e);
@@ -109,7 +110,8 @@ public class RMDelegationTokenSecretManager extends
     try {
       LOG.info("storing RMDelegation token with sequence number: "
           + identifier.getSequenceNumber());
-      rmContext.getStateStore().storeRMDelegationToken(identifier, renewDate);
+      rm.getRMContext().getStateStore().storeRMDelegationToken(identifier,
+          renewDate);
     } catch (Exception e) {
       LOG.error("Error in storing RMDelegationToken with sequence number: "
           + identifier.getSequenceNumber());
@@ -123,7 +125,7 @@ public class RMDelegationTokenSecretManager extends
     try {
       LOG.info("updating RMDelegation token with sequence number: "
           + id.getSequenceNumber());
-      rmContext.getStateStore().updateRMDelegationToken(id, renewDate);
+      rm.getRMContext().getStateStore().updateRMDelegationToken(id, renewDate);
     } catch (Exception e) {
       LOG.error("Error in updating persisted RMDelegationToken" +
                 " with sequence number: " + id.getSequenceNumber());
@@ -137,7 +139,7 @@ public class RMDelegationTokenSecretManager extends
     try {
       LOG.info("removing RMDelegation token with sequence number: "
           + ident.getSequenceNumber());
-      rmContext.getStateStore().removeRMDelegationToken(ident);
+      rm.getRMContext().getStateStore().removeRMDelegationToken(ident);
     } catch (Exception e) {
       LOG.error("Error in removing RMDelegationToken with sequence number: "
           + ident.getSequenceNumber());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/722ee841/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/ContainerPage.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/ContainerPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/ContainerPage.java
index 2cd209b..0204989 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/ContainerPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/ContainerPage.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.yarn.server.resourcemanager.webapp;
 
 import static org.apache.hadoop.yarn.util.StringHelper.join;
 
-import org.apache.hadoop.yarn.server.webapp.ContainerBlock;
 import org.apache.hadoop.yarn.webapp.SubView;
 import org.apache.hadoop.yarn.webapp.YarnWebParams;
 
@@ -38,7 +37,7 @@ public class ContainerPage extends RmView {
 
   @Override
   protected Class<? extends SubView> content() {
-    return ContainerBlock.class;
+    return RMContainerBlock.class;
   }
 
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/722ee841/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppAttemptBlock.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/RMAppAttemptBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppAttemptBlock.java
index 82ddb54..b26eb09 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppAttemptBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppAttemptBlock.java
@@ -23,18 +23,22 @@ import static org.apache.hadoop.yarn.webapp.view.JQueryUI._INFO_WRAP;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI._ODD;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI._TH;
 
+import java.io.IOException;
 import java.util.Collection;
 import java.util.List;
 
 import org.apache.commons.lang.StringEscapeUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainersRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerReport;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
@@ -61,7 +65,7 @@ public class RMAppAttemptBlock extends AppAttemptBlock{
 
   @Inject
   RMAppAttemptBlock(ViewContext ctx, ResourceManager rm, Configuration conf) {
-    super(rm.getClientRMService(), ctx);
+    super(null, ctx);
     this.rm = rm;
     this.conf = conf;
   }
@@ -275,4 +279,18 @@ public class RMAppAttemptBlock extends AppAttemptBlock{
     createContainerLocalityTable(html);
     createResourceRequestsTable(html);
   }
+
+  @Override
+  protected List<ContainerReport> getContainers(
+      final GetContainersRequest request) throws YarnException, IOException {
+    return rm.getClientRMService().getContainers(request).getContainerList();
+  }
+
+  @Override
+  protected ApplicationAttemptReport getApplicationAttemptReport(
+      final GetApplicationAttemptReportRequest request)
+      throws YarnException, IOException {
+    return rm.getClientRMService().getApplicationAttemptReport(request)
+        .getApplicationAttemptReport();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/722ee841/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppBlock.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/RMAppBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppBlock.java
index cd04264..e5e31e0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppBlock.java
@@ -20,15 +20,23 @@ package org.apache.hadoop.yarn.server.resourcemanager.webapp;
 
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI._INFO_WRAP;
 
+import java.io.IOException;
 import java.util.Collection;
+import java.util.List;
 import java.util.Set;
 
 import org.apache.commons.lang.StringEscapeUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.ContainerReport;
 import org.apache.hadoop.yarn.api.records.LogAggregationStatus;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppMetrics;
@@ -52,7 +60,7 @@ public class RMAppBlock extends AppBlock{
 
   @Inject
   RMAppBlock(ViewContext ctx, Configuration conf, ResourceManager rm) {
-    super(rm.getClientRMService(), ctx, conf);
+    super(null, ctx, conf);
     this.conf = conf;
     this.rm = rm;
   }
@@ -187,4 +195,29 @@ public class RMAppBlock extends AppBlock{
     }
     return rmApp.getLogAggregationStatusForAppReport();
   }
+
+  @Override
+  protected ContainerReport getContainerReport(
+      final GetContainerReportRequest request)
+      throws YarnException, IOException {
+    return rm.getClientRMService().getContainerReport(request)
+        .getContainerReport();
+  }
+
+  @Override
+  protected List<ApplicationAttemptReport> getApplicationAttemptsReport(
+      final GetApplicationAttemptsRequest request)
+      throws YarnException, IOException {
+    return rm.getClientRMService().getApplicationAttempts(request)
+        .getApplicationAttemptList();
+  }
+
+  @Override
+  protected ApplicationReport getApplicationReport(
+      final GetApplicationReportRequest request)
+      throws YarnException, IOException {
+    return rm.getClientRMService().getApplicationReport(request)
+        .getApplicationReport();
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/722ee841/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppsBlock.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/RMAppsBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppsBlock.java
index ede71e3..d0dccab 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppsBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppsBlock.java
@@ -22,14 +22,17 @@ import static org.apache.hadoop.yarn.util.StringHelper.join;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.C_PROGRESSBAR;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.C_PROGRESSBAR_VALUE;
 
+import java.io.IOException;
+import java.util.List;
 import java.util.Set;
 
 import org.apache.commons.lang.StringEscapeUtils;
 import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.yarn.api.ApplicationBaseProtocol;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
@@ -47,9 +50,8 @@ public class RMAppsBlock extends AppsBlock {
   private ResourceManager rm;
 
   @Inject
-  RMAppsBlock(ResourceManager rm, ApplicationBaseProtocol appBaseProt,
-      View.ViewContext ctx) {
-    super(appBaseProt, ctx);
+  RMAppsBlock(ResourceManager rm, View.ViewContext ctx) {
+    super(null, ctx);
     this.rm = rm;
   }
 
@@ -193,4 +195,11 @@ public class RMAppsBlock extends AppsBlock {
 
     tbody.__().__();
   }
+
+  @Override
+  protected List<ApplicationReport> getApplicationReport(
+      final GetApplicationsRequest request) throws YarnException, IOException {
+    return rm.getClientRMService().getApplications(request)
+        .getApplicationList();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/722ee841/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMContainerBlock.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/RMContainerBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMContainerBlock.java
new file mode 100644
index 0000000..f589a5d
--- /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/RMContainerBlock.java
@@ -0,0 +1,47 @@
+/**
+ * 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;
+
+import java.io.IOException;
+
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportRequest;
+import org.apache.hadoop.yarn.api.records.ContainerReport;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.server.webapp.ContainerBlock;
+
+import com.google.inject.Inject;
+
+public class RMContainerBlock extends ContainerBlock {
+
+  private final ResourceManager rm;
+
+  @Inject
+  public RMContainerBlock(ResourceManager resourceManager, ViewContext ctx) {
+    super(null, ctx);
+    this.rm = resourceManager;
+  }
+
+  @Override
+  protected ContainerReport getContainerReport(
+      final GetContainerReportRequest request)
+      throws YarnException, IOException {
+    return rm.getClientRMService().getContainerReport(request)
+        .getContainerReport();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/722ee841/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebApp.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/RMWebApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebApp.java
index 3367cf4..4e36665 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebApp.java
@@ -23,7 +23,6 @@ import static org.apache.hadoop.yarn.util.StringHelper.pajoin;
 import java.net.InetSocketAddress;
 
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
-import org.apache.hadoop.yarn.api.ApplicationBaseProtocol;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.util.RMHAUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
@@ -55,7 +54,6 @@ public class RMWebApp extends WebApp implements YarnWebParams {
 
     if (rm != null) {
       bind(ResourceManager.class).toInstance(rm);
-      bind(ApplicationBaseProtocol.class).toInstance(rm.getClientRMService());
     }
     route("/", RmController.class);
     route(pajoin("/nodes", NODE_STATE), RmController.class, "nodes");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/722ee841/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.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/RMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
index 7ffe106..6dc3d9a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
@@ -72,7 +72,12 @@ import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthentica
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainersRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
@@ -95,10 +100,12 @@ import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationPriorityRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationTimeoutsRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType;
+import org.apache.hadoop.yarn.api.records.ContainerReport;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeLabel;
@@ -223,7 +230,8 @@ public class RMWebServices extends WebServices implements RMWebServiceProtocol {
 
   @Inject
   public RMWebServices(final ResourceManager rm, Configuration conf) {
-    super(rm.getClientRMService());
+    // don't inject, always take appBaseRoot from RM.
+    super(null);
     this.rm = rm;
     this.conf = conf;
     isCentralizedNodeLabelConfiguration =
@@ -2404,4 +2412,46 @@ public class RMWebServices extends WebServices implements RMWebServiceProtocol {
         app.getApplicationTimeouts().get(appTimeout.getTimeoutType()));
     return Response.status(Status.OK).entity(timeout).build();
   }
+
+  @Override
+  protected ApplicationReport getApplicationReport(
+      GetApplicationReportRequest request) throws YarnException, IOException {
+    return rm.getClientRMService().getApplicationReport(request)
+        .getApplicationReport();
+  }
+
+  @Override
+  protected List<ApplicationReport> getApplicationsReport(
+      final GetApplicationsRequest request) throws YarnException, IOException {
+    return rm.getClientRMService().getApplications(request)
+        .getApplicationList();
+  }
+
+  @Override
+  protected ApplicationAttemptReport getApplicationAttemptReport(
+      GetApplicationAttemptReportRequest request)
+      throws YarnException, IOException {
+    return rm.getClientRMService().getApplicationAttemptReport(request)
+        .getApplicationAttemptReport();
+  }
+
+  @Override
+  protected List<ApplicationAttemptReport> getApplicationAttemptsReport(
+      GetApplicationAttemptsRequest request) throws YarnException, IOException {
+    return rm.getClientRMService().getApplicationAttempts(request)
+        .getApplicationAttemptList();
+  }
+
+  @Override
+  protected ContainerReport getContainerReport(
+      GetContainerReportRequest request) throws YarnException, IOException {
+    return rm.getClientRMService().getContainerReport(request)
+        .getContainerReport();
+  }
+
+  @Override
+  protected List<ContainerReport> getContainersReport(
+      GetContainersRequest request) throws YarnException, IOException {
+    return rm.getClientRMService().getContainers(request).getContainerList();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/722ee841/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMTokens.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/TestClientRMTokens.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMTokens.java
index 65145a4..06c1c42 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMTokens.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMTokens.java
@@ -533,8 +533,11 @@ public class TestClientRMTokens {
   private static RMDelegationTokenSecretManager
       createRMDelegationTokenSecretManager(long secretKeyInterval,
           long tokenMaxLifetime, long tokenRenewInterval) {
+    ResourceManager rm = mock(ResourceManager.class);
     RMContext rmContext = mock(RMContext.class);
     when(rmContext.getStateStore()).thenReturn(new NullRMStateStore());
+    when(rm.getRMContext()).thenReturn(rmContext);
+    when(rmContext.getResourceManager()).thenReturn(rm);
 
     RMDelegationTokenSecretManager rmDtSecretManager =
         new RMDelegationTokenSecretManager(secretKeyInterval, tokenMaxLifetime,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/722ee841/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestTokenClientRMService.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/TestTokenClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestTokenClientRMService.java
index 78271c6..3e1ede9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestTokenClientRMService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestTokenClientRMService.java
@@ -71,8 +71,11 @@ public class TestTokenClientRMService {
 
   @BeforeClass
   public static void setupSecretManager() throws IOException {
+    ResourceManager rm = mock(ResourceManager.class);
     RMContext rmContext = mock(RMContext.class);
     when(rmContext.getStateStore()).thenReturn(new NullRMStateStore());
+    when(rm.getRMContext()).thenReturn(rmContext);
+    when(rmContext.getResourceManager()).thenReturn(rm);
     dtsm =
         new RMDelegationTokenSecretManager(60000, 60000, 60000, 60000,
             rmContext);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/722ee841/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestAppPage.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/TestAppPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestAppPage.java
index a009bc0..d9ed073 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestAppPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestAppPage.java
@@ -23,7 +23,6 @@ import static org.mockito.Mockito.when;
 
 import java.io.IOException;
 
-import org.apache.hadoop.yarn.api.ApplicationBaseProtocol;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -80,15 +79,13 @@ public class TestAppPage {
                 try {
                   ResourceManager rm = TestRMWebApp.mockRm(rmContext);
                   binder.bind(ResourceManager.class).toInstance(rm);
-                  binder.bind(ApplicationBaseProtocol.class).toInstance(
-                    rm.getClientRMService());
                 } catch (IOException e) {
                   throw new IllegalStateException(e);
                 }
               }
             });
     
-    AppBlock instance = injector.getInstance(AppBlock.class);
+    AppBlock instance = injector.getInstance(RMAppBlock.class);
     instance.set(YarnWebParams.APPLICATION_ID, APP_ID.toString());
     instance.render();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/722ee841/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebApp.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/TestRMWebApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebApp.java
index 8a65040..b125608 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebApp.java
@@ -32,7 +32,6 @@ import java.util.Map;
 import java.util.concurrent.ConcurrentMap;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.api.ApplicationBaseProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@@ -103,8 +102,6 @@ public class TestRMWebApp {
         try {
           ResourceManager mockRm = mockRm(3, 1, 2, 8*GiB);
           binder.bind(ResourceManager.class).toInstance(mockRm);
-          binder.bind(ApplicationBaseProtocol.class)
-              .toInstance(mockRm.getClientRMService());
         } catch (IOException e) {
           throw new IllegalStateException(e);
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/722ee841/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebAppFairScheduler.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/TestRMWebAppFairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebAppFairScheduler.java
index 78fadef..8c00b39 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebAppFairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebAppFairScheduler.java
@@ -24,7 +24,6 @@ import com.google.inject.Injector;
 import com.google.inject.Module;
 
 import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.yarn.api.ApplicationBaseProtocol;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -75,8 +74,6 @@ public class TestRMWebAppFairScheduler {
                   mockRm(rmContext);
               binder.bind(ResourceManager.class).toInstance
                   (mockRmWithFairScheduler);
-              binder.bind(ApplicationBaseProtocol.class).toInstance(
-                mockRmWithFairScheduler.getClientRMService());
             } catch (IOException e) {
               throw new IllegalStateException(e);
             }
@@ -115,9 +112,6 @@ public class TestRMWebAppFairScheduler {
                   mockRmWithApps(rmContext);
               binder.bind(ResourceManager.class).toInstance
                   (mockRmWithFairScheduler);
-              binder.bind(ApplicationBaseProtocol.class).toInstance(
-                  mockRmWithFairScheduler.getClientRMService());
-
             } catch (IOException e) {
               throw new IllegalStateException(e);
             }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/722ee841/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRedirectionErrorPage.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/TestRedirectionErrorPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRedirectionErrorPage.java
index 408dc9b..eb0a62d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRedirectionErrorPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRedirectionErrorPage.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.yarn.server.resourcemanager.webapp;
 
 import java.io.IOException;
 
-import org.apache.hadoop.yarn.api.ApplicationBaseProtocol;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
@@ -52,8 +51,6 @@ public class TestRedirectionErrorPage {
             try {
               ResourceManager rm = TestRMWebApp.mockRm(rmContext);
               binder.bind(ResourceManager.class).toInstance(rm);
-              binder.bind(ApplicationBaseProtocol.class).toInstance(
-                  rm.getClientRMService());
             } catch (IOException e) {
               throw new IllegalStateException(e);
             }


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


[37/50] [abbrv] hadoop git commit: HADOOP-14850. Read HttpServer2 resources directly from the source tree (if exists). Contributed by Elek, Marton.

Posted by as...@apache.org.
HADOOP-14850. Read HttpServer2 resources directly from the source tree (if exists). Contributed by Elek, Marton.


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

Branch: refs/heads/YARN-5972
Commit: e8278b02a45d16569fdebfd1ac36b2e648ad1e1e
Parents: 1f53ae7
Author: Anu Engineer <ae...@apache.org>
Authored: Fri Sep 8 10:01:01 2017 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Fri Sep 8 10:01:01 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/http/HttpServer2.java     | 30 ++++++++++++++++----
 1 file changed, 24 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e8278b02/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java
index a450f66..7f1362c 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java
@@ -27,6 +27,7 @@ import java.io.InterruptedIOException;
 import java.io.PrintStream;
 import java.net.BindException;
 import java.net.InetSocketAddress;
+import java.net.MalformedURLException;
 import java.net.URI;
 import java.net.URL;
 import java.util.ArrayList;
@@ -993,14 +994,31 @@ public final class HttpServer2 implements FilterContainer {
    * Get the pathname to the webapps files.
    * @param appName eg "secondary" or "datanode"
    * @return the pathname as a URL
-   * @throws FileNotFoundException if 'webapps' directory cannot be found on CLASSPATH.
+   * @throws FileNotFoundException if 'webapps' directory cannot be found
+   *   on CLASSPATH or in the development location.
    */
   protected String getWebAppsPath(String appName) throws FileNotFoundException {
-    URL url = getClass().getClassLoader().getResource("webapps/" + appName);
-    if (url == null)
-      throw new FileNotFoundException("webapps/" + appName
-          + " not found in CLASSPATH");
-    String urlString = url.toString();
+    URL resourceUrl = null;
+    File webResourceDevLocation = new File("src/main/webapps", appName);
+    if (webResourceDevLocation.exists()) {
+      LOG.info("Web server is in development mode. Resources "
+          + "will be read from the source tree.");
+      try {
+        resourceUrl = webResourceDevLocation.getParentFile().toURI().toURL();
+      } catch (MalformedURLException e) {
+        throw new FileNotFoundException("Mailformed URL while finding the "
+            + "web resource dir:" + e.getMessage());
+      }
+    } else {
+      resourceUrl =
+          getClass().getClassLoader().getResource("webapps/" + appName);
+
+      if (resourceUrl == null) {
+        throw new FileNotFoundException("webapps/" + appName +
+            " not found in CLASSPATH");
+      }
+    }
+    String urlString = resourceUrl.toString();
     return urlString.substring(0, urlString.lastIndexOf('/'));
   }
 


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


[13/50] [abbrv] hadoop git commit: YARN-6992. Kill application button is visible even if the application is FINISHED in RM UI. Contributed by Suma Shivaprasad.

Posted by as...@apache.org.
YARN-6992. Kill application button is visible even if the application is FINISHED in RM UI. Contributed by Suma Shivaprasad.


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

Branch: refs/heads/YARN-5972
Commit: 6f101e7df10d477cf7469f6979d0cb8269315a3e
Parents: b6e7d13
Author: Sunil G <su...@apache.org>
Authored: Thu Sep 7 13:16:20 2017 +0530
Committer: Sunil G <su...@apache.org>
Committed: Thu Sep 7 13:16:20 2017 +0530

----------------------------------------------------------------------
 .../java/org/apache/hadoop/yarn/server/webapp/AppBlock.java | 9 ++++++++-
 1 file changed, 8 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f101e7d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
index 693aa04..95bc0aa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
@@ -144,7 +144,8 @@ public class AppBlock extends HtmlBlock {
         && webUiType.equals(YarnWebParams.RM_WEB_UI)
         && conf.getBoolean(YarnConfiguration.RM_WEBAPP_UI_ACTIONS_ENABLED,
           YarnConfiguration.DEFAULT_RM_WEBAPP_UI_ACTIONS_ENABLED)
-            && !unsecuredUIForSecuredCluster) {
+            && !unsecuredUIForSecuredCluster
+            && !isAppInFinalState(app)) {
       // Application Kill
       html.div()
         .button()
@@ -419,4 +420,10 @@ public class AppBlock extends HtmlBlock {
     }
     return ret;
   }
+
+  private boolean isAppInFinalState(AppInfo app) {
+    return app.getAppState() == YarnApplicationState.FINISHED
+        || app.getAppState() == YarnApplicationState.FAILED
+        || app.getAppState() == YarnApplicationState.KILLED;
+  }
 }


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