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 ar...@apache.org on 2016/01/29 20:06:52 UTC

[01/50] hadoop git commit: HDFS-9646. ErasureCodingWorker may fail when recovering data blocks with length less than the first internal block. Contributed by Jing Zhao.

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-1312 0fd3f16be -> a0efd5328


HDFS-9646. ErasureCodingWorker may fail when recovering data blocks with length less than the first internal block. Contributed by Jing Zhao.


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

Branch: refs/heads/HDFS-1312
Commit: 95363bcc7dae28ba9ae2cd7ee9a258fcb58cd932
Parents: 34a3900
Author: Jing Zhao <ji...@apache.org>
Authored: Fri Jan 22 09:46:02 2016 -0800
Committer: Jing Zhao <ji...@apache.org>
Committed: Fri Jan 22 09:46:02 2016 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DFSInputStream.java  |  48 ++++--
 .../hadoop/hdfs/DFSStripedInputStream.java      |   2 +-
 .../hadoop/hdfs/util/StripedBlockUtil.java      |  13 +-
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../erasurecode/ErasureCodingWorker.java        | 146 +++++++++++-----
 .../server/protocol/BlockECRecoveryCommand.java |   2 +-
 .../hdfs/TestReadStripedFileWithDecoding.java   |  17 +-
 .../hadoop/hdfs/TestRecoverStripedFile.java     | 172 ++++++++++++-------
 8 files changed, 270 insertions(+), 133 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/95363bcc/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
index 3de60b2..3c91ca1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
@@ -32,7 +32,6 @@ import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
-import java.util.Map.Entry;
 import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CancellationException;
@@ -909,7 +908,8 @@ public class DFSInputStream extends FSInputStream
     }
   }
 
-  protected synchronized int readWithStrategy(ReaderStrategy strategy, int off, int len) throws IOException {
+  protected synchronized int readWithStrategy(ReaderStrategy strategy, int off,
+      int len) throws IOException {
     dfsClient.checkOpen();
     if (closed.get()) {
       throw new IOException("Stream closed");
@@ -959,7 +959,7 @@ public class DFSInputStream extends FSInputStream
           // Check if need to report block replicas corruption either read
           // was successful or ChecksumException occured.
           reportCheckSumFailure(corruptedBlockMap,
-              currentLocatedBlock.getLocations().length);
+              currentLocatedBlock.getLocations().length, false);
         }
       }
     }
@@ -1492,7 +1492,8 @@ public class DFSInputStream extends FSInputStream
         // Check and report if any block replicas are corrupted.
         // BlockMissingException may be caught if all block replicas are
         // corrupted.
-        reportCheckSumFailure(corruptedBlockMap, blk.getLocations().length);
+        reportCheckSumFailure(corruptedBlockMap, blk.getLocations().length,
+            false);
       }
 
       remaining -= bytesToRead;
@@ -1508,6 +1509,7 @@ public class DFSInputStream extends FSInputStream
 
   /**
    * DFSInputStream reports checksum failure.
+   * For replicated blocks, we have the following logic:
    * Case I : client has tried multiple data nodes and at least one of the
    * attempts has succeeded. We report the other failures as corrupted block to
    * namenode.
@@ -1515,29 +1517,39 @@ public class DFSInputStream extends FSInputStream
    * only report if the total number of replica is 1. We do not
    * report otherwise since this maybe due to the client is a handicapped client
    * (who can not read).
+   *
+   * For erasure-coded blocks, each block in corruptedBlockMap is an internal
+   * block in a block group, and there is usually only one DataNode
+   * corresponding to each internal block. For this case we simply report the
+   * corrupted blocks to NameNode and ignore the above logic.
+   *
    * @param corruptedBlockMap map of corrupted blocks
    * @param dataNodeCount number of data nodes who contains the block replicas
    */
   protected void reportCheckSumFailure(
       Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap,
-      int dataNodeCount) {
+      int dataNodeCount, boolean isStriped) {
     if (corruptedBlockMap.isEmpty()) {
       return;
     }
-    Iterator<Entry<ExtendedBlock, Set<DatanodeInfo>>> it = corruptedBlockMap
-        .entrySet().iterator();
-    Entry<ExtendedBlock, Set<DatanodeInfo>> entry = it.next();
-    ExtendedBlock blk = entry.getKey();
-    Set<DatanodeInfo> dnSet = entry.getValue();
-    if (((dnSet.size() < dataNodeCount) && (dnSet.size() > 0))
-        || ((dataNodeCount == 1) && (dnSet.size() == dataNodeCount))) {
-      DatanodeInfo[] locs = new DatanodeInfo[dnSet.size()];
-      int i = 0;
-      for (DatanodeInfo dn:dnSet) {
-        locs[i++] = dn;
+    List<LocatedBlock> reportList = new ArrayList<>(corruptedBlockMap.size());
+    for (Map.Entry<ExtendedBlock, Set<DatanodeInfo>> entry :
+        corruptedBlockMap.entrySet()) {
+      ExtendedBlock blk = entry.getKey();
+      Set<DatanodeInfo> dnSet = entry.getValue();
+      if (isStriped || ((dnSet.size() < dataNodeCount) && (dnSet.size() > 0))
+          || ((dataNodeCount == 1) && (dnSet.size() == dataNodeCount))) {
+        DatanodeInfo[] locs = new DatanodeInfo[dnSet.size()];
+        int i = 0;
+        for (DatanodeInfo dn:dnSet) {
+          locs[i++] = dn;
+        }
+        reportList.add(new LocatedBlock(blk, locs));
       }
-      LocatedBlock [] lblocks = { new LocatedBlock(blk, locs) };
-      dfsClient.reportChecksumFailure(src, lblocks);
+    }
+    if (reportList.size() > 0) {
+      dfsClient.reportChecksumFailure(src,
+          reportList.toArray(new LocatedBlock[reportList.size()]));
     }
     corruptedBlockMap.clear();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/95363bcc/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
index 7433256..d15e536 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
@@ -451,7 +451,7 @@ public class DFSStripedInputStream extends DFSInputStream {
         // Check if need to report block replicas corruption either read
         // was successful or ChecksumException occured.
         reportCheckSumFailure(corruptedBlockMap,
-            currentLocatedBlock.getLocations().length);
+            currentLocatedBlock.getLocations().length, true);
       }
     }
     return -1;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/95363bcc/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
index e8653c8..dbd53a3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
@@ -22,7 +22,6 @@ import com.google.common.annotations.VisibleForTesting;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSStripedOutputStream;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -35,6 +34,8 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
 import org.apache.hadoop.security.token.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.nio.ByteBuffer;
 import java.util.*;
@@ -72,6 +73,8 @@ import java.util.concurrent.TimeUnit;
 @InterfaceAudience.Private
 public class StripedBlockUtil {
 
+  public static final Logger LOG = LoggerFactory.getLogger(StripedBlockUtil.class);
+
   /**
    * This method parses a striped block group into individual blocks.
    *
@@ -221,15 +224,11 @@ public class StripedBlockUtil {
         return new StripingChunkReadResult(StripingChunkReadResult.TIMEOUT);
       }
     } catch (ExecutionException e) {
-      if (DFSClient.LOG.isDebugEnabled()) {
-        DFSClient.LOG.debug("Exception during striped read task", e);
-      }
+      LOG.debug("Exception during striped read task", e);
       return new StripingChunkReadResult(futures.remove(future),
           StripingChunkReadResult.FAILED);
     } catch (CancellationException e) {
-      if (DFSClient.LOG.isDebugEnabled()) {
-        DFSClient.LOG.debug("Exception during striped read task", e);
-      }
+      LOG.debug("Exception during striped read task", e);
       return new StripingChunkReadResult(futures.remove(future),
           StripingChunkReadResult.CANCELLED);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/95363bcc/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 43adf62..940fa90 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -406,6 +406,9 @@ Trunk (Unreleased)
     HDFS-9615. Fix variable name typo in DFSConfigKeys. (Ray Chiang via
     Arpit Agarwal)
 
+    HDFS-9646. ErasureCodingWorker may fail when recovering data blocks with
+    length less than the first internal block. (jing9)
+
     BREAKDOWN OF HDFS-7285 SUBTASKS AND RELATED JIRAS
 
       HDFS-7347. Configurable erasure coding policy for individual files and

http://git-wip-us.apache.org/repos/asf/hadoop/blob/95363bcc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
index 5588eec..6ad7164 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
@@ -32,8 +32,10 @@ import java.util.BitSet;
 import java.util.Collection;
 import java.util.EnumSet;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CompletionService;
 import java.util.concurrent.ExecutorCompletionService;
@@ -46,6 +48,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.ChecksumException;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.BlockReader;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -121,9 +124,8 @@ public final class ErasureCodingWorker {
   }
 
   private void initializeStripedReadThreadPool(int num) {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Using striped reads; pool threads=" + num);
-    }
+    LOG.debug("Using striped reads; pool threads=" + num);
+
     STRIPED_READ_THREAD_POOL = new ThreadPoolExecutor(1, num, 60,
         TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
         new Daemon.DaemonFactory() {
@@ -148,9 +150,7 @@ public final class ErasureCodingWorker {
   }
 
   private void initializeStripedBlkRecoveryThreadPool(int num) {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Using striped block recovery; pool threads=" + num);
-    }
+    LOG.debug("Using striped block recovery; pool threads=" + num);
     STRIPED_BLK_RECOVERY_THREAD_POOL = new ThreadPoolExecutor(2, num, 60,
         TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>(),
         new Daemon.DaemonFactory() {
@@ -368,11 +368,11 @@ public final class ErasureCodingWorker {
      * @return StripedReader
      */
     private StripedReader addStripedReader(int i, long offsetInBlock) {
-      StripedReader reader = new StripedReader(liveIndices[i]);
+      final ExtendedBlock block = getBlock(blockGroup, liveIndices[i]);
+      StripedReader reader = new StripedReader(liveIndices[i], block, sources[i]);
       stripedReaders.add(reader);
 
-      BlockReader blockReader = newBlockReader(
-          getBlock(blockGroup, liveIndices[i]), offsetInBlock, sources[i]);
+      BlockReader blockReader = newBlockReader(block, offsetInBlock, sources[i]);
       if (blockReader != null) {
         initChecksumAndBufferSizeIfNeeded(blockReader);
         reader.blockReader = blockReader;
@@ -435,19 +435,27 @@ public final class ErasureCodingWorker {
           throw new IOException(error);
         }
 
-        long firstStripedBlockLength = getBlockLen(blockGroup, 0);
-        while (positionInBlock < firstStripedBlockLength) {
-          int toRead = Math.min(
-              bufferSize, (int)(firstStripedBlockLength - positionInBlock));
+        long maxTargetLength = 0;
+        for (short targetIndex : targetIndices) {
+          maxTargetLength = Math.max(maxTargetLength,
+              getBlockLen(blockGroup, targetIndex));
+        }
+        while (positionInBlock < maxTargetLength) {
+          final int toRecover = (int) Math.min(
+              bufferSize, maxTargetLength - positionInBlock);
           // step1: read from minimum source DNs required for reconstruction.
-          //   The returned success list is the source DNs we do real read from
-          success = readMinimumStripedData4Recovery(success);
+          // The returned success list is the source DNs we do real read from
+          Map<ExtendedBlock, Set<DatanodeInfo>> corruptionMap = new HashMap<>();
+          try {
+            success = readMinimumStripedData4Recovery(success, toRecover,
+                corruptionMap);
+          } finally {
+            // report corrupted blocks to NN
+            reportCorruptedBlocks(corruptionMap);
+          }
 
           // step2: decode to reconstruct targets
-          long remaining = firstStripedBlockLength - positionInBlock;
-          int toRecoverLen = remaining < bufferSize ? 
-              (int)remaining : bufferSize;
-          recoverTargets(success, targetsStatus, toRecoverLen);
+          recoverTargets(success, targetsStatus, toRecover);
 
           // step3: transfer data
           if (transferData2Targets(targetsStatus) == 0) {
@@ -456,7 +464,7 @@ public final class ErasureCodingWorker {
           }
 
           clearBuffers();
-          positionInBlock += toRead;
+          positionInBlock += toRecover;
         }
 
         endTargetBlocks(targetsStatus);
@@ -513,10 +521,11 @@ public final class ErasureCodingWorker {
       }
     }
 
-    private long getReadLength(int index) {
+    /** the reading length should not exceed the length for recovery */
+    private int getReadLength(int index, int recoverLength) {
       long blockLen = getBlockLen(blockGroup, index);
       long remaining = blockLen - positionInBlock;
-      return remaining > bufferSize ? bufferSize : remaining;
+      return (int) Math.min(remaining, recoverLength);
     }
 
     /**
@@ -529,11 +538,15 @@ public final class ErasureCodingWorker {
      * operations and next iteration read.
      * 
      * @param success the initial success list of source DNs we think best
+     * @param recoverLength the length to recover.
      * @return updated success list of source DNs we do real read
      * @throws IOException
      */
-    private int[] readMinimumStripedData4Recovery(final int[] success)
+    private int[] readMinimumStripedData4Recovery(final int[] success,
+        int recoverLength, Map<ExtendedBlock, Set<DatanodeInfo>> corruptionMap)
         throws IOException {
+      Preconditions.checkArgument(recoverLength >= 0 &&
+          recoverLength <= bufferSize);
       int nsuccess = 0;
       int[] newSuccess = new int[minRequiredSources];
       BitSet used = new BitSet(sources.length);
@@ -543,9 +556,11 @@ public final class ErasureCodingWorker {
        */
       for (int i = 0; i < minRequiredSources; i++) {
         StripedReader reader = stripedReaders.get(success[i]);
-        if (getReadLength(liveIndices[success[i]]) > 0) {
-          Callable<Void> readCallable = readFromBlock(
-              reader.blockReader, reader.buffer);
+        final int toRead = getReadLength(liveIndices[success[i]],
+            recoverLength);
+        if (toRead > 0) {
+          Callable<Void> readCallable = readFromBlock(reader, reader.buffer,
+              toRead, corruptionMap);
           Future<Void> f = readService.submit(readCallable);
           futures.put(f, success[i]);
         } else {
@@ -570,10 +585,10 @@ public final class ErasureCodingWorker {
             StripedReader failedReader = stripedReaders.get(result.index);
             closeBlockReader(failedReader.blockReader);
             failedReader.blockReader = null;
-            resultIndex = scheduleNewRead(used);
+            resultIndex = scheduleNewRead(used, recoverLength, corruptionMap);
           } else if (result.state == StripingChunkReadResult.TIMEOUT) {
             // If timeout, we also schedule a new read.
-            resultIndex = scheduleNewRead(used);
+            resultIndex = scheduleNewRead(used, recoverLength, corruptionMap);
           }
           if (resultIndex >= 0) {
             newSuccess[nsuccess++] = resultIndex;
@@ -601,6 +616,9 @@ public final class ErasureCodingWorker {
     }
     
     private void paddingBufferToLen(ByteBuffer buffer, int len) {
+      if (len > buffer.limit()) {
+        buffer.limit(len);
+      }
       int toPadding = len - buffer.position();
       for (int i = 0; i < toPadding; i++) {
         buffer.put((byte) 0);
@@ -648,8 +666,8 @@ public final class ErasureCodingWorker {
       int m = 0;
       for (int i = 0; i < targetBuffers.length; i++) {
         if (targetsStatus[i]) {
+          targetBuffers[i].limit(toRecoverLen);
           outputs[m++] = targetBuffers[i];
-          outputs[i].limit(toRecoverLen);
         }
       }
       decoder.decode(inputs, erasedIndices, outputs);
@@ -658,7 +676,7 @@ public final class ErasureCodingWorker {
         if (targetsStatus[i]) {
           long blockLen = getBlockLen(blockGroup, targetIndices[i]);
           long remaining = blockLen - positionInBlock;
-          if (remaining < 0) {
+          if (remaining <= 0) {
             targetBuffers[i].limit(0);
           } else if (remaining < toRecoverLen) {
             targetBuffers[i].limit((int)remaining);
@@ -678,16 +696,19 @@ public final class ErasureCodingWorker {
      * @param used the used source DNs in this iteration.
      * @return the array index of source DN if don't need to do real read.
      */
-    private int scheduleNewRead(BitSet used) {
+    private int scheduleNewRead(BitSet used, int recoverLength,
+        Map<ExtendedBlock, Set<DatanodeInfo>> corruptionMap) {
       StripedReader reader = null;
       // step1: initially we may only have <code>minRequiredSources</code>
       // number of StripedReader, and there may be some source DNs we never 
       // read before, so will try to create StripedReader for one new source DN
       // and try to read from it. If found, go to step 3.
       int m = stripedReaders.size();
+      int toRead = 0;
       while (reader == null && m < sources.length) {
         reader = addStripedReader(m, positionInBlock);
-        if (getReadLength(liveIndices[m]) > 0) {
+        toRead = getReadLength(liveIndices[m], recoverLength);
+        if (toRead > 0) {
           if (reader.blockReader == null) {
             reader = null;
             m++;
@@ -706,12 +727,14 @@ public final class ErasureCodingWorker {
       for (int i = 0; reader == null && i < stripedReaders.size(); i++) {
         if (!used.get(i)) {
           StripedReader r = stripedReaders.get(i);
-          if (getReadLength(liveIndices[i]) > 0) {
+          toRead = getReadLength(liveIndices[i], recoverLength);
+          if (toRead > 0) {
             closeBlockReader(r.blockReader);
             r.blockReader = newBlockReader(
                 getBlock(blockGroup, liveIndices[i]), positionInBlock,
                 sources[i]);
             if (r.blockReader != null) {
+              r.buffer.position(0);
               m = i;
               reader = r;
             }
@@ -725,8 +748,8 @@ public final class ErasureCodingWorker {
 
       // step3: schedule if find a correct source DN and need to do real read.
       if (reader != null) {
-        Callable<Void> readCallable = readFromBlock(
-            reader.blockReader, reader.buffer);
+        Callable<Void> readCallable = readFromBlock(reader, reader.buffer,
+            toRead, corruptionMap);
         Future<Void> f = readService.submit(readCallable);
         futures.put(f, m);
         used.set(m);
@@ -742,15 +765,22 @@ public final class ErasureCodingWorker {
       }
     }
 
-    private Callable<Void> readFromBlock(final BlockReader reader,
-        final ByteBuffer buf) {
+    private Callable<Void> readFromBlock(final StripedReader reader,
+        final ByteBuffer buf, final int length,
+        final Map<ExtendedBlock, Set<DatanodeInfo>> corruptionMap) {
       return new Callable<Void>() {
 
         @Override
         public Void call() throws Exception {
           try {
-            actualReadFromBlock(reader, buf);
+            buf.limit(length);
+            actualReadFromBlock(reader.blockReader, buf);
             return null;
+          } catch (ChecksumException e) {
+            LOG.warn("Found Checksum error for " + reader.block + " from "
+                + reader.source + " at " + e.getPos());
+            addCorruptedBlock(reader.block, reader.source, corruptionMap);
+            throw e;
           } catch (IOException e) {
             LOG.info(e.getMessage());
             throw e;
@@ -760,6 +790,30 @@ public final class ErasureCodingWorker {
       };
     }
 
+    private void reportCorruptedBlocks(
+        Map<ExtendedBlock, Set<DatanodeInfo>> corruptionMap) throws IOException {
+      if (!corruptionMap.isEmpty()) {
+        for (Map.Entry<ExtendedBlock, Set<DatanodeInfo>> entry :
+            corruptionMap.entrySet()) {
+          for (DatanodeInfo dnInfo : entry.getValue()) {
+            datanode.reportRemoteBadBlock(dnInfo, entry.getKey());
+          }
+        }
+      }
+    }
+
+    private void addCorruptedBlock(ExtendedBlock blk, DatanodeInfo node,
+        Map<ExtendedBlock, Set<DatanodeInfo>> corruptionMap) {
+      Set<DatanodeInfo> dnSet = corruptionMap.get(blk);
+      if (dnSet == null) {
+        dnSet = new HashSet<>();
+        corruptionMap.put(blk, dnSet);
+      }
+      if (!dnSet.contains(node)) {
+        dnSet.add(node);
+      }
+    }
+
     /**
      * Read bytes from block
      */
@@ -900,14 +954,14 @@ public final class ErasureCodingWorker {
       }
 
       if (zeroStripeBuffers != null) {
-        for (int i = 0; i < zeroStripeBuffers.length; i++) {
-          zeroStripeBuffers[i].clear();
+        for (ByteBuffer zeroStripeBuffer : zeroStripeBuffers) {
+          zeroStripeBuffer.clear();
         }
       }
 
-      for (int i = 0; i < targetBuffers.length; i++) {
-        if (targetBuffers[i] != null) {
-          targetBuffers[i].clear();
+      for (ByteBuffer targetBuffer : targetBuffers) {
+        if (targetBuffer != null) {
+          targetBuffer.clear();
         }
       }
     }
@@ -998,9 +1052,13 @@ public final class ErasureCodingWorker {
     private final short index; // internal block index
     private BlockReader blockReader;
     private ByteBuffer buffer;
+    private final ExtendedBlock block;
+    private final DatanodeInfo source;
 
-    private StripedReader(short index) {
+    StripedReader(short index, ExtendedBlock block, DatanodeInfo source) {
       this.index = index;
+      this.block = block;
+      this.source = source;
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/95363bcc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java
index 1cb74b3..d0c1786 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java
@@ -136,7 +136,7 @@ public class BlockECRecoveryCommand extends DatanodeCommand {
           .append("Recovering ").append(block).append(" From: ")
           .append(Arrays.asList(sources)).append(" To: [")
           .append(Arrays.asList(targets)).append(")\n")
-          .append(" Block Indices: ").append(Arrays.asList(liveBlockIndices))
+          .append(" Block Indices: ").append(Arrays.toString(liveBlockIndices))
           .toString();
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/95363bcc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java
index 32b0216..b0af50e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.Path;
@@ -29,12 +30,16 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.log4j.Level;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -51,6 +56,14 @@ import static org.apache.hadoop.hdfs.StripedFileTestUtil.numDNs;
 public class TestReadStripedFileWithDecoding {
   static final Log LOG = LogFactory.getLog(TestReadStripedFileWithDecoding.class);
 
+  static {
+    ((Log4JLogger)LogFactory.getLog(BlockPlacementPolicy.class))
+        .getLogger().setLevel(Level.ALL);
+    GenericTestUtils.setLogLevel(BlockManager.LOG, Level.ALL);
+    GenericTestUtils.setLogLevel(BlockManager.blockLog, Level.ALL);
+    GenericTestUtils.setLogLevel(NameNode.stateChangeLog, Level.ALL);
+  }
+
   private MiniDFSCluster cluster;
   private DistributedFileSystem fs;
   private final short dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS;
@@ -66,9 +79,9 @@ public class TestReadStripedFileWithDecoding {
     Configuration conf = new HdfsConfiguration();
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_STREAMS_HARD_LIMIT_KEY, 0);
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, false);
-    cluster = new MiniDFSCluster.Builder(new HdfsConfiguration())
-        .numDataNodes(numDNs).build();
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
     cluster.getFileSystem().getClient().setErasureCodingPolicy("/", null);
     fs = cluster.getFileSystem();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/95363bcc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java
index 21352b5..ca9d933 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java
@@ -23,11 +23,12 @@ import static org.junit.Assert.assertTrue;
 import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.BitSet;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.ThreadLocalRandom;
+import java.util.Random;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -38,6 +39,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
@@ -64,17 +66,25 @@ public class TestRecoverStripedFile {
 
   static {
     GenericTestUtils.setLogLevel(DFSClient.LOG, Level.ALL);
+    GenericTestUtils.setLogLevel(BlockManager.LOG, Level.ALL);
+    GenericTestUtils.setLogLevel(BlockManager.blockLog, Level.ALL);
+  }
+
+  enum RecoveryType {
+    DataOnly,
+    ParityOnly,
+    Any
   }
 
   private MiniDFSCluster cluster;
-  private Configuration conf;
   private DistributedFileSystem fs;
   // Map: DatanodeID -> datanode index in cluster
-  private Map<DatanodeID, Integer> dnMap = new HashMap<DatanodeID, Integer>();
+  private Map<DatanodeID, Integer> dnMap = new HashMap<>();
+  private final Random random = new Random();
 
   @Before
   public void setup() throws IOException {
-    conf = new Configuration();
+    final Configuration conf = new Configuration();
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
     conf.setInt(DFSConfigKeys.DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_KEY,
         cellSize - 1);
@@ -104,75 +114,140 @@ public class TestRecoverStripedFile {
   @Test(timeout = 120000)
   public void testRecoverOneParityBlock() throws Exception {
     int fileLen = 10 * blockSize + blockSize/10;
-    assertFileBlocksRecovery("/testRecoverOneParityBlock", fileLen, 0, 1);
+    assertFileBlocksRecovery("/testRecoverOneParityBlock", fileLen,
+        RecoveryType.ParityOnly, 1);
   }
   
   @Test(timeout = 120000)
   public void testRecoverOneParityBlock1() throws Exception {
     int fileLen = cellSize + cellSize/10;
-    assertFileBlocksRecovery("/testRecoverOneParityBlock1", fileLen, 0, 1);
+    assertFileBlocksRecovery("/testRecoverOneParityBlock1", fileLen,
+        RecoveryType.ParityOnly, 1);
   }
   
   @Test(timeout = 120000)
   public void testRecoverOneParityBlock2() throws Exception {
     int fileLen = 1;
-    assertFileBlocksRecovery("/testRecoverOneParityBlock2", fileLen, 0, 1);
+    assertFileBlocksRecovery("/testRecoverOneParityBlock2", fileLen,
+        RecoveryType.ParityOnly, 1);
   }
   
   @Test(timeout = 120000)
   public void testRecoverOneParityBlock3() throws Exception {
     int fileLen = 3 * blockSize + blockSize/10;
-    assertFileBlocksRecovery("/testRecoverOneParityBlock3", fileLen, 0, 1);
+    assertFileBlocksRecovery("/testRecoverOneParityBlock3", fileLen,
+        RecoveryType.ParityOnly, 1);
   }
   
   @Test(timeout = 120000)
   public void testRecoverThreeParityBlocks() throws Exception {
     int fileLen = 10 * blockSize + blockSize/10;
-    assertFileBlocksRecovery("/testRecoverThreeParityBlocks", fileLen, 0, 3);
+    assertFileBlocksRecovery("/testRecoverThreeParityBlocks", fileLen,
+        RecoveryType.ParityOnly, 3);
   }
   
   @Test(timeout = 120000)
   public void testRecoverThreeDataBlocks() throws Exception {
     int fileLen = 10 * blockSize + blockSize/10;
-    assertFileBlocksRecovery("/testRecoverThreeDataBlocks", fileLen, 1, 3);
+    assertFileBlocksRecovery("/testRecoverThreeDataBlocks", fileLen,
+        RecoveryType.DataOnly, 3);
   }
   
   @Test(timeout = 120000)
   public void testRecoverThreeDataBlocks1() throws Exception {
     int fileLen = 3 * blockSize + blockSize/10;
-    assertFileBlocksRecovery("/testRecoverThreeDataBlocks1", fileLen, 1, 3);
+    assertFileBlocksRecovery("/testRecoverThreeDataBlocks1", fileLen,
+        RecoveryType.DataOnly, 3);
   }
   
   @Test(timeout = 120000)
   public void testRecoverOneDataBlock() throws Exception {
     int fileLen = 10 * blockSize + blockSize/10;
-    assertFileBlocksRecovery("/testRecoverOneDataBlock", fileLen, 1, 1);
+    assertFileBlocksRecovery("/testRecoverOneDataBlock", fileLen,
+        RecoveryType.DataOnly, 1);
   }
   
   @Test(timeout = 120000)
   public void testRecoverOneDataBlock1() throws Exception {
     int fileLen = cellSize + cellSize/10;
-    assertFileBlocksRecovery("/testRecoverOneDataBlock1", fileLen, 1, 1);
+    assertFileBlocksRecovery("/testRecoverOneDataBlock1", fileLen,
+        RecoveryType.DataOnly, 1);
   }
   
   @Test(timeout = 120000)
   public void testRecoverOneDataBlock2() throws Exception {
     int fileLen = 1;
-    assertFileBlocksRecovery("/testRecoverOneDataBlock2", fileLen, 1, 1);
+    assertFileBlocksRecovery("/testRecoverOneDataBlock2", fileLen,
+        RecoveryType.DataOnly, 1);
   }
   
   @Test(timeout = 120000)
   public void testRecoverAnyBlocks() throws Exception {
     int fileLen = 3 * blockSize + blockSize/10;
-    assertFileBlocksRecovery("/testRecoverAnyBlocks", fileLen, 2, 2);
+    assertFileBlocksRecovery("/testRecoverAnyBlocks", fileLen,
+        RecoveryType.Any, 2);
   }
   
   @Test(timeout = 120000)
   public void testRecoverAnyBlocks1() throws Exception {
     int fileLen = 10 * blockSize + blockSize/10;
-    assertFileBlocksRecovery("/testRecoverAnyBlocks1", fileLen, 2, 3);
+    assertFileBlocksRecovery("/testRecoverAnyBlocks1", fileLen,
+        RecoveryType.Any, 3);
   }
-  
+
+  private int[] generateDeadDnIndices(RecoveryType type, int deadNum,
+      byte[] indices) {
+    List<Integer> deadList = new ArrayList<>(deadNum);
+    while (deadList.size() < deadNum) {
+      int dead = random.nextInt(indices.length);
+      boolean isOfType = true;
+      if (type == RecoveryType.DataOnly) {
+        isOfType = indices[dead] < dataBlkNum;
+      } else if (type == RecoveryType.ParityOnly) {
+        isOfType = indices[dead] >= dataBlkNum;
+      }
+      if (isOfType && !deadList.contains(dead)) {
+        deadList.add(dead);
+      }
+    }
+    int[] d = new int[deadNum];
+    for (int i = 0; i < deadNum; i++) {
+      d[i] = deadList.get(i);
+    }
+    return d;
+  }
+
+  private void shutdownDataNodes(DataNode dn) throws IOException {
+    /*
+     * Kill the datanode which contains one replica
+     * We need to make sure it dead in namenode: clear its update time and
+     * trigger NN to check heartbeat.
+     */
+    dn.shutdown();
+    cluster.setDataNodeDead(dn.getDatanodeId());
+  }
+
+  private int generateErrors(Map<ExtendedBlock, DataNode> corruptTargets,
+      RecoveryType type)
+    throws IOException {
+    int stoppedDN = 0;
+    for (Map.Entry<ExtendedBlock, DataNode> target : corruptTargets.entrySet()) {
+      if (stoppedDN == 0 || type != RecoveryType.DataOnly
+          || random.nextBoolean()) {
+        // stop at least one DN to trigger recovery
+        LOG.info("Note: stop DataNode " + target.getValue().getDisplayName()
+            + " with internal block " + target.getKey());
+        shutdownDataNodes(target.getValue());
+        stoppedDN++;
+      } else { // corrupt the data on the DN
+        LOG.info("Note: corrupt data on " + target.getValue().getDisplayName()
+            + " with internal block " + target.getKey());
+        cluster.corruptReplica(target.getValue(), target.getKey());
+      }
+    }
+    return stoppedDN;
+  }
+
   /**
    * Test the file blocks recovery.
    * 1. Check the replica is recovered in the target datanode, 
@@ -180,11 +255,7 @@ public class TestRecoverStripedFile {
    * 2. Read the file and verify content. 
    */
   private void assertFileBlocksRecovery(String fileName, int fileLen,
-      int recovery, int toRecoverBlockNum) throws Exception {
-    if (recovery != 0 && recovery != 1 && recovery != 2) {
-      Assert.fail("Invalid recovery: 0 is to recovery parity blocks,"
-          + "1 is to recovery data blocks, 2 is any.");
-    }
+      RecoveryType type, int toRecoverBlockNum) throws Exception {
     if (toRecoverBlockNum < 1 || toRecoverBlockNum > parityBlkNum) {
       Assert.fail("toRecoverBlockNum should be between 1 ~ " + parityBlkNum);
     }
@@ -192,7 +263,7 @@ public class TestRecoverStripedFile {
     Path file = new Path(fileName);
 
     final byte[] data = new byte[fileLen];
-    ThreadLocalRandom.current().nextBytes(data);
+    Arrays.fill(data, (byte) 1);
     DFSTestUtil.writeFile(fs, file, data);
     StripedFileTestUtil.waitBlockGroupsReported(fs, fileName);
 
@@ -209,26 +280,10 @@ public class TestRecoverStripedFile {
     for (DatanodeInfo storageInfo : storageInfos) {
       bitset.set(dnMap.get(storageInfo));
     }
-    
-    int[] toDead = new int[toRecoverBlockNum];
-    int n = 0;
-    for (int i = 0; i < indices.length; i++) {
-      if (n < toRecoverBlockNum) {
-        if (recovery == 0) {
-          if (indices[i] >= dataBlkNum) {
-            toDead[n++] = i;
-          }
-        } else if (recovery == 1) {
-          if (indices[i] < dataBlkNum) {
-            toDead[n++] = i;
-          }
-        } else {
-          toDead[n++] = i;
-        }
-      } else {
-        break;
-      }
-    }
+
+    int[] dead = generateDeadDnIndices(type, toRecoverBlockNum, indices);
+    LOG.info("Note: indices == " + Arrays.toString(indices)
+        + ". Generate errors on datanodes: " + Arrays.toString(dead));
     
     DatanodeInfo[] dataDNs = new DatanodeInfo[toRecoverBlockNum];
     int[] deadDnIndices = new int[toRecoverBlockNum];
@@ -236,46 +291,41 @@ public class TestRecoverStripedFile {
     File[] replicas = new File[toRecoverBlockNum];
     File[] metadatas = new File[toRecoverBlockNum];
     byte[][] replicaContents = new byte[toRecoverBlockNum][];
+    Map<ExtendedBlock, DataNode> errorMap = new HashMap<>(dead.length);
     for (int i = 0; i < toRecoverBlockNum; i++) {
-      dataDNs[i] = storageInfos[toDead[i]];
+      dataDNs[i] = storageInfos[dead[i]];
       deadDnIndices[i] = dnMap.get(dataDNs[i]);
-      
+
       // Check the block replica file on deadDn before it dead.
       blocks[i] = StripedBlockUtil.constructInternalBlock(
-          lastBlock.getBlock(), cellSize, dataBlkNum, indices[toDead[i]]);
+          lastBlock.getBlock(), cellSize, dataBlkNum, indices[dead[i]]);
+      errorMap.put(blocks[i], cluster.getDataNodes().get(deadDnIndices[i]));
       replicas[i] = cluster.getBlockFile(deadDnIndices[i], blocks[i]);
       metadatas[i] = cluster.getBlockMetadataFile(deadDnIndices[i], blocks[i]);
       // the block replica on the datanode should be the same as expected
       assertEquals(replicas[i].length(), 
           StripedBlockUtil.getInternalBlockLength(
-          lastBlock.getBlockSize(), cellSize, dataBlkNum, indices[toDead[i]]));
+          lastBlock.getBlockSize(), cellSize, dataBlkNum, indices[dead[i]]));
       assertTrue(metadatas[i].getName().
           endsWith(blocks[i].getGenerationStamp() + ".meta"));
+      LOG.info("replica " + i + " locates in file: " + replicas[i]);
       replicaContents[i] = DFSTestUtil.readFileAsBytes(replicas[i]);
     }
     
     int cellsNum = (fileLen - 1) / cellSize + 1;
     int groupSize = Math.min(cellsNum, dataBlkNum) + parityBlkNum;
 
-    for (int i = 0; i < toRecoverBlockNum; i++) {
-      /*
-       * Kill the datanode which contains one replica
-       * We need to make sure it dead in namenode: clear its update time and
-       * trigger NN to check heartbeat.
-       */
-      DataNode dn = cluster.getDataNodes().get(deadDnIndices[i]);
-      dn.shutdown();
-      cluster.setDataNodeDead(dn.getDatanodeId());
-    }
+    // shutdown datanodes or generate corruption
+    int stoppedDN = generateErrors(errorMap, type);
 
     // Check the locatedBlocks of the file again
     locatedBlocks = getLocatedBlocks(file);
     lastBlock = (LocatedStripedBlock)locatedBlocks.getLastLocatedBlock();
     storageInfos = lastBlock.getLocations();
-    assertEquals(storageInfos.length, groupSize - toRecoverBlockNum);
+    assertEquals(storageInfos.length, groupSize - stoppedDN);
 
     int[] targetDNs = new int[dnNum - groupSize];
-    n = 0;
+    int n = 0;
     for (int i = 0; i < dnNum; i++) {
       if (!bitset.get(i)) { // not contain replica of the block.
         targetDNs[n++] = i;
@@ -289,9 +339,11 @@ public class TestRecoverStripedFile {
     // Check the replica on the new target node.
     for (int i = 0; i < toRecoverBlockNum; i++) {
       File replicaAfterRecovery = cluster.getBlockFile(targetDNs[i], blocks[i]);
+      LOG.info("replica after recovery " + replicaAfterRecovery);
       File metadataAfterRecovery =
           cluster.getBlockMetadataFile(targetDNs[i], blocks[i]);
       assertEquals(replicaAfterRecovery.length(), replicas[i].length());
+      LOG.info("replica before " + replicas[i]);
       assertTrue(metadataAfterRecovery.getName().
           endsWith(blocks[i].getGenerationStamp() + ".meta"));
       byte[] replicaContentAfterRecovery =
@@ -366,7 +418,7 @@ public class TestRecoverStripedFile {
     BlockECRecoveryInfo invalidECInfo = new BlockECRecoveryInfo(
         new ExtendedBlock("bp-id", 123456), dataDNs, dnStorageInfo, liveIndices,
         ErasureCodingPolicyManager.getSystemDefaultPolicy());
-    List<BlockECRecoveryInfo> ecTasks = new ArrayList<BlockECRecoveryInfo>();
+    List<BlockECRecoveryInfo> ecTasks = new ArrayList<>();
     ecTasks.add(invalidECInfo);
     dataNode.getErasureCodingWorker().processErasureCodingTasks(ecTasks);
   }


[22/50] hadoop git commit: HDFS-9541. Add hdfsStreamBuilder API to libhdfs to support defaultBlockSizes greater than 2 GB. Contributed by Colin Patrick McCabe.

Posted by ar...@apache.org.
HDFS-9541. Add hdfsStreamBuilder API to libhdfs to support defaultBlockSizes greater than 2 GB. Contributed by Colin Patrick McCabe.

Change-Id: Ifce1b9be534dc8f7e9d2634cd60e423921b9810f


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

Branch: refs/heads/HDFS-1312
Commit: cf8af7bb459b21babaad2d972330a3b4c6bb222d
Parents: d0d7c22
Author: Zhe Zhang <zh...@apache.org>
Authored: Tue Jan 26 11:24:57 2016 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Tue Jan 26 11:24:57 2016 -0800

----------------------------------------------------------------------
 .../src/main/native/libhdfs/hdfs.c              | 98 +++++++++++++++++++-
 .../src/main/native/libhdfs/include/hdfs/hdfs.h | 89 +++++++++++++++++-
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +
 3 files changed, 187 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf8af7bb/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/hdfs.c
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/hdfs.c b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/hdfs.c
index c5aad1d..4618dbb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/hdfs.c
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/hdfs.c
@@ -836,9 +836,95 @@ static jthrowable getDefaultBlockSize(JNIEnv *env, jobject jFS,
     return NULL;
 }
 
-hdfsFile hdfsOpenFile(hdfsFS fs, const char *path, int flags, 
+hdfsFile hdfsOpenFile(hdfsFS fs, const char *path, int flags,
                       int bufferSize, short replication, tSize blockSize)
 {
+    struct hdfsStreamBuilder *bld = hdfsStreamBuilderAlloc(fs, path, flags);
+    if (bufferSize != 0) {
+      hdfsStreamBuilderSetBufferSize(bld, bufferSize);
+    }
+    if (replication != 0) {
+      hdfsStreamBuilderSetReplication(bld, replication);
+    }
+    if (blockSize != 0) {
+      hdfsStreamBuilderSetDefaultBlockSize(bld, blockSize);
+    }
+    return hdfsStreamBuilderBuild(bld);
+}
+
+struct hdfsStreamBuilder {
+    hdfsFS fs;
+    int flags;
+    int32_t bufferSize;
+    int16_t replication;
+    int64_t defaultBlockSize;
+    char path[1];
+};
+
+struct hdfsStreamBuilder *hdfsStreamBuilderAlloc(hdfsFS fs,
+                                            const char *path, int flags)
+{
+    int path_len = strlen(path);
+    struct hdfsStreamBuilder *bld;
+
+    // sizeof(hdfsStreamBuilder->path) includes one byte for the string
+    // terminator
+    bld = malloc(sizeof(struct hdfsStreamBuilder) + path_len);
+    if (!bld) {
+        errno = ENOMEM;
+        return NULL;
+    }
+    bld->fs = fs;
+    bld->flags = flags;
+    bld->bufferSize = 0;
+    bld->replication = 0;
+    bld->defaultBlockSize = 0;
+    memcpy(bld->path, path, path_len);
+    bld->path[path_len] = '\0';
+    return bld;
+}
+
+void hdfsStreamBuilderFree(struct hdfsStreamBuilder *bld)
+{
+    free(bld);
+}
+
+int hdfsStreamBuilderSetBufferSize(struct hdfsStreamBuilder *bld,
+                                   int32_t bufferSize)
+{
+    if ((bld->flags & O_ACCMODE) != O_WRONLY) {
+        errno = EINVAL;
+        return -1;
+    }
+    bld->bufferSize = bufferSize;
+    return 0;
+}
+
+int hdfsStreamBuilderSetReplication(struct hdfsStreamBuilder *bld,
+                                    int16_t replication)
+{
+    if ((bld->flags & O_ACCMODE) != O_WRONLY) {
+        errno = EINVAL;
+        return -1;
+    }
+    bld->replication = replication;
+    return 0;
+}
+
+int hdfsStreamBuilderSetDefaultBlockSize(struct hdfsStreamBuilder *bld,
+                                         int64_t defaultBlockSize)
+{
+    if ((bld->flags & O_ACCMODE) != O_WRONLY) {
+        errno = EINVAL;
+        return -1;
+    }
+    bld->defaultBlockSize = defaultBlockSize;
+    return 0;
+}
+
+static hdfsFile hdfsOpenFileImpl(hdfsFS fs, const char *path, int flags,
+                  int32_t bufferSize, int16_t replication, int64_t blockSize)
+{
     /*
       JAVA EQUIVALENT:
        File f = new File(path);
@@ -1037,6 +1123,16 @@ done:
     return file;
 }
 
+hdfsFile hdfsStreamBuilderBuild(struct hdfsStreamBuilder *bld)
+{
+    hdfsFile file = hdfsOpenFileImpl(bld->fs, bld->path, bld->flags,
+                  bld->bufferSize, bld->replication, bld->defaultBlockSize);
+    int prevErrno = errno;
+    hdfsStreamBuilderFree(bld);
+    errno = prevErrno;
+    return file;
+}
+
 int hdfsTruncateFile(hdfsFS fs, const char* path, tOffset newlength)
 {
     jobject jFS = (jobject)fs;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf8af7bb/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/include/hdfs/hdfs.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/include/hdfs/hdfs.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/include/hdfs/hdfs.h
index c1515d7..c856928 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/include/hdfs/hdfs.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/include/hdfs/hdfs.h
@@ -80,6 +80,7 @@ extern  "C" {
         kObjectKindFile = 'F',
         kObjectKindDirectory = 'D',
     } tObjectKind;
+    struct hdfsStreamBuilder;
 
 
     /**
@@ -376,9 +377,11 @@ extern  "C" {
     LIBHDFS_EXTERNAL
     int hdfsDisconnect(hdfsFS fs);
         
-
     /** 
      * hdfsOpenFile - Open a hdfs file in given mode.
+     * @deprecated    Use the hdfsStreamBuilder functions instead.
+     * This function does not support setting block sizes bigger than 2 GB.
+     *
      * @param fs The configured filesystem handle.
      * @param path The full path to the file.
      * @param flags - an | of bits/fcntl.h file flags - supported flags are O_RDONLY, O_WRONLY (meaning create or overwrite i.e., implies O_TRUNCAT), 
@@ -388,7 +391,9 @@ extern  "C" {
      * @param replication Block replication - pass 0 if you want to use
      * the default configured values.
      * @param blocksize Size of block - pass 0 if you want to use the
-     * default configured values.
+     * default configured values.  Note that if you want a block size bigger
+     * than 2 GB, you must use the hdfsStreamBuilder API rather than this
+     * deprecated function.
      * @return Returns the handle to the open file or NULL on error.
      */
     LIBHDFS_EXTERNAL
@@ -396,6 +401,86 @@ extern  "C" {
                           int bufferSize, short replication, tSize blocksize);
 
     /**
+     * hdfsStreamBuilderAlloc - Allocate an HDFS stream builder.
+     *
+     * @param fs The configured filesystem handle.
+     * @param path The full path to the file.  Will be deep-copied.
+     * @param flags The open flags, as in hdfsOpenFile.
+     * @return Returns the hdfsStreamBuilder, or NULL on error.
+     */
+    LIBHDFS_EXTERNAL
+    struct hdfsStreamBuilder *hdfsStreamBuilderAlloc(hdfsFS fs,
+                                      const char *path, int flags);
+
+    /**
+     * hdfsStreamBuilderFree - Free an HDFS file builder.
+     *
+     * It is normally not necessary to call this function since
+     * hdfsStreamBuilderBuild frees the builder.
+     *
+     * @param bld The hdfsStreamBuilder to free.
+     */
+    LIBHDFS_EXTERNAL
+    void hdfsStreamBuilderFree(struct hdfsStreamBuilder *bld);
+
+    /**
+     * hdfsStreamBuilderSetBufferSize - Set the stream buffer size.
+     *
+     * @param bld The hdfs stream builder.
+     * @param bufferSize The buffer size to set.
+     *
+     * @return 0 on success, or -1 on error.  Errno will be set on error.
+     */
+    LIBHDFS_EXTERNAL
+    int hdfsStreamBuilderSetBufferSize(struct hdfsStreamBuilder *bld,
+                                       int32_t bufferSize);
+
+    /**
+     * hdfsStreamBuilderSetReplication - Set the replication for the stream.
+     * This is only relevant for output streams, which will create new blocks.
+     *
+     * @param bld The hdfs stream builder.
+     * @param replication The replication to set.
+     *
+     * @return 0 on success, or -1 on error.  Errno will be set on error.
+     *              If you call this on an input stream builder, you will get
+     *              EINVAL, because this configuration is not relevant to input
+     *              streams.
+     */
+    LIBHDFS_EXTERNAL
+    int hdfsStreamBuilderSetReplication(struct hdfsStreamBuilder *bld,
+                                        int16_t replication);
+
+    /**
+     * hdfsStreamBuilderSetDefaultBlockSize - Set the default block size for
+     * the stream.  This is only relevant for output streams, which will create
+     * new blocks.
+     *
+     * @param bld The hdfs stream builder.
+     * @param defaultBlockSize The default block size to set.
+     *
+     * @return 0 on success, or -1 on error.  Errno will be set on error.
+     *              If you call this on an input stream builder, you will get
+     *              EINVAL, because this configuration is not relevant to input
+     *              streams.
+     */
+    LIBHDFS_EXTERNAL
+    int hdfsStreamBuilderSetDefaultBlockSize(struct hdfsStreamBuilder *bld,
+                                       int64_t defaultBlockSize);
+
+    /**
+     * hdfsStreamBuilderBuild - Build the stream by calling open or create.
+     *
+     * @param bld The hdfs stream builder.  This pointer will be freed, whether
+     *            or not the open succeeds.
+     *
+     * @return the stream pointer on success, or NULL on error.  Errno will be
+     * set on error.
+     */
+    LIBHDFS_EXTERNAL
+    hdfsFile hdfsStreamBuilderBuild(struct hdfsStreamBuilder *bld);
+
+    /**
      * hdfsTruncateFile - Truncate a hdfs file to given lenght.
      * @param fs The configured filesystem handle.
      * @param path The full path to the file.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf8af7bb/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 56a85f4..097c051 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -956,6 +956,9 @@ Release 2.9.0 - UNRELEASED
     HDFS-9094. Add command line option to ask NameNode reload
     configuration. (Xiaobing Zhou via Arpit Agarwal)
 
+    HDFS-9541. Add hdfsStreamBuilder API to libhdfs to support defaultBlockSizes
+    greater than 2 GB. (cmccabe via zhz)
+
   OPTIMIZATIONS
 
   BUG FIXES


[04/50] hadoop git commit: YARN-4598. Invalid event: RESOURCE_FAILED at CONTAINER_CLEANEDUP_AFTER_KILL. Contributed by tangshangwen

Posted by ar...@apache.org.
YARN-4598. Invalid event: RESOURCE_FAILED at CONTAINER_CLEANEDUP_AFTER_KILL. Contributed by tangshangwen


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

Branch: refs/heads/HDFS-1312
Commit: 46e5ea81e0e41646ba02391aea31481ead28aaf8
Parents: e91e8b7
Author: Jason Lowe <jl...@apache.org>
Authored: Fri Jan 22 21:55:01 2016 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Fri Jan 22 21:55:01 2016 +0000

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                | 6 ++++++
 .../nodemanager/containermanager/container/ContainerImpl.java  | 1 +
 2 files changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/46e5ea81/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 0ceb905..f840a9e 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -1389,6 +1389,9 @@ Release 2.7.3 - UNRELEASED
     YARN-4610. Reservations continue looking for one app causes other apps to
     starve (jlowe)
 
+    YARN-4598. Invalid event: RESOURCE_FAILED at
+    CONTAINER_CLEANEDUP_AFTER_KILL (tangshangwen via jlowe)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -2278,6 +2281,9 @@ Release 2.6.4 - UNRELEASED
     YARN-4581. AHS writer thread leak makes RM crash while RM is recovering.
     (sandflee via junping_du)
 
+    YARN-4598. Invalid event: RESOURCE_FAILED at
+    CONTAINER_CLEANEDUP_AFTER_KILL (tangshangwen via jlowe)
+
 Release 2.6.3 - 2015-12-17
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/46e5ea81/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 3c49489..fb1728a 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
@@ -328,6 +328,7 @@ public class ContainerImpl implements Container {
     .addTransition(ContainerState.CONTAINER_CLEANEDUP_AFTER_KILL,
         ContainerState.CONTAINER_CLEANEDUP_AFTER_KILL,
         EnumSet.of(ContainerEventType.KILL_CONTAINER,
+            ContainerEventType.RESOURCE_FAILED,
             ContainerEventType.CONTAINER_EXITED_WITH_SUCCESS,
             ContainerEventType.CONTAINER_EXITED_WITH_FAILURE))
 


[29/50] hadoop git commit: YARN-4462. FairScheduler: Disallow preemption from a queue. (Tao Jie via kasha)

Posted by ar...@apache.org.
YARN-4462. FairScheduler: Disallow preemption from a queue. (Tao Jie via kasha)


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

Branch: refs/heads/HDFS-1312
Commit: fb238d7e5dcd96466c8938b13ca7f13cedecb08a
Parents: 2e8ab3d
Author: Karthik Kambatla <ka...@apache.org>
Authored: Wed Jan 27 11:47:29 2016 -0800
Committer: Karthik Kambatla <ka...@apache.org>
Committed: Wed Jan 27 12:29:06 2016 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   2 +
 .../scheduler/fair/AllocationConfiguration.java |  11 +-
 .../fair/AllocationFileLoaderService.java       |  16 +-
 .../scheduler/fair/FSParentQueue.java           |   8 +
 .../resourcemanager/scheduler/fair/FSQueue.java |  11 +-
 .../webapp/FairSchedulerPage.java               |   1 +
 .../webapp/dao/FairSchedulerQueueInfo.java      |   7 +
 .../scheduler/fair/TestFairScheduler.java       | 327 +++++++++++++++++++
 8 files changed, 377 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb238d7e/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 2fbecdb..2fae034 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -67,6 +67,8 @@ Release 2.9.0 - UNRELEASED
     YARN-1856. Added cgroups based memory monitoring for containers as another
     alternative to custom memory-monitoring. (Varun Vasudev via vinodkv)
 
+    YARN-4462. FairScheduler: Disallow preemption from a queue. (Tao Jie via kasha)
+
   IMPROVEMENTS
 
     YARN-4072. ApplicationHistoryServer, WebAppProxyServer, NodeManager and

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb238d7e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfiguration.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/fair/AllocationConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfiguration.java
index bf4eae8..180ae49 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfiguration.java
@@ -98,6 +98,8 @@ public class AllocationConfiguration extends ReservationSchedulerConfiguration {
   // Reservation system configuration
   private ReservationQueueConfiguration globalReservationQueueConfig;
 
+  private final Set<String> nonPreemptableQueues;
+
   public AllocationConfiguration(Map<String, Resource> minQueueResources,
       Map<String, Resource> maxQueueResources,
       Map<String, Integer> queueMaxApps, Map<String, Integer> userMaxApps,
@@ -114,7 +116,8 @@ public class AllocationConfiguration extends ReservationSchedulerConfiguration {
       QueuePlacementPolicy placementPolicy,
       Map<FSQueueType, Set<String>> configuredQueues,
       ReservationQueueConfiguration globalReservationQueueConfig,
-      Set<String> reservableQueues) {
+      Set<String> reservableQueues,
+      Set<String> nonPreemptableQueues) {
     this.minQueueResources = minQueueResources;
     this.maxQueueResources = maxQueueResources;
     this.queueMaxApps = queueMaxApps;
@@ -135,6 +138,7 @@ public class AllocationConfiguration extends ReservationSchedulerConfiguration {
     this.globalReservationQueueConfig = globalReservationQueueConfig;
     this.placementPolicy = placementPolicy;
     this.configuredQueues = configuredQueues;
+    this.nonPreemptableQueues = nonPreemptableQueues;
   }
   
   public AllocationConfiguration(Configuration conf) {
@@ -161,6 +165,7 @@ public class AllocationConfiguration extends ReservationSchedulerConfiguration {
     }
     placementPolicy = QueuePlacementPolicy.fromConfiguration(conf,
         configuredQueues);
+    nonPreemptableQueues = new HashSet<String>();
   }
   
   /**
@@ -210,6 +215,10 @@ public class AllocationConfiguration extends ReservationSchedulerConfiguration {
         -1f : fairSharePreemptionThreshold;
   }
 
+  public boolean isPreemptable(String queueName) {
+    return !nonPreemptableQueues.contains(queueName);
+  }
+
   public ResourceWeights getQueueWeight(String queue) {
     ResourceWeights weight = queueWeights.get(queue);
     return (weight == null) ? ResourceWeights.NEUTRAL : weight;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb238d7e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationFileLoaderService.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/fair/AllocationFileLoaderService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationFileLoaderService.java
index 9a31be3..d6012af 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationFileLoaderService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationFileLoaderService.java
@@ -224,6 +224,7 @@ public class AllocationFileLoaderService extends AbstractService {
     Map<String, Map<QueueACL, AccessControlList>> queueAcls =
         new HashMap<String, Map<QueueACL, AccessControlList>>();
     Set<String> reservableQueues = new HashSet<String>();
+    Set<String> nonPreemptableQueues = new HashSet<String>();
     int userMaxAppsDefault = Integer.MAX_VALUE;
     int queueMaxAppsDefault = Integer.MAX_VALUE;
     Resource queueMaxResourcesDefault = Resources.unbounded();
@@ -360,7 +361,7 @@ public class AllocationFileLoaderService extends AbstractService {
           queueMaxApps, userMaxApps, queueMaxAMShares, queueWeights,
           queuePolicies, minSharePreemptionTimeouts, fairSharePreemptionTimeouts,
           fairSharePreemptionThresholds, queueAcls, configuredQueues,
-          reservableQueues);
+          reservableQueues, nonPreemptableQueues);
     }
 
     // Load placement policy and pass it configured queues
@@ -409,7 +410,7 @@ public class AllocationFileLoaderService extends AbstractService {
         defaultSchedPolicy, minSharePreemptionTimeouts,
         fairSharePreemptionTimeouts, fairSharePreemptionThresholds, queueAcls,
         newPlacementPolicy, configuredQueues, globalReservationQueueConfig,
-        reservableQueues);
+        reservableQueues, nonPreemptableQueues);
     
     lastSuccessfulReload = clock.getTime();
     lastReloadAttemptFailed = false;
@@ -431,7 +432,8 @@ public class AllocationFileLoaderService extends AbstractService {
       Map<String, Float> fairSharePreemptionThresholds,
       Map<String, Map<QueueACL, AccessControlList>> queueAcls,
       Map<FSQueueType, Set<String>> configuredQueues,
-      Set<String> reservableQueues)
+      Set<String> reservableQueues,
+      Set<String> nonPreemptableQueues)
       throws AllocationConfigurationException {
     String queueName = element.getAttribute("name").trim();
 
@@ -508,13 +510,19 @@ public class AllocationFileLoaderService extends AbstractService {
         isLeaf = false;
         reservableQueues.add(queueName);
         configuredQueues.get(FSQueueType.PARENT).add(queueName);
+      } else if ("allowPreemptionFrom".equals(field.getTagName())) {
+        String text = ((Text)field.getFirstChild()).getData().trim();
+        if (!Boolean.parseBoolean(text)) {
+          nonPreemptableQueues.add(queueName);
+        }
       } else if ("queue".endsWith(field.getTagName()) || 
           "pool".equals(field.getTagName())) {
         loadQueue(queueName, field, minQueueResources, maxQueueResources,
             queueMaxApps, userMaxApps, queueMaxAMShares, queueWeights,
             queuePolicies, minSharePreemptionTimeouts,
             fairSharePreemptionTimeouts, fairSharePreemptionThresholds,
-            queueAcls, configuredQueues, reservableQueues);
+            queueAcls, configuredQueues, reservableQueues,
+            nonPreemptableQueues);
         isLeaf = false;
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb238d7e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSParentQueue.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/fair/FSParentQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSParentQueue.java
index febe050..a028422 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSParentQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSParentQueue.java
@@ -260,6 +260,14 @@ public class FSParentQueue extends FSQueue {
     readLock.lock();
     try {
       for (FSQueue queue : childQueues) {
+        // Skip selection for non-preemptable queue
+        if (!queue.isPreemptable()) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("skipping from queue=" + getName()
+                + " because it's a non-preemptable queue");
+          }
+          continue;
+        }
         if (candidateQueue == null ||
             comparator.compare(queue, candidateQueue) > 0) {
           candidateQueue = queue;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb238d7e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.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/fair/FSQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java
index 713bdca..f82411d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java
@@ -62,6 +62,7 @@ public abstract class FSQueue implements Queue, Schedulable {
   private long fairSharePreemptionTimeout = Long.MAX_VALUE;
   private long minSharePreemptionTimeout = Long.MAX_VALUE;
   private float fairSharePreemptionThreshold = 0.5f;
+  private boolean preemptable = true;
 
   public FSQueue(String name, FairScheduler scheduler, FSParentQueue parent) {
     this.name = name;
@@ -235,6 +236,10 @@ public abstract class FSQueue implements Queue, Schedulable {
     this.fairSharePreemptionThreshold = fairSharePreemptionThreshold;
   }
 
+  public boolean isPreemptable() {
+    return preemptable;
+  }
+
   /**
    * Recomputes the shares for all child queues and applications based on this
    * queue's current share
@@ -242,7 +247,8 @@ public abstract class FSQueue implements Queue, Schedulable {
   public abstract void recomputeShares();
 
   /**
-   * Update the min/fair share preemption timeouts and threshold for this queue.
+   * Update the min/fair share preemption timeouts, threshold and preemption
+   * disabled flag for this queue.
    */
   public void updatePreemptionVariables() {
     // For min share timeout
@@ -263,6 +269,9 @@ public abstract class FSQueue implements Queue, Schedulable {
     if (fairSharePreemptionThreshold < 0 && parent != null) {
       fairSharePreemptionThreshold = parent.getFairSharePreemptionThreshold();
     }
+    // For option whether allow preemption from this queue
+    preemptable = scheduler.getAllocationConfiguration()
+        .isPreemptable(getName());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb238d7e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerPage.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/FairSchedulerPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerPage.java
index 5ff9422..689622f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerPage.java
@@ -81,6 +81,7 @@ public class FairSchedulerPage extends RmView {
       }
       ri._(STEADY_FAIR_SHARE + ":", qinfo.getSteadyFairShare().toString());
       ri._(INSTANTANEOUS_FAIR_SHARE + ":", qinfo.getFairShare().toString());
+      ri._("Preemptable:", qinfo.isPreemptable());
       html._(InfoBlock.class);
 
       // clear the info contents so this queue's info doesn't accumulate into another queue's info

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb238d7e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/FairSchedulerQueueInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/FairSchedulerQueueInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/FairSchedulerQueueInfo.java
index ee37f18..e02df65 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/FairSchedulerQueueInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/FairSchedulerQueueInfo.java
@@ -65,6 +65,8 @@ public class FairSchedulerQueueInfo {
   private String queueName;
   private String schedulingPolicy;
 
+  private boolean preemptable;
+
   private FairSchedulerQueueInfoList childQueues;
 
   public FairSchedulerQueueInfo() {
@@ -108,6 +110,7 @@ public class FairSchedulerQueueInfo {
       return;
     }
 
+    preemptable = queue.isPreemptable();
     childQueues = getChildQueues(queue, scheduler);
   }
 
@@ -228,4 +231,8 @@ public class FairSchedulerQueueInfo {
     return childQueues != null ? childQueues.getQueueInfoList() :
         new ArrayList<FairSchedulerQueueInfo>();
   }
+
+  public boolean isPreemptable() {
+    return preemptable;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb238d7e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.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/fair/TestFairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
index 1b1418a..fac28b7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
@@ -2493,6 +2493,333 @@ public class TestFairScheduler extends FairSchedulerTestBase {
   }
 
   @Test
+  /**
+   * Tests the decision to preempt tasks respect to non-preemptable queues
+   * 1, Queues as follow:
+   *   queueA(non-preemptable)
+   *   queueB(preemptable)
+   *   parentQueue(non-preemptable)
+   *     --queueC(preemptable)
+   *   queueD(preemptable)
+   * 2, Submit request to queueA, queueB, queueC, and all of them are over MinShare
+   * 3, Now all resource are occupied
+   * 4, Submit request to queueD, and need to preempt resource from other queues
+   * 5, Only preemptable queue(queueB) would be preempted.
+   */
+  public void testPreemptionDecisionWithNonPreemptableQueue() throws Exception {
+    conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
+    ControlledClock clock = new ControlledClock();
+    scheduler.setClock(clock);
+
+    PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
+    out.println("<?xml version=\"1.0\"?>");
+    out.println("<allocations>");
+    out.println("<queue name=\"default\">");
+    out.println("<maxResources>0mb,0vcores</maxResources>");
+    out.println("</queue>");
+    out.println("<queue name=\"queueA\">");
+    out.println("<weight>.25</weight>");
+    out.println("<minResources>1024mb,0vcores</minResources>");
+    out.println("<allowPreemptionFrom>false</allowPreemptionFrom>");
+    out.println("</queue>");
+    out.println("<queue name=\"queueB\">");
+    out.println("<weight>.25</weight>");
+    out.println("<minResources>1024mb,0vcores</minResources>");
+    out.println("</queue>");
+    out.println("<queue name=\"parentQueue\">");
+    out.println("<allowPreemptionFrom>false</allowPreemptionFrom>");
+    out.println("<queue name=\"queueC\">");
+    out.println("<weight>.25</weight>");
+    out.println("<minResources>1024mb,0vcores</minResources>");
+    out.println("</queue>");
+    out.println("</queue>");
+    out.println("<queue name=\"queueD\">");
+    out.println("<weight>.25</weight>");
+    out.println("<minResources>2048mb,0vcores</minResources>");
+    out.println("</queue>");
+    out.println("<defaultMinSharePreemptionTimeout>5</defaultMinSharePreemptionTimeout>");
+    out.println("<defaultFairSharePreemptionTimeout>10</defaultFairSharePreemptionTimeout>");
+    out.println("<defaultFairSharePreemptionThreshold>.5</defaultFairSharePreemptionThreshold>");
+    out.println("</allocations>");
+    out.close();
+
+    scheduler.init(conf);
+    scheduler.start();
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+
+    // Create four nodes(3G each)
+    RMNode node1 =
+        MockNodes.newNodeInfo(1, Resources.createResource(3 * 1024, 3), 1,
+            "127.0.0.1");
+    NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
+    scheduler.handle(nodeEvent1);
+
+    RMNode node2 =
+        MockNodes.newNodeInfo(1, Resources.createResource(3 * 1024, 3), 2,
+            "127.0.0.2");
+    NodeAddedSchedulerEvent nodeEvent2 = new NodeAddedSchedulerEvent(node2);
+    scheduler.handle(nodeEvent2);
+
+    RMNode node3 =
+        MockNodes.newNodeInfo(1, Resources.createResource(3 * 1024, 3), 3,
+            "127.0.0.3");
+    NodeAddedSchedulerEvent nodeEvent3 = new NodeAddedSchedulerEvent(node3);
+    scheduler.handle(nodeEvent3);
+
+    RMNode node4 =
+        MockNodes.newNodeInfo(1, Resources.createResource(3 * 1024, 3), 4,
+            "127.0.0.4");
+    NodeAddedSchedulerEvent nodeEvent4 = new NodeAddedSchedulerEvent(node4);
+    scheduler.handle(nodeEvent4);
+
+    // Submit apps to queueA, queueB, queueC,
+    // now all resource of the cluster is occupied
+    ApplicationAttemptId app1 =
+        createSchedulingRequest(1 * 1024, "queueA", "user1", 4, 1);
+    ApplicationAttemptId app2 =
+        createSchedulingRequest(1 * 1024, "queueB", "user1", 4, 2);
+    ApplicationAttemptId app3 =
+        createSchedulingRequest(1 * 1024, "parentQueue.queueC", "user1", 4, 3);
+
+    scheduler.update();
+
+    // Sufficient node check-ins to fully schedule containers
+    for (int i = 0; i < 3; i++) {
+      NodeUpdateSchedulerEvent nodeUpdate1 = new NodeUpdateSchedulerEvent(node1);
+      scheduler.handle(nodeUpdate1);
+
+      NodeUpdateSchedulerEvent nodeUpdate2 = new NodeUpdateSchedulerEvent(node2);
+      scheduler.handle(nodeUpdate2);
+
+      NodeUpdateSchedulerEvent nodeUpdate3 = new NodeUpdateSchedulerEvent(node3);
+      scheduler.handle(nodeUpdate3);
+
+      NodeUpdateSchedulerEvent nodeUpdate4 = new NodeUpdateSchedulerEvent(node4);
+      scheduler.handle(nodeUpdate4);
+    }
+
+    assertEquals(4, scheduler.getSchedulerApp(app1).getLiveContainers().size());
+    assertEquals(4, scheduler.getSchedulerApp(app2).getLiveContainers().size());
+    assertEquals(4, scheduler.getSchedulerApp(app3).getLiveContainers().size());
+
+    // Now new requests arrive from queues D
+    ApplicationAttemptId app4 =
+        createSchedulingRequest(1 * 1024, "queueD", "user1", 4, 1);
+    scheduler.update();
+    FSLeafQueue schedD =
+        scheduler.getQueueManager().getLeafQueue("queueD", true);
+
+    // After minSharePreemptionTime has passed, 2G resource should preempted from
+    // queueB to queueD
+    clock.tickSec(6);
+    assertEquals(2048,
+        scheduler.resourceDeficit(schedD, clock.getTime()).getMemory());
+
+    scheduler.preemptResources(Resources.createResource(2 * 1024));
+    // now only app2 is selected to be preempted
+    assertTrue("App2 should have container to be preempted",
+        !Collections.disjoint(
+            scheduler.getSchedulerApp(app2).getLiveContainers(),
+            scheduler.getSchedulerApp(app2).getPreemptionContainers()));
+    assertTrue("App1 should not have container to be preempted",
+        Collections.disjoint(
+            scheduler.getSchedulerApp(app1).getLiveContainers(),
+            scheduler.getSchedulerApp(app1).getPreemptionContainers()));
+    assertTrue("App3 should not have container to be preempted",
+        Collections.disjoint(
+            scheduler.getSchedulerApp(app3).getLiveContainers(),
+            scheduler.getSchedulerApp(app3).getPreemptionContainers()));
+    // Pretend 20 seconds have passed
+    clock.tickSec(20);
+    scheduler.preemptResources(Resources.createResource(2 * 1024));
+    for (int i = 0; i < 3; i++) {
+      NodeUpdateSchedulerEvent nodeUpdate1 = new NodeUpdateSchedulerEvent(node1);
+      scheduler.handle(nodeUpdate1);
+
+      NodeUpdateSchedulerEvent nodeUpdate2 = new NodeUpdateSchedulerEvent(node2);
+      scheduler.handle(nodeUpdate2);
+
+      NodeUpdateSchedulerEvent nodeUpdate3 = new NodeUpdateSchedulerEvent(node3);
+      scheduler.handle(nodeUpdate3);
+
+      NodeUpdateSchedulerEvent nodeUpdate4 = new NodeUpdateSchedulerEvent(node4);
+      scheduler.handle(nodeUpdate4);
+    }
+    // after preemption
+    assertEquals(4, scheduler.getSchedulerApp(app1).getLiveContainers().size());
+    assertEquals(2, scheduler.getSchedulerApp(app2).getLiveContainers().size());
+    assertEquals(4, scheduler.getSchedulerApp(app3).getLiveContainers().size());
+    assertEquals(2, scheduler.getSchedulerApp(app4).getLiveContainers().size());
+  }
+
+  @Test
+  /**
+   * Tests the decision to preempt tasks when allowPreemptionFrom is set false on
+   * all queues.
+   * Then none of them would be preempted actually.
+   * 1, Queues as follow:
+   *   queueA(non-preemptable)
+   *   queueB(non-preemptable)
+   *   parentQueue(non-preemptable)
+   *     --queueC(preemptable)
+   *   parentQueue(preemptable)
+   *     --queueD(non-preemptable)
+   * 2, Submit request to queueB, queueC, queueD, and all of them are over MinShare
+   * 3, Now all resource are occupied
+   * 4, Submit request to queueA, and need to preempt resource from other queues
+   * 5, None of queues would be preempted.
+   */
+  public void testPreemptionDecisionWhenPreemptionDisabledOnAllQueues()
+      throws Exception {
+    conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
+    ControlledClock clock = new ControlledClock();
+    scheduler.setClock(clock);
+
+    PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
+    out.println("<?xml version=\"1.0\"?>");
+    out.println("<allocations>");
+    out.println("<queue name=\"default\">");
+    out.println("<maxResources>0mb,0vcores</maxResources>");
+    out.println("</queue>");
+    out.println("<queue name=\"queueA\">");
+    out.println("<weight>.25</weight>");
+    out.println("<minResources>2048mb,0vcores</minResources>");
+    out.println("<allowPreemptionFrom>false</allowPreemptionFrom>");
+    out.println("</queue>");
+    out.println("<queue name=\"queueB\">");
+    out.println("<weight>.25</weight>");
+    out.println("<minResources>1024mb,0vcores</minResources>");
+    out.println("<allowPreemptionFrom>false</allowPreemptionFrom>");
+    out.println("</queue>");
+    out.println("<queue name=\"parentQueue1\">");
+    out.println("<allowPreemptionFrom>false</allowPreemptionFrom>");
+    out.println("<queue name=\"queueC\">");
+    out.println("<weight>.25</weight>");
+    out.println("<minResources>1024mb,0vcores</minResources>");
+    out.println("</queue>");
+    out.println("</queue>");
+    out.println("<queue name=\"parentQueue2\">");
+    out.println("<queue name=\"queueD\">");
+    out.println("<weight>.25</weight>");
+    out.println("<minResources>1024mb,0vcores</minResources>");
+    out.println("<allowPreemptionFrom>false</allowPreemptionFrom>");
+    out.println("</queue>");
+    out.println("</queue>");
+    out.println("<defaultMinSharePreemptionTimeout>5</defaultMinSharePreemptionTimeout>");
+    out.println("<defaultFairSharePreemptionTimeout>10</defaultFairSharePreemptionTimeout>");
+    out.println("<defaultFairSharePreemptionThreshold>.5</defaultFairSharePreemptionThreshold>");
+    out.println("</allocations>");
+    out.close();
+
+    scheduler.init(conf);
+    scheduler.start();
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+
+    // Create four nodes(3G each)
+    RMNode node1 =
+        MockNodes.newNodeInfo(1, Resources.createResource(3 * 1024, 3), 1,
+            "127.0.0.1");
+    NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
+    scheduler.handle(nodeEvent1);
+
+    RMNode node2 =
+        MockNodes.newNodeInfo(1, Resources.createResource(3 * 1024, 3), 2,
+            "127.0.0.2");
+    NodeAddedSchedulerEvent nodeEvent2 = new NodeAddedSchedulerEvent(node2);
+    scheduler.handle(nodeEvent2);
+
+    RMNode node3 =
+        MockNodes.newNodeInfo(1, Resources.createResource(3 * 1024, 3), 3,
+            "127.0.0.3");
+    NodeAddedSchedulerEvent nodeEvent3 = new NodeAddedSchedulerEvent(node3);
+    scheduler.handle(nodeEvent3);
+
+    RMNode node4 =
+        MockNodes.newNodeInfo(1, Resources.createResource(3 * 1024, 3), 4,
+            "127.0.0.4");
+    NodeAddedSchedulerEvent nodeEvent4 = new NodeAddedSchedulerEvent(node4);
+    scheduler.handle(nodeEvent4);
+
+    // Submit apps to queueB, queueC, queueD
+    // now all resource of the cluster is occupied
+
+    ApplicationAttemptId app1 =
+        createSchedulingRequest(1 * 1024, "queueB", "user1", 4, 1);
+    ApplicationAttemptId app2 =
+        createSchedulingRequest(1 * 1024, "parentQueue1.queueC", "user1", 4, 2);
+    ApplicationAttemptId app3 =
+        createSchedulingRequest(1 * 1024, "parentQueue2.queueD", "user1", 4, 3);
+    scheduler.update();
+
+    // Sufficient node check-ins to fully schedule containers
+    for (int i = 0; i < 3; i++) {
+      NodeUpdateSchedulerEvent nodeUpdate1 = new NodeUpdateSchedulerEvent(node1);
+      scheduler.handle(nodeUpdate1);
+
+      NodeUpdateSchedulerEvent nodeUpdate2 = new NodeUpdateSchedulerEvent(node2);
+      scheduler.handle(nodeUpdate2);
+
+      NodeUpdateSchedulerEvent nodeUpdate3 = new NodeUpdateSchedulerEvent(node3);
+      scheduler.handle(nodeUpdate3);
+
+      NodeUpdateSchedulerEvent nodeUpdate4 = new NodeUpdateSchedulerEvent(node4);
+      scheduler.handle(nodeUpdate4);
+    }
+
+    assertEquals(4, scheduler.getSchedulerApp(app1).getLiveContainers().size());
+    assertEquals(4, scheduler.getSchedulerApp(app2).getLiveContainers().size());
+    assertEquals(4, scheduler.getSchedulerApp(app3).getLiveContainers().size());
+
+    // Now new requests arrive from queues A
+    ApplicationAttemptId app4 =
+        createSchedulingRequest(1 * 1024, "queueA", "user1", 4, 1);
+    scheduler.update();
+    FSLeafQueue schedA =
+        scheduler.getQueueManager().getLeafQueue("queueA", true);
+
+    // After minSharePreemptionTime has passed, resource deficit is 2G
+    clock.tickSec(6);
+    assertEquals(2048,
+        scheduler.resourceDeficit(schedA, clock.getTime()).getMemory());
+
+    scheduler.preemptResources(Resources.createResource(2 * 1024));
+    // now none app is selected to be preempted
+    assertTrue("App1 should have container to be preempted",
+        Collections.disjoint(
+            scheduler.getSchedulerApp(app1).getLiveContainers(),
+            scheduler.getSchedulerApp(app1).getPreemptionContainers()));
+    assertTrue("App2 should not have container to be preempted",
+        Collections.disjoint(
+            scheduler.getSchedulerApp(app2).getLiveContainers(),
+            scheduler.getSchedulerApp(app2).getPreemptionContainers()));
+    assertTrue("App3 should not have container to be preempted",
+        Collections.disjoint(
+            scheduler.getSchedulerApp(app3).getLiveContainers(),
+            scheduler.getSchedulerApp(app3).getPreemptionContainers()));
+    // Pretend 20 seconds have passed
+    clock.tickSec(20);
+    scheduler.preemptResources(Resources.createResource(2 * 1024));
+    for (int i = 0; i < 3; i++) {
+      NodeUpdateSchedulerEvent nodeUpdate1 = new NodeUpdateSchedulerEvent(node1);
+      scheduler.handle(nodeUpdate1);
+
+      NodeUpdateSchedulerEvent nodeUpdate2 = new NodeUpdateSchedulerEvent(node2);
+      scheduler.handle(nodeUpdate2);
+
+      NodeUpdateSchedulerEvent nodeUpdate3 = new NodeUpdateSchedulerEvent(node3);
+      scheduler.handle(nodeUpdate3);
+
+      NodeUpdateSchedulerEvent nodeUpdate4 = new NodeUpdateSchedulerEvent(node4);
+      scheduler.handle(nodeUpdate4);
+    }
+    // after preemption
+    assertEquals(4, scheduler.getSchedulerApp(app1).getLiveContainers().size());
+    assertEquals(4, scheduler.getSchedulerApp(app2).getLiveContainers().size());
+    assertEquals(4, scheduler.getSchedulerApp(app3).getLiveContainers().size());
+    assertEquals(0, scheduler.getSchedulerApp(app4).getLiveContainers().size());
+  }
+
+  @Test
   public void testBackwardsCompatiblePreemptionConfiguration() throws Exception {
     conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
 


[32/50] hadoop git commit: HDFS-9677. Rename generationStampV1/generationStampV2 to legacyGenerationStamp/generationStamp. Contributed by Mingliang Liu.

Posted by ar...@apache.org.
HDFS-9677. Rename generationStampV1/generationStampV2 to legacyGenerationStamp/generationStamp. Contributed by Mingliang Liu.


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

Branch: refs/heads/HDFS-1312
Commit: ec25c7f9c7e60c077d8c4143253c20445fcdaecf
Parents: 3a95713
Author: Jing Zhao <ji...@apache.org>
Authored: Wed Jan 27 16:34:40 2016 -0800
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Jan 27 16:34:40 2016 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +
 .../server/blockmanagement/BlockIdManager.java  | 83 ++++++++++----------
 .../OutOfLegacyGenerationStampsException.java   | 38 +++++++++
 .../OutOfV1GenerationStampsException.java       | 38 ---------
 .../hdfs/server/common/HdfsServerConstants.java |  3 +-
 .../hadoop/hdfs/server/namenode/FSEditLog.java  |  4 +-
 .../hdfs/server/namenode/FSEditLogLoader.java   |  4 +-
 .../hdfs/server/namenode/FSImageFormat.java     | 12 +--
 .../server/namenode/FSImageFormatProtobuf.java  | 12 +--
 .../hdfs/server/namenode/FSNamesystem.java      |  4 +-
 .../hadoop-hdfs/src/main/proto/fsimage.proto    |  4 +-
 .../blockmanagement/TestSequentialBlockId.java  | 18 ++---
 .../hdfs/server/namenode/TestEditLog.java       |  6 +-
 .../hdfs/server/namenode/TestFileTruncate.java  |  4 +-
 .../hdfs/server/namenode/TestSaveNamespace.java |  2 +-
 15 files changed, 119 insertions(+), 116 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec25c7f9/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 097c051..7e75558 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -959,6 +959,9 @@ Release 2.9.0 - UNRELEASED
     HDFS-9541. Add hdfsStreamBuilder API to libhdfs to support defaultBlockSizes
     greater than 2 GB. (cmccabe via zhz)
 
+    HDFS-9677. Rename generationStampV1/generationStampV2 to
+    legacyGenerationStamp/generationStamp. (Mingliang Liu via jing9)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec25c7f9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
index 9c71287..3f21d9b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
@@ -36,11 +36,11 @@ public class BlockIdManager {
    * The global generation stamp for legacy blocks with randomly
    * generated block IDs.
    */
-  private final GenerationStamp generationStampV1 = new GenerationStamp();
+  private final GenerationStamp legacyGenerationStamp = new GenerationStamp();
   /**
    * The global generation stamp for this file system.
    */
-  private final GenerationStamp generationStampV2 = new GenerationStamp();
+  private final GenerationStamp generationStamp = new GenerationStamp();
   /**
    * The value of the generation stamp when the first switch to sequential
    * block IDs was made. Blocks with generation stamps below this value
@@ -49,7 +49,7 @@ public class BlockIdManager {
    * (or initialized as an offset from the V1 (legacy) generation stamp on
    * upgrade).
    */
-  private long generationStampV1Limit;
+  private long legacyGenerationStampLimit;
   /**
    * The global block ID space for this file system.
    */
@@ -57,7 +57,8 @@ public class BlockIdManager {
   private final SequentialBlockGroupIdGenerator blockGroupIdGenerator;
 
   public BlockIdManager(BlockManager blockManager) {
-    this.generationStampV1Limit = HdfsConstants.GRANDFATHER_GENERATION_STAMP;
+    this.legacyGenerationStampLimit =
+        HdfsConstants.GRANDFATHER_GENERATION_STAMP;
     this.blockIdGenerator = new SequentialBlockIdGenerator(blockManager);
     this.blockGroupIdGenerator = new SequentialBlockGroupIdGenerator(blockManager);
   }
@@ -68,14 +69,14 @@ public class BlockIdManager {
    * Should be invoked only during the first upgrade to
    * sequential block IDs.
    */
-  public long upgradeGenerationStampToV2() {
-    Preconditions.checkState(generationStampV2.getCurrentValue() ==
+  public long upgradeLegacyGenerationStamp() {
+    Preconditions.checkState(generationStamp.getCurrentValue() ==
       GenerationStamp.LAST_RESERVED_STAMP);
-    generationStampV2.skipTo(generationStampV1.getCurrentValue() +
-      HdfsServerConstants.RESERVED_GENERATION_STAMPS_V1);
+    generationStamp.skipTo(legacyGenerationStamp.getCurrentValue() +
+      HdfsServerConstants.RESERVED_LEGACY_GENERATION_STAMPS);
 
-    generationStampV1Limit = generationStampV2.getCurrentValue();
-    return generationStampV2.getCurrentValue();
+    legacyGenerationStampLimit = generationStamp.getCurrentValue();
+    return generationStamp.getCurrentValue();
   }
 
   /**
@@ -84,10 +85,10 @@ public class BlockIdManager {
    *
    * @param stamp set generation stamp limit to this value
    */
-  public void setGenerationStampV1Limit(long stamp) {
-    Preconditions.checkState(generationStampV1Limit == HdfsConstants
-      .GRANDFATHER_GENERATION_STAMP);
-    generationStampV1Limit = stamp;
+  public void setLegacyGenerationStampLimit(long stamp) {
+    Preconditions.checkState(legacyGenerationStampLimit ==
+        HdfsConstants.GRANDFATHER_GENERATION_STAMP);
+    legacyGenerationStampLimit = stamp;
   }
 
   /**
@@ -95,7 +96,7 @@ public class BlockIdManager {
    * and random block IDs.
    */
   public long getGenerationStampAtblockIdSwitch() {
-    return generationStampV1Limit;
+    return legacyGenerationStampLimit;
   }
 
   @VisibleForTesting
@@ -138,58 +139,58 @@ public class BlockIdManager {
   /**
    * Sets the current generation stamp for legacy blocks
    */
-  public void setGenerationStampV1(long stamp) {
-    generationStampV1.setCurrentValue(stamp);
+  public void setLegacyGenerationStamp(long stamp) {
+    legacyGenerationStamp.setCurrentValue(stamp);
   }
 
   /**
    * Gets the current generation stamp for legacy blocks
    */
-  public long getGenerationStampV1() {
-    return generationStampV1.getCurrentValue();
+  public long getLegacyGenerationStamp() {
+    return legacyGenerationStamp.getCurrentValue();
   }
 
   /**
    * Gets the current generation stamp for this filesystem
    */
-  public void setGenerationStampV2(long stamp) {
-    generationStampV2.setCurrentValue(stamp);
+  public void setGenerationStamp(long stamp) {
+    generationStamp.setCurrentValue(stamp);
   }
 
-  public long getGenerationStampV2() {
-    return generationStampV2.getCurrentValue();
+  public long getGenerationStamp() {
+    return generationStamp.getCurrentValue();
   }
 
   /**
    * Increments, logs and then returns the stamp
    */
   long nextGenerationStamp(boolean legacyBlock) throws IOException {
-    return legacyBlock ? getNextGenerationStampV1() :
-      getNextGenerationStampV2();
+    return legacyBlock ? getNextLegacyGenerationStamp() :
+        getNextGenerationStamp();
   }
 
   @VisibleForTesting
-  long getNextGenerationStampV1() throws IOException {
-    long genStampV1 = generationStampV1.nextValue();
+  long getNextLegacyGenerationStamp() throws IOException {
+    long legacyGenStamp = legacyGenerationStamp.nextValue();
 
-    if (genStampV1 >= generationStampV1Limit) {
+    if (legacyGenStamp >= legacyGenerationStampLimit) {
       // We ran out of generation stamps for legacy blocks. In practice, it
-      // is extremely unlikely as we reserved 1T v1 generation stamps. The
+      // is extremely unlikely as we reserved 1T legacy generation stamps. The
       // result is that we can no longer append to the legacy blocks that
       // were created before the upgrade to sequential block IDs.
-      throw new OutOfV1GenerationStampsException();
+      throw new OutOfLegacyGenerationStampsException();
     }
 
-    return genStampV1;
+    return legacyGenStamp;
   }
 
   @VisibleForTesting
-  long getNextGenerationStampV2() {
-    return generationStampV2.nextValue();
+  long getNextGenerationStamp() {
+    return generationStamp.nextValue();
   }
 
-  public long getGenerationStampV1Limit() {
-    return generationStampV1Limit;
+  public long getLegacyGenerationStampLimit() {
+    return legacyGenerationStampLimit;
   }
 
   /**
@@ -200,7 +201,7 @@ public class BlockIdManager {
    * @return true if the block ID was randomly generated, false otherwise.
    */
   boolean isLegacyBlock(Block block) {
-    return block.getGenerationStamp() < getGenerationStampV1Limit();
+    return block.getGenerationStamp() < getLegacyGenerationStampLimit();
   }
 
   /**
@@ -213,18 +214,18 @@ public class BlockIdManager {
 
   boolean isGenStampInFuture(Block block) {
     if (isLegacyBlock(block)) {
-      return block.getGenerationStamp() > getGenerationStampV1();
+      return block.getGenerationStamp() > getLegacyGenerationStamp();
     } else {
-      return block.getGenerationStamp() > getGenerationStampV2();
+      return block.getGenerationStamp() > getGenerationStamp();
     }
   }
 
   void clear() {
-    generationStampV1.setCurrentValue(GenerationStamp.LAST_RESERVED_STAMP);
-    generationStampV2.setCurrentValue(GenerationStamp.LAST_RESERVED_STAMP);
+    legacyGenerationStamp.setCurrentValue(GenerationStamp.LAST_RESERVED_STAMP);
+    generationStamp.setCurrentValue(GenerationStamp.LAST_RESERVED_STAMP);
     getBlockIdGenerator().setCurrentValue(SequentialBlockIdGenerator
       .LAST_RESERVED_BLOCK_ID);
-    generationStampV1Limit = HdfsConstants.GRANDFATHER_GENERATION_STAMP;
+    legacyGenerationStampLimit = HdfsConstants.GRANDFATHER_GENERATION_STAMP;
   }
 
   public static boolean isStripedBlockID(long id) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec25c7f9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/OutOfLegacyGenerationStampsException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/OutOfLegacyGenerationStampsException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/OutOfLegacyGenerationStampsException.java
new file mode 100644
index 0000000..f81392f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/OutOfLegacyGenerationStampsException.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.server.blockmanagement;
+
+import java.io.IOException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * This exception is thrown when the name node runs out of V1 (legacy)
+ * generation stamps.
+ *
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class OutOfLegacyGenerationStampsException extends IOException {
+  private static final long serialVersionUID = 1L;
+
+  public OutOfLegacyGenerationStampsException() {
+    super("Out of V1 (legacy) generation stamps\n");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec25c7f9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/OutOfV1GenerationStampsException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/OutOfV1GenerationStampsException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/OutOfV1GenerationStampsException.java
deleted file mode 100644
index 127ad54..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/OutOfV1GenerationStampsException.java
+++ /dev/null
@@ -1,38 +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.server.blockmanagement;
-
-import java.io.IOException;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * This exception is thrown when the name node runs out of V1 generation
- * stamps.
- *
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class OutOfV1GenerationStampsException extends IOException {
-  private static final long serialVersionUID = 1L;
-
-  public OutOfV1GenerationStampsException() {
-    super("Out of V1 (legacy) generation stamps\n");
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec25c7f9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
index 13c9137..b2dda3c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
@@ -61,8 +61,7 @@ public interface HdfsServerConstants {
   // An invalid transaction ID that will never be seen in a real namesystem.
   long INVALID_TXID = -12345;
   // Number of generation stamps reserved for legacy blocks.
-  long RESERVED_GENERATION_STAMPS_V1 =
-      1024L * 1024 * 1024 * 1024;
+  long RESERVED_LEGACY_GENERATION_STAMPS = 1024L * 1024 * 1024 * 1024;
   /**
    * Current layout version for NameNode.
    * Please see {@link NameNodeLayoutVersion.Feature} on adding new layout version.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec25c7f9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
index e255cff..c8986dc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
@@ -960,7 +960,7 @@ public class FSEditLog implements LogsPurgeable {
   /**
    * Add legacy block generation stamp record to edit log
    */
-  void logGenerationStampV1(long genstamp) {
+  void logLegacyGenerationStamp(long genstamp) {
     SetGenstampV1Op op = SetGenstampV1Op.getInstance(cache.get())
         .setGenerationStamp(genstamp);
     logEdit(op);
@@ -969,7 +969,7 @@ public class FSEditLog implements LogsPurgeable {
   /**
    * Add generation stamp record to edit log
    */
-  void logGenerationStampV2(long genstamp) {
+  void logGenerationStamp(long genstamp) {
     SetGenstampV2Op op = SetGenstampV2Op.getInstance(cache.get())
         .setGenerationStamp(genstamp);
     logEdit(op);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec25c7f9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
index 1a9d7a9..25f5a4f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
@@ -589,7 +589,7 @@ public class FSEditLogLoader {
     }
     case OP_SET_GENSTAMP_V1: {
       SetGenstampV1Op setGenstampV1Op = (SetGenstampV1Op)op;
-      blockManager.getBlockIdManager().setGenerationStampV1(
+      blockManager.getBlockIdManager().setLegacyGenerationStamp(
           setGenstampV1Op.genStampV1);
       break;
     }
@@ -797,7 +797,7 @@ public class FSEditLogLoader {
     }
     case OP_SET_GENSTAMP_V2: {
       SetGenstampV2Op setGenstampV2Op = (SetGenstampV2Op) op;
-      blockManager.getBlockIdManager().setGenerationStampV2(
+      blockManager.getBlockIdManager().setGenerationStamp(
           setGenstampV2Op.genStampV2);
       break;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec25c7f9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
index a815864..e4263bd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
@@ -346,24 +346,24 @@ public class FSImageFormat {
         long genstamp = in.readLong();
         final BlockIdManager blockIdManager = namesystem.getBlockManager()
             .getBlockIdManager();
-        blockIdManager.setGenerationStampV1(genstamp);
+        blockIdManager.setLegacyGenerationStamp(genstamp);
 
         if (NameNodeLayoutVersion.supports(
             LayoutVersion.Feature.SEQUENTIAL_BLOCK_ID, imgVersion)) {
           // read the starting generation stamp for sequential block IDs
           genstamp = in.readLong();
-          blockIdManager.setGenerationStampV2(genstamp);
+          blockIdManager.setGenerationStamp(genstamp);
 
           // read the last generation stamp for blocks created after
           // the switch to sequential block IDs.
           long stampAtIdSwitch = in.readLong();
-          blockIdManager.setGenerationStampV1Limit(stampAtIdSwitch);
+          blockIdManager.setLegacyGenerationStampLimit(stampAtIdSwitch);
 
           // read the max sequential block ID.
           long maxSequentialBlockId = in.readLong();
           blockIdManager.setLastAllocatedContiguousBlockId(maxSequentialBlockId);
         } else {
-          long startingGenStamp = blockIdManager.upgradeGenerationStampToV2();
+          long startingGenStamp = blockIdManager.upgradeLegacyGenerationStamp();
           // This is an upgrade.
           LOG.info("Upgrading to sequential block IDs. Generation stamp " +
                    "for new blocks set to " + startingGenStamp);
@@ -1270,8 +1270,8 @@ public class FSImageFormat {
         out.writeLong(numINodes);
         final BlockIdManager blockIdManager = sourceNamesystem.getBlockManager()
             .getBlockIdManager();
-        out.writeLong(blockIdManager.getGenerationStampV1());
-        out.writeLong(blockIdManager.getGenerationStampV2());
+        out.writeLong(blockIdManager.getLegacyGenerationStamp());
+        out.writeLong(blockIdManager.getGenerationStamp());
         out.writeLong(blockIdManager.getGenerationStampAtblockIdSwitch());
         out.writeLong(blockIdManager.getLastAllocatedContiguousBlockId());
         out.writeLong(context.getTxId());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec25c7f9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
index 635dc34..e9bd8c5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
@@ -294,9 +294,9 @@ public final class FSImageFormatProtobuf {
     private void loadNameSystemSection(InputStream in) throws IOException {
       NameSystemSection s = NameSystemSection.parseDelimitedFrom(in);
       BlockIdManager blockIdManager = fsn.getBlockManager().getBlockIdManager();
-      blockIdManager.setGenerationStampV1(s.getGenstampV1());
-      blockIdManager.setGenerationStampV2(s.getGenstampV2());
-      blockIdManager.setGenerationStampV1Limit(s.getGenstampV1Limit());
+      blockIdManager.setLegacyGenerationStamp(s.getGenstampV1());
+      blockIdManager.setGenerationStamp(s.getGenstampV2());
+      blockIdManager.setLegacyGenerationStampLimit(s.getGenstampV1Limit());
       blockIdManager.setLastAllocatedContiguousBlockId(s.getLastAllocatedBlockId());
       if (s.hasLastAllocatedStripedBlockId()) {
         blockIdManager.setLastAllocatedStripedBlockId(
@@ -550,9 +550,9 @@ public final class FSImageFormatProtobuf {
       OutputStream out = sectionOutputStream;
       BlockIdManager blockIdManager = fsn.getBlockManager().getBlockIdManager();
       NameSystemSection.Builder b = NameSystemSection.newBuilder()
-          .setGenstampV1(blockIdManager.getGenerationStampV1())
-          .setGenstampV1Limit(blockIdManager.getGenerationStampV1Limit())
-          .setGenstampV2(blockIdManager.getGenerationStampV2())
+          .setGenstampV1(blockIdManager.getLegacyGenerationStamp())
+          .setGenstampV1Limit(blockIdManager.getLegacyGenerationStampLimit())
+          .setGenstampV2(blockIdManager.getGenerationStamp())
           .setLastAllocatedBlockId(blockIdManager.getLastAllocatedContiguousBlockId())
           .setLastAllocatedStripedBlockId(blockIdManager.getLastAllocatedStripedBlockId())
           .setTransactionId(context.getTxId());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec25c7f9/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 1693958..5e2e975 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
@@ -4575,9 +4575,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
     long gs = blockManager.nextGenerationStamp(legacyBlock);
     if (legacyBlock) {
-      getEditLog().logGenerationStampV1(gs);
+      getEditLog().logLegacyGenerationStamp(gs);
     } else {
-      getEditLog().logGenerationStampV2(gs);
+      getEditLog().logGenerationStamp(gs);
     }
 
     // NB: callers sync the log

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec25c7f9/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
index 43a1fc5..e57efee 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
@@ -67,8 +67,8 @@ message FileSummary {
  */
 message NameSystemSection {
   optional uint32 namespaceId = 1;
-  optional uint64 genstampV1 = 2;
-  optional uint64 genstampV2 = 3;
+  optional uint64 genstampV1 = 2; // legacy generation stamp
+  optional uint64 genstampV2 = 3; // generation stamp of latest version
   optional uint64 genstampV1Limit = 4;
   optional uint64 lastAllocatedBlockId = 5;
   optional uint64 transactionId = 6;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec25c7f9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockId.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockId.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockId.java
index e612ea9..bfda393 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockId.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockId.java
@@ -151,7 +151,7 @@ public class TestSequentialBlockId {
     BlockIdManager bid = mock(BlockIdManager.class);
     final long maxGenStampForLegacyBlocks = 10000;
 
-    when(bid.getGenerationStampV1Limit())
+    when(bid.getLegacyGenerationStampLimit())
         .thenReturn(maxGenStampForLegacyBlocks);
 
     Block legacyBlock = spy(new Block());
@@ -180,18 +180,18 @@ public class TestSequentialBlockId {
     // Setup a mock object and stub out a few routines to
     // retrieve the generation stamp counters.
     BlockIdManager bid = mock(BlockIdManager.class);
-    final long nextGenerationStampV1 = 5000;
-    final long nextGenerationStampV2 = 20000;
+    final long nextLegacyGenerationStamp = 5000;
+    final long nextGenerationStamp = 20000;
 
-    when(bid.getNextGenerationStampV1())
-        .thenReturn(nextGenerationStampV1);
-    when(bid.getNextGenerationStampV2())
-        .thenReturn(nextGenerationStampV2);
+    when(bid.getNextLegacyGenerationStamp())
+        .thenReturn(nextLegacyGenerationStamp);
+    when(bid.getNextGenerationStamp())
+        .thenReturn(nextGenerationStamp);
 
     // Make sure that the generation stamp is set correctly for both
     // kinds of blocks.
     when(bid.nextGenerationStamp(anyBoolean())).thenCallRealMethod();
-    assertThat(bid.nextGenerationStamp(true), is(nextGenerationStampV1));
-    assertThat(bid.nextGenerationStamp(false), is(nextGenerationStampV2));
+    assertThat(bid.nextGenerationStamp(true), is(nextLegacyGenerationStamp));
+    assertThat(bid.nextGenerationStamp(false), is(nextGenerationStamp));
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec25c7f9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
index 4b72112..14240e0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
@@ -1145,7 +1145,7 @@ public class TestEditLog {
     editlog.initJournalsForWrite();
     editlog.openForWrite(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
     for (int i = 2; i < TXNS_PER_ROLL; i++) {
-      editlog.logGenerationStampV2((long) 0);
+      editlog.logGenerationStamp((long) 0);
     }
     editlog.logSync();
     
@@ -1157,7 +1157,7 @@ public class TestEditLog {
     for (int i = 0; i < numrolls; i++) {
       editlog.rollEditLog(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
       
-      editlog.logGenerationStampV2((long) i);
+      editlog.logGenerationStamp((long) i);
       editlog.logSync();
 
       while (aborts.size() > 0 
@@ -1167,7 +1167,7 @@ public class TestEditLog {
       } 
       
       for (int j = 3; j < TXNS_PER_ROLL; j++) {
-        editlog.logGenerationStampV2((long) i);
+        editlog.logGenerationStamp((long) i);
       }
       editlog.logSync();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec25c7f9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
index f2c0330..d723525 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
@@ -1020,7 +1020,7 @@ public class TestFileTruncate {
       assertThat(truncateBlock.getNumBytes(),
           is(oldBlock.getNumBytes()));
       assertThat(truncateBlock.getGenerationStamp(),
-          is(fsn.getBlockManager().getBlockIdManager().getGenerationStampV2()));
+          is(fsn.getBlockManager().getBlockIdManager().getGenerationStamp()));
       assertThat(file.getLastBlock().getBlockUCState(),
           is(HdfsServerConstants.BlockUCState.UNDER_RECOVERY));
       long blockRecoveryId = file.getLastBlock().getUnderConstructionFeature()
@@ -1054,7 +1054,7 @@ public class TestFileTruncate {
       assertThat(truncateBlock.getNumBytes() < oldBlock.getNumBytes(),
           is(true));
       assertThat(truncateBlock.getGenerationStamp(),
-          is(fsn.getBlockManager().getBlockIdManager().getGenerationStampV2()));
+          is(fsn.getBlockManager().getBlockIdManager().getGenerationStamp()));
       assertThat(file.getLastBlock().getBlockUCState(),
           is(HdfsServerConstants.BlockUCState.UNDER_RECOVERY));
       long blockRecoveryId = file.getLastBlock().getUnderConstructionFeature()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec25c7f9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java
index 4d37d61..a03e900 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java
@@ -519,7 +519,7 @@ public class TestSaveNamespace {
     DelayAnswer delayer = new GenericTestUtils.DelayAnswer(LOG);
     BlockIdManager bid = spy(spyFsn.getBlockManager().getBlockIdManager());
     Whitebox.setInternalState(finalFsn.getBlockManager(), "blockIdManager", bid);
-    doAnswer(delayer).when(bid).getGenerationStampV2();
+    doAnswer(delayer).when(bid).getGenerationStamp();
 
     ExecutorService pool = Executors.newFixedThreadPool(2);
     


[26/50] hadoop git commit: HADOOP-12718. Incorrect error message by fs -put local dir without permission. (John Zhuge via Yongjun Zhang)

Posted by ar...@apache.org.
HADOOP-12718. Incorrect error message by fs -put local dir without permission. (John Zhuge via Yongjun Zhang)


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

Branch: refs/heads/HDFS-1312
Commit: 97056c3355810a803f07baca89b89e2bf6bb7201
Parents: c01bee0
Author: Yongjun Zhang <yz...@cloudera.com>
Authored: Wed Jan 27 08:04:25 2016 -0800
Committer: Yongjun Zhang <yz...@cloudera.com>
Committed: Wed Jan 27 08:04:25 2016 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt                 | 3 +++
 .../src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java  | 5 +++++
 2 files changed, 8 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/97056c33/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 5121a83..3b8376f 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -1648,6 +1648,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-12731. Remove useless boxing/unboxing code.
     (Kousuke Saruta via aajisaka)
 
+    HADOOP-12718. Incorrect error message by fs -put local dir without
+    permission. (John Zhuge via Yongjun Zhang)
+
 Release 2.7.3 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/97056c33/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
index 352b27a..3e984e3 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
@@ -33,6 +33,7 @@ import java.io.OutputStream;
 import java.io.FileDescriptor;
 import java.net.URI;
 import java.nio.ByteBuffer;
+import java.nio.file.AccessDeniedException;
 import java.nio.file.Files;
 import java.nio.file.NoSuchFileException;
 import java.nio.file.attribute.BasicFileAttributes;
@@ -463,6 +464,10 @@ public class RawLocalFileSystem extends FileSystem {
     if (localf.isDirectory()) {
       String[] names = localf.list();
       if (names == null) {
+        if (!localf.canRead()) {
+          throw new AccessDeniedException("cannot open directory " + f +
+              ": Permission denied");
+        }
         return null;
       }
       results = new FileStatus[names.length];


[21/50] hadoop git commit: MAPREDUCE-6610. JobHistoryEventHandler should not swallow timeline response. Contributed by Li Lu

Posted by ar...@apache.org.
MAPREDUCE-6610. JobHistoryEventHandler should not swallow timeline response. Contributed by Li Lu


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

Branch: refs/heads/HDFS-1312
Commit: d0d7c221682a88ac6e11e9b7c07513e369104b10
Parents: 45c763a
Author: Jian He <ji...@apache.org>
Authored: Mon Jan 25 23:01:03 2016 -0800
Committer: Jian He <ji...@apache.org>
Committed: Mon Jan 25 23:01:03 2016 -0800

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt              |  3 +++
 .../jobhistory/JobHistoryEventHandler.java        | 18 +++++++++++++++++-
 2 files changed, 20 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d0d7c221/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 8f35c6f..68564b6 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -698,6 +698,9 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6614. Remove unnecessary code in TestMapreduceConfigFields.
     (Kai Sasaki via aajisaka)
 
+    MAPREDUCE-6610. JobHistoryEventHandler should not swallow timeline response
+    (Li Lu via jianhe)
+
 Release 2.7.3 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d0d7c221/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java
index dd0de2a..63e3333 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java
@@ -23,6 +23,7 @@ import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.Iterator;
+import java.util.List;
 import java.util.Map;
 import java.util.Timer;
 import java.util.TimerTask;
@@ -63,6 +64,7 @@ import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
 import org.apache.hadoop.yarn.client.api.TimelineClient;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.EventHandler;
@@ -1012,7 +1014,21 @@ public class JobHistoryEventHandler extends AbstractService
     }
 
     try {
-      timelineClient.putEntities(tEntity);
+      TimelinePutResponse response = timelineClient.putEntities(tEntity);
+      List<TimelinePutResponse.TimelinePutError> errors = response.getErrors();
+      if (errors.size() == 0) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Timeline entities are successfully put in event " + event
+              .getEventType());
+        }
+      } else {
+        for (TimelinePutResponse.TimelinePutError error : errors) {
+          LOG.error(
+              "Error when publishing entity [" + error.getEntityType() + ","
+                  + error.getEntityId() + "], server side error code: "
+                  + error.getErrorCode());
+        }
+      }
     } catch (IOException ex) {
       LOG.error("Error putting entity " + tEntity.getEntityId() + " to Timeline"
       + "Server", ex);


[44/50] hadoop git commit: YARN-4543. Fix random test failure in TestNodeStatusUpdater.testStopReentrant. (Akihiro Suda via rohithsharmaks)

Posted by ar...@apache.org.
YARN-4543. Fix random test failure in TestNodeStatusUpdater.testStopReentrant. (Akihiro Suda via rohithsharmaks)


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

Branch: refs/heads/HDFS-1312
Commit: ac686668031ee9837deed3f3566f09f33c437870
Parents: 09d831c
Author: Rohith Sharma K S <ro...@apache.org>
Authored: Fri Jan 29 12:29:54 2016 +0530
Committer: Rohith Sharma K S <ro...@apache.org>
Committed: Fri Jan 29 12:29:54 2016 +0530

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                             | 3 +++
 .../yarn/server/nodemanager/TestNodeStatusUpdater.java      | 9 ++++++++-
 2 files changed, 11 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac686668/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index ee57e4b..dd61f2a 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -173,6 +173,9 @@ Release 2.9.0 - UNRELEASED
     YARN-4519. Potential deadlock of CapacityScheduler between decrease container
     and assign containers. (Meng Ding via jianhe)
 
+    YARN-4543. Fix random test failure in TestNodeStatusUpdater.testStopReentrant
+    (Akihiro Suda via rohithsharmaks)
+
 Release 2.8.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac686668/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.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/TestNodeStatusUpdater.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java
index a8066c1..9e6868d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java
@@ -1269,7 +1269,14 @@ public class TestNodeStatusUpdater {
     }
 
     Assert.assertEquals(STATE.STOPPED, nm.getServiceState());
-    Assert.assertEquals(numCleanups.get(), 1);
+
+    // It further takes a while after NM reached the STOPPED state.
+    waitCount = 0;
+    while (numCleanups.get() == 0 && waitCount++ != 20) {
+      LOG.info("Waiting for NM shutdown..");
+      Thread.sleep(1000);
+    }
+    Assert.assertEquals(1, numCleanups.get());
   }
 
   @Test


[02/50] hadoop git commit: HDFS-9525. hadoop utilities need to support provided delegation tokens (HeeSoo Kim via aw)

Posted by ar...@apache.org.
HDFS-9525. hadoop utilities need to support provided delegation tokens (HeeSoo Kim via aw)


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

Branch: refs/heads/HDFS-1312
Commit: d22c4239a40a1c7ed49c06038138f0e3f387b4a0
Parents: 95363bc
Author: Allen Wittenauer <aw...@apache.org>
Authored: Fri Jan 22 12:15:22 2016 -0800
Committer: Allen Wittenauer <aw...@apache.org>
Committed: Fri Jan 22 12:15:22 2016 -0800

----------------------------------------------------------------------
 .../fs/CommonConfigurationKeysPublic.java       |  3 ++
 .../hadoop/security/UserGroupInformation.java   | 22 +++++++++
 .../src/main/resources/core-default.xml         |  6 +++
 .../security/TestUserGroupInformation.java      | 48 +++++++++++++++++++-
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      | 14 +++---
 .../hdfs/web/resources/DelegationParam.java     |  5 +-
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../apache/hadoop/hdfs/web/TestWebHdfsUrl.java  |  5 +-
 8 files changed, 92 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d22c4239/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
index c9f758b..648ad59 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
@@ -311,6 +311,9 @@ public class CommonConfigurationKeysPublic {
   /** See <a href="{@docRoot}/../core-default.html">core-default.xml</a> */
   public static final String HADOOP_SECURITY_DNS_NAMESERVER_KEY =
     "hadoop.security.dns.nameserver";
+  /** See <a href="{@docRoot}/../core-default.html">core-default.xml</a> */
+  public static final String HADOOP_TOKEN_FILES =
+      "hadoop.token.files";
 
   /** See <a href="{@docRoot}/../core-default.html">core-default.xml</a> */
   public static final String HADOOP_KERBEROS_MIN_SECONDS_BEFORE_RELOGIN =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d22c4239/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
index 28014bf..d33e1aa 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.security;
 import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_USER_GROUP_METRICS_PERCENTILES_INTERVALS;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_KERBEROS_MIN_SECONDS_BEFORE_RELOGIN;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_KERBEROS_MIN_SECONDS_BEFORE_RELOGIN_DEFAULT;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_TOKEN_FILES;
 import static org.apache.hadoop.util.PlatformName.IBM_JAVA;
 
 import java.io.File;
@@ -70,6 +71,7 @@ import org.apache.hadoop.security.authentication.util.KerberosUtil;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -821,6 +823,26 @@ public class UserGroupInformation {
       }
       loginUser = proxyUser == null ? realUser : createProxyUser(proxyUser, realUser);
 
+      String tokenFileLocation = System.getProperty(HADOOP_TOKEN_FILES);
+      if (tokenFileLocation == null) {
+        tokenFileLocation = conf.get(HADOOP_TOKEN_FILES);
+      }
+      if (tokenFileLocation != null) {
+        for (String tokenFileName:
+             StringUtils.getTrimmedStrings(tokenFileLocation)) {
+          if (tokenFileName.length() > 0) {
+            File tokenFile = new File(tokenFileName);
+            if (tokenFile.exists() && tokenFile.isFile()) {
+              Credentials cred = Credentials.readTokenStorageFile(
+                  tokenFile, conf);
+              loginUser.addCredentials(cred);
+            } else {
+              LOG.info("tokenFile("+tokenFileName+") does not exist");
+            }
+          }
+        }
+      }
+
       String fileLocation = System.getenv(HADOOP_TOKEN_FILE_LOCATION);
       if (fileLocation != null) {
         // Load the token storage file and put all of the tokens into the

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d22c4239/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 bf60a25..c25f49e 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
@@ -466,6 +466,12 @@ for ldap providers in the same way as above does.
   <description>Maps kerberos principals to local user names</description>
 </property>
 
+<property>
+  <name>hadoop.token.files</name>
+  <value></value>
+  <description>List of token cache files that have delegation tokens for hadoop service</description>
+</property>
+
 <!-- i/o properties -->
 <property>
   <name>io.file.buffer.size</name>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d22c4239/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java
index 54cfc2d..6abe78c 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java
@@ -18,6 +18,7 @@ package org.apache.hadoop.security;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.security.SaslRpcServer.AuthMethod;
@@ -35,6 +36,7 @@ import javax.security.auth.login.AppConfigurationEntry;
 import javax.security.auth.login.LoginContext;
 
 import java.io.BufferedReader;
+import java.io.File;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.lang.reflect.Method;
@@ -848,7 +850,9 @@ public class TestUserGroupInformation {
    */
   @Test
   public void testPrivateTokenExclusion() throws Exception  {
-    UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+    UserGroupInformation ugi =
+        UserGroupInformation.createUserForTesting(
+            "privateUser", new String[] { "PRIVATEUSERS" });
     TestTokenIdentifier tokenId = new TestTokenIdentifier();
     Token<TestTokenIdentifier> token = new Token<TestTokenIdentifier>(
             tokenId.getBytes(), "password".getBytes(),
@@ -928,4 +932,46 @@ public class TestUserGroupInformation {
       }
     }
   }
+
+  @Test
+  public void testExternalTokenFiles() throws Exception {
+    StringBuilder tokenFullPathnames = new StringBuilder();
+    String tokenFilenames = "token1,token2";
+    String tokenFiles[] = StringUtils.getTrimmedStrings(tokenFilenames);
+    final File testDir = new File("target",
+        TestUserGroupInformation.class.getName() + "-tmpDir").getAbsoluteFile();
+    String testDirPath = testDir.getAbsolutePath();
+
+    // create path for token files
+    for (String tokenFile: tokenFiles) {
+      if (tokenFullPathnames.length() > 0) {
+        tokenFullPathnames.append(",");
+      }
+      tokenFullPathnames.append(testDirPath).append("/").append(tokenFile);
+    }
+
+    // create new token and store it
+    TestTokenIdentifier tokenId = new TestTokenIdentifier();
+    Credentials cred1 = new Credentials();
+    Token<TestTokenIdentifier> token1 = new Token<TestTokenIdentifier>(
+            tokenId.getBytes(), "password".getBytes(),
+            tokenId.getKind(), new Text("token-service1"));
+    cred1.addToken(token1.getService(), token1);
+    cred1.writeTokenStorageFile(new Path(testDirPath, tokenFiles[0]), conf);
+
+    Credentials cred2 = new Credentials();
+    Token<TestTokenIdentifier> token2 = new Token<TestTokenIdentifier>(
+            tokenId.getBytes(), "password".getBytes(),
+            tokenId.getKind(), new Text("token-service2"));
+    cred2.addToken(token2.getService(), token2);
+    cred2.writeTokenStorageFile(new Path(testDirPath, tokenFiles[1]), conf);
+
+    // set property for token external token files
+    System.setProperty("hadoop.token.files", tokenFullPathnames.toString());
+    UserGroupInformation.setLoginUser(null);
+    UserGroupInformation tokenUgi = UserGroupInformation.getLoginUser();
+    Collection<Token<?>> credsugiTokens = tokenUgi.getTokens();
+    assertTrue(credsugiTokens.contains(token1));
+    assertTrue(credsugiTokens.contains(token2));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d22c4239/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
index f9c2c6e..d806d55 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
@@ -238,7 +238,7 @@ public class WebHdfsFileSystem extends FileSystem
   // the first getAuthParams() for a non-token op will either get the
   // internal token from the ugi or lazy fetch one
   protected synchronized Token<?> getDelegationToken() throws IOException {
-    if (canRefreshDelegationToken && delegationToken == null) {
+    if (delegationToken == null) {
       Token<?> token = tokenSelector.selectToken(
           new Text(getCanonicalServiceName()), ugi.getTokens());
       // ugi tokens are usually indicative of a task which can't
@@ -248,11 +248,13 @@ public class WebHdfsFileSystem extends FileSystem
         LOG.debug("Using UGI token: {}", token);
         canRefreshDelegationToken = false;
       } else {
-        token = getDelegationToken(null);
-        if (token != null) {
-          LOG.debug("Fetched new token: {}", token);
-        } else { // security is disabled
-          canRefreshDelegationToken = false;
+        if (canRefreshDelegationToken) {
+          token = getDelegationToken(null);
+          if (token != null) {
+            LOG.debug("Fetched new token: {}", token);
+          } else { // security is disabled
+            canRefreshDelegationToken = false;
+          }
         }
       }
       setDelegationToken(token);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d22c4239/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/DelegationParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/DelegationParam.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/DelegationParam.java
index 5329580..fda14438 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/DelegationParam.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/DelegationParam.java
@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.hdfs.web.resources;
 
-import org.apache.hadoop.security.UserGroupInformation;
-
 /** Represents delegation token used for authentication. */
 public class DelegationParam extends StringParam {
   /** Parameter name. */
@@ -33,8 +31,7 @@ public class DelegationParam extends StringParam {
    * @param str a string representation of the parameter value.
    */
   public DelegationParam(final String str) {
-    super(DOMAIN, UserGroupInformation.isSecurityEnabled()
-        && str != null && !str.equals(DEFAULT)? str: null);
+    super(DOMAIN, str != null && !str.equals(DEFAULT)? str: null);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d22c4239/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 940fa90..74b8aac 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -55,6 +55,9 @@ Trunk (Unreleased)
     HDFS-9057. allow/disallow snapshots via webhdfs
     (Bramma Reddy Battula via vinayakumarb)
 
+    HDFS-9525. hadoop utilities need to support provided delegation
+    tokens (HeeSoo Kim via aw)
+
   IMPROVEMENTS
 
     HDFS-4665. Move TestNetworkTopologyWithNodeGroup to common.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d22c4239/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java
index 2913a97..24c13af 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java
@@ -195,7 +195,7 @@ public class TestWebHdfsUrl {
     checkQueryParams(
         new String[]{
             GetOpParam.Op.GETFILESTATUS.toQueryString(),
-            new UserParam(ugi.getShortUserName()).toString()
+            new DelegationParam(tokenString).toString()
         },
         fileStatusUrl);    
   }
@@ -280,8 +280,7 @@ public class TestWebHdfsUrl {
     checkQueryParams(
         new String[]{
             GetOpParam.Op.GETFILESTATUS.toQueryString(),
-            new UserParam(ugi.getRealUser().getShortUserName()).toString(),
-            new DoAsParam(ugi.getShortUserName()).toString()
+            new DelegationParam(tokenString).toString()
         },
         fileStatusUrl);    
   }


[13/50] hadoop git commit: HADOOP-12715. TestValueQueue#testgetAtMostPolicyALL fails intermittently. Contributed by Xiao Chen.

Posted by ar...@apache.org.
HADOOP-12715. TestValueQueue#testgetAtMostPolicyALL fails intermittently. Contributed by Xiao Chen.


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

Branch: refs/heads/HDFS-1312
Commit: 6eacdea0e475b4fff91cedce5005a7c11749cf64
Parents: 56a0c17
Author: Walter Su <wa...@apache.org>
Authored: Mon Jan 25 19:30:04 2016 +0800
Committer: Walter Su <wa...@apache.org>
Committed: Mon Jan 25 19:30:04 2016 +0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 +
 .../hadoop/crypto/key/TestValueQueue.java       | 65 +++++++++++++++-----
 2 files changed, 52 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6eacdea0/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 2fc8ab4..9606296 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -2496,6 +2496,9 @@ Release 2.6.4 - UNRELEASED
     HADOOP-12736. TestTimedOutTestsListener#testThreadDumpAndDeadlocks
     sometimes times out. (Xiao Chen via aajisaka)
 
+    HADOOP-12715. TestValueQueue#testgetAtMostPolicyALL fails intermittently.
+    (Xiao Chen via waltersu4549)
+
 Release 2.6.3 - 2015-12-17
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6eacdea0/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestValueQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestValueQueue.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestValueQueue.java
index 8e3a093..5eae9a0 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestValueQueue.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestValueQueue.java
@@ -19,18 +19,24 @@ package org.apache.hadoop.crypto.key;
 
 import java.io.IOException;
 import java.util.Queue;
+import java.util.concurrent.ExecutionException;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.TimeUnit;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.crypto.key.kms.ValueQueue;
 import org.apache.hadoop.crypto.key.kms.ValueQueue.QueueRefiller;
 import org.apache.hadoop.crypto.key.kms.ValueQueue.SyncGenerationPolicy;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Assert;
 import org.junit.Test;
 
+import com.google.common.base.Supplier;
 import com.google.common.collect.Sets;
 
 public class TestValueQueue {
+  Logger LOG = LoggerFactory.getLogger(TestValueQueue.class);
 
   private static class FillInfo {
     final int num;
@@ -60,7 +66,7 @@ public class TestValueQueue {
   /**
    * Verifies that Queue is initially filled to "numInitValues"
    */
-  @Test
+  @Test(timeout=30000)
   public void testInitFill() throws Exception {
     MockFiller filler = new MockFiller();
     ValueQueue<String> vq =
@@ -74,7 +80,7 @@ public class TestValueQueue {
   /**
    * Verifies that Queue is initialized (Warmed-up) for provided keys
    */
-  @Test
+  @Test(timeout=30000)
   public void testWarmUp() throws Exception {
     MockFiller filler = new MockFiller();
     ValueQueue<String> vq =
@@ -97,7 +103,7 @@ public class TestValueQueue {
    * Verifies that the refill task is executed after "checkInterval" if
    * num values below "lowWatermark"
    */
-  @Test
+  @Test(timeout=30000)
   public void testRefill() throws Exception {
     MockFiller filler = new MockFiller();
     ValueQueue<String> vq =
@@ -116,7 +122,7 @@ public class TestValueQueue {
    * Verifies that the No refill Happens after "checkInterval" if
    * num values above "lowWatermark"
    */
-  @Test
+  @Test(timeout=30000)
   public void testNoRefill() throws Exception {
     MockFiller filler = new MockFiller();
     ValueQueue<String> vq =
@@ -131,29 +137,56 @@ public class TestValueQueue {
   /**
    * Verify getAtMost when SyncGeneration Policy = ALL
    */
-  @Test
+  @Test(timeout=30000)
   public void testgetAtMostPolicyALL() throws Exception {
     MockFiller filler = new MockFiller();
-    ValueQueue<String> vq =
+    final ValueQueue<String> vq =
         new ValueQueue<String>(10, 0.1f, 300, 1,
             SyncGenerationPolicy.ALL, filler);
     Assert.assertEquals("test", vq.getNext("k1"));
     Assert.assertEquals(1, filler.getTop().num);
-    // Drain completely
-    Assert.assertEquals(10, vq.getAtMost("k1", 10).size());
-    // Synchronous call
-    Assert.assertEquals(10, filler.getTop().num);
-    // Ask for more... return all
-    Assert.assertEquals(19, vq.getAtMost("k1", 19).size());
+
+    // Synchronous call:
+    // 1. Synchronously fill returned list
+    // 2. Start another async task to fill the queue in the cache
+    Assert.assertEquals("Failed in sync call.", 10,
+        vq.getAtMost("k1", 10).size());
+    Assert.assertEquals("Sync call filler got wrong number.", 10,
+        filler.getTop().num);
+
+    // Wait for the async task to finish
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        try {
+          int size = vq.getSize("k1");
+          if (size != 10) {
+            LOG.info("Current ValueQueue size is " + size);
+            return false;
+          }
+          return true;
+        } catch (ExecutionException e) {
+          LOG.error("Exception when getSize.", e);
+          return false;
+        }
+      }
+    }, 100, 3000);
+    Assert.assertEquals("Failed in async call.", 10, filler.getTop().num);
+
+    // Drain completely after filled by the async thread
+    Assert.assertEquals("Failed to drain completely after async.", 10,
+        vq.getAtMost("k1", 10).size());
     // Synchronous call (No Async call since num > lowWatermark)
-    Assert.assertEquals(19, filler.getTop().num);
+    Assert.assertEquals("Failed to get all 19.", 19,
+        vq.getAtMost("k1", 19).size());
+    Assert.assertEquals("Failed in sync call.", 19, filler.getTop().num);
     vq.shutdown();
   }
 
   /**
    * Verify getAtMost when SyncGeneration Policy = ALL
    */
-  @Test
+  @Test(timeout=30000)
   public void testgetAtMostPolicyATLEAST_ONE() throws Exception {
     MockFiller filler = new MockFiller();
     ValueQueue<String> vq =
@@ -171,7 +204,7 @@ public class TestValueQueue {
   /**
    * Verify getAtMost when SyncGeneration Policy = LOW_WATERMARK
    */
-  @Test
+  @Test(timeout=30000)
   public void testgetAtMostPolicyLOW_WATERMARK() throws Exception {
     MockFiller filler = new MockFiller();
     ValueQueue<String> vq =
@@ -188,7 +221,7 @@ public class TestValueQueue {
     vq.shutdown();
   }
 
-  @Test
+  @Test(timeout=30000)
   public void testDrain() throws Exception {
     MockFiller filler = new MockFiller();
     ValueQueue<String> vq =


[20/50] hadoop git commit: HDFS-9690. ClientProtocol.addBlock is not idempotent after HDFS-8071.

Posted by ar...@apache.org.
HDFS-9690. ClientProtocol.addBlock is not idempotent after HDFS-8071.


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

Branch: refs/heads/HDFS-1312
Commit: 45c763ad6171bc7808c2ddcb9099a4215113da2a
Parents: bd909ed
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Tue Jan 26 11:20:13 2016 +0800
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Tue Jan 26 11:20:13 2016 +0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../hdfs/server/namenode/FSDirWriteFileOp.java  | 11 +++---
 .../hadoop/hdfs/TestDFSClientRetries.java       | 36 +++++++++++++++-----
 3 files changed, 35 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/45c763ad/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index a14a1d8..56a85f4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -2696,6 +2696,9 @@ Release 2.7.3 - UNRELEASED
     HDFS-9672. o.a.h.hdfs.TestLeaseRecovery2 fails intermittently (Mingliang Liu
     via jitendra)
 
+    HDFS-9690. ClientProtocol.addBlock is not idempotent after HDFS-8071.
+    (szetszwo)
+
 Release 2.7.2 - 2016-01-25
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45c763ad/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 914fbd9..6ba8e1c 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
@@ -184,17 +184,16 @@ class FSDirWriteFileOp {
     src = fsn.dir.resolvePath(pc, src, pathComponents);
     FileState fileState = analyzeFileState(fsn, src, fileId, clientName,
                                            previous, onRetryBlock);
-    final INodeFile pendingFile = fileState.inode;
-    // Check if the penultimate block is minimally replicated
-    if (!fsn.checkFileProgress(src, pendingFile, false)) {
-      throw new NotReplicatedYetException("Not replicated yet: " + src);
-    }
-
     if (onRetryBlock[0] != null && onRetryBlock[0].getLocations().length > 0) {
       // This is a retry. No need to generate new locations.
       // Use the last block if it has locations.
       return null;
     }
+
+    final INodeFile pendingFile = fileState.inode;
+    if (!fsn.checkFileProgress(src, pendingFile, false)) {
+      throw new NotReplicatedYetException("Not replicated yet: " + src);
+    }
     if (pendingFile.getBlocks().length >= fsn.maxBlocksPerFile) {
       throw new IOException("File has reached the limit on maximum number of"
           + " blocks (" + DFSConfigKeys.DFS_NAMENODE_MAX_BLOCKS_PER_FILE_KEY

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45c763ad/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
index e41c06a..1f783f6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
@@ -432,19 +432,37 @@ public class TestDFSClientRetries {
       // Make the call to addBlock() get called twice, as if it were retried
       // due to an IPC issue.
       doAnswer(new Answer<LocatedBlock>() {
-        @Override
-        public LocatedBlock answer(InvocationOnMock invocation) throws Throwable {
-          LocatedBlock ret = (LocatedBlock) invocation.callRealMethod();
+        private int getBlockCount(LocatedBlock ret) throws IOException {
           LocatedBlocks lb = cluster.getNameNodeRpc().getBlockLocations(src, 0, Long.MAX_VALUE);
-          int blockCount = lb.getLocatedBlocks().size();
           assertEquals(lb.getLastLocatedBlock().getBlock(), ret.getBlock());
-          
+          return lb.getLocatedBlocks().size();
+        }
+
+        @Override
+        public LocatedBlock answer(InvocationOnMock invocation) throws Throwable {
+          LOG.info("Called addBlock: "
+              + Arrays.toString(invocation.getArguments()));
+
+          // call first time
+          // warp NotReplicatedYetException with RemoteException as rpc does.
+          final LocatedBlock ret;
+          try {
+            ret = (LocatedBlock) invocation.callRealMethod();
+          } catch(NotReplicatedYetException e) {
+            throw new RemoteException(e.getClass().getName(), e.getMessage());
+          }
+          final int blockCount = getBlockCount(ret);
+
           // Retrying should result in a new block at the end of the file.
           // (abandoning the old one)
-          LocatedBlock ret2 = (LocatedBlock) invocation.callRealMethod();
-          lb = cluster.getNameNodeRpc().getBlockLocations(src, 0, Long.MAX_VALUE);
-          int blockCount2 = lb.getLocatedBlocks().size();
-          assertEquals(lb.getLastLocatedBlock().getBlock(), ret2.getBlock());
+          // It should not have NotReplicatedYetException.
+          final LocatedBlock ret2;
+          try {
+            ret2 = (LocatedBlock) invocation.callRealMethod();
+          } catch(NotReplicatedYetException e) {
+            throw new AssertionError("Unexpected exception", e);
+          }
+          final int blockCount2 = getBlockCount(ret2);
 
           // We shouldn't have gained an extra block by the RPC.
           assertEquals(blockCount, blockCount2);


[37/50] hadoop git commit: YARN-4643. Container recovery is broken with delegating container runtime. Contributed by Sidharta Seethana

Posted by ar...@apache.org.
YARN-4643. Container recovery is broken with delegating container runtime. Contributed by Sidharta Seethana


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

Branch: refs/heads/HDFS-1312
Commit: 61382ff8fabc76b3a51f227646573cdf367fea1a
Parents: ef343be
Author: Jason Lowe <jl...@apache.org>
Authored: Thu Jan 28 18:59:35 2016 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Thu Jan 28 18:59:35 2016 +0000

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                             | 3 +++
 .../containermanager/launcher/RecoveredContainerLaunch.java | 7 ++++---
 .../yarn/server/nodemanager/TestLinuxContainerExecutor.java | 9 +++++++++
 3 files changed, 16 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/61382ff8/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index c8a8c06..1b57a3d 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -1345,6 +1345,9 @@ Release 2.8.0 - UNRELEASED
     YARN-4520. Finished app info is unnecessarily persisted in NM state-store
     if container is acquired but not lunched on this node. (sandflee via jianeh)
 
+    YARN-4643. Container recovery is broken with delegating container runtime
+    (Sidharta Seethana via jlowe)
+
 Release 2.7.3 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/61382ff8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/RecoveredContainerLaunch.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/RecoveredContainerLaunch.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/RecoveredContainerLaunch.java
index d7b9ae2..66f5a2a 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/RecoveredContainerLaunch.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/RecoveredContainerLaunch.java
@@ -83,9 +83,10 @@ public class RecoveredContainerLaunch extends ContainerLaunch {
         exec.activateContainer(containerId, pidFilePath);
         retCode = exec.reacquireContainer(
             new ContainerReacquisitionContext.Builder()
-            .setUser(container.getUser())
-            .setContainerId(containerId)
-            .build());
+                .setContainer(container)
+                .setUser(container.getUser())
+                .setContainerId(containerId)
+                .build());
       } else {
         LOG.warn("Unable to locate pid file for container " + containerIdStr);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/61382ff8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestLinuxContainerExecutor.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/TestLinuxContainerExecutor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestLinuxContainerExecutor.java
index 58debc9..88ebf8d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestLinuxContainerExecutor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestLinuxContainerExecutor.java
@@ -629,7 +629,16 @@ public class TestLinuxContainerExecutor {
     } catch (IOException e) {
       // expected if LCE isn't setup right, but not necessary for this test
     }
+
+    Container container = mock(Container.class);
+    ContainerLaunchContext context = mock(ContainerLaunchContext.class);
+    HashMap<String, String> env = new HashMap<>();
+
+    when(container.getLaunchContext()).thenReturn(context);
+    when(context.getEnvironment()).thenReturn(env);
+
     lce.reacquireContainer(new ContainerReacquisitionContext.Builder()
+        .setContainer(container)
         .setUser("foouser")
         .setContainerId(cid)
         .build());


[15/50] hadoop git commit: YARN-4520. Finished app info is unnecessarily persisted in NM state-store if container is acquired but not lunched on this node. Contributed by sandflee

Posted by ar...@apache.org.
YARN-4520. Finished app info is unnecessarily persisted in NM state-store if container is acquired but not lunched on this node. Contributed by sandflee


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

Branch: refs/heads/HDFS-1312
Commit: 992dd2f783fc051c32727d4a45a5c61c22bf5640
Parents: d62b4a4
Author: Jian He <ji...@apache.org>
Authored: Mon Jan 25 15:35:51 2016 -0800
Committer: Jian He <ji...@apache.org>
Committed: Mon Jan 25 15:36:14 2016 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                | 3 +++
 .../nodemanager/containermanager/ContainerManagerImpl.java     | 6 ++++++
 2 files changed, 9 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/992dd2f7/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 8ece214..e5049d9 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -1327,6 +1327,9 @@ Release 2.8.0 - UNRELEASED
 
     YARN-4592. Remove unused GetContainerStatus proto. (Chang Li via aajisaka)
 
+    YARN-4520. Finished app info is unnecessarily persisted in NM state-store
+    if container is acquired but not lunched on this node. (sandflee via jianeh)
+
 Release 2.7.3 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/992dd2f7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.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/ContainerManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
index f44de59..d0663d5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
@@ -1310,6 +1310,12 @@ public class ContainerManagerImpl extends CompositeService implements
       CMgrCompletedAppsEvent appsFinishedEvent =
           (CMgrCompletedAppsEvent) event;
       for (ApplicationId appID : appsFinishedEvent.getAppsToCleanup()) {
+        Application app = this.context.getApplications().get(appID);
+        if (app == null) {
+          LOG.warn("couldn't find application " + appID + " while processing"
+              + " FINISH_APPS event");
+          continue;
+        }
         String diagnostic = "";
         if (appsFinishedEvent.getReason() == CMgrCompletedAppsEvent.Reason.ON_SHUTDOWN) {
           diagnostic = "Application killed on shutdown";


[19/50] hadoop git commit: HDFS-8999. Allow a file to be closed with COMMITTED but not yet COMPLETE blocks.

Posted by ar...@apache.org.
HDFS-8999. Allow a file to be closed with COMMITTED but not yet COMPLETE blocks.


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

Branch: refs/heads/HDFS-1312
Commit: bd909ed9f2d853f614f04a50e2230a7932732776
Parents: 2085e60
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Tue Jan 26 10:32:51 2016 +0800
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Tue Jan 26 10:32:51 2016 +0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/DFSClient.java  | 42 ++++++++---
 .../org/apache/hadoop/hdfs/DataStreamer.java    |  3 +-
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |  4 ++
 .../server/blockmanagement/BlockManager.java    | 38 ++++------
 .../hdfs/server/namenode/FSDirAppendOp.java     | 15 +++-
 .../hdfs/server/namenode/FSDirWriteFileOp.java  |  4 +-
 .../hdfs/server/namenode/FSEditLogLoader.java   |  8 +--
 .../hdfs/server/namenode/FSNamesystem.java      | 74 +++++++++++++++-----
 .../hadoop/hdfs/server/namenode/INodeFile.java  | 56 +++++++++++----
 .../hdfs/server/namenode/LeaseManager.java      | 17 ++---
 .../org/apache/hadoop/hdfs/TestFileAppend.java  | 56 ++++++++++++++-
 .../server/namenode/TestFSEditLogLoader.java    | 23 +++---
 .../hdfs/server/namenode/TestFSImage.java       |  4 +-
 .../hdfs/server/namenode/TestINodeFile.java     |  6 +-
 .../hdfs/server/namenode/TestLeaseManager.java  |  4 +-
 16 files changed, 255 insertions(+), 102 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bd909ed9/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index 02ef47e..9c00ea7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -119,6 +119,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.EncryptionZoneIterator;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
@@ -160,10 +161,10 @@ import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.MD5Hash;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.io.retry.LossyRetryInvocationHandler;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.ipc.RetriableException;
 import org.apache.hadoop.ipc.RpcNoSuchMethodException;
 import org.apache.hadoop.net.DNS;
 import org.apache.hadoop.net.NetUtils;
@@ -178,16 +179,15 @@ import org.apache.hadoop.util.DataChecksum.Type;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.Time;
 import org.apache.htrace.core.TraceScope;
+import org.apache.htrace.core.Tracer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.net.InetAddresses;
-import org.apache.htrace.core.Tracer;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /********************************************************
  * DFSClient can connect to a Hadoop Filesystem and
@@ -1291,17 +1291,43 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     }
   }
 
+  /**
+   * Invoke namenode append RPC.
+   * It retries in case of {@link BlockNotYetCompleteException}.
+   */
+  private LastBlockWithStatus callAppend(String src,
+      EnumSetWritable<CreateFlag> flag) throws IOException {
+    final long startTime = Time.monotonicNow();
+    for(;;) {
+      try {
+        return namenode.append(src, clientName, flag);
+      } catch(RemoteException re) {
+        if (Time.monotonicNow() - startTime > 5000
+            || !RetriableException.class.getName().equals(
+                re.getClassName())) {
+          throw re;
+        }
+
+        try { // sleep and retry
+          Thread.sleep(500);
+        } catch (InterruptedException e) {
+          throw DFSUtilClient.toInterruptedIOException("callAppend", e);
+        }
+      }
+    }
+  }
+
   /** Method to get stream returned by append call */
   private DFSOutputStream callAppend(String src, EnumSet<CreateFlag> flag,
       Progressable progress, String[] favoredNodes) throws IOException {
     CreateFlag.validateForAppend(flag);
     try {
-      LastBlockWithStatus blkWithStatus = namenode.append(src, clientName,
+      final LastBlockWithStatus blkWithStatus = callAppend(src,
           new EnumSetWritable<>(flag, CreateFlag.class));
       HdfsFileStatus status = blkWithStatus.getFileStatus();
       if (status == null) {
-        DFSClient.LOG.debug("NameNode is on an older version, request file " +
-            "info with additional RPC call for file: " + src);
+        LOG.debug("NameNode is on an older version, request file " +
+            "info with additional RPC call for file: {}", src);
         status = getFileInfo(src);
       }
       return DFSOutputStream.newStreamForAppend(this, src, flag, progress,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bd909ed9/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
index c249338..bac4d12 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
@@ -475,8 +475,7 @@ class DataStreamer extends Daemon {
     setPipeline(lastBlock);
     if (nodes.length < 1) {
       throw new IOException("Unable to retrieve blocks locations " +
-          " for last block " + block +
-          "of file " + src);
+          " for last block " + block + " of file " + src);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bd909ed9/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 68d5de6..a14a1d8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1910,6 +1910,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-9436. Make NNThroughputBenchmark$BlockReportStats run with 10
     datanodes by default. (Mingliang Liu via shv)
 
+    HDFS-8999. Allow a file to be closed with COMMITTED but not yet COMPLETE
+    blocks.  (szetszwo)
+
   BUG FIXES
 
     HDFS-7501. TransactionsSinceLastCheckpoint can be negative on SBNs.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bd909ed9/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 9dd251f..5217740 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
@@ -203,6 +203,10 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_NAMENODE_REPLICATION_MIN_KEY =
       HdfsClientConfigKeys.DeprecatedKeys.DFS_NAMENODE_REPLICATION_MIN_KEY;
   public static final int     DFS_NAMENODE_REPLICATION_MIN_DEFAULT = 1;
+  public static final String  DFS_NAMENODE_FILE_CLOSE_NUM_COMMITTED_ALLOWED_KEY
+      = "dfs.namenode.file.close.num-committed-allowed";
+  public static final int     DFS_NAMENODE_FILE_CLOSE_NUM_COMMITTED_ALLOWED_DEFAULT
+      = 0;
   public static final String  DFS_NAMENODE_STRIPE_MIN_KEY = "dfs.namenode.stripe.min";
   public static final int     DFS_NAMENODE_STRIPE_MIN_DEFAULT = 1;
   public static final String  DFS_NAMENODE_SAFEMODE_REPLICATION_MIN_KEY =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bd909ed9/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 d255471..a76429e 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
@@ -641,6 +641,10 @@ public class BlockManager implements BlockStatsMXBean {
     }
   }
 
+  public short getMinReplication() {
+    return minReplication;
+  }
+
   public short getMinStorageNum(BlockInfo block) {
     if (block.isStriped()) {
       return ((BlockInfoStriped) block).getRealDataBlockNum();
@@ -703,8 +707,8 @@ public class BlockManager implements BlockStatsMXBean {
     
     final boolean b = commitBlock(lastBlock, commitBlock);
     if (hasMinStorage(lastBlock)) {
-      if (b && !bc.isStriped()) {
-        addExpectedReplicasToPending(lastBlock);
+      if (b) {
+        addExpectedReplicasToPending(lastBlock, bc);
       }
       completeBlock(lastBlock, false);
     }
@@ -716,6 +720,12 @@ public class BlockManager implements BlockStatsMXBean {
    * pendingReplications in order to keep ReplicationMonitor from scheduling
    * the block.
    */
+  public void addExpectedReplicasToPending(BlockInfo blk, BlockCollection bc) {
+    if (!bc.isStriped()) {
+      addExpectedReplicasToPending(blk);
+    }
+  }
+
   private void addExpectedReplicasToPending(BlockInfo lastBlock) {
     DatanodeStorageInfo[] expectedStorages =
         lastBlock.getUnderConstructionFeature().getExpectedStorageLocations();
@@ -2844,9 +2854,7 @@ public class BlockManager implements BlockStatsMXBean {
 
     if(storedBlock.getBlockUCState() == BlockUCState.COMMITTED &&
         hasMinStorage(storedBlock, numLiveReplicas)) {
-      if (!bc.isStriped()) {
-        addExpectedReplicasToPending(storedBlock);
-      }
+      addExpectedReplicasToPending(storedBlock, bc);
       completeBlock(storedBlock, false);
     } else if (storedBlock.isComplete() && result == AddBlockResult.ADDED) {
       // check whether safe replication is reached for the block
@@ -3825,26 +3833,6 @@ public class BlockManager implements BlockStatsMXBean {
     }
   }
 
-  /**
-   * Check that the indicated blocks are present and
-   * replicated.
-   */
-  public boolean checkBlocksProperlyReplicated(
-      String src, BlockInfo[] blocks) {
-    for (BlockInfo b: blocks) {
-      if (!b.isComplete()) {
-        final int numNodes = b.numNodes();
-        final int min = getMinStorageNum(b);
-        final BlockUCState state = b.getBlockUCState();
-        LOG.info("BLOCK* " + b + " is not COMPLETE (ucState = " + state
-            + ", replication# = " + numNodes + (numNodes < min ? " < " : " >= ")
-            + " minimum = " + min + ") in file " + src);
-        return false;
-      }
-    }
-    return true;
-  }
-
   /** 
    * @return 0 if the block is not found;
    *         otherwise, return the replication factor of the block.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bd909ed9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAppendOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAppendOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAppendOp.java
index ae84f39..88d706b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAppendOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAppendOp.java
@@ -33,8 +33,10 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem.RecoverLeaseOp;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion.Feature;
+import org.apache.hadoop.ipc.RetriableException;
 
 import com.google.common.base.Preconditions;
 
@@ -126,10 +128,17 @@ final class FSDirAppendOp {
 
       final BlockInfo lastBlock = file.getLastBlock();
       // Check that the block has at least minimum replication.
-      if (lastBlock != null && lastBlock.isComplete()
+      if (lastBlock != null) {
+        if (lastBlock.getBlockUCState() == BlockUCState.COMMITTED) {
+          throw new RetriableException(
+              new NotReplicatedYetException("append: lastBlock="
+                  + lastBlock + " of src=" + path
+                  + " is COMMITTED but not yet COMPLETE."));
+        } else if (lastBlock.isComplete()
           && !blockManager.isSufficientlyReplicated(lastBlock)) {
-        throw new IOException("append: lastBlock=" + lastBlock + " of src="
-            + path + " is not sufficiently replicated yet.");
+          throw new IOException("append: lastBlock=" + lastBlock + " of src="
+              + path + " is not sufficiently replicated yet.");
+        }
       }
       lb = prepareFileForAppend(fsn, iip, holder, clientMachine, newBlock,
           true, logRetryCache);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bd909ed9/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 0a71d78..914fbd9 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
@@ -789,8 +789,10 @@ class FSDirWriteFileOp {
       return false;
     }
 
+    fsn.addCommittedBlocksToPending(pendingFile);
+
     fsn.finalizeINodeFileUnderConstruction(src, pendingFile,
-        Snapshot.CURRENT_STATE_ID);
+        Snapshot.CURRENT_STATE_ID, true);
     return true;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bd909ed9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
index 5d27786..1a9d7a9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.TruncateOp;
 import static org.apache.hadoop.hdfs.server.namenode.FSImageFormat.renameReservedPathsOnUpgrade;
 import static org.apache.hadoop.util.Time.monotonicNow;
 
@@ -29,7 +28,6 @@ import java.util.EnumMap;
 import java.util.EnumSet;
 import java.util.List;
 
-import com.google.common.collect.Lists;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -94,6 +92,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetStoragePolicyOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetXAttrOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SymlinkOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.TimesOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.TruncateOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateBlocksOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateMasterKeyOp;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
@@ -457,8 +456,9 @@ public class FSEditLogLoader {
       // One might expect that you could use removeLease(holder, path) here,
       // but OP_CLOSE doesn't serialize the holder. So, remove the inode.
       if (file.isUnderConstruction()) {
-        fsNamesys.leaseManager.removeLeases(Lists.newArrayList(file.getId()));
-        file.toCompleteFile(file.getModificationTime());
+        fsNamesys.getLeaseManager().removeLease(file.getId());
+        file.toCompleteFile(file.getModificationTime(), 0,
+            fsNamesys.getBlockManager().getMinReplication());
       }
       break;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bd909ed9/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 afa41c1..1693958 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
@@ -459,6 +459,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
   private final long minBlockSize;         // minimum block size
   final long maxBlocksPerFile;     // maximum # of blocks per file
+  private final int numCommittedAllowed;
 
   /** Lock to protect FSNamesystem. */
   private final FSNamesystemLock fsLock;
@@ -756,6 +757,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           DFSConfigKeys.DFS_NAMENODE_MIN_BLOCK_SIZE_DEFAULT);
       this.maxBlocksPerFile = conf.getLong(DFSConfigKeys.DFS_NAMENODE_MAX_BLOCKS_PER_FILE_KEY,
           DFSConfigKeys.DFS_NAMENODE_MAX_BLOCKS_PER_FILE_DEFAULT);
+      this.numCommittedAllowed = conf.getInt(
+          DFSConfigKeys.DFS_NAMENODE_FILE_CLOSE_NUM_COMMITTED_ALLOWED_KEY,
+          DFSConfigKeys.DFS_NAMENODE_FILE_CLOSE_NUM_COMMITTED_ALLOWED_DEFAULT);
 
       this.dtpReplaceDatanodeOnFailure = ReplaceDatanodeOnFailure.get(conf);
       
@@ -2594,18 +2598,37 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   boolean checkFileProgress(String src, INodeFile v, boolean checkall) {
     assert hasReadLock();
     if (checkall) {
-      return blockManager.checkBlocksProperlyReplicated(src, v
-          .getBlocks());
+      return checkBlocksComplete(src, true, v.getBlocks());
     } else {
-      // check the penultimate block of this file
-      BlockInfo b = v.getPenultimateBlock();
-      return b == null ||
-          blockManager.checkBlocksProperlyReplicated(
-              src, new BlockInfo[] { b });
+      final BlockInfo[] blocks = v.getBlocks();
+      final int i = blocks.length - numCommittedAllowed - 2;
+      return i < 0 || blocks[i] == null
+          || checkBlocksComplete(src, false, blocks[i]);
     }
   }
 
   /**
+   * Check if the blocks are COMPLETE;
+   * it may allow the last block to be COMMITTED.
+   */
+  private boolean checkBlocksComplete(String src, boolean allowCommittedBlock,
+      BlockInfo... blocks) {
+    final int n = allowCommittedBlock? numCommittedAllowed: 0;
+    for(int i = 0; i < blocks.length; i++) {
+      final short min = blockManager.getMinStorageNum(blocks[i]);
+      final String err = INodeFile.checkBlockComplete(blocks, i, n, min);
+      if (err != null) {
+        final int numNodes = blocks[i].numNodes();
+        LOG.info("BLOCK* " + err + "(numNodes= " + numNodes
+            + (numNodes < min ? " < " : " >= ")
+            + " minimum = " + min + ") in file " + src);
+        return false;
+      }
+    }
+    return true;
+  }
+
+  /**
    * Change the indicated filename. 
    * @deprecated Use {@link #renameTo(String, String, boolean,
    * Options.Rename...)} instead.
@@ -2735,7 +2758,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       List<INode> removedINodes,
       final boolean acquireINodeMapLock) {
     assert hasWriteLock();
-    leaseManager.removeLeases(removedUCFiles);
+    for(long i : removedUCFiles) {
+      leaseManager.removeLease(i);
+    }
     // remove inodes from inodesMap
     if (removedINodes != null) {
       if (acquireINodeMapLock) {
@@ -2994,7 +3019,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     // then reap lease immediately and close the file.
     if(nrCompleteBlocks == nrBlocks) {
       finalizeINodeFileUnderConstruction(src, pendingFile,
-          iip.getLatestSnapshotId());
+          iip.getLatestSnapshotId(), false);
       NameNode.stateChangeLog.warn("BLOCK*"
         + " internalReleaseLease: All existing blocks are COMPLETE,"
         + " lease removed, file closed.");
@@ -3033,7 +3058,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       if(penultimateBlockMinStorage &&
           blockManager.hasMinStorage(lastBlock)) {
         finalizeINodeFileUnderConstruction(src, pendingFile,
-            iip.getLatestSnapshotId());
+            iip.getLatestSnapshotId(), false);
         NameNode.stateChangeLog.warn("BLOCK*"
           + " internalReleaseLease: Committed blocks are minimally replicated,"
           + " lease removed, file closed.");
@@ -3077,7 +3102,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         // We can remove this block and close the file.
         pendingFile.removeLastBlock(lastBlock);
         finalizeINodeFileUnderConstruction(src, pendingFile,
-            iip.getLatestSnapshotId());
+            iip.getLatestSnapshotId(), false);
         NameNode.stateChangeLog.warn("BLOCK* internalReleaseLease: "
             + "Removed empty last block and closed file.");
         return true;
@@ -3163,8 +3188,23 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
   }
 
-  void finalizeINodeFileUnderConstruction(
-      String src, INodeFile pendingFile, int latestSnapshot) throws IOException {
+  void addCommittedBlocksToPending(final INodeFile pendingFile) {
+    final BlockInfo[] blocks = pendingFile.getBlocks();
+    int i = blocks.length - numCommittedAllowed;
+    if (i < 0) {
+      i = 0;
+    }
+    for(; i < blocks.length; i++) {
+      final BlockInfo b = blocks[i];
+      if (b != null && b.getBlockUCState() == BlockUCState.COMMITTED) {
+        // b is COMMITTED but not yet COMPLETE, add it to pending replication.
+        blockManager.addExpectedReplicasToPending(b, pendingFile);
+      }
+    }
+  }
+
+  void finalizeINodeFileUnderConstruction(String src, INodeFile pendingFile,
+      int latestSnapshot, boolean allowCommittedBlock) throws IOException {
     assert hasWriteLock();
 
     FileUnderConstructionFeature uc = pendingFile.getFileUnderConstructionFeature();
@@ -3179,7 +3219,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     // The file is no longer pending.
     // Create permanent INode, update blocks. No need to replace the inode here
     // since we just remove the uc feature from pendingFile
-    pendingFile.toCompleteFile(now());
+    pendingFile.toCompleteFile(now(),
+        allowCommittedBlock? numCommittedAllowed: 0,
+        blockManager.getMinReplication());
 
     // close file and persist block allocations for this file
     closeFile(src, pendingFile);
@@ -3412,8 +3454,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     commitOrCompleteLastBlock(pendingFile, iip, storedBlock);
 
     //remove lease, close file
-    finalizeINodeFileUnderConstruction(src, pendingFile,
-        Snapshot.findLatestSnapshot(pendingFile, Snapshot.CURRENT_STATE_ID));
+    int s = Snapshot.findLatestSnapshot(pendingFile, Snapshot.CURRENT_STATE_ID);
+    finalizeINodeFileUnderConstruction(src, pendingFile, s, false);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bd909ed9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
index 962a282..353f29b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
@@ -224,28 +224,56 @@ public class INodeFile extends INodeWithAdditionalFields
    * Convert the file to a complete file, i.e., to remove the Under-Construction
    * feature.
    */
-  public INodeFile toCompleteFile(long mtime) {
-    Preconditions.checkState(isUnderConstruction(),
-        "file is no longer under construction");
-    FileUnderConstructionFeature uc = getFileUnderConstructionFeature();
-    if (uc != null) {
-      assertAllBlocksComplete();
-      removeFeature(uc);
-      this.setModificationTime(mtime);
-    }
-    return this;
+  void toCompleteFile(long mtime, int numCommittedAllowed, short minReplication) {
+    final FileUnderConstructionFeature uc = getFileUnderConstructionFeature();
+    Preconditions.checkNotNull(uc, "File %s is not under construction", this);
+    assertAllBlocksComplete(numCommittedAllowed, minReplication);
+    removeFeature(uc);
+    setModificationTime(mtime);
   }
 
   /** Assert all blocks are complete. */
-  private void assertAllBlocksComplete() {
+  private void assertAllBlocksComplete(int numCommittedAllowed,
+      short minReplication) {
     if (blocks == null) {
       return;
     }
     for (int i = 0; i < blocks.length; i++) {
-      Preconditions.checkState(blocks[i].isComplete(), "Failed to finalize"
-          + " %s %s since blocks[%s] is non-complete, where blocks=%s.",
-          getClass().getSimpleName(), this, i, Arrays.asList(blocks));
+      final String err = checkBlockComplete(blocks, i, numCommittedAllowed,
+          minReplication);
+      Preconditions.checkState(err == null,
+          "Unexpected block state: %s, file=%s (%s), blocks=%s (i=%s)",
+          err, this, getClass().getSimpleName(), Arrays.asList(blocks), i);
+    }
+  }
+
+  /**
+   * Check if the i-th block is COMPLETE;
+   * when the i-th block is the last block, it may be allowed to be COMMITTED.
+   *
+   * @return null if the block passes the check;
+   *              otherwise, return an error message.
+   */
+  static String checkBlockComplete(BlockInfo[] blocks, int i,
+      int numCommittedAllowed, short minReplication) {
+    final BlockInfo b = blocks[i];
+    final BlockUCState state = b.getBlockUCState();
+    if (state == BlockUCState.COMPLETE) {
+      return null;
+    }
+    if (b.isStriped() || i < blocks.length - numCommittedAllowed) {
+      return b + " is " + state + " but not COMPLETE";
     }
+    if (state != BlockUCState.COMMITTED) {
+      return b + " is " + state + " but neither COMPLETE nor COMMITTED";
+    }
+    final int numExpectedLocations
+        = b.getUnderConstructionFeature().getNumExpectedLocations();
+    if (numExpectedLocations <= minReplication) {
+      return b + " is " + state + " but numExpectedLocations = "
+          + numExpectedLocations + " <= minReplication = " + minReplication;
+    }
+    return null;
   }
 
   @Override // BlockCollection

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bd909ed9/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 cec9313..e97aa53 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
@@ -160,6 +160,13 @@ public class LeaseManager {
     return lease;
   }
 
+  synchronized void removeLease(long inodeId) {
+    final Lease lease = leasesById.get(inodeId);
+    if (lease != null) {
+      removeLease(lease, inodeId);
+    }
+  }
+
   /**
    * Remove the specified lease and src.
    */
@@ -298,16 +305,6 @@ public class LeaseManager {
     }
   }
 
-  @VisibleForTesting
-  synchronized void removeLeases(Collection<Long> inodes) {
-    for (long inode : inodes) {
-      Lease lease = leasesById.get(inode);
-      if (lease != null) {
-        removeLease(lease, inode);
-      }
-    }
-  }
-
   public void setLeasePeriod(long softLimit, long hardLimit) {
     this.softLimit = softLimit;
     this.hardLimit = hardLimit; 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bd909ed9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java
index 11d1b18..56b6590 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java
@@ -27,10 +27,12 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.EnumSet;
 import java.util.List;
+import java.util.concurrent.ThreadLocalRandom;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.HardLink;
@@ -41,12 +43,12 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
-import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetTestUtil;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.util.Time;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -55,6 +57,8 @@ import org.junit.Test;
  * support HDFS appends.
  */
 public class TestFileAppend{
+  private static final long RANDOM_TEST_RUNTIME = 10000;
+
   final boolean simulatedStorage = false;
 
   private static byte[] fileContents = null;
@@ -381,6 +385,56 @@ public class TestFileAppend{
     }
   }
 
+
+  @Test
+  public void testMultipleAppends() throws Exception {
+    final long startTime = Time.monotonicNow();
+    final Configuration conf = new HdfsConfiguration();
+    conf.setInt(
+        DFSConfigKeys.DFS_NAMENODE_FILE_CLOSE_NUM_COMMITTED_ALLOWED_KEY, 1);
+
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(4).build();
+    final DistributedFileSystem fs = cluster.getFileSystem();
+    try {
+      final Path p = new Path("/testMultipleAppend/foo");
+      final int blockSize = 1 << 16;
+      final byte[] data = AppendTestUtil.initBuffer(blockSize);
+
+      // create an empty file.
+      fs.create(p, true, 4096, (short)3, blockSize).close();
+
+      int fileLen = 0;
+      for(int i = 0;
+          i < 10 || Time.monotonicNow() - startTime < RANDOM_TEST_RUNTIME;
+          i++) {
+        int appendLen = ThreadLocalRandom.current().nextInt(100) + 1;
+        if (fileLen + appendLen > data.length) {
+          break;
+        }
+
+        AppendTestUtil.LOG.info(i + ") fileLen="  + fileLen
+            + ", appendLen=" + appendLen);
+        final FSDataOutputStream out = fs.append(p);
+        out.write(data, fileLen, appendLen);
+        out.close();
+        fileLen += appendLen;
+      }
+
+      Assert.assertEquals(fileLen, fs.getFileStatus(p).getLen());
+      final byte[] actual = new byte[fileLen];
+      final FSDataInputStream in = fs.open(p);
+      in.readFully(actual);
+      in.close();
+      for(int i = 0; i < fileLen; i++) {
+        Assert.assertEquals(data[i], actual[i]);
+      }
+    } finally {
+      fs.close();
+      cluster.shutdown();
+    }
+  }
+
   /** Tests appending after soft-limit expires. */
   @Test
   public void testAppendAfterSoftLimit() 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bd909ed9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
index 4152712..47c3ace 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hdfs.server.namenode;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-import static org.junit.Assert.assertFalse;
 import static org.mockito.Mockito.doNothing;
 import static org.mockito.Mockito.spy;
 
@@ -42,20 +41,19 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.StripedFileTestUtil;
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogLoader.EditLogValidation;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
-import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.PathUtils;
@@ -473,7 +471,7 @@ public class TestFSEditLogLoader {
       file.toUnderConstruction(clientName, clientMachine);
       file.addBlock(stripedBlk);
       fns.getEditLog().logAddBlock(testFilePath, file);
-      file.toCompleteFile(System.currentTimeMillis());
+      TestINodeFile.toCompleteFile(file);
 
       //If the block by loaded is the same as above it means that
       //we have successfully applied the edit log to the fsimage.
@@ -539,7 +537,7 @@ public class TestFSEditLogLoader {
       file.toUnderConstruction(clientName, clientMachine);
       file.addBlock(stripedBlk);
       fns.getEditLog().logAddBlock(testFilePath, file);
-      file.toCompleteFile(System.currentTimeMillis());
+      TestINodeFile.toCompleteFile(file);
       fns.enterSafeMode(false);
       fns.saveNamespace(0, 0);
       fns.leaveSafeMode(false);
@@ -551,7 +549,7 @@ public class TestFSEditLogLoader {
       file.getLastBlock().setNumBytes(newBlkNumBytes);
       file.getLastBlock().setGenerationStamp(newTimestamp);
       fns.getEditLog().logUpdateBlocks(testFilePath, file, true);
-      file.toCompleteFile(System.currentTimeMillis());
+      TestINodeFile.toCompleteFile(file);
 
       //After the namenode restarts if the block by loaded is the same as above
       //(new block size and timestamp) it means that we have successfully
@@ -616,7 +614,7 @@ public class TestFSEditLogLoader {
       file.toUnderConstruction(clientName, clientMachine);
       file.addBlock(cBlk);
       fns.getEditLog().logAddBlock(testFilePath, file);
-      file.toCompleteFile(System.currentTimeMillis());
+      TestINodeFile.toCompleteFile(file);
       cluster.restartNameNodes();
       cluster.waitActive();
       fns = cluster.getNamesystem();
@@ -662,7 +660,7 @@ public class TestFSEditLogLoader {
       INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath);
       file.toUnderConstruction(clientName, clientMachine);
       file.addBlock(cBlk);
-      file.toCompleteFile(System.currentTimeMillis());
+      TestINodeFile.toCompleteFile(file);
 
       long newBlkNumBytes = 1024*8;
       long newTimestamp = 1426222918+3600;
@@ -671,7 +669,7 @@ public class TestFSEditLogLoader {
       file.getLastBlock().setNumBytes(newBlkNumBytes);
       file.getLastBlock().setGenerationStamp(newTimestamp);
       fns.getEditLog().logUpdateBlocks(testFilePath, file, true);
-      file.toCompleteFile(System.currentTimeMillis());
+      TestINodeFile.toCompleteFile(file);
       cluster.restartNameNodes();
       cluster.waitActive();
       fns = cluster.getNamesystem();
@@ -685,5 +683,4 @@ public class TestFSEditLogLoader {
       }
     }
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bd909ed9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
index 0c7398c..8e39bca 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
@@ -509,7 +509,7 @@ public class TestFSImage {
       INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath);
       file.toUnderConstruction(clientName, clientMachine);
       file.addBlock(cBlk);
-      file.toCompleteFile(System.currentTimeMillis());
+      TestINodeFile.toCompleteFile(file);
       fns.enterSafeMode(false);
       fns.saveNamespace(0, 0);
       cluster.restartNameNodes();
@@ -617,7 +617,7 @@ public class TestFSImage {
       INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath);
       file.toUnderConstruction(clientName, clientMachine);
       file.addBlock(cBlk);
-      file.toCompleteFile(System.currentTimeMillis());
+      TestINodeFile.toCompleteFile(file);
 
       fs.createSnapshot(d,"testHasNonEcBlockUsingStripeID");
       fs.truncate(p,0);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bd909ed9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
index 98e8426..68519ab 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
@@ -94,6 +94,10 @@ public class TestINodeFile {
         (short)3, 1024L);
   }
 
+  static void toCompleteFile(INodeFile file) {
+    file.toCompleteFile(Time.now(), 0, (short)1);
+  }
+
   INodeFile createINodeFile(short replication, long preferredBlockSize) {
     return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
         null, replication, preferredBlockSize);
@@ -1130,7 +1134,7 @@ public class TestINodeFile {
     assertEquals(clientName, uc.getClientName());
     assertEquals(clientMachine, uc.getClientMachine());
 
-    file.toCompleteFile(Time.now());
+    toCompleteFile(file);
     assertFalse(file.isUnderConstruction());
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bd909ed9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestLeaseManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestLeaseManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestLeaseManager.java
index de30161..3bb7bb7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestLeaseManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestLeaseManager.java
@@ -51,8 +51,8 @@ public class TestLeaseManager {
     }
 
     assertEquals(4, lm.getINodeIdWithLeases().size());
-    synchronized (lm) {
-      lm.removeLeases(ids);
+    for (long id : ids) {
+      lm.removeLease(id);
     }
     assertEquals(0, lm.getINodeIdWithLeases().size());
   }


[10/50] hadoop git commit: HADOOP-12731. Remove useless boxing/unboxing code. Contributed by Kousuke Saruta.

Posted by ar...@apache.org.
HADOOP-12731. Remove useless boxing/unboxing code. Contributed by Kousuke Saruta.


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

Branch: refs/heads/HDFS-1312
Commit: 736eb17a796a1c1ad5f4db2c6a64f6752db7bec3
Parents: 2b83329
Author: Akira Ajisaka <aa...@apache.org>
Authored: Mon Jan 25 13:47:29 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Mon Jan 25 13:47:29 2016 +0900

----------------------------------------------------------------------
 .../util/TestZKSignerSecretProvider.java        |  2 +-
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 ++
 .../java/org/apache/hadoop/fs/FileContext.java  |  4 +-
 .../org/apache/hadoop/ha/SshFenceByTcpPort.java |  4 +-
 .../apache/hadoop/io/AbstractMapWritable.java   | 54 +++++++-------------
 .../hadoop/security/ShellBasedIdMapping.java    |  9 ++--
 .../java/org/apache/hadoop/util/bloom/Key.java  |  4 +-
 .../apache/hadoop/ha/TestSshFenceByTcpPort.java |  8 +--
 .../apache/hadoop/test/GenericTestUtils.java    |  2 +-
 .../gridmix/DistributedCacheEmulator.java       |  4 +-
 .../hadoop/mapred/gridmix/CommonJobTest.java    |  2 +-
 .../mapred/gridmix/TestPseudoLocalFs.java       |  2 +-
 .../apache/hadoop/streaming/DelayEchoApp.java   |  2 +-
 13 files changed, 43 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/736eb17a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestZKSignerSecretProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestZKSignerSecretProvider.java b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestZKSignerSecretProvider.java
index 4f8b5ae..8211314 100644
--- a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestZKSignerSecretProvider.java
+++ b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestZKSignerSecretProvider.java
@@ -35,7 +35,7 @@ public class TestZKSignerSecretProvider {
 
   // rollover every 2 sec
   private final int timeout = 4000;
-  private final long rolloverFrequency = Long.valueOf(timeout / 2);
+  private final long rolloverFrequency = timeout / 2;
 
   @Before
   public void setup() throws Exception {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/736eb17a/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 36cac2f..3db68fb 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -1645,6 +1645,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-12730. Hadoop streaming -mapper and -reducer options are wrongly
     documented as required. (Kengo Seki via aajisaka)
 
+    HADOOP-12731. Remove useless boxing/unboxing code.
+    (Kousuke Saruta via aajisaka)
+
 Release 2.7.3 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/736eb17a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
index 2456154..d96abad 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
@@ -761,7 +761,7 @@ public class FileContext {
       @Override
       public Boolean next(final AbstractFileSystem fs, final Path p) 
         throws IOException, UnresolvedLinkException {
-        return Boolean.valueOf(fs.delete(p, recursive));
+        return fs.delete(p, recursive);
       }
     }.resolve(this, absF);
   }
@@ -895,7 +895,7 @@ public class FileContext {
       @Override
       public Boolean next(final AbstractFileSystem fs, final Path p) 
         throws IOException, UnresolvedLinkException {
-        return Boolean.valueOf(fs.setReplication(p, replication));
+        return fs.setReplication(p, replication);
       }
     }.resolve(this, absF);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/736eb17a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/SshFenceByTcpPort.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/SshFenceByTcpPort.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/SshFenceByTcpPort.java
index 90eb915..5815564 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/SshFenceByTcpPort.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/SshFenceByTcpPort.java
@@ -256,10 +256,10 @@ public class SshFenceByTcpPort extends Configured
       }
     }
 
-    private Integer parseConfiggedPort(String portStr)
+    private int parseConfiggedPort(String portStr)
         throws BadFencingConfigurationException {
       try {
-        return Integer.valueOf(portStr);
+        return Integer.parseInt(portStr);
       } catch (NumberFormatException nfe) {
         throw new BadFencingConfigurationException(
             "Port number '" + portStr + "' invalid");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/736eb17a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/AbstractMapWritable.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/AbstractMapWritable.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/AbstractMapWritable.java
index cc1e517..7dd9e69 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/AbstractMapWritable.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/AbstractMapWritable.java
@@ -116,55 +116,39 @@ public abstract class AbstractMapWritable implements Writable, Configurable {
         DataInputBuffer in = new DataInputBuffer();
         in.reset(out.getData(), out.getLength());
         readFields(in);
-        
+
       } catch (IOException e) {
         throw new IllegalArgumentException("map cannot be copied: " +
             e.getMessage());
       }
-      
+
     } else {
       throw new IllegalArgumentException("source map cannot be null");
     }
   }
-  
+
   /** constructor. */
   protected AbstractMapWritable() {
     this.conf = new AtomicReference<Configuration>();
 
-    addToMap(ArrayWritable.class,
-        Byte.valueOf(Integer.valueOf(-127).byteValue())); 
-    addToMap(BooleanWritable.class,
-        Byte.valueOf(Integer.valueOf(-126).byteValue()));
-    addToMap(BytesWritable.class,
-        Byte.valueOf(Integer.valueOf(-125).byteValue()));
-    addToMap(FloatWritable.class,
-        Byte.valueOf(Integer.valueOf(-124).byteValue()));
-    addToMap(IntWritable.class,
-        Byte.valueOf(Integer.valueOf(-123).byteValue()));
-    addToMap(LongWritable.class,
-        Byte.valueOf(Integer.valueOf(-122).byteValue()));
-    addToMap(MapWritable.class,
-        Byte.valueOf(Integer.valueOf(-121).byteValue()));
-    addToMap(MD5Hash.class,
-        Byte.valueOf(Integer.valueOf(-120).byteValue()));
-    addToMap(NullWritable.class,
-        Byte.valueOf(Integer.valueOf(-119).byteValue()));
-    addToMap(ObjectWritable.class,
-        Byte.valueOf(Integer.valueOf(-118).byteValue()));
-    addToMap(SortedMapWritable.class,
-        Byte.valueOf(Integer.valueOf(-117).byteValue()));
-    addToMap(Text.class,
-        Byte.valueOf(Integer.valueOf(-116).byteValue()));
-    addToMap(TwoDArrayWritable.class,
-        Byte.valueOf(Integer.valueOf(-115).byteValue()));
-    
-    // UTF8 is deprecated so we don't support it
+    addToMap(ArrayWritable.class, (byte)-127);
+    addToMap(BooleanWritable.class, (byte)-126);
+    addToMap(BytesWritable.class, (byte)-125);
+    addToMap(FloatWritable.class, (byte)-124);
+    addToMap(IntWritable.class, (byte)-123);
+    addToMap(LongWritable.class, (byte)-122);
+    addToMap(MapWritable.class, (byte)-121);
+    addToMap(MD5Hash.class, (byte)-120);
+    addToMap(NullWritable.class, (byte)-119);
+    addToMap(ObjectWritable.class, (byte)-118);
+    addToMap(SortedMapWritable.class, (byte)-117);
+    addToMap(Text.class, (byte)-116);
+    addToMap(TwoDArrayWritable.class, (byte)-115);
 
-    addToMap(VIntWritable.class,
-        Byte.valueOf(Integer.valueOf(-114).byteValue()));
-    addToMap(VLongWritable.class,
-        Byte.valueOf(Integer.valueOf(-113).byteValue()));
+    // UTF8 is deprecated so we don't support it
 
+    addToMap(VIntWritable.class, (byte)-114);
+    addToMap(VLongWritable.class, (byte)-113);
   }
 
   /** @return the conf */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/736eb17a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ShellBasedIdMapping.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ShellBasedIdMapping.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ShellBasedIdMapping.java
index fd362d0..20bccd5 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ShellBasedIdMapping.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ShellBasedIdMapping.java
@@ -203,9 +203,8 @@ public class ShellBasedIdMapping implements IdMappingServiceProvider {
    * Integer, e.g. 4294967294 maps to -2 and 4294967295 maps to -1.
    */
   private static Integer parseId(final String idStr) {
-    Long longVal = Long.parseLong(idStr);
-    int intVal = longVal.intValue();
-    return Integer.valueOf(intVal);
+    long longVal = Long.parseLong(idStr);
+    return Integer.valueOf((int)longVal);
   }
   
   /**
@@ -605,8 +604,8 @@ public class ShellBasedIdMapping implements IdMappingServiceProvider {
         // We know the line is fine to parse without error checking like this
         // since it matched the regex above.
         String firstComponent = lineMatcher.group(1);
-        int remoteId = parseId(lineMatcher.group(2));
-        int localId = parseId(lineMatcher.group(3));
+        Integer remoteId = parseId(lineMatcher.group(2));
+        Integer localId = parseId(lineMatcher.group(3));
         if (firstComponent.equals("uid")) {
           uidMapping.put(localId, remoteId);
         } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/736eb17a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/bloom/Key.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/bloom/Key.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/bloom/Key.java
index 7ac134c..e8ad18c 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/bloom/Key.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/bloom/Key.java
@@ -177,8 +177,8 @@ public class Key implements WritableComparable<Key> {
     }
     
     if (result == 0) {
-      result = Double.valueOf(this.weight - other.weight).intValue();
+      result = (int)(this.weight - other.weight);
     }
     return result;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/736eb17a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestSshFenceByTcpPort.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestSshFenceByTcpPort.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestSshFenceByTcpPort.java
index 4796fe6..fb238fe 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestSshFenceByTcpPort.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestSshFenceByTcpPort.java
@@ -34,20 +34,20 @@ public class TestSshFenceByTcpPort {
   static {
     ((Log4JLogger)SshFenceByTcpPort.LOG).getLogger().setLevel(Level.ALL);
   }
-  
+
   private static String TEST_FENCING_HOST = System.getProperty(
       "test.TestSshFenceByTcpPort.host", "localhost");
   private static final String TEST_FENCING_PORT = System.getProperty(
       "test.TestSshFenceByTcpPort.port", "8020");
   private static final String TEST_KEYFILE = System.getProperty(
       "test.TestSshFenceByTcpPort.key");
-  
+
   private static final InetSocketAddress TEST_ADDR =
     new InetSocketAddress(TEST_FENCING_HOST,
-      Integer.valueOf(TEST_FENCING_PORT));
+      Integer.parseInt(TEST_FENCING_PORT));
   private static final HAServiceTarget TEST_TARGET =
     new DummyHAService(HAServiceState.ACTIVE, TEST_ADDR);
-  
+
   /**
    *  Connect to Google's DNS server - not running ssh!
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/736eb17a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java
index 9c14bb4..7e494a8 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java
@@ -479,6 +479,6 @@ public abstract class GenericTestUtils {
    */
   public static void assumeInNativeProfile() {
     Assume.assumeTrue(
-        Boolean.valueOf(System.getProperty("runningWithNative", "false")));
+        Boolean.parseBoolean(System.getProperty("runningWithNative", "false")));
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/736eb17a/hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/DistributedCacheEmulator.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/DistributedCacheEmulator.java b/hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/DistributedCacheEmulator.java
index d30a51d..72027c1 100644
--- a/hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/DistributedCacheEmulator.java
+++ b/hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/DistributedCacheEmulator.java
@@ -330,7 +330,7 @@ class DistributedCacheEmulator {
         // Check if visibilities are available because older hadoop versions
         // didn't have public, private Distributed Caches separately.
         boolean visibility =
-            (visibilities == null) ? true : Boolean.valueOf(visibilities[i]);
+            (visibilities == null) || Boolean.parseBoolean(visibilities[i]);
         if (isLocalDistCacheFile(files[i], user, visibility)) {
           // local FS based distributed cache file.
           // Create this file on the pseudo local FS on the fly (i.e. when the
@@ -514,7 +514,7 @@ class DistributedCacheEmulator {
           // Check if visibilities are available because older hadoop versions
           // didn't have public, private Distributed Caches separately.
           boolean visibility =
-            (visibilities == null) ? true : Boolean.valueOf(visibilities[i]);
+              (visibilities == null) || Boolean.parseBoolean(visibilities[i]);
           if (isLocalDistCacheFile(files[i], user, visibility)) {
             // local FS based distributed cache file.
             // Create this file on the pseudo local FS.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/736eb17a/hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/CommonJobTest.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/CommonJobTest.java b/hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/CommonJobTest.java
index 24e5aad..6a45ba7 100644
--- a/hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/CommonJobTest.java
+++ b/hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/CommonJobTest.java
@@ -172,7 +172,7 @@ public class CommonJobTest {
         assertTrue("Gridmix job name is not in the expected format.",
                 jobName.equals(GridmixJob.JOB_NAME_PREFIX + jobSeqNum));
         final FileStatus stat = GridmixTestUtils.dfs.getFileStatus(new Path(
-                GridmixTestUtils.DEST, "" + Integer.valueOf(jobSeqNum)));
+                GridmixTestUtils.DEST, "" + Integer.parseInt(jobSeqNum)));
         assertEquals("Wrong owner for " + jobName, spec.getUser(),
                 stat.getOwner());
         final int nMaps = spec.getNumberMaps();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/736eb17a/hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestPseudoLocalFs.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestPseudoLocalFs.java b/hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestPseudoLocalFs.java
index b9c2728..a607ece 100644
--- a/hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestPseudoLocalFs.java
+++ b/hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestPseudoLocalFs.java
@@ -95,7 +95,7 @@ public class TestPseudoLocalFs {
 
       // validate fileSize
       String[] parts = path.toUri().getPath().split("\\.");
-      long expectedFileSize = Long.valueOf(parts[parts.length - 1]);
+      long expectedFileSize = Long.parseLong(parts[parts.length - 1]);
       assertEquals("Invalid file size.", expectedFileSize, stat.getLen());
     } else {
       assertTrue("getFileStatus() did not throw Exception for invalid file "

http://git-wip-us.apache.org/repos/asf/hadoop/blob/736eb17a/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/DelayEchoApp.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/DelayEchoApp.java b/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/DelayEchoApp.java
index 546b7eb..a912dc9 100644
--- a/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/DelayEchoApp.java
+++ b/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/DelayEchoApp.java
@@ -45,7 +45,7 @@ public class DelayEchoApp {
     int seconds = 5;
     if (args.length >= 1) {
       try {
-        seconds = Integer.valueOf(args[0]);
+        seconds = Integer.parseInt(args[0]);
       } catch (NumberFormatException e) {
         // just use default 5.
       }


[41/50] hadoop git commit: HADOOP-12702. Add an HDFS metrics sink. (Daniel Templeton via kasha)

Posted by ar...@apache.org.
HADOOP-12702. Add an HDFS metrics sink. (Daniel Templeton via kasha)


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

Branch: refs/heads/HDFS-1312
Commit: ee005e010cff3f97a5daa8000ac2cd151e2631ca
Parents: 7f46636
Author: Karthik Kambatla <ka...@cloudera.com>
Authored: Thu Jan 28 17:43:17 2016 -0800
Committer: Karthik Kambatla <ka...@cloudera.com>
Committed: Thu Jan 28 17:43:17 2016 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |   2 +
 .../metrics2/sink/RollingFileSystemSink.java    | 420 +++++++++++++++
 .../sink/RollingFileSystemSinkTestBase.java     | 506 +++++++++++++++++++
 .../sink/TestRollingFileSystemSink.java         | 156 ++++++
 4 files changed, 1084 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee005e01/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 4da20e0..4d01857 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -650,6 +650,8 @@ Release 2.9.0 - UNRELEASED
 
   NEW FEATURES
 
+    HADOOP-12702. Add an HDFS metrics sink. (Daniel Templeton via kasha)
+
   IMPROVEMENTS
 
     HADOOP-12321. Make JvmPauseMonitor an AbstractService.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee005e01/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/RollingFileSystemSink.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/RollingFileSystemSink.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/RollingFileSystemSink.java
new file mode 100644
index 0000000..8271362
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/RollingFileSystemSink.java
@@ -0,0 +1,420 @@
+/**
+ * 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.metrics2.sink;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.net.InetAddress;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.nio.charset.StandardCharsets;
+import java.util.Date;
+import java.util.TimeZone;
+
+import org.apache.commons.configuration.SubsetConfiguration;
+import org.apache.commons.lang.time.FastDateFormat;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+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.metrics2.AbstractMetric;
+import org.apache.hadoop.metrics2.MetricsException;
+import org.apache.hadoop.metrics2.MetricsRecord;
+import org.apache.hadoop.metrics2.MetricsSink;
+import org.apache.hadoop.metrics2.MetricsTag;
+
+/**
+ * This class is a metrics sink that uses
+ * {@link org.apache.hadoop.fs.FileSystem} to write the metrics logs.  Every
+ * hour a new directory will be created under the path specified by the
+ * <code>basepath</code> property. All metrics will be logged to a file in the
+ * current hour's directory in a file named &lt;hostname&gt;.log, where
+ * &lt;hostname&gt; is the name of the host on which the metrics logging
+ * process is running. The base path is set by the
+ * <code>&lt;prefix&gt;.sink.&lt;instance&gt;.basepath</code> property.  The
+ * time zone used to create the current hour's directory name is GMT.  If the
+ * <code>basepath</code> property isn't specified, it will default to
+ * &quot;/tmp&quot;, which is the temp directory on whatever default file
+ * system is configured for the cluster.
+ *
+ * The <code>&lt;prefix&gt;.sink.&lt;instance&gt;.ignore-error</code> property
+ * controls whether an exception is thrown when an error is encountered writing
+ * a log file.  The default value is <code>true</code>.  When set to
+ * <code>false</code>, file errors are quietly swallowed.
+ *
+ * The primary use of this class is for logging to HDFS.  As it uses
+ * {@link org.apache.hadoop.fs.FileSystem} to access the target file system,
+ * however, it can be used to write to the local file system, Amazon S3, or any
+ * other supported file system.  The base path for the sink will determine the
+ * file system used.  An unqualified path will write to the default file system
+ * set by the configuration.
+ *
+ * Not all file systems support the ability to append to files.  In file systems
+ * without the ability to append to files, only one writer can write to a file
+ * at a time.  To allow for concurrent writes from multiple daemons on a single
+ * host, the <code>source</code> property should be set to the name of the
+ * source daemon, e.g. <i>namenode</i>.  The value of the <code>source</code>
+ * property should typically be the same as the property's prefix.  If this
+ * property is not set, the source is taken to be <i>unknown</i>.
+ *
+ * Instead of appending to an existing file, by default the sink
+ * will create a new file with a suffix of &quot;.&lt;n&gt;&quet;, where
+ * <i>n</i> is the next lowest integer that isn't already used in a file name,
+ * similar to the Hadoop daemon logs.  NOTE: the file with the <b>highest</b>
+ * sequence number is the <b>newest</b> file, unlike the Hadoop daemon logs.
+ *
+ * For file systems that allow append, the sink supports appending to the
+ * existing file instead. If the <code>allow-append</code> property is set to
+ * true, the sink will instead append to the existing file on file systems that
+ * support appends. By default, the <code>allow-append</code> property is
+ * false.
+ *
+ * Note that when writing to HDFS with <code>allow-append</code> set to true,
+ * there is a minimum acceptable number of data nodes.  If the number of data
+ * nodes drops below that minimum, the append will succeed, but reading the
+ * data will fail with an IOException in the DataStreamer class.  The minimum
+ * number of data nodes required for a successful append is generally 2 or 3.
+ *
+ * Note also that when writing to HDFS, the file size information is not updated
+ * until the file is closed (e.g. at the top of the hour) even though the data
+ * is being written successfully. This is a known HDFS limitation that exists
+ * because of the performance cost of updating the metadata.  See
+ * <a href="https://issues.apache.org/jira/browse/HDFS-5478">HDFS-5478</a>.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class RollingFileSystemSink implements MetricsSink, Closeable {
+  private static final String BASEPATH_KEY = "basepath";
+  private static final String SOURCE_KEY = "source";
+  private static final String IGNORE_ERROR_KEY = "ignore-error";
+  private static final String ALLOW_APPEND_KEY = "allow-append";
+  private static final String SOURCE_DEFAULT = "unknown";
+  private static final String BASEPATH_DEFAULT = "/tmp";
+  private static final FastDateFormat DATE_FORMAT =
+      FastDateFormat.getInstance("yyyyMMddHH", TimeZone.getTimeZone("GMT"));
+  private String source;
+  private boolean ignoreError;
+  private boolean allowAppend;
+  private Path basePath;
+  private FileSystem fileSystem;
+  // The current directory path into which we're writing files
+  private Path currentDirPath;
+  // The path to the current file into which we're writing data
+  private Path currentFilePath;
+  // The stream to which we're currently writing.
+  private PrintStream currentOutStream;
+  // We keep this only to be able to call hsynch() on it.
+  private FSDataOutputStream currentFSOutStream;
+
+  @Override
+  public void init(SubsetConfiguration conf) {
+    basePath = new Path(conf.getString(BASEPATH_KEY, BASEPATH_DEFAULT));
+    source = conf.getString(SOURCE_KEY, SOURCE_DEFAULT);
+    ignoreError = conf.getBoolean(IGNORE_ERROR_KEY, false);
+    allowAppend = conf.getBoolean(ALLOW_APPEND_KEY, false);
+
+    try {
+      fileSystem = FileSystem.get(new URI(basePath.toString()),
+          new Configuration());
+    } catch (URISyntaxException ex) {
+      throw new MetricsException("The supplied filesystem base path URI"
+          + " is not a valid URI: " + basePath.toString(), ex);
+    } catch (IOException ex) {
+      throw new MetricsException("Error connecting to file system: "
+          + basePath + " [" + ex.toString() + "]", ex);
+    }
+
+    // If we're permitted to append, check if we actually can
+    if (allowAppend) {
+      allowAppend = checkAppend(fileSystem);
+    }
+  }
+
+  /**
+   * Test whether the file system supports append and return the answer.
+   * @param fs the target file system
+   */
+  private boolean checkAppend(FileSystem fs) {
+    boolean canAppend = true;
+
+    try {
+      fs.append(basePath);
+    } catch (IOException ex) {
+      if (ex.getMessage().equals("Not supported")) {
+        canAppend = false;
+      }
+    }
+
+    return canAppend;
+  }
+
+  /**
+   * Check the current directory against the time stamp.  If they're not
+   * the same, create a new directory and a new log file in that directory.
+   *
+   * @throws MetricsException thrown if an error occurs while creating the
+   * new directory or new log file
+   */
+  private void rollLogDirIfNeeded() throws MetricsException {
+    String currentDir = DATE_FORMAT.format(new Date());
+    Path path = new Path(basePath, currentDir);
+
+    // We check whether currentOutStream is null instead of currentDirPath,
+    // because if currentDirPath is null, then currentOutStream is null, but
+    // currentOutStream can be null for other reasons.
+    if ((currentOutStream == null) || !path.equals(currentDirPath)) {
+      currentDirPath = path;
+
+      if (currentOutStream != null) {
+        currentOutStream.close();
+      }
+
+      try {
+        rollLogDir();
+      } catch (IOException ex) {
+        throwMetricsException("Failed to creating new log file", ex);
+      }
+    }
+  }
+
+  /**
+   * Create a new directory based on the current hour and a new log file in
+   * that directory.
+   *
+   * @throws IOException thrown if an error occurs while creating the
+   * new directory or new log file
+   */
+  private void rollLogDir() throws IOException {
+    String fileName =
+        source + "-" + InetAddress.getLocalHost().getHostName() + ".log";
+
+    Path targetFile = new Path(currentDirPath, fileName);
+    fileSystem.mkdirs(currentDirPath);
+
+    if (allowAppend) {
+      createOrAppendLogFile(targetFile);
+    } else {
+      createLogFile(targetFile);
+    }
+  }
+
+  /**
+   * Create a new log file and return the {@link FSDataOutputStream}. If a
+   * file with the specified path already exists, add a suffix, starting with 1
+   * and try again. Keep incrementing the suffix until a nonexistent target
+   * path is found.
+   *
+   * Once the file is open, update {@link #currentFSOutStream},
+   * {@link #currentOutStream}, and {@#link #currentFile} are set appropriately.
+   *
+   * @param initial the target path
+   * @throws IOException thrown if the call to see if the exists fails
+   */
+  private void createLogFile(Path initial) throws IOException {
+    Path currentAttempt = initial;
+    int id = 1;
+
+    while (true) {
+      // First try blindly creating the file. If we fail, it either means
+      // the file exists, or the operation actually failed.  We do it this way
+      // because if we check whether the file exists, it might still be created
+      // by the time we try to create it. Creating first works like a
+      // test-and-set.
+      try {
+        currentFSOutStream = fileSystem.create(currentAttempt, false);
+        currentOutStream = new PrintStream(currentFSOutStream, true,
+            StandardCharsets.UTF_8.name());
+        currentFilePath = currentAttempt;
+        break;
+      } catch (IOException ex) {
+        // Now we can check to see if the file exists to know why we failed
+        if (fileSystem.exists(currentAttempt)) {
+          currentAttempt = new Path(initial.toString() + "." + id);
+          id += 1;
+        } else {
+          throw ex;
+        }
+      }
+    }
+  }
+
+  /**
+   * Create a new log file and return the {@link FSDataOutputStream}. If a
+   * file with the specified path already exists, open the file for append
+   * instead.
+   *
+   * Once the file is open, update {@link #currentFSOutStream},
+   * {@link #currentOutStream}, and {@#link #currentFile} are set appropriately.
+   *
+   * @param initial the target path
+   * @throws IOException thrown if the call to see the append operation fails.
+   */
+  private void createOrAppendLogFile(Path targetFile) throws IOException {
+    // First try blindly creating the file. If we fail, it either means
+    // the file exists, or the operation actually failed.  We do it this way
+    // because if we check whether the file exists, it might still be created
+    // by the time we try to create it. Creating first works like a
+    // test-and-set.
+    try {
+      currentFSOutStream = fileSystem.create(targetFile, false);
+      currentOutStream = new PrintStream(currentFSOutStream, true,
+          StandardCharsets.UTF_8.name());
+    } catch (IOException ex) {
+      // Try appending instead.  If we fail, if means the file doesn't
+      // actually exist yet or the operation actually failed.
+      try {
+        currentFSOutStream = fileSystem.append(targetFile);
+        currentOutStream = new PrintStream(currentFSOutStream, true,
+            StandardCharsets.UTF_8.name());
+      } catch (IOException ex2) {
+        // If the original create failed for a legit but transitory
+        // reason, the append will fail because the file now doesn't exist,
+        // resulting in a confusing stack trace.  To avoid that, we set
+        // the cause of the second exception to be the first exception.
+        // It's still a tiny bit confusing, but it's enough
+        // information that someone should be able to figure it out.
+        ex2.initCause(ex);
+
+        throw ex2;
+      }
+    }
+
+    currentFilePath = targetFile;
+  }
+
+  @Override
+  public void putMetrics(MetricsRecord record) {
+    rollLogDirIfNeeded();
+
+    if (currentOutStream != null) {
+      currentOutStream.printf("%d %s.%s", record.timestamp(),
+          record.context(), record.name());
+
+      String separator = ": ";
+
+      for (MetricsTag tag : record.tags()) {
+        currentOutStream.printf("%s%s=%s", separator, tag.name(), tag.value());
+        separator = ", ";
+      }
+
+      for (AbstractMetric metric : record.metrics()) {
+        currentOutStream.printf("%s%s=%s", separator, metric.name(),
+            metric.value());
+      }
+
+      currentOutStream.println();
+
+      // If we don't hflush(), the data may not be written until the file is
+      // closed. The file won't be closed until the top of the hour *AND*
+      // another record is received. Calling hflush() makes sure that the data
+      // is complete at the top of the hour.
+      try {
+        currentFSOutStream.hflush();
+      } catch (IOException ex) {
+        throwMetricsException("Failed flushing the stream", ex);
+      }
+
+      checkForErrors("Unable to write to log file");
+    } else if (!ignoreError) {
+      throwMetricsException("Unable to write to log file");
+    }
+  }
+
+  @Override
+  public void flush() {
+    // currentOutStream is null if currentFSOutStream is null
+    if (currentFSOutStream != null) {
+      try {
+        currentFSOutStream.hflush();
+      } catch (IOException ex) {
+        throwMetricsException("Unable to flush log file", ex);
+      }
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (currentOutStream != null) {
+      currentOutStream.close();
+
+      try {
+        checkForErrors("Unable to close log file");
+      } finally {
+        // Null out the streams just in case someone tries to reuse us.
+        currentOutStream = null;
+        currentFSOutStream = null;
+      }
+    }
+  }
+
+  /**
+   * If the sink isn't set to ignore errors, throw a {@link MetricsException}
+   * if the stream encountered an exception.  The message parameter will be used
+   * as the new exception's message with the current file name
+   * ({@link #currentFilePath}) appended to it.
+   *
+   * @param message the exception message. The message will have the current
+   * file name ({@link #currentFilePath}) appended to it.
+   * @throws MetricsException thrown if there was an error and the sink isn't
+   * ignoring errors
+   */
+  private void checkForErrors(String message)
+      throws MetricsException {
+    if (!ignoreError && currentOutStream.checkError()) {
+      throw new MetricsException(message + ": " + currentFilePath);
+    }
+  }
+
+  /**
+   * If the sink isn't set to ignore errors, wrap the Throwable in a
+   * {@link MetricsException} and throw it.  The message parameter will be used
+   * as the new exception's message with the current file name
+   * ({@link #currentFilePath}) and the Throwable's string representation
+   * appended to it.
+   *
+   * @param message the exception message. The message will have the current
+   * file name ({@link #currentFilePath}) and the Throwable's string
+   * representation appended to it.
+   * @param t the Throwable to wrap
+   */
+  private void throwMetricsException(String message, Throwable t) {
+    if (!ignoreError) {
+      throw new MetricsException(message + ": " + currentFilePath + " ["
+          + t.toString() + "]", t);
+    }
+  }
+
+  /**
+   * If the sink isn't set to ignore errors, throw a new
+   * {@link MetricsException}.  The message parameter will be used  as the
+   * new exception's message with the current file name
+   * ({@link #currentFilePath}) appended to it.
+   *
+   * @param message the exception message. The message will have the current
+   * file name ({@link #currentFilePath}) appended to it.
+   */
+  private void throwMetricsException(String message) {
+    if (!ignoreError) {
+      throw new MetricsException(message + ": " + currentFilePath);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee005e01/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/sink/RollingFileSystemSinkTestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/sink/RollingFileSystemSinkTestBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/sink/RollingFileSystemSinkTestBase.java
new file mode 100644
index 0000000..3213276
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/sink/RollingFileSystemSinkTestBase.java
@@ -0,0 +1,506 @@
+/*
+ * 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.metrics2.sink;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.net.InetAddress;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.nio.charset.StandardCharsets;
+import java.text.SimpleDateFormat;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.TimeZone;
+import java.util.regex.Pattern;
+import org.apache.commons.io.FileUtils;
+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.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.metrics2.MetricsException;
+import org.apache.hadoop.metrics2.MetricsRecord;
+
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.annotation.Metric;
+import org.apache.hadoop.metrics2.annotation.Metrics;
+import org.apache.hadoop.metrics2.annotation.Metric.Type;
+import org.apache.hadoop.metrics2.impl.ConfigBuilder;
+import org.apache.hadoop.metrics2.impl.MetricsSystemImpl;
+import org.apache.hadoop.metrics2.impl.TestMetricsConfig;
+import org.apache.hadoop.metrics2.lib.MutableGaugeInt;
+import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
+import org.junit.AfterClass;
+import org.junit.Rule;
+import org.junit.rules.TestName;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * This class is a base class for testing the {@link RollingFileSystemSink}
+ * class in various contexts. It provides the a number of useful utility
+ * methods for classes that extend it.
+ */
+public class RollingFileSystemSinkTestBase {
+  protected static final File ROOT_TEST_DIR =
+      new File(System.getProperty("test.build.data", "target/"),
+        "FileSystemSinkTest");
+  protected static final SimpleDateFormat DATE_FORMAT =
+      new SimpleDateFormat("yyyyMMddHH");
+  protected static File methodDir;
+
+  /**
+   * The name of the current test method.
+   */
+  @Rule
+  public TestName methodName = new TestName();
+
+  /**
+   * A sample metric class
+   */
+  @Metrics(name="testRecord1", context="test1")
+  protected class MyMetrics1 {
+    @Metric(value={"testTag1", ""}, type=Type.TAG)
+    String testTag1() { return "testTagValue1"; }
+
+    @Metric(value={"testTag2", ""}, type=Type.TAG)
+    String gettestTag2() { return "testTagValue2"; }
+
+    @Metric(value={"testMetric1", "An integer gauge"}, always=true)
+    MutableGaugeInt testMetric1;
+
+    @Metric(value={"testMetric2", "A long gauge"}, always=true)
+    MutableGaugeLong testMetric2;
+
+    public MyMetrics1 registerWith(MetricsSystem ms) {
+      return ms.register(methodName.getMethodName() + "-m1", null, this);
+    }
+  }
+
+  /**
+   * Another sample metrics class
+   */
+  @Metrics(name="testRecord2", context="test1")
+  protected class MyMetrics2 {
+    @Metric(value={"testTag22", ""}, type=Type.TAG)
+    String testTag1() { return "testTagValue22"; }
+
+    public MyMetrics2 registerWith(MetricsSystem ms) {
+      return ms.register(methodName.getMethodName() + "-m2", null, this);
+    }
+  }
+
+  /**
+   * Set the date format's timezone to GMT.
+   */
+  @BeforeClass
+  public static void setTZ() {
+    DATE_FORMAT.setTimeZone(TimeZone.getTimeZone("GMT"));
+  }
+
+  /**
+   * Delete the test directory for this test.
+   * @throws IOException thrown if the delete fails
+   */
+  @AfterClass
+  public static void deleteBaseDir() throws IOException {
+    FileUtils.deleteDirectory(ROOT_TEST_DIR);
+  }
+
+  /**
+   * Create the test directory for this test.
+   * @throws IOException thrown if the create fails
+   */
+  @Before
+  public void createMethodDir() throws IOException {
+    methodDir = new File(ROOT_TEST_DIR, methodName.getMethodName());
+
+    methodDir.mkdirs();
+  }
+
+  /**
+   * Set up the metrics system, start it, and return it.
+   * @param path the base path for the sink
+   * @param ignoreErrors whether the sink should ignore errors
+   * @param allowAppend whether the sink is allowed to append to existing files
+   * @return the metrics system
+   */
+  protected MetricsSystem initMetricsSystem(String path, boolean ignoreErrors,
+      boolean allowAppend) {
+    // If the prefix is not lower case, the metrics system won't be able to
+    // read any of the properties.
+    final String prefix = methodName.getMethodName().toLowerCase();
+
+    new ConfigBuilder().add("*.period", 10000)
+        .add(prefix + ".sink.mysink0.class", ErrorSink.class.getName())
+        .add(prefix + ".sink.mysink0.basepath", path)
+        .add(prefix + ".sink.mysink0.source", "testsrc")
+        .add(prefix + ".sink.mysink0.context", "test1")
+        .add(prefix + ".sink.mysink0.ignore-error", ignoreErrors)
+        .add(prefix + ".sink.mysink0.allow-append", allowAppend)
+        .save(TestMetricsConfig.getTestFilename("hadoop-metrics2-" + prefix));
+
+    MetricsSystemImpl ms = new MetricsSystemImpl(prefix);
+
+    ms.start();
+
+    return ms;
+  }
+
+  /**
+   * Helper method that writes metrics files to a target path, reads those
+   * files, and returns the contents of all files as a single string. This
+   * method will assert that the correct number of files is found.
+   *
+   * @param ms an initialized MetricsSystem to use
+   * @param path the target path from which to read the logs
+   * @param count the number of log files to expect
+   * @return the contents of the log files
+   * @throws IOException when the log file can't be read
+   * @throws URISyntaxException when the target path is an invalid URL
+   */
+  protected String doWriteTest(MetricsSystem ms, String path, int count)
+      throws IOException, URISyntaxException {
+    final String then = DATE_FORMAT.format(new Date());
+
+    MyMetrics1 mm1 = new MyMetrics1().registerWith(ms);
+    new MyMetrics2().registerWith(ms);
+
+    mm1.testMetric1.incr();
+    mm1.testMetric2.incr(2);
+
+    ms.publishMetricsNow(); // publish the metrics
+    ms.stop();
+    ms.shutdown();
+
+    return readLogFile(path, then, count);
+  }
+
+  /**
+   * Read the log files at the target path and return the contents as a single
+   * string. This method will assert that the correct number of files is found.
+   *
+   * @param path the target path
+   * @param then when the test method began. Used to find the log directory in
+   * the case that the test run crosses the top of the hour.
+   * @param count the number of log files to expect
+   * @return
+   * @throws IOException
+   * @throws URISyntaxException
+   */
+  protected String readLogFile(String path, String then, int count)
+      throws IOException, URISyntaxException {
+    final String now = DATE_FORMAT.format(new Date());
+    final String logFile =
+        "testsrc-" + InetAddress.getLocalHost().getHostName() + ".log";
+    FileSystem fs = FileSystem.get(new URI(path), new Configuration());
+    StringBuilder metrics = new StringBuilder();
+    boolean found = false;
+
+    for (FileStatus status : fs.listStatus(new Path(path))) {
+      Path logDir = status.getPath();
+
+      // There are only two possible valid log directory names: the time when
+      // the test started and the current time.  Anything else can be ignored.
+      if (now.equals(logDir.getName()) || then.equals(logDir.getName())) {
+        readLogData(fs, findMostRecentLogFile(fs, new Path(logDir, logFile)),
+            metrics);
+        assertFileCount(fs, logDir, count);
+        found = true;
+      }
+    }
+
+    assertTrue("No valid log directories found", found);
+
+    return metrics.toString();
+  }
+
+  /**
+   * Read the target log file and append its contents to the StringBuilder.
+   * @param fs the target FileSystem
+   * @param logFile the target file path
+   * @param metrics where to append the file contents
+   * @throws IOException thrown if the file cannot be read
+   */
+  protected void readLogData(FileSystem fs, Path logFile, StringBuilder metrics)
+      throws IOException {
+    FSDataInputStream fsin = fs.open(logFile);
+    BufferedReader in = new BufferedReader(new InputStreamReader(fsin,
+        StandardCharsets.UTF_8));
+    String line = null;
+
+    while ((line = in.readLine()) != null) {
+      metrics.append(line).append("\n");
+    }
+  }
+
+  /**
+   * Return the path to the log file to use, based on the target path.
+   * @param fs the target FileSystem
+   * @param initial the path from which to start
+   * @return the path to use
+   * @throws IOException thrown if testing for file existence fails.
+   */
+  protected Path findMostRecentLogFile(FileSystem fs, Path initial)
+      throws IOException {
+    Path logFile = null;
+    Path nextLogFile = initial;
+    int id = 1;
+
+    do {
+      logFile = nextLogFile;
+      nextLogFile = new Path(initial.toString() + "." + id);
+      id += 1;
+    } while (fs.exists(nextLogFile));
+    return logFile;
+  }
+
+  /**
+   * Assert that the given contents match what is expected from the test
+   * metrics.
+   *
+   * @param contents the file contents to test
+   */
+  protected void assertMetricsContents(String contents) {
+    // Note that in the below expression we allow tags and metrics to go in
+    // arbitrary order, but the records must be in order.
+    final Pattern expectedContentPattern = Pattern.compile(
+        "^\\d+\\s+test1.testRecord1:\\s+Context=test1,\\s+"
+        + "(testTag1=testTagValue1,\\s+testTag2=testTagValue2|"
+        + "testTag2=testTagValue2,\\s+testTag1=testTagValue1),"
+        + "\\s+Hostname=.*,\\s+"
+        + "(testMetric1=1,\\s+testMetric2=2|testMetric2=2,\\s+testMetric1=1)"
+        + "[\\n\\r]*^\\d+\\s+test1.testRecord2:\\s+Context=test1,"
+        + "\\s+testTag22=testTagValue22,\\s+Hostname=.*$[\\n\\r]*",
+         Pattern.MULTILINE);
+
+    assertTrue("Sink did not produce the expected output. Actual output was: "
+        + contents, expectedContentPattern.matcher(contents).matches());
+  }
+
+  /**
+   * Assert that the given contents match what is expected from the test
+   * metrics when there is pre-existing data.
+   *
+   * @param contents the file contents to test
+   */
+  protected void assertExtraContents(String contents) {
+    // Note that in the below expression we allow tags and metrics to go in
+    // arbitrary order, but the records must be in order.
+    final Pattern expectedContentPattern = Pattern.compile(
+        "Extra stuff[\\n\\r]*"
+        + "^\\d+\\s+test1.testRecord1:\\s+Context=test1,\\s+"
+        + "(testTag1=testTagValue1,\\s+testTag2=testTagValue2|"
+        + "testTag2=testTagValue2,\\s+testTag1=testTagValue1),"
+        + "\\s+Hostname=.*,\\s+"
+        + "(testMetric1=1,\\s+testMetric2=2|testMetric2=2,\\s+testMetric1=1)"
+        + "[\\n\\r]*^\\d+\\s+test1.testRecord2:\\s+Context=test1,"
+        + "\\s+testTag22=testTagValue22,\\s+Hostname=.*$[\\n\\r]*",
+         Pattern.MULTILINE);
+
+    assertTrue("Sink did not produce the expected output. Actual output was: "
+        + contents, expectedContentPattern.matcher(contents).matches());
+  }
+
+  /**
+   * Call {@link #doWriteTest} after pre-creating the log file and filling it
+   * with junk data.
+   *
+   * @param path the base path for the test
+   * @param ignoreErrors whether to ignore errors
+   * @param allowAppend whether to allow appends
+   * @param count the number of files to expect
+   * @return the contents of the final log file
+   * @throws IOException if a file system operation fails
+   * @throws InterruptedException if interrupted while calling
+   * {@link #getNowNotTopOfHour()}
+   * @throws URISyntaxException if the path is not a valid URI
+   */
+  protected String doAppendTest(String path, boolean ignoreErrors,
+      boolean allowAppend, int count)
+      throws IOException, InterruptedException, URISyntaxException {
+    preCreateLogFile(path);
+
+    return doWriteTest(initMetricsSystem(path, ignoreErrors, allowAppend),
+        path, count);
+  }
+
+  /**
+   * Create a file at the target path with some known data in it:
+   * &quot;Extra stuff&quot;.
+   *
+   * If the test run is happening within 20 seconds of the top of the hour,
+   * this method will sleep until the top of the hour.
+   *
+   * @param path the target path under which to create the directory for the
+   * current hour that will contain the log file.
+   *
+   * @throws IOException thrown if the file creation fails
+   * @throws InterruptedException thrown if interrupted while waiting for the
+   * top of the hour.
+   * @throws URISyntaxException thrown if the path isn't a valid URI
+   */
+  protected void preCreateLogFile(String path)
+      throws IOException, InterruptedException, URISyntaxException {
+    preCreateLogFile(path, 1);
+  }
+
+  /**
+   * Create files at the target path with some known data in them.  Each file
+   * will have the same content: &quot;Extra stuff&quot;.
+   *
+   * If the test run is happening within 20 seconds of the top of the hour,
+   * this method will sleep until the top of the hour.
+   *
+   * @param path the target path under which to create the directory for the
+   * current hour that will contain the log files.
+   * @param numFiles the number of log files to create
+   * @throws IOException thrown if the file creation fails
+   * @throws InterruptedException thrown if interrupted while waiting for the
+   * top of the hour.
+   * @throws URISyntaxException thrown if the path isn't a valid URI
+   */
+  protected void preCreateLogFile(String path, int numFiles)
+      throws IOException, InterruptedException, URISyntaxException {
+    Calendar now = getNowNotTopOfHour();
+
+    FileSystem fs = FileSystem.get(new URI(path), new Configuration());
+    Path dir = new Path(path, DATE_FORMAT.format(now.getTime()));
+
+    fs.mkdirs(dir);
+
+    Path file = new Path(dir,
+        "testsrc-" + InetAddress.getLocalHost().getHostName() + ".log");
+
+    // Create the log file to force the sink to append
+    try (FSDataOutputStream out = fs.create(file)) {
+      out.write("Extra stuff\n".getBytes());
+      out.flush();
+    }
+
+    if (numFiles > 1) {
+      int count = 1;
+
+      while (count < numFiles) {
+        file = new Path(dir, "testsrc-"
+            + InetAddress.getLocalHost().getHostName() + ".log." + count);
+
+        // Create the log file to force the sink to append
+        try (FSDataOutputStream out = fs.create(file)) {
+          out.write("Extra stuff\n".getBytes());
+          out.flush();
+        }
+
+        count += 1;
+      }
+    }
+  }
+
+  /**
+   * Return a calendar based on the current time.  If the current time is very
+   * near the top of the hour (less than 20 seconds), sleep until the new hour
+   * before returning a new Calendar instance.
+   *
+   * @return a new Calendar instance that isn't near the top of the hour
+   * @throws InterruptedException if interrupted while sleeping
+   */
+  public Calendar getNowNotTopOfHour() throws InterruptedException {
+    Calendar now = Calendar.getInstance(TimeZone.getTimeZone("GMT"));
+
+    // If we're at the very top of the hour, sleep until the next hour
+    // so that we don't get confused by the directory rolling
+    if ((now.get(Calendar.MINUTE) == 59) && (now.get(Calendar.SECOND) > 40)) {
+      Thread.sleep((61 - now.get(Calendar.SECOND)) * 1000L);
+      now.setTime(new Date());
+    }
+
+    return now;
+  }
+
+  /**
+   * Assert that the number of log files in the target directory is as expected.
+   * @param fs the target FileSystem
+   * @param dir the target directory path
+   * @param expected the expected number of files
+   * @throws IOException thrown if listing files fails
+   */
+  public void assertFileCount(FileSystem fs, Path dir, int expected)
+      throws IOException {
+    RemoteIterator<LocatedFileStatus> i = fs.listFiles(dir, true);
+    int count = 0;
+
+    while (i.hasNext()) {
+      i.next();
+      count++;
+    }
+
+    assertTrue("The sink created additional unexpected log files. " + count
+        + "files were created", expected >= count);
+    assertTrue("The sink created too few log files. " + count + "files were "
+        + "created", expected <= count);
+  }
+
+  /**
+   * This class is a {@link RollingFileSystemSink} wrapper that tracks whether
+   * an exception has been thrown during operations.
+   */
+  public static class ErrorSink extends RollingFileSystemSink {
+    public static volatile boolean errored = false;
+
+    @Override
+    public void putMetrics(MetricsRecord record) {
+      try {
+        super.putMetrics(record);
+      } catch (MetricsException ex) {
+        errored = true;
+
+        throw new MetricsException(ex);
+      }
+    }
+
+    @Override
+    public void close() throws IOException {
+      try {
+        super.close();
+      } catch (MetricsException ex) {
+        errored = true;
+
+        throw new MetricsException(ex);
+      }
+    }
+
+    @Override
+    public void flush() {
+      try {
+        super.flush();
+      } catch (MetricsException ex) {
+        errored = true;
+
+        throw new MetricsException(ex);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee005e01/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/sink/TestRollingFileSystemSink.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/sink/TestRollingFileSystemSink.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/sink/TestRollingFileSystemSink.java
new file mode 100644
index 0000000..da63235
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/sink/TestRollingFileSystemSink.java
@@ -0,0 +1,156 @@
+/*
+ * 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.metrics2.sink;
+
+import org.apache.hadoop.metrics2.MetricsSystem;
+
+import org.junit.Test;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test the {@link RollingFileSystemSink} class in the context of the local file
+ * system.
+ */
+public class TestRollingFileSystemSink extends RollingFileSystemSinkTestBase {
+  /**
+   * Test writing logs to the local file system.
+   * @throws Exception when things break
+   */
+  @Test
+  public void testWrite() throws Exception {
+    String path = methodDir.getAbsolutePath();
+    MetricsSystem ms = initMetricsSystem(path, false, false);
+
+    assertMetricsContents(doWriteTest(ms, path, 1));
+  }
+
+  /**
+   * Test writing logs to the local file system with the sink set to ignore
+   * errors.
+   * @throws Exception when things break
+   */
+  @Test
+  public void testSilentWrite() throws Exception {
+    String path = methodDir.getAbsolutePath();
+    MetricsSystem ms = initMetricsSystem(path, true, false);
+
+    assertMetricsContents(doWriteTest(ms, path, 1));
+  }
+
+  /**
+   * Test writing logs to HDFS when the log file already exists.
+   *
+   * @throws Exception when things break
+   */
+  @Test
+  public void testExistingWrite() throws Exception {
+    String path = methodDir.getAbsolutePath();
+
+    assertMetricsContents(doAppendTest(path, false, false, 2));
+  }
+
+  /**
+   * Test writing logs to HDFS when the log file and the .1 log file already
+   * exist.
+   *
+   * @throws Exception when things break
+   */
+  @Test
+  public void testExistingWrite2() throws Exception {
+    String path = methodDir.getAbsolutePath();
+    MetricsSystem ms = initMetricsSystem(path, false, false);
+
+    preCreateLogFile(path, 2);
+
+    assertMetricsContents(doWriteTest(ms, path, 3));
+  }
+
+  /**
+   * Test writing logs to HDFS with ignore errors enabled when
+   * the log file already exists.
+   *
+   * @throws Exception when things break
+   */
+  @Test
+  public void testSilentExistingWrite() throws Exception {
+    String path = methodDir.getAbsolutePath();
+
+    assertMetricsContents(doAppendTest(path, false, false, 2));
+  }
+
+  /**
+   * Test that writing fails when the directory isn't writable.
+   */
+  @Test
+  public void testFailedWrite() {
+    String path = methodDir.getAbsolutePath();
+    MetricsSystem ms = initMetricsSystem(path, false, false);
+
+    new MyMetrics1().registerWith(ms);
+
+    methodDir.setWritable(false);
+    ErrorSink.errored = false;
+
+    try {
+      // publish the metrics
+      ms.publishMetricsNow();
+
+      assertTrue("No exception was generated while writing metrics "
+          + "even though the target directory was not writable",
+          ErrorSink.errored);
+
+      ms.stop();
+      ms.shutdown();
+    } finally {
+      // Make sure the dir is writable again so we can delete it at the end
+      methodDir.setWritable(true);
+    }
+  }
+
+  /**
+   * Test that writing fails silently when the directory is not writable.
+   */
+  @Test
+  public void testSilentFailedWrite() {
+    String path = methodDir.getAbsolutePath();
+    MetricsSystem ms = initMetricsSystem(path, true, false);
+
+    new MyMetrics1().registerWith(ms);
+
+    methodDir.setWritable(false);
+    ErrorSink.errored = false;
+
+    try {
+      // publish the metrics
+      ms.publishMetricsNow();
+
+      assertFalse("An exception was generated while writing metrics "
+          + "when the target directory was not writable, even though the "
+          + "sink is set to ignore errors",
+          ErrorSink.errored);
+
+      ms.stop();
+      ms.shutdown();
+    } finally {
+      // Make sure the dir is writable again so we can delete it at the end
+      methodDir.setWritable(true);
+    }
+  }
+}


[06/50] hadoop git commit: YARN-4614. Fix random failure in TestApplicationPriority#testApplicationPriorityAllocationWithChangeInPriority. (Sunil G via rohithsharmaks)

Posted by ar...@apache.org.
YARN-4614. Fix random failure in TestApplicationPriority#testApplicationPriorityAllocationWithChangeInPriority. (Sunil G via rohithsharmaks)


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

Branch: refs/heads/HDFS-1312
Commit: 99829eb221482928d8a1b148ae3c802cc7c9253e
Parents: 618bfd6
Author: rohithsharmaks <ro...@apache.org>
Authored: Sat Jan 23 07:56:15 2016 +0530
Committer: rohithsharmaks <ro...@apache.org>
Committed: Sat Jan 23 07:56:57 2016 +0530

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                   | 3 +++
 .../scheduler/capacity/TestApplicationPriority.java               | 1 -
 2 files changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/99829eb2/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 8e87f4a..1e9f83c 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -1319,6 +1319,9 @@ Release 2.8.0 - UNRELEASED
     YARN-4605. Spelling mistake in the help message of "yarn applicationattempt"
     command. (Weiwei Yang via aajisaka)
 
+    YARN-4614. Fix random failure in TestApplicationPriority#testApplicationPriority
+    AllocationWithChangeInPriority. (Sunil G via rohithsharmaks)
+
 Release 2.7.3 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99829eb2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationPriority.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/TestApplicationPriority.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/TestApplicationPriority.java
index e32a33b..1569a12 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/TestApplicationPriority.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/TestApplicationPriority.java
@@ -490,7 +490,6 @@ public class TestApplicationPriority {
     RMApp app2 = rm.submitApp(1 * GB, appPriority2);
 
     // kick the scheduler, 1 GB which was free is given to AM of App2
-    nm1.nodeHeartbeat(true);
     MockAM am2 = MockRM.launchAM(app2, rm, nm1);
     am2.registerAppAttempt();
 


[43/50] hadoop git commit: YARN-4219. addendum patch to fix javadoc errors

Posted by ar...@apache.org.
YARN-4219. addendum patch to fix javadoc errors


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

Branch: refs/heads/HDFS-1312
Commit: 09d831c95ba18e2892cddd749f6e06f112dda7f5
Parents: f67149a
Author: Rohith Sharma K S <ro...@apache.org>
Authored: Fri Jan 29 11:51:47 2016 +0530
Committer: Rohith Sharma K S <ro...@apache.org>
Committed: Fri Jan 29 11:51:47 2016 +0530

----------------------------------------------------------------------
 .../hadoop/yarn/server/timeline/LevelDBCacheTimelineStore.java   | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/09d831c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/LevelDBCacheTimelineStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/LevelDBCacheTimelineStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/LevelDBCacheTimelineStore.java
index 976241f..3ff5dd7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/LevelDBCacheTimelineStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/LevelDBCacheTimelineStore.java
@@ -46,11 +46,11 @@ import java.util.Map;
  * There are two partitions of the key space. One partition is to store a
  * entity id to start time mapping:
  *
- * i!ENTITY_ID!ENTITY_TYPE -> ENTITY_START_TIME
+ * i!ENTITY_ID!ENTITY_TYPE to ENTITY_START_TIME
  *
  * The other partition is to store the actual data:
  *
- * e!START_TIME!ENTITY_ID!ENTITY_TYPE -> ENTITY_BYTES
+ * e!START_TIME!ENTITY_ID!ENTITY_TYPE to ENTITY_BYTES
  *
  * This storage does not have any garbage collection mechanism, and is designed
  * mainly for caching usages.


[11/50] hadoop git commit: HADOOP-12736. TestTimedOutTestsListener#testThreadDumpAndDeadlocks sometimes times out. Contributed by Xiao Chen.

Posted by ar...@apache.org.
HADOOP-12736. TestTimedOutTestsListener#testThreadDumpAndDeadlocks sometimes times out. Contributed by Xiao Chen.


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

Branch: refs/heads/HDFS-1312
Commit: 643227927a7d7974655627d7e97aae42600692ae
Parents: 736eb17
Author: Akira Ajisaka <aa...@apache.org>
Authored: Mon Jan 25 13:59:25 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Mon Jan 25 13:59:25 2016 +0900

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt                   | 3 +++
 .../java/org/apache/hadoop/test/TestTimedOutTestsListener.java    | 2 +-
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/64322792/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 3db68fb..2fc8ab4 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -2493,6 +2493,9 @@ Release 2.6.4 - UNRELEASED
     HADOOP-12706. TestLocalFsFCStatistics#testStatisticsThreadLocalDataCleanUp
     times out occasionally (Sangjin Lee and Colin Patrick McCabe via jlowe)
 
+    HADOOP-12736. TestTimedOutTestsListener#testThreadDumpAndDeadlocks
+    sometimes times out. (Xiao Chen via aajisaka)
+
 Release 2.6.3 - 2015-12-17
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/64322792/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestTimedOutTestsListener.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestTimedOutTestsListener.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestTimedOutTestsListener.java
index 62748b4..1334f1c 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestTimedOutTestsListener.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestTimedOutTestsListener.java
@@ -144,7 +144,7 @@ public class TestTimedOutTestsListener {
   
   }
 
-  @Test(timeout=500)
+  @Test(timeout=30000)
   public void testThreadDumpAndDeadlocks() throws Exception {
     new Deadlock();
     String s = null;


[25/50] hadoop git commit: YARN-4573. Fix test failure in TestRMAppTransitions#testAppRunningKill and testAppKilledKilled. (Takashi Ohnishi via rohithsharmaks)

Posted by ar...@apache.org.
YARN-4573. Fix test failure in TestRMAppTransitions#testAppRunningKill and testAppKilledKilled. (Takashi Ohnishi via rohithsharmaks)


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

Branch: refs/heads/HDFS-1312
Commit: c01bee010832ca31d8e60e5461181cdf05140602
Parents: 4efdf3a
Author: Rohith Sharma K S <ro...@apache.org>
Authored: Wed Jan 27 08:23:02 2016 +0530
Committer: Rohith Sharma K S <ro...@apache.org>
Committed: Wed Jan 27 08:23:02 2016 +0530

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                             | 3 +++
 .../server/resourcemanager/rmapp/TestRMAppTransitions.java  | 9 +++++----
 2 files changed, 8 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c01bee01/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 435eb68..2fbecdb 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -162,6 +162,9 @@ Release 2.9.0 - UNRELEASED
     YARN-4612. Fix rumen and scheduler load simulator handle killed tasks properly.
     (Ming Ma via xgong)
 
+    YARN-4573. Fix test failure in TestRMAppTransitions#testAppRunningKill and
+    testAppKilledKilled. (Takashi Ohnishi via rohithsharmaks)
+
 Release 2.8.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c01bee01/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.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/TestRMAppTransitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
index f2f09de..293c0b6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
@@ -275,7 +275,7 @@ public class TestRMAppTransitions {
   // Test expected newly created app state
   private static void testAppStartState(ApplicationId applicationId, 
       String user, String name, String queue, RMApp application) {
-    Assert.assertTrue("application start time is not greater then 0", 
+    Assert.assertTrue("application start time is not greater than 0",
         application.getStartTime() > 0);
     Assert.assertTrue("application start time is before currentTime", 
         application.getStartTime() <= System.currentTimeMillis());
@@ -300,7 +300,7 @@ public class TestRMAppTransitions {
 
   // test to make sure times are set when app finishes
   private static void assertStartTimeSet(RMApp application) {
-    Assert.assertTrue("application start time is not greater then 0", 
+    Assert.assertTrue("application start time is not greater than 0",
         application.getStartTime() > 0);
     Assert.assertTrue("application start time is before currentTime", 
         application.getStartTime() <= System.currentTimeMillis());
@@ -319,9 +319,9 @@ public class TestRMAppTransitions {
   // test to make sure times are set when app finishes
   private void assertTimesAtFinish(RMApp application) {
     assertStartTimeSet(application);
-    Assert.assertTrue("application finish time is not greater then 0",
+    Assert.assertTrue("application finish time is not greater than 0",
         (application.getFinishTime() > 0));
-    Assert.assertTrue("application finish time is not >= then start time",
+    Assert.assertTrue("application finish time is not >= than start time",
         (application.getFinishTime() >= application.getStartTime()));
   }
 
@@ -364,6 +364,7 @@ public class TestRMAppTransitions {
     application.getCurrentAppAttempt().handle(
         new RMAppAttemptEvent(application.getCurrentAppAttempt().getAppAttemptId(),
             RMAppAttemptEventType.ATTEMPT_UPDATE_SAVED));
+    rmDispatcher.await();
   }
 
   protected RMApp testCreateAppNewSaving(


[05/50] hadoop git commit: YARN-4496. Improve HA ResourceManager Failover detection on the client. Contributed by Jian He

Posted by ar...@apache.org.
YARN-4496. Improve HA ResourceManager Failover detection on the client.
Contributed by Jian He


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

Branch: refs/heads/HDFS-1312
Commit: 618bfd6ac2a5b62d39e9bed80f75362bafc0ef28
Parents: 46e5ea8
Author: Xuan <xg...@apache.org>
Authored: Fri Jan 22 18:20:38 2016 -0800
Committer: Xuan <xg...@apache.org>
Committed: Fri Jan 22 18:20:38 2016 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 ...stHedgingRequestRMFailoverProxyProvider.java |  98 ++++++++++
 .../ConfiguredRMFailoverProxyProvider.java      |   6 +-
 .../org/apache/hadoop/yarn/client/RMProxy.java  |  33 ++--
 .../RequestHedgingRMFailoverProxyProvider.java  | 194 +++++++++++++++++++
 .../nodemanager/TestNodeStatusUpdater.java      |   7 +-
 .../hadoop/yarn/server/MiniYARNCluster.java     |   1 +
 7 files changed, 323 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/618bfd6a/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index f840a9e..8e87f4a 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -104,6 +104,9 @@ Release 2.9.0 - UNRELEASED
     YARN-4603. FairScheduler should mention user requested queuename in error 
     message when failed in queue ACL check. (Tao Jie via kasha)
 
+    YARN-4496. Improve HA ResourceManager Failover detection on the client.
+    (Jian He via xgong)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/618bfd6a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestHedgingRequestRMFailoverProxyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestHedgingRequestRMFailoverProxyProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestHedgingRequestRMFailoverProxyProvider.java
new file mode 100644
index 0000000..6fd6591
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestHedgingRequestRMFailoverProxyProvider.java
@@ -0,0 +1,98 @@
+/**
+ * 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.client;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ha.HAServiceProtocol;
+import org.apache.hadoop.yarn.client.api.YarnClient;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.MiniYARNCluster;
+import org.apache.hadoop.yarn.server.resourcemanager.HATestUtil;
+import org.junit.Assert;
+import org.junit.Test;
+
+
+public class TestHedgingRequestRMFailoverProxyProvider {
+
+  @Test
+  public void testHedgingRequestProxyProvider() throws Exception {
+    final MiniYARNCluster cluster =
+        new MiniYARNCluster("testHedgingRequestProxyProvider", 5, 0, 1, 1);
+    Configuration conf = new YarnConfiguration();
+
+    conf.setBoolean(YarnConfiguration.RM_HA_ENABLED, true);
+    conf.setBoolean(YarnConfiguration.AUTO_FAILOVER_ENABLED, false);
+    conf.set(YarnConfiguration.RM_CLUSTER_ID, "cluster1");
+    conf.set(YarnConfiguration.RM_HA_IDS, "rm1,rm2,rm3,rm4,rm5");
+
+    conf.set(YarnConfiguration.CLIENT_FAILOVER_PROXY_PROVIDER,
+        RequestHedgingRMFailoverProxyProvider.class.getName());
+    conf.setLong(YarnConfiguration.RESOURCEMANAGER_CONNECT_RETRY_INTERVAL_MS,
+        2000);
+
+    HATestUtil.setRpcAddressForRM("rm1", 10000, conf);
+    HATestUtil.setRpcAddressForRM("rm2", 20000, conf);
+    HATestUtil.setRpcAddressForRM("rm3", 30000, conf);
+    HATestUtil.setRpcAddressForRM("rm4", 40000, conf);
+    HATestUtil.setRpcAddressForRM("rm5", 50000, conf);
+    conf.setBoolean(YarnConfiguration.YARN_MINICLUSTER_FIXED_PORTS, true);
+
+    cluster.init(conf);
+    cluster.start();
+
+    final YarnClient client = YarnClient.createYarnClient();
+    client.init(conf);
+    client.start();
+
+    // Transition rm5 to active;
+    long start = System.currentTimeMillis();
+    makeRMActive(cluster, 4);
+    // client will retry until the rm becomes active.
+    client.getAllQueues();
+    long end = System.currentTimeMillis();
+    System.out.println("Client call succeeded at " + end);
+    // should return the response fast
+    Assert.assertTrue(end - start <= 10000);
+
+    // transition rm5 to standby
+    cluster.getResourceManager(4).getRMContext().getRMAdminService()
+        .transitionToStandby(new HAServiceProtocol.StateChangeRequestInfo(
+            HAServiceProtocol.RequestSource.REQUEST_BY_USER));
+
+    makeRMActive(cluster, 2);
+    client.getAllQueues();
+    cluster.stop();
+  }
+
+  private void makeRMActive(final MiniYARNCluster cluster, final int index) {
+    Thread t = new Thread() {
+      @Override public void run() {
+        try {
+          System.out.println("Transition rm" + index + " to active");
+          cluster.getResourceManager(index).getRMContext().getRMAdminService()
+              .transitionToActive(new HAServiceProtocol.StateChangeRequestInfo(
+                  HAServiceProtocol.RequestSource.REQUEST_BY_USER));
+        } catch (Exception e) {
+          e.printStackTrace();
+        }
+      }
+    };
+    t.start();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/618bfd6a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/ConfiguredRMFailoverProxyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/ConfiguredRMFailoverProxyProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/ConfiguredRMFailoverProxyProvider.java
index 5577d20..8676db2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/ConfiguredRMFailoverProxyProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/ConfiguredRMFailoverProxyProvider.java
@@ -45,8 +45,8 @@ public class ConfiguredRMFailoverProxyProvider<T>
   private int currentProxyIndex = 0;
   Map<String, T> proxies = new HashMap<String, T>();
 
-  private RMProxy<T> rmProxy;
-  private Class<T> protocol;
+  protected RMProxy<T> rmProxy;
+  protected Class<T> protocol;
   protected YarnConfiguration conf;
   protected String[] rmServiceIds;
 
@@ -71,7 +71,7 @@ public class ConfiguredRMFailoverProxyProvider<T>
             YarnConfiguration.DEFAULT_CLIENT_FAILOVER_RETRIES_ON_SOCKET_TIMEOUTS));
   }
 
-  private T getProxyInternal() {
+  protected T getProxyInternal() {
     try {
       final InetSocketAddress rmAddress = rmProxy.getRMAddress(conf, protocol);
       return RMProxy.getProxy(conf, protocol, rmAddress);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/618bfd6a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/RMProxy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/RMProxy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/RMProxy.java
index 3779ce5..3ab06bd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/RMProxy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/RMProxy.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.io.retry.RetryPolicies;
 import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.io.retry.RetryProxy;
 import org.apache.hadoop.ipc.RetriableException;
+import org.apache.hadoop.ipc.StandbyException;
 import org.apache.hadoop.net.ConnectTimeoutException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.ReflectionUtils;
@@ -77,6 +78,7 @@ public class RMProxy<T> {
   }
 
   /**
+   * Currently, used by Client and AM only
    * Create a proxy for the specified protocol. For non-HA,
    * this is a direct connection to the ResourceManager address. When HA is
    * enabled, the proxy handles the failover between the ResourceManagers as
@@ -88,12 +90,12 @@ public class RMProxy<T> {
     YarnConfiguration conf = (configuration instanceof YarnConfiguration)
         ? (YarnConfiguration) configuration
         : new YarnConfiguration(configuration);
-    RetryPolicy retryPolicy =
-        createRetryPolicy(conf);
-    return createRMProxy(conf, protocol, instance, retryPolicy);
+    RetryPolicy retryPolicy = createRetryPolicy(conf, HAUtil.isHAEnabled(conf));
+    return newProxyInstance(conf, protocol, instance, retryPolicy);
   }
 
   /**
+   * Currently, used by NodeManagers only.
    * Create a proxy for the specified protocol. For non-HA,
    * this is a direct connection to the ResourceManager address. When HA is
    * enabled, the proxy handles the failover between the ResourceManagers as
@@ -106,12 +108,12 @@ public class RMProxy<T> {
     YarnConfiguration conf = (configuration instanceof YarnConfiguration)
         ? (YarnConfiguration) configuration
         : new YarnConfiguration(configuration);
-    RetryPolicy retryPolicy =
-        createRetryPolicy(conf, retryTime, retryInterval);
-    return createRMProxy(conf, protocol, instance, retryPolicy);
+    RetryPolicy retryPolicy = createRetryPolicy(conf, retryTime, retryInterval,
+        HAUtil.isHAEnabled(conf));
+    return newProxyInstance(conf, protocol, instance, retryPolicy);
   }
 
-  private static <T> T createRMProxy(final YarnConfiguration conf,
+  private static <T> T newProxyInstance(final YarnConfiguration conf,
       final Class<T> protocol, RMProxy instance, RetryPolicy retryPolicy)
           throws IOException{
     if (HAUtil.isHAEnabled(conf)) {
@@ -144,7 +146,7 @@ public class RMProxy<T> {
   @Deprecated
   public static <T> T createRMProxy(final Configuration conf,
       final Class<T> protocol, InetSocketAddress rmAddress) throws IOException {
-    RetryPolicy retryPolicy = createRetryPolicy(conf);
+    RetryPolicy retryPolicy = createRetryPolicy(conf, HAUtil.isHAEnabled(conf));
     T proxy = RMProxy.<T>getProxy(conf, protocol, rmAddress);
     LOG.info("Connecting to ResourceManager at " + rmAddress);
     return (T) RetryProxy.create(protocol, proxy, retryPolicy);
@@ -194,7 +196,8 @@ public class RMProxy<T> {
    */
   @Private
   @VisibleForTesting
-  public static RetryPolicy createRetryPolicy(Configuration conf) {
+  public static RetryPolicy createRetryPolicy(Configuration conf,
+      boolean isHAEnabled) {
     long rmConnectWaitMS =
         conf.getLong(
             YarnConfiguration.RESOURCEMANAGER_CONNECT_MAX_WAIT_MS,
@@ -204,16 +207,17 @@ public class RMProxy<T> {
             YarnConfiguration.RESOURCEMANAGER_CONNECT_RETRY_INTERVAL_MS,
             YarnConfiguration
                 .DEFAULT_RESOURCEMANAGER_CONNECT_RETRY_INTERVAL_MS);
-    return createRetryPolicy(
-        conf, rmConnectWaitMS, rmConnectionRetryIntervalMS);
+
+    return createRetryPolicy(conf, rmConnectWaitMS, rmConnectionRetryIntervalMS,
+        isHAEnabled);
   }
 
   /**
    * Fetch retry policy from Configuration and create the
    * retry policy with specified retryTime and retry interval.
    */
-  private static RetryPolicy createRetryPolicy(Configuration conf,
-      long retryTime, long retryInterval) {
+  protected static RetryPolicy createRetryPolicy(Configuration conf,
+      long retryTime, long retryInterval, boolean isHAEnabled) {
     long rmConnectWaitMS = retryTime;
     long rmConnectionRetryIntervalMS = retryInterval;
 
@@ -236,7 +240,7 @@ public class RMProxy<T> {
     }
 
     // Handle HA case first
-    if (HAUtil.isHAEnabled(conf)) {
+    if (isHAEnabled) {
       final long failoverSleepBaseMs = conf.getLong(
           YarnConfiguration.CLIENT_FAILOVER_SLEEPTIME_BASE_MS,
           rmConnectionRetryIntervalMS);
@@ -287,6 +291,7 @@ public class RMProxy<T> {
     exceptionToPolicyMap.put(ConnectTimeoutException.class, retryPolicy);
     exceptionToPolicyMap.put(RetriableException.class, retryPolicy);
     exceptionToPolicyMap.put(SocketException.class, retryPolicy);
+    exceptionToPolicyMap.put(StandbyException.class, retryPolicy);
     // YARN-4288: local IOException is also possible.
     exceptionToPolicyMap.put(IOException.class, retryPolicy);
     // Not retry on remote IO exception.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/618bfd6a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/RequestHedgingRMFailoverProxyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/RequestHedgingRMFailoverProxyProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/RequestHedgingRMFailoverProxyProvider.java
new file mode 100644
index 0000000..dc8d19b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/RequestHedgingRMFailoverProxyProvider.java
@@ -0,0 +1,194 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <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.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.client;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.retry.MultiException;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.io.retry.RetryProxy;
+import org.apache.hadoop.yarn.conf.HAUtil;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+
+import java.io.IOException;
+import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.lang.reflect.Proxy;
+import java.net.InetSocketAddress;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+/**
+ * A FailoverProxyProvider implementation that technically does not "failover"
+ * per-se. It constructs a wrapper proxy that sends the request to ALL
+ * underlying proxies simultaneously. Each proxy inside the wrapper proxy will
+ * retry the corresponding target. It assumes the in an HA setup, there will
+ * be only one Active, and the active should respond faster than any configured
+ * standbys. Once it receives a response from any one of the configred proxies,
+ * outstanding requests to other proxies are immediately cancelled.
+ */
+public class RequestHedgingRMFailoverProxyProvider<T>
+    extends ConfiguredRMFailoverProxyProvider<T> {
+
+  private static final Log LOG =
+      LogFactory.getLog(RequestHedgingRMFailoverProxyProvider.class);
+
+  private volatile String successfulProxy = null;
+  private ProxyInfo<T> wrappedProxy = null;
+  private Map<String, T> nonRetriableProxy = new HashMap<>();
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public void init(Configuration configuration, RMProxy<T> rmProxy,
+      Class<T> protocol) {
+    super.init(configuration, rmProxy, protocol);
+    Map<String, ProxyInfo<T>> retriableProxies = new HashMap<>();
+
+    String originalId = HAUtil.getRMHAId(conf);
+    for (String rmId : rmServiceIds) {
+      conf.set(YarnConfiguration.RM_HA_ID, rmId);
+      nonRetriableProxy.put(rmId, super.getProxyInternal());
+
+      T proxy = createRetriableProxy();
+      ProxyInfo<T> pInfo = new ProxyInfo<T>(proxy, rmId);
+      retriableProxies.put(rmId, pInfo);
+    }
+    conf.set(YarnConfiguration.RM_HA_ID, originalId);
+
+    T proxyInstance = (T) Proxy.newProxyInstance(
+        RMRequestHedgingInvocationHandler.class.getClassLoader(),
+        new Class<?>[] {protocol},
+        new RMRequestHedgingInvocationHandler(retriableProxies));
+    String combinedInfo = Arrays.toString(rmServiceIds);
+    wrappedProxy = new ProxyInfo<T>(proxyInstance, combinedInfo);
+    LOG.info("Created wrapped proxy for " + combinedInfo);
+  }
+
+  @SuppressWarnings("unchecked")
+  protected T createRetriableProxy() {
+    try {
+      // Create proxy that can retry exceptions properly.
+      RetryPolicy retryPolicy = RMProxy.createRetryPolicy(conf, false);
+      InetSocketAddress rmAddress = rmProxy.getRMAddress(conf, protocol);
+      T proxy = RMProxy.<T>getProxy(conf, protocol, rmAddress);
+      return (T) RetryProxy.create(protocol, proxy, retryPolicy);
+    } catch (IOException ioe) {
+      LOG.error("Unable to create proxy to the ResourceManager " + HAUtil
+          .getRMHAId(conf), ioe);
+      return null;
+    }
+  }
+
+  class RMRequestHedgingInvocationHandler implements InvocationHandler {
+
+    final private Map<String, ProxyInfo<T>> allProxies;
+
+    public RMRequestHedgingInvocationHandler(
+        Map<String, ProxyInfo<T>> allProxies) {
+      this.allProxies = new HashMap<>(allProxies);
+    }
+
+    protected Object invokeMethod(Object proxy, Method method, Object[] args)
+        throws Throwable {
+      try {
+        return method.invoke(proxy, args);
+      } catch (InvocationTargetException ex) {
+        throw ex.getCause();
+      }
+    }
+
+    /**
+     * Creates a Executor and invokes all proxies concurrently.
+     */
+    @Override
+    public Object invoke(Object proxy, final Method method,
+        final Object[] args) throws Throwable {
+      if (successfulProxy != null) {
+        return invokeMethod(nonRetriableProxy.get(successfulProxy), method, args);
+      }
+
+      ExecutorService executor = null;
+      CompletionService<Object> completionService;
+      try {
+        Map<Future<Object>, ProxyInfo<T>> proxyMap = new HashMap<>();
+        int numAttempts = 0;
+        executor = Executors.newFixedThreadPool(allProxies.size());
+        completionService = new ExecutorCompletionService<>(executor);
+        for (final ProxyInfo<T> pInfo : allProxies.values()) {
+          Callable<Object> c = new Callable<Object>() {
+            @Override public Object call() throws Exception {
+              return method.invoke(pInfo.proxy, args);
+            }
+          };
+          proxyMap.put(completionService.submit(c), pInfo);
+          numAttempts++;
+        }
+
+        Map<String, Exception> badResults = new HashMap<>();
+        while (numAttempts > 0) {
+          Future<Object> callResultFuture = completionService.take();
+          String pInfo = proxyMap.get(callResultFuture).proxyInfo;
+          Object retVal;
+          try {
+            retVal = callResultFuture.get();
+            successfulProxy = pInfo;
+            LOG.info("Invocation successful on [" + pInfo + "]");
+            return retVal;
+          } catch (Exception ex) {
+            LOG.warn("Invocation returned exception on " + "[" + pInfo + "]");
+            badResults.put(pInfo, ex);
+            numAttempts--;
+          }
+        }
+
+        // At this point we should have All bad results (Exceptions)
+        // Or should have returned with successful result.
+        if (badResults.size() == 1) {
+          throw badResults.values().iterator().next();
+        } else {
+          throw new MultiException(badResults);
+        }
+      } finally {
+        if (executor != null) {
+          executor.shutdownNow();
+        }
+      }
+    }
+  }
+
+  @Override
+  public ProxyInfo<T> getProxy() {
+    return wrappedProxy;
+  }
+
+  @Override
+  public void performFailover(T currentProxy) {
+    LOG.info("Connection lost, trying to fail over.");
+    successfulProxy = null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/618bfd6a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.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/TestNodeStatusUpdater.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java
index 90804b8..a8066c1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java
@@ -74,6 +74,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.SignalContainerCommand;
 import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.client.RMProxy;
+import org.apache.hadoop.yarn.conf.HAUtil;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
@@ -443,7 +444,8 @@ public class TestNodeStatusUpdater {
 
     @Override
     protected ResourceTracker getRMClient() throws IOException {
-      RetryPolicy retryPolicy = RMProxy.createRetryPolicy(conf);
+      RetryPolicy retryPolicy = RMProxy.createRetryPolicy(conf,
+          HAUtil.isHAEnabled(conf));
       resourceTracker =
           (ResourceTracker) RetryProxy.create(ResourceTracker.class,
             new MyResourceTracker6(rmStartIntervalMS, rmNeverStart),
@@ -476,7 +478,8 @@ public class TestNodeStatusUpdater {
 
     @Override
     protected ResourceTracker getRMClient() {
-      RetryPolicy retryPolicy = RMProxy.createRetryPolicy(conf);
+      RetryPolicy retryPolicy = RMProxy.createRetryPolicy(conf,
+          HAUtil.isHAEnabled(conf));
       return (ResourceTracker) RetryProxy.create(ResourceTracker.class,
         resourceTracker, retryPolicy);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/618bfd6a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
index 68c9efd..630b7ef 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
@@ -457,6 +457,7 @@ public class MiniYARNCluster extends CompositeService {
     protected synchronized void serviceStart() throws Exception {
       startResourceManager(index);
       Configuration conf = resourceManagers[index].getConfig();
+      LOG.info("Starting resourcemanager " + index);
       LOG.info("MiniYARN ResourceManager address: " +
           conf.get(YarnConfiguration.RM_ADDRESS));
       LOG.info("MiniYARN ResourceManager web address: " + WebAppUtils


[17/50] hadoop git commit: HDFS-9672. o.a.h.hdfs.TestLeaseRecovery2 fails intermittently. Contributed by Mingliang Liu.

Posted by ar...@apache.org.
HDFS-9672. o.a.h.hdfs.TestLeaseRecovery2 fails intermittently. Contributed by Mingliang Liu.


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

Branch: refs/heads/HDFS-1312
Commit: e8650fea1f0837026cbb36ae8bf51c6133259809
Parents: ec4d2d9
Author: Jitendra Pandey <ji...@apache.org>
Authored: Mon Jan 25 15:42:25 2016 -0800
Committer: Jitendra Pandey <ji...@apache.org>
Committed: Mon Jan 25 16:08:46 2016 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../apache/hadoop/hdfs/TestLeaseRecovery2.java  | 48 ++++++++++++++------
 2 files changed, 37 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e8650fea/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index f35ae3d..68d5de6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -2690,6 +2690,9 @@ Release 2.7.3 - UNRELEASED
     HDFS-9625. set replication for empty file failed when set storage policy
     (DENG FEI via vinayakumarb)
 
+    HDFS-9672. o.a.h.hdfs.TestLeaseRecovery2 fails intermittently (Mingliang Liu
+    via jitendra)
+
 Release 2.7.2 - 2016-01-25
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e8650fea/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery2.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery2.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery2.java
index 13e8644..e8cd476 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery2.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery2.java
@@ -21,11 +21,14 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.spy;
 
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 
+import com.google.common.base.Supplier;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -42,6 +45,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLog;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
@@ -49,10 +53,11 @@ import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.log4j.Level;
-import org.junit.AfterClass;
+import org.junit.After;
 import org.junit.Assert;
-import org.junit.BeforeClass;
+import org.junit.Before;
 import org.junit.Test;
+import org.mockito.Mockito;
 
 public class TestLeaseRecovery2 {
   
@@ -85,12 +90,15 @@ public class TestLeaseRecovery2 {
    * 
    * @throws IOException
    */
-  @BeforeClass
-  public static void startUp() throws IOException {
+  @Before
+  public void startUp() throws IOException {
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
     conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
 
-    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(5).build();
+    cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(5)
+        .checkExitOnShutdown(false)
+        .build();
     cluster.waitActive();
     dfs = cluster.getFileSystem();
   }
@@ -99,8 +107,8 @@ public class TestLeaseRecovery2 {
    * stop the cluster
    * @throws IOException
    */
-  @AfterClass
-  public static void tearDown() throws IOException {
+  @After
+  public void tearDown() throws IOException {
     if (cluster != null) {
       IOUtils.closeStream(dfs);
       cluster.shutdown();
@@ -419,17 +427,17 @@ public class TestLeaseRecovery2 {
    * 
    * @throws Exception
    */
-  @Test
+  @Test(timeout = 30000)
   public void testHardLeaseRecoveryAfterNameNodeRestart() throws Exception {
     hardLeaseRecoveryRestartHelper(false, -1);
   }
 
-  @Test
+  @Test(timeout = 30000)
   public void testHardLeaseRecoveryAfterNameNodeRestart2() throws Exception {
     hardLeaseRecoveryRestartHelper(false, 1535);
   }
 
-  @Test
+  @Test(timeout = 30000)
   public void testHardLeaseRecoveryWithRenameAfterNameNodeRestart()
       throws Exception {
     hardLeaseRecoveryRestartHelper(true, -1);
@@ -489,10 +497,22 @@ public class TestLeaseRecovery2 {
     cluster.setLeasePeriod(LONG_LEASE_PERIOD, SHORT_LEASE_PERIOD);
     
     // Make sure lease recovery begins.
-    Thread.sleep(HdfsServerConstants.NAMENODE_LEASE_RECHECK_INTERVAL * 2);
-    
-    checkLease(fileStr, size);
-    
+    final String path = fileStr;
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        return HdfsServerConstants.NAMENODE_LEASE_HOLDER.equals(
+            NameNodeAdapter.getLeaseHolderForPath(cluster.getNameNode(), path));
+      }
+    }, (int)SHORT_LEASE_PERIOD, (int)SHORT_LEASE_PERIOD * 10);
+
+    // Normally, the in-progress edit log would be finalized by
+    // FSEditLog#endCurrentLogSegment.  For testing purposes, we
+    // disable that here.
+    FSEditLog spyLog = spy(cluster.getNameNode().getFSImage().getEditLog());
+    doNothing().when(spyLog).endCurrentLogSegment(Mockito.anyBoolean());
+    DFSTestUtil.setEditLogForTesting(cluster.getNamesystem(), spyLog);
+
     cluster.restartNameNode(false);
     
     checkLease(fileStr, size);


[48/50] hadoop git commit: YARN-4647. Make RegisterNodeManagerRequestPBImpl thread-safe. (kasha)

Posted by ar...@apache.org.
YARN-4647. Make RegisterNodeManagerRequestPBImpl thread-safe. (kasha)


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

Branch: refs/heads/HDFS-1312
Commit: c9a09d6926b258e205a4ff7998ce5a86bf5dbe3b
Parents: a277bdc
Author: Karthik Kambatla <ka...@apache.org>
Authored: Fri Jan 29 08:12:54 2016 -0800
Committer: Karthik Kambatla <ka...@apache.org>
Committed: Fri Jan 29 08:12:54 2016 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  2 +
 .../pb/RegisterNodeManagerRequestPBImpl.java    | 75 +++++++++++---------
 2 files changed, 42 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9a09d69/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 138e581..76cad7f 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -113,6 +113,8 @@ Release 2.9.0 - UNRELEASED
     integrated ResourceHandler mechanism, and also deprecated the old
     LCEResourceHandler inteface hierarchy. (Varun Vasudev via vinodkv)
 
+    YARN-4647. Make RegisterNodeManagerRequestPBImpl thread-safe. (kasha)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9a09d69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RegisterNodeManagerRequestPBImpl.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/RegisterNodeManagerRequestPBImpl.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/RegisterNodeManagerRequestPBImpl.java
index 5b0e0a1..2a1a268 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/RegisterNodeManagerRequestPBImpl.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/RegisterNodeManagerRequestPBImpl.java
@@ -65,14 +65,14 @@ public class RegisterNodeManagerRequestPBImpl extends RegisterNodeManagerRequest
     viaProto = true;
   }
   
-  public RegisterNodeManagerRequestProto getProto() {
-      mergeLocalToProto();
+  public synchronized RegisterNodeManagerRequestProto getProto() {
+    mergeLocalToProto();
     proto = viaProto ? proto : builder.build();
     viaProto = true;
     return proto;
   }
 
-  private void mergeLocalToBuilder() {
+  private synchronized void mergeLocalToBuilder() {
     if (this.containerStatuses != null) {
       addNMContainerStatusesToProto();
     }
@@ -107,15 +107,16 @@ public class RegisterNodeManagerRequestPBImpl extends RegisterNodeManagerRequest
   }
 
     
-  private void mergeLocalToProto() {
-    if (viaProto) 
+  private synchronized void mergeLocalToProto() {
+    if (viaProto) {
       maybeInitBuilder();
+    }
     mergeLocalToBuilder();
     proto = builder.build();
     viaProto = true;
   }
 
-  private void maybeInitBuilder() {
+  private synchronized void maybeInitBuilder() {
     if (viaProto || builder == null) {
       builder = RegisterNodeManagerRequestProto.newBuilder(proto);
     }
@@ -124,7 +125,7 @@ public class RegisterNodeManagerRequestPBImpl extends RegisterNodeManagerRequest
     
   
   @Override
-  public Resource getResource() {
+  public synchronized Resource getResource() {
     RegisterNodeManagerRequestProtoOrBuilder p = viaProto ? proto : builder;
     if (this.resource != null) {
       return this.resource;
@@ -137,7 +138,7 @@ public class RegisterNodeManagerRequestPBImpl extends RegisterNodeManagerRequest
   }
 
   @Override
-  public void setResource(Resource resource) {
+  public synchronized void setResource(Resource resource) {
     maybeInitBuilder();
     if (resource == null) 
       builder.clearResource();
@@ -145,7 +146,7 @@ public class RegisterNodeManagerRequestPBImpl extends RegisterNodeManagerRequest
   }
 
   @Override
-  public NodeId getNodeId() {
+  public synchronized NodeId getNodeId() {
     RegisterNodeManagerRequestProtoOrBuilder p = viaProto ? proto : builder;
     if (this.nodeId != null) {
       return this.nodeId;
@@ -158,15 +159,16 @@ public class RegisterNodeManagerRequestPBImpl extends RegisterNodeManagerRequest
   }
 
   @Override
-  public void setNodeId(NodeId nodeId) {
+  public synchronized void setNodeId(NodeId nodeId) {
     maybeInitBuilder();
-    if (nodeId == null) 
+    if (nodeId == null) {
       builder.clearNodeId();
+    }
     this.nodeId = nodeId;
   }
 
   @Override
-  public int getHttpPort() {
+  public synchronized int getHttpPort() {
     RegisterNodeManagerRequestProtoOrBuilder p = viaProto ? proto : builder;
     if (!p.hasHttpPort()) {
       return 0;
@@ -175,18 +177,18 @@ public class RegisterNodeManagerRequestPBImpl extends RegisterNodeManagerRequest
   }
 
   @Override
-  public void setHttpPort(int httpPort) {
+  public synchronized void setHttpPort(int httpPort) {
     maybeInitBuilder();
     builder.setHttpPort(httpPort);
   }
   
   @Override
-  public List<ApplicationId> getRunningApplications() {
+  public synchronized List<ApplicationId> getRunningApplications() {
     initRunningApplications();
     return runningApplications;
   }
   
-  private void initRunningApplications() {
+  private synchronized void initRunningApplications() {
     if (this.runningApplications != null) {
       return;
     }
@@ -199,7 +201,7 @@ public class RegisterNodeManagerRequestPBImpl extends RegisterNodeManagerRequest
   }
 
   @Override
-  public void setRunningApplications(List<ApplicationId> apps) {
+  public synchronized void setRunningApplications(List<ApplicationId> apps) {
     if (apps == null) {
       return;
     }
@@ -207,7 +209,7 @@ public class RegisterNodeManagerRequestPBImpl extends RegisterNodeManagerRequest
     this.runningApplications.addAll(apps);
   }
   
-  private void addRunningApplicationsToProto() {
+  private synchronized void addRunningApplicationsToProto() {
     maybeInitBuilder();
     builder.clearRunningApplications();
     if (runningApplications == null) {
@@ -241,12 +243,12 @@ public class RegisterNodeManagerRequestPBImpl extends RegisterNodeManagerRequest
   }
 
   @Override
-  public List<NMContainerStatus> getNMContainerStatuses() {
+  public synchronized List<NMContainerStatus> getNMContainerStatuses() {
     initContainerRecoveryReports();
     return containerStatuses;
   }
   
-  private void initContainerRecoveryReports() {
+  private synchronized void initContainerRecoveryReports() {
     if (this.containerStatuses != null) {
       return;
     }
@@ -259,7 +261,7 @@ public class RegisterNodeManagerRequestPBImpl extends RegisterNodeManagerRequest
   }
 
   @Override
-  public void setContainerStatuses(
+  public synchronized void setContainerStatuses(
       List<NMContainerStatus> containerReports) {
     if (containerReports == null) {
       return;
@@ -284,7 +286,7 @@ public class RegisterNodeManagerRequestPBImpl extends RegisterNodeManagerRequest
   }
   
   @Override
-  public String getNMVersion() {
+  public synchronized String getNMVersion() {
     RegisterNodeManagerRequestProtoOrBuilder p = viaProto ? proto : builder;
     if (!p.hasNmVersion()) {
       return "";
@@ -293,25 +295,25 @@ public class RegisterNodeManagerRequestPBImpl extends RegisterNodeManagerRequest
   }
 
   @Override
-  public void setNMVersion(String version) {
+  public synchronized void setNMVersion(String version) {
     maybeInitBuilder();
     builder.setNmVersion(version);
   }
   
   @Override
-  public Set<NodeLabel> getNodeLabels() {
+  public synchronized Set<NodeLabel> getNodeLabels() {
     initNodeLabels();
     return this.labels;
   }
 
   @Override
-  public void setNodeLabels(Set<NodeLabel> nodeLabels) {
+  public synchronized void setNodeLabels(Set<NodeLabel> nodeLabels) {
     maybeInitBuilder();
     builder.clearNodeLabels();
     this.labels = nodeLabels;
   }
   
-  private void initNodeLabels() {
+  private synchronized void initNodeLabels() {
     if (this.labels != null) {
       return;
     }
@@ -327,43 +329,46 @@ public class RegisterNodeManagerRequestPBImpl extends RegisterNodeManagerRequest
     }
   }
 
-  private NodeLabelPBImpl convertFromProtoFormat(NodeLabelProto p) {
+  private static NodeLabelPBImpl convertFromProtoFormat(NodeLabelProto p) {
     return new NodeLabelPBImpl(p);
   }
 
-  private NodeLabelProto convertToProtoFormat(NodeLabel t) {
+  private static NodeLabelProto convertToProtoFormat(NodeLabel t) {
     return ((NodeLabelPBImpl)t).getProto();
   }
 
-  private ApplicationIdPBImpl convertFromProtoFormat(ApplicationIdProto p) {
+  private static ApplicationIdPBImpl convertFromProtoFormat(
+      ApplicationIdProto p) {
     return new ApplicationIdPBImpl(p);
   }
 
-  private ApplicationIdProto convertToProtoFormat(ApplicationId t) {
+  private static ApplicationIdProto convertToProtoFormat(ApplicationId t) {
     return ((ApplicationIdPBImpl)t).getProto();
   }
 
-  private NodeIdPBImpl convertFromProtoFormat(NodeIdProto p) {
+  private static NodeIdPBImpl convertFromProtoFormat(NodeIdProto p) {
     return new NodeIdPBImpl(p);
   }
 
-  private NodeIdProto convertToProtoFormat(NodeId t) {
+  private static NodeIdProto convertToProtoFormat(NodeId t) {
     return ((NodeIdPBImpl)t).getProto();
   }
 
-  private ResourcePBImpl convertFromProtoFormat(ResourceProto p) {
+  private static ResourcePBImpl convertFromProtoFormat(ResourceProto p) {
     return new ResourcePBImpl(p);
   }
 
-  private ResourceProto convertToProtoFormat(Resource t) {
+  private static ResourceProto convertToProtoFormat(Resource t) {
     return ((ResourcePBImpl)t).getProto();
   }
 
-  private NMContainerStatusPBImpl convertFromProtoFormat(NMContainerStatusProto c) {
+  private static NMContainerStatusPBImpl convertFromProtoFormat(
+      NMContainerStatusProto c) {
     return new NMContainerStatusPBImpl(c);
   }
   
-  private NMContainerStatusProto convertToProtoFormat(NMContainerStatus c) {
+  private static NMContainerStatusProto convertToProtoFormat(
+      NMContainerStatus c) {
     return ((NMContainerStatusPBImpl)c).getProto();
   }
 }
\ No newline at end of file


[49/50] hadoop git commit: HDFS-9647. DiskBalancer: Add getRuntimeSettings. (Contributed by Anu Engineer)

Posted by ar...@apache.org.
HDFS-9647. DiskBalancer: Add getRuntimeSettings. (Contributed by Anu Engineer)


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

Branch: refs/heads/HDFS-1312
Commit: e4e585a635b723ca92b80310bb69744a44698568
Parents: 0fd3f16
Author: Arpit Agarwal <ar...@apache.org>
Authored: Fri Jan 29 11:05:53 2016 -0800
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Fri Jan 29 11:05:53 2016 -0800

----------------------------------------------------------------------
 .../hdfs/protocol/ClientDatanodeProtocol.java   | 10 ++++++
 .../ClientDatanodeProtocolTranslatorPB.java     | 17 +++++++++-
 .../src/main/proto/ClientDatanodeProtocol.proto | 19 +++++++++++
 .../hadoop-hdfs/HDFS-1312_CHANGES.txt           |  3 ++
 ...tDatanodeProtocolServerSideTranslatorPB.java | 24 ++++++++++++--
 .../hadoop/hdfs/server/datanode/DataNode.java   | 14 ++++++++
 .../diskbalancer/DiskBalancerConstants.java     | 35 ++++++++++++++++++++
 .../diskbalancer/TestDiskBalancerRPC.java       | 16 ++++++---
 8 files changed, 130 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e4e585a6/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
index e2f2491..706bc04 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
@@ -176,4 +176,14 @@ public interface ClientDatanodeProtocol {
    * Gets the status of an executing diskbalancer Plan.
    */
   WorkStatus queryDiskBalancerPlan() throws IOException;
+
+  /**
+   * Gets a run-time configuration value from running diskbalancer instance.
+   * For example : Disk Balancer bandwidth of a running instance.
+   *
+   * @param key runtime configuration key
+   * @return value of the key as a string.
+   * @throws IOException - Throws if there is no such key
+   */
+  String getDiskBalancerSetting(String key) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e4e585a6/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
index 2331fa8..04f21e0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
@@ -55,6 +55,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.Submit
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.CancelPlanRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.QueryPlanStatusRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.QueryPlanStatusResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DiskBalancerSettingRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DiskBalancerSettingResponseProto;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.datanode.WorkStatus;
 import org.apache.hadoop.ipc.ProtobufHelper;
@@ -353,8 +355,8 @@ public class ClientDatanodeProtocolTranslatorPB implements
 
   /**
    * Cancels an executing disk balancer plan.
-   * @param planID - A SHA512 hash of the plan string.
    *
+   * @param planID - A SHA512 hash of the plan string.
    * @throws IOException on error
    */
   @Override
@@ -387,4 +389,17 @@ public class ClientDatanodeProtocolTranslatorPB implements
       throw ProtobufHelper.getRemoteException(e);
     }
   }
+
+  @Override
+  public String getDiskBalancerSetting(String key) throws IOException {
+    try {
+      DiskBalancerSettingRequestProto request =
+          DiskBalancerSettingRequestProto.newBuilder().setKey(key).build();
+      DiskBalancerSettingResponseProto response =
+          rpcProxy.getDiskBalancerSetting(NULL_CONTROLLER, request);
+      return response.hasValue() ? response.getValue() : null;
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e4e585a6/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientDatanodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientDatanodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientDatanodeProtocol.proto
index 5187fa7..175a571 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientDatanodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientDatanodeProtocol.proto
@@ -190,7 +190,21 @@ message QueryPlanStatusResponseProto {
   optional string status = 2;
   optional string planID = 3;
   optional string currentStatus = 4;
+}
+
+/**
+  * This message sends a request to data node get a specific setting
+  * that is used by disk balancer.
+  */
+message DiskBalancerSettingRequestProto {
+  required string key = 1;
+}
 
+/**
+ * Response that describes the value of requested disk balancer setting.
+ */
+message DiskBalancerSettingResponseProto {
+  required string value = 1;
 }
 
 /**
@@ -265,4 +279,9 @@ service ClientDatanodeProtocolService {
    */
   rpc queryDiskBalancerPlan(QueryPlanStatusRequestProto)
       returns (QueryPlanStatusResponseProto);
+  /**
+   *  Gets run-time settings of Disk Balancer.
+   */
+  rpc getDiskBalancerSetting(DiskBalancerSettingRequestProto)
+      returns(DiskBalancerSettingResponseProto);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e4e585a6/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
index c6a5554..d3bdedf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
@@ -21,3 +21,6 @@ HDFS-1312 Change Log
 
     HDFS-9645. DiskBalancer: Add Query RPC. (Anu Engineer via Arpit Agarwal)
 
+    HDFS-9647. DiskBalancer: Add getRuntimeSettings. (Anu Engineer
+    via Arpit Agarwal)
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e4e585a6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
index 98e3d0b..adb9888 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
@@ -51,7 +51,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.Cancel
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.CancelPlanResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.QueryPlanStatusRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.QueryPlanStatusResponseProto;
-
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DiskBalancerSettingRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DiskBalancerSettingResponseProto;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
 import org.apache.hadoop.hdfs.server.datanode.WorkStatus;
@@ -264,7 +265,7 @@ public class ClientDatanodeProtocolServerSideTranslatorPB implements
     try {
       impl.cancelDiskBalancePlan(request.getPlanID());
       return CancelPlanResponseProto.newBuilder().build();
-    }catch (Exception e) {
+    } catch (Exception e) {
       throw new ServiceException(e);
     }
   }
@@ -274,7 +275,7 @@ public class ClientDatanodeProtocolServerSideTranslatorPB implements
    */
   @Override
   public QueryPlanStatusResponseProto queryDiskBalancerPlan(
-      RpcController controller,  QueryPlanStatusRequestProto request)
+      RpcController controller, QueryPlanStatusRequestProto request)
       throws ServiceException {
     try {
       WorkStatus result = impl.queryDiskBalancerPlan();
@@ -289,4 +290,21 @@ public class ClientDatanodeProtocolServerSideTranslatorPB implements
       throw new ServiceException(e);
     }
   }
+
+  /**
+   * Returns a run-time setting from diskbalancer like Bandwidth.
+   */
+  @Override
+  public DiskBalancerSettingResponseProto getDiskBalancerSetting(
+      RpcController controller, DiskBalancerSettingRequestProto request)
+      throws ServiceException {
+    try {
+      String val = impl.getDiskBalancerSetting(request.getKey());
+      return DiskBalancerSettingResponseProto.newBuilder()
+          .setValue(val)
+          .build();
+    } catch (Exception e) {
+      throw new ServiceException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e4e585a6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index 7e2f378..b78f18e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -3250,4 +3250,18 @@ public class DataNode extends ReconfigurableBase
     checkSuperuserPrivilege();
     throw new DiskbalancerException("Not Implemented", 0);
   }
+
+  /**
+   * Gets a run-time configuration value from running diskbalancer instance. For
+   * example : Disk Balancer bandwidth of a running instance.
+   *
+   * @param key - String that represents the run time key value.
+   * @return value of the key as a string.
+   * @throws IOException - Throws if there is no such key
+   */
+  @Override
+  public String getDiskBalancerSetting(String key) throws IOException {
+    checkSuperuserPrivilege();
+    throw new DiskbalancerException("Not Implemented", 0);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e4e585a6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerConstants.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerConstants.java
new file mode 100644
index 0000000..553827e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerConstants.java
@@ -0,0 +1,35 @@
+/**
+ * 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
+ * <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. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.hdfs.server.diskbalancer;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Constants used by Disk Balancer.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public final class DiskBalancerConstants {
+  public static final String DISKBALANCER_BANDWIDTH = "DiskBalancerBandwidth";
+  public static final String DISKBALANCER_VOLUME_NAME =
+      "DiskBalancerVolumeName";
+
+  // never constructed.
+  private DiskBalancerConstants() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e4e585a6/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
index a127816..143b776 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
@@ -60,7 +60,7 @@ public class TestDiskBalancerRPC {
   }
 
   @Test
-  public void TestSubmitTestRpc() throws Exception {
+  public void testSubmitTestRpc() throws Exception {
     final int dnIndex = 0;
     cluster.restartDataNode(dnIndex);
     cluster.waitActive();
@@ -91,7 +91,7 @@ public class TestDiskBalancerRPC {
   }
 
   @Test
-  public void TestCancelTestRpc() throws Exception {
+  public void testCancelTestRpc() throws Exception {
     final int dnIndex = 0;
     cluster.restartDataNode(dnIndex);
     cluster.waitActive();
@@ -122,11 +122,10 @@ public class TestDiskBalancerRPC {
     }
     thrown.expect(DiskbalancerException.class);
     dataNode.cancelDiskBalancePlan(planHash);
-
   }
 
   @Test
-  public void TestQueryTestRpc() throws Exception {
+  public void testQueryTestRpc() throws Exception {
     final int dnIndex = 0;
     cluster.restartDataNode(dnIndex);
     cluster.waitActive();
@@ -162,4 +161,13 @@ public class TestDiskBalancerRPC {
     thrown.expect(DiskbalancerException.class);
     dataNode.queryDiskBalancerPlan();
   }
+
+  @Test
+  public void testgetDiskBalancerSetting() throws Exception {
+    final int dnIndex = 0;
+    DataNode dataNode = cluster.getDataNodes().get(dnIndex);
+    thrown.expect(DiskbalancerException.class);
+    dataNode.getDiskBalancerSetting(
+        DiskBalancerConstants.DISKBALANCER_BANDWIDTH);
+  }
 }


[12/50] hadoop git commit: YARN-4592. Remove unused GetContainerStatus proto. Contributed by Chang Li.

Posted by ar...@apache.org.
YARN-4592. Remove unused GetContainerStatus proto. Contributed by Chang Li.


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

Branch: refs/heads/HDFS-1312
Commit: 56a0c175082b2b62698f65c1769a64224b3fb821
Parents: 6432279
Author: Akira Ajisaka <aa...@apache.org>
Authored: Mon Jan 25 15:20:29 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Mon Jan 25 15:20:29 2016 +0900

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                               | 2 ++
 .../hadoop-yarn-api/src/main/proto/yarn_service_protos.proto  | 7 -------
 2 files changed, 2 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/56a0c175/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 6674194..8ece214 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -1325,6 +1325,8 @@ Release 2.8.0 - UNRELEASED
     YARN-4614. Fix random failure in TestApplicationPriority#testApplicationPriority
     AllocationWithChangeInPriority. (Sunil G via rohithsharmaks)
 
+    YARN-4592. Remove unused GetContainerStatus proto. (Chang Li via aajisaka)
+
 Release 2.7.3 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/56a0c175/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
index 115df9a..eae840b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
@@ -262,13 +262,6 @@ message StopContainerRequestProto {
 message StopContainerResponseProto {
 }
 
-message GetContainerStatusRequestProto {
-  optional ContainerIdProto container_id = 1;
-}
-
-message GetContainerStatusResponseProto {
-  optional ContainerStatusProto status = 1;
-}
 
 //// bulk API records
 message StartContainersRequestProto {


[42/50] hadoop git commit: HDFS-7764. DirectoryScanner shouldn't abort the scan if one directory had an error (Rakesh R via cmccabe)

Posted by ar...@apache.org.
HDFS-7764. DirectoryScanner shouldn't abort the scan if one directory had an error (Rakesh R via cmccabe)


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

Branch: refs/heads/HDFS-1312
Commit: f67149ab08bb49381def6c535ab4c4610e0a4221
Parents: ee005e0
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Thu Jan 28 19:54:50 2016 -0800
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Thu Jan 28 19:54:50 2016 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +
 .../hdfs/server/datanode/DirectoryScanner.java  | 72 +++++++++++++-------
 .../server/datanode/TestDirectoryScanner.java   | 50 ++++++++++++++
 3 files changed, 100 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f67149ab/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index a51dc15..9b80aa1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -962,6 +962,9 @@ Release 2.9.0 - UNRELEASED
     HDFS-9677. Rename generationStampV1/generationStampV2 to
     legacyGenerationStamp/generationStamp. (Mingliang Liu via jing9)
 
+    HDFS-7764. DirectoryScanner shouldn't abort the scan if one directory had
+    an error (Rakesh R via cmccabe)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f67149ab/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
index 392c121..083ca31 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
@@ -19,8 +19,10 @@ package org.apache.hadoop.hdfs.server.datanode;
 
 import com.google.common.annotations.VisibleForTesting;
 import java.io.File;
+import java.io.FilenameFilter;
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
@@ -42,12 +44,12 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.StopWatch;
 import org.apache.hadoop.util.Time;
@@ -727,18 +729,20 @@ public class DirectoryScanner implements Runnable {
 
       for (Entry<Integer, Future<ScanInfoPerBlockPool>> report :
           compilersInProgress.entrySet()) {
+        Integer index = report.getKey();
         try {
-          dirReports[report.getKey()] = report.getValue().get();
+          dirReports[index] = report.getValue().get();
 
           // If our compiler threads were interrupted, give up on this run
-          if (dirReports[report.getKey()] == null) {
+          if (dirReports[index] == null) {
             dirReports = null;
             break;
           }
         } catch (Exception ex) {
-          LOG.error("Error compiling report", ex);
-          // Propagate ex to DataBlockScanner to deal with
-          throw new RuntimeException(ex);
+          FsVolumeSpi fsVolumeSpi = volumes.get(index);
+          LOG.error("Error compiling report for the volume, StorageId: "
+              + fsVolumeSpi.getStorageID(), ex);
+          // Continue scanning the other volumes
         }
       }
     } catch (IOException e) {
@@ -747,7 +751,9 @@ public class DirectoryScanner implements Runnable {
     if (dirReports != null) {
       // Compile consolidated report for all the volumes
       for (ScanInfoPerBlockPool report : dirReports) {
-        list.addAll(report);
+        if(report != null){
+          list.addAll(report);
+        }
       }
     }
     return list.toSortedArrays();
@@ -837,12 +843,11 @@ public class DirectoryScanner implements Runnable {
         File bpFinalizedDir, File dir, LinkedList<ScanInfo> report)
         throws InterruptedException {
 
-      File[] files;
-
       throttle();
 
+      List <String> fileNames;
       try {
-        files = FileUtil.listFiles(dir);
+        fileNames = IOUtils.listDirectory(dir, BlockDirFilter.INSTANCE);
       } catch (IOException ioe) {
         LOG.warn("Exception occured while compiling report: ", ioe);
         // Initiate a check on disk failure.
@@ -850,44 +855,50 @@ public class DirectoryScanner implements Runnable {
         // Ignore this directory and proceed.
         return report;
       }
-      Arrays.sort(files);
+      Collections.sort(fileNames);
+
       /*
        * Assumption: In the sorted list of files block file appears immediately
        * before block metadata file. This is true for the current naming
        * convention for block file blk_<blockid> and meta file
        * blk_<blockid>_<genstamp>.meta
        */
-      for (int i = 0; i < files.length; i++) {
+      for (int i = 0; i < fileNames.size(); i++) {
         // Make sure this thread can make a timely exit. With a low throttle
         // rate, completing a run can take a looooong time.
         if (Thread.interrupted()) {
           throw new InterruptedException();
         }
 
-        if (files[i].isDirectory()) {
-          compileReport(vol, bpFinalizedDir, files[i], report);
+        File file = new File(dir, fileNames.get(i));
+        if (file.isDirectory()) {
+          compileReport(vol, bpFinalizedDir, file, report);
           continue;
         }
-        if (!Block.isBlockFilename(files[i])) {
-          if (isBlockMetaFile(Block.BLOCK_FILE_PREFIX, files[i].getName())) {
-            long blockId = Block.getBlockId(files[i].getName());
-            verifyFileLocation(files[i].getParentFile(), bpFinalizedDir,
+        if (!Block.isBlockFilename(file)) {
+          if (isBlockMetaFile(Block.BLOCK_FILE_PREFIX, file.getName())) {
+            long blockId = Block.getBlockId(file.getName());
+            verifyFileLocation(file.getParentFile(), bpFinalizedDir,
                 blockId);
-            report.add(new ScanInfo(blockId, null, files[i], vol));
+            report.add(new ScanInfo(blockId, null, file, vol));
           }
           continue;
         }
-        File blockFile = files[i];
-        long blockId = Block.filename2id(blockFile.getName());
+        File blockFile = file;
+        long blockId = Block.filename2id(file.getName());
         File metaFile = null;
 
         // Skip all the files that start with block name until
         // getting to the metafile for the block
-        while (i + 1 < files.length && files[i + 1].isFile()
-            && files[i + 1].getName().startsWith(blockFile.getName())) {
+        while (i + 1 < fileNames.size()) {
+          File blkMetaFile = new File(dir, fileNames.get(i + 1));
+          if (!(blkMetaFile.isFile()
+              && blkMetaFile.getName().startsWith(blockFile.getName()))) {
+            break;
+          }
           i++;
-          if (isBlockMetaFile(blockFile.getName(), files[i].getName())) {
-            metaFile = files[i];
+          if (isBlockMetaFile(blockFile.getName(), blkMetaFile.getName())) {
+            metaFile = blkMetaFile;
             break;
           }
         }
@@ -946,4 +957,15 @@ public class DirectoryScanner implements Runnable {
       perfTimer.reset().start();
     }
   }
+
+  private enum BlockDirFilter implements FilenameFilter {
+    INSTANCE;
+
+    @Override
+    public boolean accept(File dir, String name) {
+      return name.startsWith(DataStorage.BLOCK_SUBDIR_PREFIX)
+          || name.startsWith(DataStorage.STORAGE_DIR_FINALIZED)
+          || name.startsWith(Block.BLOCK_FILE_PREFIX);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f67149ab/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
index d030144..d860107 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
@@ -30,6 +30,8 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.nio.channels.ClosedChannelException;
 import java.nio.channels.FileChannel;
+import java.util.ArrayList;
+import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Random;
@@ -54,6 +56,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi.FsVolumeReferences;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetTestUtil;
@@ -63,6 +66,7 @@ import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.Time;
 import org.junit.Before;
 import org.junit.Test;
+import org.mockito.Mockito;
 
 /**
  * Tests {@link DirectoryScanner} handling of differences
@@ -934,4 +938,50 @@ public class TestDirectoryScanner {
         new File(TEST_VOLUME.getFinalizedDir(BPID_2).getAbsolutePath(),
             "blk_567__1004.meta"));
   }
+
+  /**
+   * Test the behavior of exception handling during directory scan operation.
+   * Directory scanner shouldn't abort the scan on every directory just because
+   * one had an error.
+   */
+  @Test(timeout = 60000)
+  public void testExceptionHandlingWhileDirectoryScan() throws Exception {
+    cluster = new MiniDFSCluster.Builder(CONF).build();
+    try {
+      cluster.waitActive();
+      bpid = cluster.getNamesystem().getBlockPoolId();
+      fds = DataNodeTestUtils.getFSDataset(cluster.getDataNodes().get(0));
+      client = cluster.getFileSystem().getClient();
+      CONF.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THREADS_KEY, 1);
+      DataNode dataNode = cluster.getDataNodes().get(0);
+
+      // Add files with 2 blocks
+      createFile(GenericTestUtils.getMethodName(), BLOCK_LENGTH * 2, false);
+
+      // Inject error on #getFinalizedDir() so that ReportCompiler#call() will
+      // hit exception while preparing the block info report list.
+      List<FsVolumeSpi> volumes = new ArrayList<>();
+      Iterator<FsVolumeSpi> iterator = fds.getFsVolumeReferences().iterator();
+      while (iterator.hasNext()) {
+        FsVolumeSpi volume = iterator.next();
+        FsVolumeSpi spy = Mockito.spy(volume);
+        Mockito.doThrow(new IOException("Error while getFinalizedDir"))
+            .when(spy).getFinalizedDir(volume.getBlockPoolList()[0]);
+        volumes.add(spy);
+      }
+      FsVolumeReferences volReferences = new FsVolumeReferences(volumes);
+      FsDatasetSpi<? extends FsVolumeSpi> spyFds = Mockito.spy(fds);
+      Mockito.doReturn(volReferences).when(spyFds).getFsVolumeReferences();
+
+      scanner = new DirectoryScanner(dataNode, spyFds, CONF);
+      scanner.setRetainDiffs(true);
+      scanner.reconcile();
+    } finally {
+      if (scanner != null) {
+        scanner.shutdown();
+        scanner = null;
+      }
+      cluster.shutdown();
+    }
+  }
 }


[38/50] hadoop git commit: YARN-4219. New levelDB cache storage for timeline v1.5. Contributed by Li Lu

Posted by ar...@apache.org.
YARN-4219. New levelDB cache storage for timeline v1.5. Contributed by
Li Lu


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

Branch: refs/heads/HDFS-1312
Commit: 9fab22b36673e7f1a0bb629d2c07966ac2482e99
Parents: 61382ff
Author: Xuan <xg...@apache.org>
Authored: Thu Jan 28 14:24:22 2016 -0800
Committer: Xuan <xg...@apache.org>
Committed: Thu Jan 28 14:24:22 2016 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   2 +
 .../hadoop/yarn/conf/YarnConfiguration.java     |   7 +
 .../src/main/resources/yarn-default.xml         |   8 +
 .../timeline/KeyValueBasedTimelineStore.java    | 574 +++++++++++++++++++
 .../server/timeline/MemoryTimelineStore.java    | 491 ++--------------
 .../timeline/TimelineStoreMapAdapter.java       |  60 ++
 .../yarn/server/timeline/util/LeveldbUtils.java |   7 +
 .../pom.xml                                     |   4 +
 .../yarn/server/timeline/EntityCacheItem.java   |   3 +-
 .../timeline/LevelDBCacheTimelineStore.java     | 316 ++++++++++
 .../server/timeline/PluginStoreTestUtils.java   |   2 +-
 .../timeline/TestLevelDBCacheTimelineStore.java |  94 +++
 12 files changed, 1114 insertions(+), 454 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fab22b3/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 1b57a3d..8eaed42 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -384,6 +384,8 @@ Release 2.8.0 - UNRELEASED
     YARN-4265. Provide new timeline plugin storage to support fine-grained entity
     caching. (Li Lu and Jason Lowe via junping_du)
 
+    YARN-4219. New levelDB cache storage for timeline v1.5. (Li Lu via xgong)
+
   IMPROVEMENTS
 
     YARN-644. Basic null check is not performed on passed in arguments before

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fab22b3/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 e214a86..d84c155 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
@@ -1702,6 +1702,13 @@ public class YarnConfiguration extends Configuration {
       DEFAULT_TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_RETRY_POLICY_SPEC =
       "2000, 500";
 
+  public static final String TIMELINE_SERVICE_LEVELDB_CACHE_READ_CACHE_SIZE =
+      TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_PREFIX
+          + "leveldb-cache-read-cache-size";
+
+  public static final long
+      DEFAULT_TIMELINE_SERVICE_LEVELDB_CACHE_READ_CACHE_SIZE = 10 * 1024 * 1024;
+
   public static final String TIMELINE_SERVICE_CLIENT_FD_FLUSH_INTERVAL_SECS =
       TIMELINE_SERVICE_CLIENT_PREFIX + "fd-flush-interval-secs";
   public static final long

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fab22b3/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 6508a2a..e33d23e 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
@@ -2036,6 +2036,14 @@
     <value>604800</value>
   </property>
 
+  <property>
+    <name>yarn.timeline-service.entity-group-fs-store.leveldb-cache-read-cache-size</name>
+    <description>
+      Read cache size for the leveldb cache storage in ATS v1.5 plugin storage.
+    </description>
+    <value>10485760</value>
+  </property>
+
   <!--  Shared Cache Configuration -->
 
   <property>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fab22b3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/KeyValueBasedTimelineStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/KeyValueBasedTimelineStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/KeyValueBasedTimelineStore.java
new file mode 100644
index 0000000..79e2bf2
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/KeyValueBasedTimelineStore.java
@@ -0,0 +1,574 @@
+/**
+ * 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.timeline;
+
+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.service.AbstractService;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineDomains;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEvents;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEvents.EventsOfOneEntity;
+import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
+import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse.TimelinePutError;
+import org.apache.hadoop.yarn.server.timeline.TimelineDataManager.CheckAcl;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.SortedSet;
+
+import static org.apache.hadoop.yarn.server.timeline.TimelineDataManager.DEFAULT_DOMAIN_ID;
+
+/**
+ * Map based implementation of {@link TimelineStore}. A hash map
+ * implementation should be connected to this implementation through a
+ * {@link TimelineStoreMapAdapter}.
+ *
+ * The methods are synchronized to avoid concurrent modifications.
+ *
+ */
+@Private
+@Unstable
+abstract class KeyValueBasedTimelineStore
+    extends AbstractService implements TimelineStore {
+
+  protected TimelineStoreMapAdapter<EntityIdentifier, TimelineEntity> entities;
+  protected TimelineStoreMapAdapter<EntityIdentifier, Long> entityInsertTimes;
+  protected TimelineStoreMapAdapter<String, TimelineDomain> domainById;
+  protected TimelineStoreMapAdapter<String, Set<TimelineDomain>> domainsByOwner;
+
+  private boolean serviceStopped = false;
+
+  private static final Log LOG
+      = LogFactory.getLog(KeyValueBasedTimelineStore.class);
+
+  public KeyValueBasedTimelineStore() {
+    super(KeyValueBasedTimelineStore.class.getName());
+  }
+
+  public KeyValueBasedTimelineStore(String name) {
+    super(name);
+  }
+
+  public synchronized boolean getServiceStopped() {
+    return serviceStopped;
+  }
+
+  @Override
+  protected synchronized void serviceStop() throws Exception {
+    serviceStopped = true;
+    super.serviceStop();
+  }
+
+  @Override
+  public synchronized TimelineEntities getEntities(String entityType, Long limit,
+      Long windowStart, Long windowEnd, String fromId, Long fromTs,
+      NameValuePair primaryFilter, Collection<NameValuePair> secondaryFilters,
+      EnumSet<Field> fields, CheckAcl checkAcl) throws IOException {
+    if (getServiceStopped()) {
+      LOG.info("Service stopped, return null for the storage");
+      return null;
+    }
+    if (limit == null) {
+      limit = DEFAULT_LIMIT;
+    }
+    if (windowStart == null) {
+      windowStart = Long.MIN_VALUE;
+    }
+    if (windowEnd == null) {
+      windowEnd = Long.MAX_VALUE;
+    }
+    if (fields == null) {
+      fields = EnumSet.allOf(Field.class);
+    }
+
+    Iterator<TimelineEntity> entityIterator = null;
+    if (fromId != null) {
+      TimelineEntity firstEntity = entities.get(new EntityIdentifier(fromId,
+          entityType));
+      if (firstEntity == null) {
+        return new TimelineEntities();
+      } else {
+        entityIterator = entities.valueSetIterator(firstEntity);
+      }
+    }
+    if (entityIterator == null) {
+      entityIterator = entities.valueSetIterator();
+    }
+
+    List<TimelineEntity> entitiesSelected = new ArrayList<TimelineEntity>();
+    while (entityIterator.hasNext()) {
+      TimelineEntity entity = entityIterator.next();
+      if (entitiesSelected.size() >= limit) {
+        break;
+      }
+      if (!entity.getEntityType().equals(entityType)) {
+        continue;
+      }
+      if (entity.getStartTime() <= windowStart) {
+        continue;
+      }
+      if (entity.getStartTime() > windowEnd) {
+        continue;
+      }
+      if (fromTs != null && entityInsertTimes.get(new EntityIdentifier(
+          entity.getEntityId(), entity.getEntityType())) > fromTs) {
+        continue;
+      }
+      if (primaryFilter != null &&
+          !KeyValueBasedTimelineStoreUtils.matchPrimaryFilter(
+              entity.getPrimaryFilters(), primaryFilter)) {
+        continue;
+      }
+      if (secondaryFilters != null) { // AND logic
+        boolean flag = true;
+        for (NameValuePair secondaryFilter : secondaryFilters) {
+          if (secondaryFilter != null && !KeyValueBasedTimelineStoreUtils
+              .matchPrimaryFilter(entity.getPrimaryFilters(), secondaryFilter)
+              && !KeyValueBasedTimelineStoreUtils.matchFilter(
+              entity.getOtherInfo(), secondaryFilter)) {
+            flag = false;
+            break;
+          }
+        }
+        if (!flag) {
+          continue;
+        }
+      }
+      if (entity.getDomainId() == null) {
+        entity.setDomainId(DEFAULT_DOMAIN_ID);
+      }
+      if (checkAcl == null || checkAcl.check(entity)) {
+        entitiesSelected.add(entity);
+      }
+    }
+    List<TimelineEntity> entitiesToReturn = new ArrayList<TimelineEntity>();
+    for (TimelineEntity entitySelected : entitiesSelected) {
+      entitiesToReturn.add(KeyValueBasedTimelineStoreUtils.maskFields(
+          entitySelected, fields));
+    }
+    Collections.sort(entitiesToReturn);
+    TimelineEntities entitiesWrapper = new TimelineEntities();
+    entitiesWrapper.setEntities(entitiesToReturn);
+    return entitiesWrapper;
+  }
+
+  @Override
+  public synchronized TimelineEntity getEntity(String entityId, String entityType,
+      EnumSet<Field> fieldsToRetrieve) {
+    if (getServiceStopped()) {
+      LOG.info("Service stopped, return null for the storage");
+      return null;
+    }
+    if (fieldsToRetrieve == null) {
+      fieldsToRetrieve = EnumSet.allOf(Field.class);
+    }
+    TimelineEntity
+        entity = entities.get(new EntityIdentifier(entityId, entityType));
+    if (entity == null) {
+      return null;
+    } else {
+      return KeyValueBasedTimelineStoreUtils.maskFields(
+          entity, fieldsToRetrieve);
+    }
+  }
+
+  @Override
+  public synchronized TimelineEvents getEntityTimelines(String entityType,
+      SortedSet<String> entityIds, Long limit, Long windowStart,
+      Long windowEnd,
+      Set<String> eventTypes) {
+    if (getServiceStopped()) {
+      LOG.info("Service stopped, return null for the storage");
+      return null;
+    }
+    TimelineEvents allEvents = new TimelineEvents();
+    if (entityIds == null) {
+      return allEvents;
+    }
+    if (limit == null) {
+      limit = DEFAULT_LIMIT;
+    }
+    if (windowStart == null) {
+      windowStart = Long.MIN_VALUE;
+    }
+    if (windowEnd == null) {
+      windowEnd = Long.MAX_VALUE;
+    }
+    for (String entityId : entityIds) {
+      EntityIdentifier entityID = new EntityIdentifier(entityId, entityType);
+      TimelineEntity entity = entities.get(entityID);
+      if (entity == null) {
+        continue;
+      }
+      EventsOfOneEntity events = new EventsOfOneEntity();
+      events.setEntityId(entityId);
+      events.setEntityType(entityType);
+      for (TimelineEvent event : entity.getEvents()) {
+        if (events.getEvents().size() >= limit) {
+          break;
+        }
+        if (event.getTimestamp() <= windowStart) {
+          continue;
+        }
+        if (event.getTimestamp() > windowEnd) {
+          continue;
+        }
+        if (eventTypes != null && !eventTypes.contains(event.getEventType())) {
+          continue;
+        }
+        events.addEvent(event);
+      }
+      allEvents.addEvent(events);
+    }
+    return allEvents;
+  }
+
+  @Override
+  public TimelineDomain getDomain(String domainId)
+      throws IOException {
+    if (getServiceStopped()) {
+      LOG.info("Service stopped, return null for the storage");
+      return null;
+    }
+    TimelineDomain domain = domainById.get(domainId);
+    if (domain == null) {
+      return null;
+    } else {
+      return KeyValueBasedTimelineStoreUtils.createTimelineDomain(
+          domain.getId(),
+          domain.getDescription(),
+          domain.getOwner(),
+          domain.getReaders(),
+          domain.getWriters(),
+          domain.getCreatedTime(),
+          domain.getModifiedTime());
+    }
+  }
+
+  @Override
+  public TimelineDomains getDomains(String owner)
+      throws IOException {
+    if (getServiceStopped()) {
+      LOG.info("Service stopped, return null for the storage");
+      return null;
+    }
+    List<TimelineDomain> domains = new ArrayList<TimelineDomain>();
+    Set<TimelineDomain> domainsOfOneOwner = domainsByOwner.get(owner);
+    if (domainsOfOneOwner == null) {
+      return new TimelineDomains();
+    }
+    for (TimelineDomain domain : domainsByOwner.get(owner)) {
+      TimelineDomain domainToReturn = KeyValueBasedTimelineStoreUtils
+          .createTimelineDomain(
+              domain.getId(),
+              domain.getDescription(),
+              domain.getOwner(),
+              domain.getReaders(),
+              domain.getWriters(),
+              domain.getCreatedTime(),
+              domain.getModifiedTime());
+      domains.add(domainToReturn);
+    }
+    Collections.sort(domains, new Comparator<TimelineDomain>() {
+      @Override
+      public int compare(
+          TimelineDomain domain1, TimelineDomain domain2) {
+         int result = domain2.getCreatedTime().compareTo(
+             domain1.getCreatedTime());
+         if (result == 0) {
+           return domain2.getModifiedTime().compareTo(
+               domain1.getModifiedTime());
+         } else {
+           return result;
+         }
+      }
+    });
+    TimelineDomains domainsToReturn = new TimelineDomains();
+    domainsToReturn.addDomains(domains);
+    return domainsToReturn;
+  }
+
+  @Override
+  public synchronized TimelinePutResponse put(TimelineEntities data) {
+    TimelinePutResponse response = new TimelinePutResponse();
+    if (getServiceStopped()) {
+      LOG.info("Service stopped, return null for the storage");
+      TimelinePutError error = new TimelinePutError();
+      error.setErrorCode(TimelinePutError.IO_EXCEPTION);
+      response.addError(error);
+      return response;
+    }
+    for (TimelineEntity entity : data.getEntities()) {
+      EntityIdentifier entityId =
+          new EntityIdentifier(entity.getEntityId(), entity.getEntityType());
+      // store entity info in memory
+      TimelineEntity existingEntity = entities.get(entityId);
+      boolean needsPut = false;
+      if (existingEntity == null) {
+        existingEntity = new TimelineEntity();
+        existingEntity.setEntityId(entity.getEntityId());
+        existingEntity.setEntityType(entity.getEntityType());
+        existingEntity.setStartTime(entity.getStartTime());
+        if (entity.getDomainId() == null ||
+            entity.getDomainId().length() == 0) {
+          TimelinePutError error = new TimelinePutError();
+          error.setEntityId(entityId.getId());
+          error.setEntityType(entityId.getType());
+          error.setErrorCode(TimelinePutError.NO_DOMAIN);
+          response.addError(error);
+          continue;
+        }
+        existingEntity.setDomainId(entity.getDomainId());
+        // insert a new entity to the storage, update insert time map
+        entityInsertTimes.put(entityId, System.currentTimeMillis());
+        needsPut = true;
+      }
+      if (entity.getEvents() != null) {
+        if (existingEntity.getEvents() == null) {
+          existingEntity.setEvents(entity.getEvents());
+        } else {
+          existingEntity.addEvents(entity.getEvents());
+        }
+        Collections.sort(existingEntity.getEvents());
+        needsPut = true;
+      }
+      // check startTime
+      if (existingEntity.getStartTime() == null) {
+        if (existingEntity.getEvents() == null
+            || existingEntity.getEvents().isEmpty()) {
+          TimelinePutError error = new TimelinePutError();
+          error.setEntityId(entityId.getId());
+          error.setEntityType(entityId.getType());
+          error.setErrorCode(TimelinePutError.NO_START_TIME);
+          response.addError(error);
+          entities.remove(entityId);
+          entityInsertTimes.remove(entityId);
+          continue;
+        } else {
+          Long min = Long.MAX_VALUE;
+          for (TimelineEvent e : entity.getEvents()) {
+            if (min > e.getTimestamp()) {
+              min = e.getTimestamp();
+            }
+          }
+          existingEntity.setStartTime(min);
+          needsPut = true;
+        }
+      }
+      if (entity.getPrimaryFilters() != null) {
+        if (existingEntity.getPrimaryFilters() == null) {
+          existingEntity.setPrimaryFilters(new HashMap<String, Set<Object>>());
+        }
+        for (Entry<String, Set<Object>> pf :
+            entity.getPrimaryFilters().entrySet()) {
+          for (Object pfo : pf.getValue()) {
+            existingEntity.addPrimaryFilter(pf.getKey(),
+                KeyValueBasedTimelineStoreUtils.compactNumber(pfo));
+            needsPut = true;
+          }
+        }
+      }
+      if (entity.getOtherInfo() != null) {
+        if (existingEntity.getOtherInfo() == null) {
+          existingEntity.setOtherInfo(new HashMap<String, Object>());
+        }
+        for (Entry<String, Object> info : entity.getOtherInfo().entrySet()) {
+          existingEntity.addOtherInfo(info.getKey(),
+              KeyValueBasedTimelineStoreUtils.compactNumber(info.getValue()));
+          needsPut = true;
+        }
+      }
+      if (needsPut) {
+        entities.put(entityId, existingEntity);
+      }
+
+      // relate it to other entities
+      if (entity.getRelatedEntities() == null) {
+        continue;
+      }
+      for (Entry<String, Set<String>> partRelatedEntities : entity
+          .getRelatedEntities().entrySet()) {
+        if (partRelatedEntities == null) {
+          continue;
+        }
+        for (String idStr : partRelatedEntities.getValue()) {
+          EntityIdentifier relatedEntityId =
+              new EntityIdentifier(idStr, partRelatedEntities.getKey());
+          TimelineEntity relatedEntity = entities.get(relatedEntityId);
+          if (relatedEntity != null) {
+            if (relatedEntity.getDomainId().equals(
+                existingEntity.getDomainId())) {
+              relatedEntity.addRelatedEntity(
+                  existingEntity.getEntityType(), existingEntity.getEntityId());
+              entities.put(relatedEntityId, relatedEntity);
+            } else {
+              // in this case the entity will be put, but the relation will be
+              // ignored
+              TimelinePutError error = new TimelinePutError();
+              error.setEntityType(existingEntity.getEntityType());
+              error.setEntityId(existingEntity.getEntityId());
+              error.setErrorCode(TimelinePutError.FORBIDDEN_RELATION);
+              response.addError(error);
+            }
+          } else {
+            relatedEntity = new TimelineEntity();
+            relatedEntity.setEntityId(relatedEntityId.getId());
+            relatedEntity.setEntityType(relatedEntityId.getType());
+            relatedEntity.setStartTime(existingEntity.getStartTime());
+            relatedEntity.addRelatedEntity(existingEntity.getEntityType(),
+                existingEntity.getEntityId());
+            relatedEntity.setDomainId(existingEntity.getDomainId());
+            entities.put(relatedEntityId, relatedEntity);
+            entityInsertTimes.put(relatedEntityId, System.currentTimeMillis());
+          }
+        }
+      }
+    }
+    return response;
+  }
+
+  public void put(TimelineDomain domain) throws IOException {
+    if (getServiceStopped()) {
+      LOG.info("Service stopped, return null for the storage");
+      return;
+    }
+    TimelineDomain domainToReplace =
+        domainById.get(domain.getId());
+    Long currentTimestamp = System.currentTimeMillis();
+    TimelineDomain domainToStore
+        = KeyValueBasedTimelineStoreUtils.createTimelineDomain(
+        domain.getId(), domain.getDescription(), domain.getOwner(),
+        domain.getReaders(), domain.getWriters(),
+        (domainToReplace == null ?
+            currentTimestamp : domainToReplace.getCreatedTime()),
+        currentTimestamp);
+    domainById.put(domainToStore.getId(), domainToStore);
+    Set<TimelineDomain> domainsByOneOwner =
+        domainsByOwner.get(domainToStore.getOwner());
+    if (domainsByOneOwner == null) {
+      domainsByOneOwner = new HashSet<TimelineDomain>();
+      domainsByOwner.put(domainToStore.getOwner(), domainsByOneOwner);
+    }
+    if (domainToReplace != null) {
+      domainsByOneOwner.remove(domainToReplace);
+    }
+    domainsByOneOwner.add(domainToStore);
+  }
+
+  private static class KeyValueBasedTimelineStoreUtils {
+
+    static TimelineDomain createTimelineDomain(
+        String id, String description, String owner,
+        String readers, String writers,
+        Long createdTime, Long modifiedTime) {
+      TimelineDomain domainToStore = new TimelineDomain();
+      domainToStore.setId(id);
+      domainToStore.setDescription(description);
+      domainToStore.setOwner(owner);
+      domainToStore.setReaders(readers);
+      domainToStore.setWriters(writers);
+      domainToStore.setCreatedTime(createdTime);
+      domainToStore.setModifiedTime(modifiedTime);
+      return domainToStore;
+    }
+
+    static TimelineEntity maskFields(
+        TimelineEntity entity, EnumSet<Field> fields) {
+      // Conceal the fields that are not going to be exposed
+      TimelineEntity entityToReturn = new TimelineEntity();
+      entityToReturn.setEntityId(entity.getEntityId());
+      entityToReturn.setEntityType(entity.getEntityType());
+      entityToReturn.setStartTime(entity.getStartTime());
+      entityToReturn.setDomainId(entity.getDomainId());
+      // Deep copy
+      if (fields.contains(Field.EVENTS)) {
+        entityToReturn.addEvents(entity.getEvents());
+      } else if (fields.contains(Field.LAST_EVENT_ONLY)) {
+        entityToReturn.addEvent(entity.getEvents().get(0));
+      } else {
+        entityToReturn.setEvents(null);
+      }
+      if (fields.contains(Field.RELATED_ENTITIES)) {
+        entityToReturn.addRelatedEntities(entity.getRelatedEntities());
+      } else {
+        entityToReturn.setRelatedEntities(null);
+      }
+      if (fields.contains(Field.PRIMARY_FILTERS)) {
+        entityToReturn.addPrimaryFilters(entity.getPrimaryFilters());
+      } else {
+        entityToReturn.setPrimaryFilters(null);
+      }
+      if (fields.contains(Field.OTHER_INFO)) {
+        entityToReturn.addOtherInfo(entity.getOtherInfo());
+      } else {
+        entityToReturn.setOtherInfo(null);
+      }
+      return entityToReturn;
+    }
+
+    static boolean matchFilter(Map<String, Object> tags,
+        NameValuePair filter) {
+      Object value = tags.get(filter.getName());
+      if (value == null) { // doesn't have the filter
+        return false;
+      } else if (!value.equals(filter.getValue())) { // doesn't match the filter
+        return false;
+      }
+      return true;
+    }
+
+    static boolean matchPrimaryFilter(Map<String, Set<Object>> tags,
+        NameValuePair filter) {
+      Set<Object> value = tags.get(filter.getName());
+      if (value == null) { // doesn't have the filter
+        return false;
+      } else {
+        return value.contains(filter.getValue());
+      }
+    }
+
+    static Object compactNumber(Object o) {
+      if (o instanceof Long) {
+        Long l = (Long) o;
+        if (l >= Integer.MIN_VALUE && l <= Integer.MAX_VALUE) {
+          return l.intValue();
+        }
+      }
+      return o;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fab22b3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/MemoryTimelineStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/MemoryTimelineStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/MemoryTimelineStore.java
index 3489114..5c2db00 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/MemoryTimelineStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/MemoryTimelineStore.java
@@ -18,39 +18,14 @@
 
 package org.apache.hadoop.yarn.server.timeline;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.EnumSet;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.Iterator;
-import java.util.List;
 import java.util.Map;
-import java.util.Map.Entry;
-import java.util.PriorityQueue;
-import java.util.Set;
-import java.util.SortedSet;
 import java.util.TreeSet;
 
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.service.AbstractService;
-import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
-import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
-import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
-import org.apache.hadoop.yarn.api.records.timeline.TimelineEvents;
-import org.apache.hadoop.yarn.api.records.timeline.TimelineEvents.EventsOfOneEntity;
-import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain;
-import org.apache.hadoop.yarn.api.records.timeline.TimelineDomains;
-import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
-import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse.TimelinePutError;
-import org.apache.hadoop.yarn.server.timeline.TimelineDataManager.CheckAcl;
-
-import static org.apache.hadoop.yarn.server.timeline.TimelineDataManager.DEFAULT_DOMAIN_ID;
-
 /**
  * In-memory implementation of {@link TimelineStore}. This
  * implementation is for test purpose only. If users improperly instantiate it,
@@ -62,448 +37,60 @@ import static org.apache.hadoop.yarn.server.timeline.TimelineDataManager.DEFAULT
  */
 @Private
 @Unstable
-public class MemoryTimelineStore
-    extends AbstractService implements TimelineStore {
-
-  private Map<EntityIdentifier, TimelineEntity> entities =
-      new HashMap<EntityIdentifier, TimelineEntity>();
-  private Map<EntityIdentifier, Long> entityInsertTimes =
-      new HashMap<EntityIdentifier, Long>();
-  private Map<String, TimelineDomain> domainsById =
-      new HashMap<String, TimelineDomain>();
-  private Map<String, Set<TimelineDomain>> domainsByOwner =
-      new HashMap<String, Set<TimelineDomain>>();
+public class MemoryTimelineStore extends KeyValueBasedTimelineStore {
 
-  public MemoryTimelineStore() {
-    super(MemoryTimelineStore.class.getName());
-  }
+  static class HashMapStoreAdapter<K, V>
+      implements TimelineStoreMapAdapter<K, V> {
+    Map<K, V> internalMap = new HashMap<>();
 
-  @Override
-  public synchronized TimelineEntities getEntities(String entityType, Long limit,
-      Long windowStart, Long windowEnd, String fromId, Long fromTs,
-      NameValuePair primaryFilter, Collection<NameValuePair> secondaryFilters,
-      EnumSet<Field> fields, CheckAcl checkAcl) throws IOException {
-    if (limit == null) {
-      limit = DEFAULT_LIMIT;
-    }
-    if (windowStart == null) {
-      windowStart = Long.MIN_VALUE;
-    }
-    if (windowEnd == null) {
-      windowEnd = Long.MAX_VALUE;
-    }
-    if (fields == null) {
-      fields = EnumSet.allOf(Field.class);
+    @Override
+    public V get(K key) {
+      return internalMap.get(key);
     }
 
-    Iterator<TimelineEntity> entityIterator = null;
-    if (fromId != null) {
-      TimelineEntity firstEntity = entities.get(new EntityIdentifier(fromId,
-          entityType));
-      if (firstEntity == null) {
-        return new TimelineEntities();
-      } else {
-        entityIterator = new TreeSet<TimelineEntity>(entities.values())
-            .tailSet(firstEntity, true).iterator();
-      }
-    }
-    if (entityIterator == null) {
-      entityIterator = new PriorityQueue<TimelineEntity>(entities.values())
-          .iterator();
+    @Override
+    public void put(K key, V value) {
+      internalMap.put(key, value);
     }
 
-    List<TimelineEntity> entitiesSelected = new ArrayList<TimelineEntity>();
-    while (entityIterator.hasNext()) {
-      TimelineEntity entity = entityIterator.next();
-      if (entitiesSelected.size() >= limit) {
-        break;
-      }
-      if (!entity.getEntityType().equals(entityType)) {
-        continue;
-      }
-      if (entity.getStartTime() <= windowStart) {
-        continue;
-      }
-      if (entity.getStartTime() > windowEnd) {
-        continue;
-      }
-      if (fromTs != null && entityInsertTimes.get(new EntityIdentifier(
-          entity.getEntityId(), entity.getEntityType())) > fromTs) {
-        continue;
-      }
-      if (primaryFilter != null &&
-          !matchPrimaryFilter(entity.getPrimaryFilters(), primaryFilter)) {
-        continue;
-      }
-      if (secondaryFilters != null) { // AND logic
-        boolean flag = true;
-        for (NameValuePair secondaryFilter : secondaryFilters) {
-          if (secondaryFilter != null && !matchPrimaryFilter(
-              entity.getPrimaryFilters(), secondaryFilter) &&
-              !matchFilter(entity.getOtherInfo(), secondaryFilter)) {
-            flag = false;
-            break;
-          }
-        }
-        if (!flag) {
-          continue;
-        }
-      }
-      if (entity.getDomainId() == null) {
-        entity.setDomainId(DEFAULT_DOMAIN_ID);
-      }
-      if (checkAcl == null || checkAcl.check(entity)) {
-        entitiesSelected.add(entity);
-      }
-    }
-    List<TimelineEntity> entitiesToReturn = new ArrayList<TimelineEntity>();
-    for (TimelineEntity entitySelected : entitiesSelected) {
-      entitiesToReturn.add(maskFields(entitySelected, fields));
+    @Override
+    public void remove(K key) {
+      internalMap.remove(key);
     }
-    Collections.sort(entitiesToReturn);
-    TimelineEntities entitiesWrapper = new TimelineEntities();
-    entitiesWrapper.setEntities(entitiesToReturn);
-    return entitiesWrapper;
-  }
-
-  @Override
-  public synchronized TimelineEntity getEntity(String entityId, String entityType,
-      EnumSet<Field> fieldsToRetrieve) {
-    if (fieldsToRetrieve == null) {
-      fieldsToRetrieve = EnumSet.allOf(Field.class);
-    }
-    TimelineEntity entity = entities.get(new EntityIdentifier(entityId, entityType));
-    if (entity == null) {
-      return null;
-    } else {
-      return maskFields(entity, fieldsToRetrieve);
-    }
-  }
 
-  @Override
-  public synchronized TimelineEvents getEntityTimelines(String entityType,
-      SortedSet<String> entityIds, Long limit, Long windowStart,
-      Long windowEnd,
-      Set<String> eventTypes) {
-    TimelineEvents allEvents = new TimelineEvents();
-    if (entityIds == null) {
-      return allEvents;
+    @Override
+    public Iterator<V>
+    valueSetIterator() {
+      return new TreeSet<>(internalMap.values()).iterator();
     }
-    if (limit == null) {
-      limit = DEFAULT_LIMIT;
-    }
-    if (windowStart == null) {
-      windowStart = Long.MIN_VALUE;
-    }
-    if (windowEnd == null) {
-      windowEnd = Long.MAX_VALUE;
-    }
-    for (String entityId : entityIds) {
-      EntityIdentifier entityID = new EntityIdentifier(entityId, entityType);
-      TimelineEntity entity = entities.get(entityID);
-      if (entity == null) {
-        continue;
-      }
-      EventsOfOneEntity events = new EventsOfOneEntity();
-      events.setEntityId(entityId);
-      events.setEntityType(entityType);
-      for (TimelineEvent event : entity.getEvents()) {
-        if (events.getEvents().size() >= limit) {
-          break;
-        }
-        if (event.getTimestamp() <= windowStart) {
-          continue;
-        }
-        if (event.getTimestamp() > windowEnd) {
-          continue;
-        }
-        if (eventTypes != null && !eventTypes.contains(event.getEventType())) {
-          continue;
-        }
-        events.addEvent(event);
-      }
-      allEvents.addEvent(events);
-    }
-    return allEvents;
-  }
 
-  @Override
-  public TimelineDomain getDomain(String domainId)
-      throws IOException {
-    TimelineDomain domain = domainsById.get(domainId);
-    if (domain == null) {
-      return null;
-    } else {
-      return createTimelineDomain(
-          domain.getId(),
-          domain.getDescription(),
-          domain.getOwner(),
-          domain.getReaders(),
-          domain.getWriters(),
-          domain.getCreatedTime(),
-          domain.getModifiedTime());
-    }
-  }
-
-  @Override
-  public TimelineDomains getDomains(String owner)
-      throws IOException {
-    List<TimelineDomain> domains = new ArrayList<TimelineDomain>();
-    Set<TimelineDomain> domainsOfOneOwner = domainsByOwner.get(owner);
-    if (domainsOfOneOwner == null) {
-      return new TimelineDomains();
-    }
-    for (TimelineDomain domain : domainsByOwner.get(owner)) {
-      TimelineDomain domainToReturn = createTimelineDomain(
-          domain.getId(),
-          domain.getDescription(),
-          domain.getOwner(),
-          domain.getReaders(),
-          domain.getWriters(),
-          domain.getCreatedTime(),
-          domain.getModifiedTime());
-      domains.add(domainToReturn);
-    }
-    Collections.sort(domains, new Comparator<TimelineDomain>() {
-      @Override
-      public int compare(
-          TimelineDomain domain1, TimelineDomain domain2) {
-         int result = domain2.getCreatedTime().compareTo(
-             domain1.getCreatedTime());
-         if (result == 0) {
-           return domain2.getModifiedTime().compareTo(
-               domain1.getModifiedTime());
-         } else {
-           return result;
-         }
-      }
-    });
-    TimelineDomains domainsToReturn = new TimelineDomains();
-    domainsToReturn.addDomains(domains);
-    return domainsToReturn;
-  }
-
-  @Override
-  public synchronized TimelinePutResponse put(TimelineEntities data) {
-    TimelinePutResponse response = new TimelinePutResponse();
-    for (TimelineEntity entity : data.getEntities()) {
-      EntityIdentifier entityId =
-          new EntityIdentifier(entity.getEntityId(), entity.getEntityType());
-      // store entity info in memory
-      TimelineEntity existingEntity = entities.get(entityId);
-      if (existingEntity == null) {
-        existingEntity = new TimelineEntity();
-        existingEntity.setEntityId(entity.getEntityId());
-        existingEntity.setEntityType(entity.getEntityType());
-        existingEntity.setStartTime(entity.getStartTime());
-        if (entity.getDomainId() == null ||
-            entity.getDomainId().length() == 0) {
-          TimelinePutError error = new TimelinePutError();
-          error.setEntityId(entityId.getId());
-          error.setEntityType(entityId.getType());
-          error.setErrorCode(TimelinePutError.NO_DOMAIN);
-          response.addError(error);
-          continue;
-        }
-        existingEntity.setDomainId(entity.getDomainId());
-        entities.put(entityId, existingEntity);
-        entityInsertTimes.put(entityId, System.currentTimeMillis());
-      }
-      if (entity.getEvents() != null) {
-        if (existingEntity.getEvents() == null) {
-          existingEntity.setEvents(entity.getEvents());
-        } else {
-          existingEntity.addEvents(entity.getEvents());
-        }
-        Collections.sort(existingEntity.getEvents());
-      }
-      // check startTime
-      if (existingEntity.getStartTime() == null) {
-        if (existingEntity.getEvents() == null
-            || existingEntity.getEvents().isEmpty()) {
-          TimelinePutError error = new TimelinePutError();
-          error.setEntityId(entityId.getId());
-          error.setEntityType(entityId.getType());
-          error.setErrorCode(TimelinePutError.NO_START_TIME);
-          response.addError(error);
-          entities.remove(entityId);
-          entityInsertTimes.remove(entityId);
-          continue;
-        } else {
-          Long min = Long.MAX_VALUE;
-          for (TimelineEvent e : entity.getEvents()) {
-            if (min > e.getTimestamp()) {
-              min = e.getTimestamp();
-            }
-          }
-          existingEntity.setStartTime(min);
-        }
-      }
-      if (entity.getPrimaryFilters() != null) {
-        if (existingEntity.getPrimaryFilters() == null) {
-          existingEntity.setPrimaryFilters(new HashMap<String, Set<Object>>());
-        }
-        for (Entry<String, Set<Object>> pf :
-            entity.getPrimaryFilters().entrySet()) {
-          for (Object pfo : pf.getValue()) {
-            existingEntity.addPrimaryFilter(pf.getKey(), maybeConvert(pfo));
-          }
-        }
-      }
-      if (entity.getOtherInfo() != null) {
-        if (existingEntity.getOtherInfo() == null) {
-          existingEntity.setOtherInfo(new HashMap<String, Object>());
-        }
-        for (Entry<String, Object> info : entity.getOtherInfo().entrySet()) {
-          existingEntity.addOtherInfo(info.getKey(),
-              maybeConvert(info.getValue()));
-        }
-      }
-      // relate it to other entities
-      if (entity.getRelatedEntities() == null) {
-        continue;
-      }
-      for (Map.Entry<String, Set<String>> partRelatedEntities : entity
-          .getRelatedEntities().entrySet()) {
-        if (partRelatedEntities == null) {
-          continue;
-        }
-        for (String idStr : partRelatedEntities.getValue()) {
-          EntityIdentifier relatedEntityId =
-              new EntityIdentifier(idStr, partRelatedEntities.getKey());
-          TimelineEntity relatedEntity = entities.get(relatedEntityId);
-          if (relatedEntity != null) {
-            if (relatedEntity.getDomainId().equals(
-                existingEntity.getDomainId())) {
-              relatedEntity.addRelatedEntity(
-                  existingEntity.getEntityType(), existingEntity.getEntityId());
-            } else {
-              // in this case the entity will be put, but the relation will be
-              // ignored
-              TimelinePutError error = new TimelinePutError();
-              error.setEntityType(existingEntity.getEntityType());
-              error.setEntityId(existingEntity.getEntityId());
-              error.setErrorCode(TimelinePutError.FORBIDDEN_RELATION);
-              response.addError(error);
-            }
-          } else {
-            relatedEntity = new TimelineEntity();
-            relatedEntity.setEntityId(relatedEntityId.getId());
-            relatedEntity.setEntityType(relatedEntityId.getType());
-            relatedEntity.setStartTime(existingEntity.getStartTime());
-            relatedEntity.addRelatedEntity(existingEntity.getEntityType(),
-                existingEntity.getEntityId());
-            relatedEntity.setDomainId(existingEntity.getDomainId());
-            entities.put(relatedEntityId, relatedEntity);
-            entityInsertTimes.put(relatedEntityId, System.currentTimeMillis());
+    @Override
+    @SuppressWarnings("unchecked")
+    public Iterator<V> valueSetIterator(V minV) {
+      if (minV instanceof Comparable) {
+        TreeSet<V> tempTreeSet = new TreeSet<>();
+        for (V value : internalMap.values()) {
+          if (((Comparable) value).compareTo(minV) >= 0) {
+            tempTreeSet.add(value);
           }
         }
+        return tempTreeSet.iterator();
+      } else {
+        return valueSetIterator();
       }
     }
-    return response;
-  }
-
-  public void put(TimelineDomain domain) throws IOException {
-    TimelineDomain domainToReplace =
-        domainsById.get(domain.getId());
-    Long currentTimestamp = System.currentTimeMillis();
-    TimelineDomain domainToStore = createTimelineDomain(
-        domain.getId(), domain.getDescription(), domain.getOwner(),
-        domain.getReaders(), domain.getWriters(),
-        (domainToReplace == null ?
-            currentTimestamp : domainToReplace.getCreatedTime()),
-        currentTimestamp);
-    domainsById.put(domainToStore.getId(), domainToStore);
-    Set<TimelineDomain> domainsByOneOwner =
-        domainsByOwner.get(domainToStore.getOwner());
-    if (domainsByOneOwner == null) {
-      domainsByOneOwner = new HashSet<TimelineDomain>();
-      domainsByOwner.put(domainToStore.getOwner(), domainsByOneOwner);
-    }
-    if (domainToReplace != null) {
-      domainsByOneOwner.remove(domainToReplace);
-    }
-    domainsByOneOwner.add(domainToStore);
-  }
-
-  private static TimelineDomain createTimelineDomain(
-      String id, String description, String owner,
-      String readers, String writers,
-      Long createdTime, Long modifiedTime) {
-    TimelineDomain domainToStore = new TimelineDomain();
-    domainToStore.setId(id);
-    domainToStore.setDescription(description);
-    domainToStore.setOwner(owner);
-    domainToStore.setReaders(readers);
-    domainToStore.setWriters(writers);
-    domainToStore.setCreatedTime(createdTime);
-    domainToStore.setModifiedTime(modifiedTime);
-    return domainToStore;
-  }
-
-  private static TimelineEntity maskFields(
-      TimelineEntity entity, EnumSet<Field> fields) {
-    // Conceal the fields that are not going to be exposed
-    TimelineEntity entityToReturn = new TimelineEntity();
-    entityToReturn.setEntityId(entity.getEntityId());
-    entityToReturn.setEntityType(entity.getEntityType());
-    entityToReturn.setStartTime(entity.getStartTime());
-    entityToReturn.setDomainId(entity.getDomainId());
-    // Deep copy
-    if (fields.contains(Field.EVENTS)) {
-      entityToReturn.addEvents(entity.getEvents());
-    } else if (fields.contains(Field.LAST_EVENT_ONLY)) {
-      entityToReturn.addEvent(entity.getEvents().get(0));
-    } else {
-      entityToReturn.setEvents(null);
-    }
-    if (fields.contains(Field.RELATED_ENTITIES)) {
-      entityToReturn.addRelatedEntities(entity.getRelatedEntities());
-    } else {
-      entityToReturn.setRelatedEntities(null);
-    }
-    if (fields.contains(Field.PRIMARY_FILTERS)) {
-      entityToReturn.addPrimaryFilters(entity.getPrimaryFilters());
-    } else {
-      entityToReturn.setPrimaryFilters(null);
-    }
-    if (fields.contains(Field.OTHER_INFO)) {
-      entityToReturn.addOtherInfo(entity.getOtherInfo());
-    } else {
-      entityToReturn.setOtherInfo(null);
-    }
-    return entityToReturn;
   }
 
-  private static boolean matchFilter(Map<String, Object> tags,
-      NameValuePair filter) {
-    Object value = tags.get(filter.getName());
-    if (value == null) { // doesn't have the filter
-      return false;
-    } else if (!value.equals(filter.getValue())) { // doesn't match the filter
-      return false;
-    }
-    return true;
-  }
-
-  private static boolean matchPrimaryFilter(Map<String, Set<Object>> tags,
-      NameValuePair filter) {
-    Set<Object> value = tags.get(filter.getName());
-    if (value == null) { // doesn't have the filter
-      return false;
-    } else {
-      return value.contains(filter.getValue());
-    }
+  public MemoryTimelineStore() {
+    this(MemoryTimelineStore.class.getName());
   }
 
-  private static Object maybeConvert(Object o) {
-    if (o instanceof Long) {
-      Long l = (Long)o;
-      if (l >= Integer.MIN_VALUE && l <= Integer.MAX_VALUE) {
-        return l.intValue();
-      }
-    }
-    return o;
+  public MemoryTimelineStore(String name) {
+    super(name);
+    entities = new HashMapStoreAdapter<>();
+    entityInsertTimes = new HashMapStoreAdapter<>();
+    domainById = new HashMapStoreAdapter<>();
+    domainsByOwner = new HashMapStoreAdapter<>();
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fab22b3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineStoreMapAdapter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineStoreMapAdapter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineStoreMapAdapter.java
new file mode 100644
index 0000000..175ed0b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineStoreMapAdapter.java
@@ -0,0 +1,60 @@
+/**
+ * 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.timeline;
+
+import java.util.Iterator;
+
+/**
+ * An adapter for map timeline store implementations
+ * @param <K> the type of the key set
+ * @param <V> the type of the value set
+ */
+interface TimelineStoreMapAdapter<K, V> {
+  /**
+   * @param key
+   * @return map(key)
+   */
+  V get(K key);
+
+  /**
+   * Add mapping key->value in the map
+   * @param key
+   * @param value
+   */
+  void put(K key, V value);
+
+  /**
+   * Remove mapping with key keyToRemove
+   * @param keyToRemove
+   */
+  void remove(K keyToRemove);
+
+  /**
+   * @return the iterator of the value set of the map
+   */
+  Iterator<V> valueSetIterator();
+
+  /**
+   * Return the iterator of the value set of the map, starting from minV if type
+   * V is comparable.
+   * @param minV
+   * @return
+   */
+  Iterator<V> valueSetIterator(V minV);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fab22b3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/util/LeveldbUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/util/LeveldbUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/util/LeveldbUtils.java
index 5638581..82c7f26 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/util/LeveldbUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/util/LeveldbUtils.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.server.timeline.util;
 
 
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.io.WritableComparator;
 
 import java.io.IOException;
@@ -177,4 +178,10 @@ public class LeveldbUtils {
         prefixlen) == 0;
   }
 
+  /**
+   * Default permission mask for the level db dir
+   */
+  public static final FsPermission LEVELDB_DIR_UMASK = FsPermission
+      .createImmutable((short) 0700);
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fab22b3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/pom.xml
index 385ba5d..71f76d3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/pom.xml
@@ -132,5 +132,9 @@
       <artifactId>mockito-all</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
+    </dependency>
   </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fab22b3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/EntityCacheItem.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/EntityCacheItem.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/EntityCacheItem.java
index 37a1d8d..efbf994 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/EntityCacheItem.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/EntityCacheItem.java
@@ -102,7 +102,8 @@ public class EntityCacheItem {
       }
       if (!appLogs.getDetailLogs().isEmpty()) {
         if (store == null) {
-          store = new MemoryTimelineStore();
+          store = new LevelDBCacheTimelineStore(groupId.toString(),
+              "LeveldbCache." + groupId);
           store.init(config);
           store.start();
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fab22b3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/LevelDBCacheTimelineStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/LevelDBCacheTimelineStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/LevelDBCacheTimelineStore.java
new file mode 100644
index 0000000..976241f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/LevelDBCacheTimelineStore.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.server.timeline;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+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;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.timeline.util.LeveldbUtils;
+import org.fusesource.leveldbjni.JniDBFactory;
+import org.iq80.leveldb.DB;
+import org.iq80.leveldb.DBIterator;
+import org.iq80.leveldb.Options;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+
+/**
+ * LevelDB implementation of {@link KeyValueBasedTimelineStore}. This
+ * implementation stores the entity hash map into a LevelDB instance.
+ * There are two partitions of the key space. One partition is to store a
+ * entity id to start time mapping:
+ *
+ * i!ENTITY_ID!ENTITY_TYPE -> ENTITY_START_TIME
+ *
+ * The other partition is to store the actual data:
+ *
+ * e!START_TIME!ENTITY_ID!ENTITY_TYPE -> ENTITY_BYTES
+ *
+ * This storage does not have any garbage collection mechanism, and is designed
+ * mainly for caching usages.
+ */
+@Private
+@Unstable
+public class LevelDBCacheTimelineStore extends KeyValueBasedTimelineStore {
+  private static final Log LOG
+      = LogFactory.getLog(LevelDBCacheTimelineStore.class);
+  private static final String CACHED_LDB_FILE_PREFIX = "-timeline-cache.ldb";
+  private String dbId;
+  private DB entityDb;
+  private Configuration configuration;
+
+  public LevelDBCacheTimelineStore(String id, String name) {
+    super(name);
+    dbId = id;
+    entityInsertTimes = new MemoryTimelineStore.HashMapStoreAdapter<>();
+    domainById = new MemoryTimelineStore.HashMapStoreAdapter<>();
+    domainsByOwner = new MemoryTimelineStore.HashMapStoreAdapter<>();
+  }
+
+  public LevelDBCacheTimelineStore(String id) {
+    this(id, LevelDBCacheTimelineStore.class.getName());
+  }
+
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+    configuration = conf;
+    Options options = new Options();
+    options.createIfMissing(true);
+    options.cacheSize(conf.getLong(
+        YarnConfiguration.TIMELINE_SERVICE_LEVELDB_CACHE_READ_CACHE_SIZE,
+        YarnConfiguration.
+            DEFAULT_TIMELINE_SERVICE_LEVELDB_CACHE_READ_CACHE_SIZE));
+    JniDBFactory factory = new JniDBFactory();
+    Path dbPath = new Path(
+        conf.get(YarnConfiguration.TIMELINE_SERVICE_LEVELDB_PATH),
+        dbId + CACHED_LDB_FILE_PREFIX);
+    FileSystem localFS = null;
+
+    try {
+      localFS = FileSystem.getLocal(conf);
+      if (!localFS.exists(dbPath)) {
+        if (!localFS.mkdirs(dbPath)) {
+          throw new IOException("Couldn't create directory for leveldb " +
+              "timeline store " + dbPath);
+        }
+        localFS.setPermission(dbPath, LeveldbUtils.LEVELDB_DIR_UMASK);
+      }
+    } finally {
+      IOUtils.cleanup(LOG, localFS);
+    }
+    LOG.info("Using leveldb path " + dbPath);
+    entityDb = factory.open(new File(dbPath.toString()), options);
+    entities = new LevelDBMapAdapter<>(entityDb);
+
+    super.serviceInit(conf);
+  }
+
+  @Override
+  protected synchronized void serviceStop() throws Exception {
+    IOUtils.cleanup(LOG, entityDb);
+    Path dbPath = new Path(
+        configuration.get(YarnConfiguration.TIMELINE_SERVICE_LEVELDB_PATH),
+        dbId + CACHED_LDB_FILE_PREFIX);
+    FileSystem localFS = null;
+    try {
+      localFS = FileSystem.getLocal(configuration);
+      if (!localFS.delete(dbPath, true)) {
+          throw new IOException("Couldn't delete data file for leveldb " +
+              "timeline store " + dbPath);
+      }
+    } finally {
+      IOUtils.cleanup(LOG, localFS);
+    }
+    super.serviceStop();
+  }
+
+  /**
+   * A specialized hash map storage that uses LevelDB for storing entity id to
+   * entity mappings.
+   *
+   * @param <K> an {@link EntityIdentifier} typed hash key
+   * @param <V> a {@link TimelineEntity} typed value
+   */
+  static class LevelDBMapAdapter<K extends EntityIdentifier,
+      V extends TimelineEntity> implements TimelineStoreMapAdapter<K, V> {
+    private static final String TIME_INDEX_PREFIX = "i";
+    private static final String ENTITY_STORAGE_PREFIX = "e";
+    DB entityDb;
+
+    public LevelDBMapAdapter(DB currLevelDb) {
+      entityDb = currLevelDb;
+    }
+
+    @Override
+    public V get(K entityId) {
+      V result = null;
+      // Read the start time from the index
+      byte[] startTimeBytes = entityDb.get(getStartTimeKey(entityId));
+      if (startTimeBytes == null) {
+        return null;
+      }
+
+      // Build the key for the entity storage and read it
+      try {
+        result = getEntityForKey(getEntityKey(entityId, startTimeBytes));
+      } catch (IOException e) {
+        LOG.error("GenericObjectMapper cannot read key from key "
+            + entityId.toString()
+            + " into an object. Read aborted! ");
+        LOG.error(e.getMessage());
+      }
+
+      return result;
+    }
+
+    @Override
+    public void put(K entityId, V entity) {
+      Long startTime = entity.getStartTime();
+      if (startTime == null) {
+        startTime = System.currentTimeMillis();
+      }
+      // Build the key for the entity storage and read it
+      byte[] startTimeBytes = GenericObjectMapper.writeReverseOrderedLong(
+          startTime);
+      try {
+        byte[] valueBytes = GenericObjectMapper.write(entity);
+        entityDb.put(getEntityKey(entityId, startTimeBytes), valueBytes);
+      } catch (IOException e) {
+        LOG.error("GenericObjectMapper cannot write "
+            + entity.getClass().getName()
+            + " into a byte array. Write aborted! ");
+        LOG.error(e.getMessage());
+      }
+
+      // Build the key for the start time index
+      entityDb.put(getStartTimeKey(entityId), startTimeBytes);
+    }
+
+    @Override
+    public void remove(K entityId) {
+      // Read the start time from the index (key starts with an "i") then delete
+      // the record
+      LeveldbUtils.KeyBuilder startTimeKeyBuilder
+          = LeveldbUtils.KeyBuilder.newInstance();
+      startTimeKeyBuilder.add(TIME_INDEX_PREFIX).add(entityId.getId())
+          .add(entityId.getType());
+      byte[] startTimeBytes = entityDb.get(startTimeKeyBuilder.getBytes());
+      if (startTimeBytes == null) {
+        return;
+      }
+      entityDb.delete(startTimeKeyBuilder.getBytes());
+
+      // Build the key for the entity storage and delete it
+      entityDb.delete(getEntityKey(entityId, startTimeBytes));
+    }
+
+    @Override
+    public Iterator<V> valueSetIterator() {
+      return getIterator(null, Long.MAX_VALUE);
+    }
+
+    @Override
+    public Iterator<V> valueSetIterator(V minV) {
+      return getIterator(
+          new EntityIdentifier(minV.getEntityId(), minV.getEntityType()),
+          minV.getStartTime());
+    }
+
+    private Iterator<V> getIterator(
+        EntityIdentifier startId, long startTimeMax) {
+
+      final DBIterator internalDbIterator = entityDb.iterator();
+
+      // we need to iterate from the first element with key greater than or
+      // equal to ENTITY_STORAGE_PREFIX!maxTS(!startId), but stop on the first
+      // key who does not have prefix ENTITY_STORATE_PREFIX
+
+      // decide end prefix
+      LeveldbUtils.KeyBuilder entityPrefixKeyBuilder
+          = LeveldbUtils.KeyBuilder.newInstance();
+      entityPrefixKeyBuilder.add(ENTITY_STORAGE_PREFIX);
+      final byte[] prefixBytes = entityPrefixKeyBuilder.getBytesForLookup();
+      // decide start prefix on top of end prefix and seek
+      final byte[] startTimeBytes
+          = GenericObjectMapper.writeReverseOrderedLong(startTimeMax);
+      entityPrefixKeyBuilder.add(startTimeBytes, true);
+      if (startId != null) {
+        entityPrefixKeyBuilder.add(startId.getId());
+      }
+      final byte[] startPrefixBytes
+          = entityPrefixKeyBuilder.getBytesForLookup();
+      internalDbIterator.seek(startPrefixBytes);
+
+      return new Iterator<V>() {
+        @Override
+        public boolean hasNext() {
+          if (!internalDbIterator.hasNext()) {
+            return false;
+          }
+          Map.Entry<byte[], byte[]> nextEntry = internalDbIterator.peekNext();
+          if (LeveldbUtils.prefixMatches(
+              prefixBytes, prefixBytes.length, nextEntry.getKey())) {
+            return true;
+          }
+          return false;
+        }
+
+        @Override
+        public V next() {
+          if (hasNext()) {
+            Map.Entry<byte[], byte[]> nextRaw = internalDbIterator.next();
+            try {
+              V result = getEntityForKey(nextRaw.getKey());
+              return result;
+            } catch (IOException e) {
+              LOG.error("GenericObjectMapper cannot read key from key "
+                  + nextRaw.getKey()
+                  + " into an object. Read aborted! ");
+              LOG.error(e.getMessage());
+            }
+          }
+          return null;
+        }
+
+        // We do not support remove operations within one iteration
+        @Override
+        public void remove() {
+          LOG.error("LevelDB map adapter does not support iterate-and-remove"
+              + " use cases. ");
+        }
+      };
+    }
+
+    @SuppressWarnings("unchecked")
+    private V getEntityForKey(byte[] key) throws IOException {
+      byte[] resultRaw = entityDb.get(key);
+      if (resultRaw == null) {
+        return null;
+      }
+      ObjectMapper entityMapper = new ObjectMapper();
+      return (V) entityMapper.readValue(resultRaw, TimelineEntity.class);
+    }
+
+    private byte[] getStartTimeKey(K entityId) {
+      LeveldbUtils.KeyBuilder startTimeKeyBuilder
+          = LeveldbUtils.KeyBuilder.newInstance();
+      startTimeKeyBuilder.add(TIME_INDEX_PREFIX).add(entityId.getId())
+          .add(entityId.getType());
+      return startTimeKeyBuilder.getBytes();
+    }
+
+    private byte[] getEntityKey(K entityId, byte[] startTimeBytes) {
+      LeveldbUtils.KeyBuilder entityKeyBuilder
+          = LeveldbUtils.KeyBuilder.newInstance();
+      entityKeyBuilder.add(ENTITY_STORAGE_PREFIX).add(startTimeBytes, true)
+          .add(entityId.getId()).add(entityId.getType());
+      return entityKeyBuilder.getBytes();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fab22b3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/PluginStoreTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/PluginStoreTestUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/PluginStoreTestUtils.java
index e0379b1..f529b59 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/PluginStoreTestUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/PluginStoreTestUtils.java
@@ -200,7 +200,7 @@ class PluginStoreTestUtils {
   }
 
   static TimelineDataManager getTdmWithMemStore(Configuration config) {
-    TimelineStore store = new MemoryTimelineStore();
+    TimelineStore store = new MemoryTimelineStore("MemoryStore.test");
     TimelineDataManager tdm = getTdmWithStore(config, store);
     return tdm;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fab22b3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/TestLevelDBCacheTimelineStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/TestLevelDBCacheTimelineStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/TestLevelDBCacheTimelineStore.java
new file mode 100644
index 0000000..66da1e0
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/TestLevelDBCacheTimelineStore.java
@@ -0,0 +1,94 @@
+/**
+ * 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.timeline;
+
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+
+public class TestLevelDBCacheTimelineStore extends TimelineStoreTestUtils {
+
+  @Before
+  public void setup() throws Exception {
+    store = new LevelDBCacheTimelineStore("app1");
+    store.init(new YarnConfiguration());
+    store.start();
+    loadTestEntityData();
+    loadVerificationEntityData();
+    loadTestDomainData();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    store.stop();
+  }
+
+  public TimelineStore getTimelineStore() {
+    return store;
+  }
+
+  @Test
+  public void testGetSingleEntity() throws IOException {
+    super.testGetSingleEntity();
+  }
+
+  @Test
+  public void testGetEntities() throws IOException {
+    super.testGetEntities();
+  }
+
+  @Test
+  public void testGetEntitiesWithFromId() throws IOException {
+    super.testGetEntitiesWithFromId();
+  }
+
+  @Test
+  public void testGetEntitiesWithFromTs() throws IOException {
+    super.testGetEntitiesWithFromTs();
+  }
+
+  @Test
+  public void testGetEntitiesWithPrimaryFilters() throws IOException {
+    super.testGetEntitiesWithPrimaryFilters();
+  }
+
+  @Test
+  public void testGetEntitiesWithSecondaryFilters() throws IOException {
+    super.testGetEntitiesWithSecondaryFilters();
+  }
+
+  @Test
+  public void testGetEvents() throws IOException {
+    super.testGetEvents();
+  }
+
+  @Test
+  public void testGetDomain() throws IOException {
+    super.testGetDomain();
+  }
+
+  @Test
+  public void testGetDomains() throws IOException {
+    super.testGetDomains();
+  }
+
+}


[03/50] hadoop git commit: YARN-4371. "yarn application -kill" should take multiple application ids. Contributed by Sunil G

Posted by ar...@apache.org.
YARN-4371. "yarn application -kill" should take multiple application ids. Contributed by Sunil G


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

Branch: refs/heads/HDFS-1312
Commit: e91e8b711c68273460b36557fc37fdfc86be097b
Parents: d22c423
Author: Jason Lowe <jl...@apache.org>
Authored: Fri Jan 22 21:36:15 2016 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Fri Jan 22 21:36:15 2016 +0000

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../hadoop/yarn/client/cli/ApplicationCLI.java  |  57 ++++++--
 .../hadoop/yarn/client/cli/TestYarnCLI.java     | 141 +++++++++++++++++--
 3 files changed, 183 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e91e8b71/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index a4f9874..0ceb905 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -733,6 +733,9 @@ Release 2.8.0 - UNRELEASED
 
     YARN-4524. Cleanup AppSchedulingInfo. (Karthik Kambatla via wangda)
 
+    YARN-4371. "yarn application -kill" should take multiple application ids
+    (Sunil G via jlowe)
+
   OPTIMIZATIONS
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e91e8b71/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 77e0688..caa4d46 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
@@ -23,6 +23,7 @@ import java.io.OutputStreamWriter;
 import java.io.PrintWriter;
 import java.nio.charset.Charset;
 import java.text.DecimalFormat;
+import java.util.Collection;
 import java.util.EnumSet;
 import java.util.HashSet;
 import java.util.List;
@@ -103,7 +104,6 @@ public class ApplicationCLI extends YarnCLI {
           + "Supports optional use of -appTypes to filter applications "
           + "based on application type, "
           + "and -appStates to filter applications based on application state.");
-      opts.addOption(KILL_CMD, true, "Kills the application.");
       opts.addOption(MOVE_TO_QUEUE_CMD, true, "Moves the application to a "
           + "different queue.");
       opts.addOption(QUEUE_CMD, true, "Works with the movetoqueue command to"
@@ -127,7 +127,12 @@ public class ApplicationCLI extends YarnCLI {
       opts.addOption(UPDATE_PRIORITY, true,
           "update priority of an application. ApplicationId can be"
               + " passed using 'appId' option.");
-      opts.getOption(KILL_CMD).setArgName("Application ID");
+      Option killOpt = new Option(KILL_CMD, true, "Kills the application. "
+          + "Set of applications can be provided separated with space");
+      killOpt.setValueSeparator(' ');
+      killOpt.setArgs(Option.UNLIMITED_VALUES);
+      killOpt.setArgName("Application ID");
+      opts.addOption(killOpt);
       opts.getOption(MOVE_TO_QUEUE_CMD).setArgName("Application ID");
       opts.getOption(QUEUE_CMD).setArgName("Queue Name");
       opts.getOption(STATUS_CMD).setArgName("Application ID");
@@ -239,15 +244,11 @@ public class ApplicationCLI extends YarnCLI {
         listContainers(cliParser.getOptionValue(LIST_CMD));
       }
     } else if (cliParser.hasOption(KILL_CMD)) {
-      if (args.length != 3) {
+      if (args.length < 3 || hasAnyOtherCLIOptions(cliParser, opts, KILL_CMD)) {
         printUsage(title, opts);
         return exitCode;
       }
-      try{
-        killApplication(cliParser.getOptionValue(KILL_CMD));
-      } catch (ApplicationNotFoundException e) {
-        return exitCode;
-      }
+      return killApplication(cliParser.getOptionValues(KILL_CMD));
     } else if (cliParser.hasOption(MOVE_TO_QUEUE_CMD)) {
       if (!cliParser.hasOption(QUEUE_CMD)) {
         printUsage(title, opts);
@@ -482,6 +483,30 @@ public class ApplicationCLI extends YarnCLI {
   }
 
   /**
+   * Kills applications with the application id as appId
+   *
+   * @param Array of applicationIds
+   * @return errorCode
+   * @throws YarnException
+   * @throws IOException
+   */
+  private int killApplication(String[] applicationIds) throws YarnException,
+      IOException {
+    int returnCode = -1;
+    for (String applicationId : applicationIds) {
+      try {
+        killApplication(applicationId);
+        returnCode = 0;
+      } catch (ApplicationNotFoundException e) {
+        // Suppress all ApplicationNotFoundException for now.
+        continue;
+      }
+    }
+
+    return returnCode;
+  }
+
+  /**
    * Kills the application with the application id as appId
    * 
    * @param applicationId
@@ -726,4 +751,20 @@ public class ApplicationCLI extends YarnCLI {
           + " as application is in final states");
     }
   }
+
+  @SuppressWarnings("unchecked")
+  private boolean hasAnyOtherCLIOptions(CommandLine cliParser, Options opts,
+      String excludeOption) {
+    Collection<Option> ops = opts.getOptions();
+    for (Option op : ops) {
+      // Skip exclude option from the option list
+      if (op.getOpt().equals(excludeOption)) {
+        continue;
+      }
+      if (cliParser.hasOption(op.getOpt())) {
+        return true;
+      }
+    }
+    return false;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e91e8b71/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 95ccccf..3783aac 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
@@ -745,14 +745,6 @@ public class TestYarnCLI {
         sysOutStream.toString());
 
     sysOutStream.reset();
-    ApplicationId applicationId = ApplicationId.newInstance(1234, 5);
-    result = cli.run(
-        new String[] {"application", "-kill", applicationId.toString(), "args" });
-    verify(spyCli).printUsage(any(String.class), any(Options.class));
-    Assert.assertEquals(createApplicationCLIHelpMessage(),
-        sysOutStream.toString());
-
-    sysOutStream.reset();
     NodeId nodeId = NodeId.newInstance("host0", 0);
     result = cli.run(
         new String[] { "application", "-status", nodeId.toString(), "args" });
@@ -878,7 +870,134 @@ public class TestYarnCLI {
       Assert.fail("Unexpected exception: " + e);
     }
   }
-  
+
+  @Test
+  public void testKillApplications() throws Exception {
+    ApplicationCLI cli = createAndGetAppCLI();
+    ApplicationId applicationId1 = ApplicationId.newInstance(1234, 5);
+    ApplicationId applicationId2 = ApplicationId.newInstance(1234, 6);
+    ApplicationId applicationId3 = ApplicationId.newInstance(1234, 7);
+    ApplicationId applicationId4 = ApplicationId.newInstance(1234, 8);
+
+    // Test Scenario 1: Both applications are FINISHED.
+    ApplicationReport newApplicationReport1 = ApplicationReport.newInstance(
+        applicationId1, ApplicationAttemptId.newInstance(applicationId1, 1),
+        "user", "queue", "appname", "host", 124, null,
+        YarnApplicationState.FINISHED, "diagnostics", "url", 0, 0,
+        FinalApplicationStatus.SUCCEEDED, null, "N/A", 0.53789f, "YARN", null);
+    ApplicationReport newApplicationReport2 = ApplicationReport.newInstance(
+        applicationId2, ApplicationAttemptId.newInstance(applicationId2, 1),
+        "user", "queue", "appname", "host", 124, null,
+        YarnApplicationState.FINISHED, "diagnostics", "url", 0, 0,
+        FinalApplicationStatus.SUCCEEDED, null, "N/A", 0.34344f, "YARN", null);
+    when(client.getApplicationReport(applicationId1)).thenReturn(
+        newApplicationReport1);
+    when(client.getApplicationReport(applicationId2)).thenReturn(
+        newApplicationReport2);
+    int result = cli.run(new String[]{"application", "-kill",
+        applicationId1.toString() + " " + applicationId2.toString()});
+    assertEquals(0, result);
+    verify(client, times(0)).killApplication(applicationId1);
+    verify(client, times(0)).killApplication(applicationId2);
+    verify(sysOut).println(
+        "Application " + applicationId1 + " has already finished ");
+    verify(sysOut).println(
+        "Application " + applicationId2 + " has already finished ");
+
+    // Test Scenario 2: Both applications are RUNNING.
+    ApplicationReport newApplicationReport3 = ApplicationReport.newInstance(
+        applicationId1, ApplicationAttemptId.newInstance(applicationId1, 1),
+        "user", "queue", "appname", "host", 124, null,
+        YarnApplicationState.RUNNING, "diagnostics", "url", 0, 0,
+        FinalApplicationStatus.SUCCEEDED, null, "N/A", 0.53789f, "YARN", null);
+    ApplicationReport newApplicationReport4 = ApplicationReport.newInstance(
+        applicationId2, ApplicationAttemptId.newInstance(applicationId2, 1),
+        "user", "queue", "appname", "host", 124, null,
+        YarnApplicationState.RUNNING, "diagnostics", "url", 0, 0,
+        FinalApplicationStatus.SUCCEEDED, null, "N/A", 0.53345f, "YARN", null);
+    when(client.getApplicationReport(applicationId1)).thenReturn(
+        newApplicationReport3);
+    when(client.getApplicationReport(applicationId2)).thenReturn(
+        newApplicationReport4);
+    result = cli.run(new String[]{"application", "-kill",
+        applicationId1.toString() + " " + applicationId2.toString()});
+    assertEquals(0, result);
+    verify(client).killApplication(applicationId1);
+    verify(client).killApplication(applicationId2);
+    verify(sysOut).println(
+        "Killing application application_1234_0005");
+    verify(sysOut).println(
+        "Killing application application_1234_0006");
+
+    // Test Scenario 3: Both applications are not present.
+    doThrow(new ApplicationNotFoundException("Application with id '"
+        + applicationId3 + "' doesn't exist in RM.")).when(client)
+        .getApplicationReport(applicationId3);
+    doThrow(new ApplicationNotFoundException("Application with id '"
+        + applicationId4 + "' doesn't exist in RM.")).when(client)
+        .getApplicationReport(applicationId4);
+    result = cli.run(new String[]{"application", "-kill",
+        applicationId3.toString() + " " + applicationId4.toString()});
+    Assert.assertNotEquals(0, result);
+    verify(sysOut).println(
+        "Application with id 'application_1234_0007' doesn't exist in RM.");
+    verify(sysOut).println(
+        "Application with id 'application_1234_0008' doesn't exist in RM.");
+
+    // Test Scenario 4: one application is not present and other RUNNING
+    doThrow(new ApplicationNotFoundException("Application with id '"
+        + applicationId3 + "' doesn't exist in RM.")).when(client)
+        .getApplicationReport(applicationId3);
+    ApplicationReport newApplicationReport5 = ApplicationReport.newInstance(
+        applicationId1, ApplicationAttemptId.newInstance(applicationId1, 1),
+        "user", "queue", "appname", "host", 124, null,
+        YarnApplicationState.RUNNING, "diagnostics", "url", 0, 0,
+        FinalApplicationStatus.SUCCEEDED, null, "N/A", 0.53345f, "YARN", null);
+    when(client.getApplicationReport(applicationId1)).thenReturn(
+        newApplicationReport5);
+    result = cli.run(new String[]{"application", "-kill",
+        applicationId3.toString() + " " + applicationId1.toString()});
+    Assert.assertEquals(0, result);
+
+    // Test Scenario 5: kill operation with some other command.
+    sysOutStream.reset();
+    result = cli.run(new String[]{"application", "--appStates", "RUNNING",
+        "-kill", applicationId3.toString() + " " + applicationId1.toString()});
+    Assert.assertEquals(-1, result);
+    Assert.assertEquals(createApplicationCLIHelpMessage(),
+        sysOutStream.toString());
+  }
+
+  @Test
+  public void testKillApplicationsOfDifferentEndStates() throws Exception {
+    ApplicationCLI cli = createAndGetAppCLI();
+    ApplicationId applicationId1 = ApplicationId.newInstance(1234, 5);
+    ApplicationId applicationId2 = ApplicationId.newInstance(1234, 6);
+
+    // Scenario: One application is FINISHED and other is RUNNING.
+    ApplicationReport newApplicationReport5 = ApplicationReport.newInstance(
+        applicationId1, ApplicationAttemptId.newInstance(applicationId1, 1),
+        "user", "queue", "appname", "host", 124, null,
+        YarnApplicationState.FINISHED, "diagnostics", "url", 0, 0,
+        FinalApplicationStatus.SUCCEEDED, null, "N/A", 0.53789f, "YARN", null);
+    ApplicationReport newApplicationReport6 = ApplicationReport.newInstance(
+        applicationId2, ApplicationAttemptId.newInstance(applicationId2, 1),
+        "user", "queue", "appname", "host", 124, null,
+        YarnApplicationState.RUNNING, "diagnostics", "url", 0, 0,
+        FinalApplicationStatus.SUCCEEDED, null, "N/A", 0.53345f, "YARN", null);
+    when(client.getApplicationReport(applicationId1)).thenReturn(
+        newApplicationReport5);
+    when(client.getApplicationReport(applicationId2)).thenReturn(
+        newApplicationReport6);
+    int result = cli.run(new String[]{"application", "-kill",
+        applicationId1.toString() + " " + applicationId2.toString()});
+    assertEquals(0, result);
+    verify(client, times(1)).killApplication(applicationId2);
+    verify(sysOut).println(
+        "Application " + applicationId1 + " has already finished ");
+    verify(sysOut).println("Killing application application_1234_0006");
+  }
+
   @Test
   public void testMoveApplicationAcrossQueues() throws Exception {
     ApplicationCLI cli = createAndGetAppCLI();
@@ -1694,7 +1813,9 @@ public class TestYarnCLI {
     pw.println("                                 based on input comma-separated list of");
     pw.println("                                 application types.");
     pw.println(" -help                           Displays help for all commands.");
-    pw.println(" -kill <Application ID>          Kills the application.");
+    pw.println(" -kill <Application ID>          Kills the application. Set of");
+    pw.println("                                 applications can be provided separated");
+    pw.println("                                 with space");
     pw.println(" -list                           List applications. Supports optional use");
     pw.println("                                 of -appTypes to filter applications based");
     pw.println("                                 on application type, and -appStates to");


[33/50] hadoop git commit: MAPREDUCE-6431. JobClient should be an AutoClosable (haibochen via rkanter)

Posted by ar...@apache.org.
MAPREDUCE-6431. JobClient should be an AutoClosable (haibochen via rkanter)


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

Branch: refs/heads/HDFS-1312
Commit: dca0dc8ac28e843acd8b79c9560245638a539fde
Parents: ec25c7f
Author: Robert Kanter <rk...@apache.org>
Authored: Wed Jan 27 17:11:07 2016 -0800
Committer: Robert Kanter <rk...@apache.org>
Committed: Wed Jan 27 17:11:07 2016 -0800

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                      |  2 ++
 .../test/java/org/apache/hadoop/mapred/TestJobClient.java | 10 ++++++++++
 .../src/main/java/org/apache/hadoop/mapred/JobClient.java |  3 ++-
 3 files changed, 14 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/dca0dc8a/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 68564b6..3f85a9b 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -311,6 +311,8 @@ Release 2.9.0 - UNRELEASED
 
   IMPROVEMENTS
 
+    MAPREDUCE-6431. JobClient should be an AutoClosable (haibochen via rkanter)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dca0dc8a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapred/TestJobClient.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapred/TestJobClient.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapred/TestJobClient.java
index b18b531..bf37b03 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapred/TestJobClient.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapred/TestJobClient.java
@@ -87,4 +87,14 @@ public class TestJobClient {
         client.getClusterHandle().getStagingAreaDir().toString()
             .equals(client.getStagingAreaDir().toString()));
   }
+
+  /**
+   * Asks the compiler to check if JobClient is AutoClosable.
+   */
+  @Test(timeout = 10000)
+  public void testAutoClosable() throws IOException {
+    Configuration conf = new Configuration();
+    try (JobClient jobClient = new JobClient(conf)) {
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dca0dc8a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobClient.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobClient.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobClient.java
index cf123c7..baa6221 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobClient.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobClient.java
@@ -137,7 +137,7 @@ import org.apache.hadoop.util.ToolRunner;
  */
 @InterfaceAudience.Public
 @InterfaceStability.Stable
-public class JobClient extends CLI {
+public class JobClient extends CLI implements AutoCloseable {
 
   @InterfaceAudience.Private
   public static final String MAPREDUCE_CLIENT_RETRY_POLICY_ENABLED_KEY =
@@ -499,6 +499,7 @@ public class JobClient extends CLI {
   /**
    * Close the <code>JobClient</code>.
    */
+  @Override
   public synchronized void close() throws IOException {
     cluster.close();
   }


[09/50] hadoop git commit: Added MAPREDUCE-6614 to 2.8.0 in CHANGES.txt.

Posted by ar...@apache.org.
Added MAPREDUCE-6614 to 2.8.0 in CHANGES.txt.


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

Branch: refs/heads/HDFS-1312
Commit: 2b833297ceb523d39e683fcd34ed8ab9b5651bcf
Parents: 10a2bc0
Author: Akira Ajisaka <aa...@apache.org>
Authored: Mon Jan 25 11:41:19 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Mon Jan 25 11:41:19 2016 +0900

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b833297/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 5eb6984..ba392c3 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -695,6 +695,9 @@ Release 2.8.0 - UNRELEASED
     mapreduce.reduce.skip.proc.count.autoincr in mapred-default.xml.
     (Kai Sasaki via aajisaka)
 
+    MAPREDUCE-6614. Remove unnecessary code in TestMapreduceConfigFields.
+    (Kai Sasaki via aajisaka)
+
 Release 2.7.3 - UNRELEASED
 
   INCOMPATIBLE CHANGES


[14/50] hadoop git commit: HDFS-9094. Add command line option to ask NameNode reload configuration. (Contributed by Xiaobing Zhou)

Posted by ar...@apache.org.
HDFS-9094. Add command line option to ask NameNode reload configuration. (Contributed by Xiaobing Zhou)


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

Branch: refs/heads/HDFS-1312
Commit: d62b4a4de75edb840df6634f49cb4beb74e3fb07
Parents: 6eacdea
Author: Arpit Agarwal <ar...@apache.org>
Authored: Mon Jan 25 12:17:05 2016 -0800
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Mon Jan 25 12:17:05 2016 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DFSUtilClient.java   |   8 +
 .../hdfs/protocol/ReconfigurationProtocol.java  |   4 +
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../ReconfigurationProtocolServerSideUtils.java |   4 +-
 .../hdfs/server/namenode/NameNodeRpcServer.java |  35 +++
 .../hdfs/server/protocol/NamenodeProtocols.java |   2 +
 .../org/apache/hadoop/hdfs/tools/DFSAdmin.java  | 254 +++++++++++++------
 .../apache/hadoop/hdfs/tools/TestDFSAdmin.java  | 162 ++++++++----
 8 files changed, 350 insertions(+), 122 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d62b4a4d/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
index 62c5d81..8f6ed14 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
@@ -38,9 +38,11 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.ReconfigurationProtocol;
 import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataEncryptionKeyFactory;
 import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient;
 import org.apache.hadoop.hdfs.protocolPB.ClientDatanodeProtocolTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.ReconfigurationProtocolTranslatorPB;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.util.IOUtilsClient;
 import org.apache.hadoop.hdfs.web.WebHdfsConstants;
@@ -496,6 +498,12 @@ public class DFSUtilClient {
     return new ClientDatanodeProtocolTranslatorPB(addr, ticket, conf, factory);
   }
 
+  public static ReconfigurationProtocol createReconfigurationProtocolProxy(
+      InetSocketAddress addr, UserGroupInformation ticket, Configuration conf,
+      SocketFactory factory) throws IOException {
+    return new ReconfigurationProtocolTranslatorPB(addr, ticket, conf, factory);
+  }
+
   /**
    * Creates a new KeyProvider from the given Configuration.
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d62b4a4d/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ReconfigurationProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ReconfigurationProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ReconfigurationProtocol.java
index 75dc877..8370438 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ReconfigurationProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ReconfigurationProtocol.java
@@ -25,6 +25,7 @@ import java.util.List;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.ReconfigurationTaskStatus;
+import org.apache.hadoop.io.retry.Idempotent;
 
 /**********************************************************************
  * ReconfigurationProtocol is used by HDFS admin to reload configuration
@@ -39,16 +40,19 @@ public interface ReconfigurationProtocol {
   /**
    * Asynchronously reload configuration on disk and apply changes.
    */
+  @Idempotent
   void startReconfiguration() throws IOException;
 
   /**
    * Get the status of the previously issued reconfig task.
    * @see {@link org.apache.hadoop.conf.ReconfigurationTaskStatus}.
    */
+  @Idempotent
   ReconfigurationTaskStatus getReconfigurationStatus() throws IOException;
 
   /**
    * Get a list of allowed properties for reconfiguration.
    */
+  @Idempotent
   List<String> listReconfigurableProperties() throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d62b4a4d/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 49b4d8a..e5285b6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -953,6 +953,9 @@ Release 2.9.0 - UNRELEASED
     HDFS-9674. The HTrace span for OpWriteBlock should record the maxWriteToDisk
     time. (cmccabe via zhz)
 
+    HDFS-9094. Add command line option to ask NameNode reload
+    configuration. (Xiaobing Zhou via Arpit Agarwal)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d62b4a4d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ReconfigurationProtocolServerSideUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ReconfigurationProtocolServerSideUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ReconfigurationProtocolServerSideUtils.java
index b2be9cd..9e24204 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ReconfigurationProtocolServerSideUtils.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ReconfigurationProtocolServerSideUtils.java
@@ -41,9 +41,7 @@ public final class ReconfigurationProtocolServerSideUtils {
           List<String> reconfigurableProperties) {
     ListReconfigurablePropertiesResponseProto.Builder builder =
         ListReconfigurablePropertiesResponseProto.newBuilder();
-    for (String name : reconfigurableProperties) {
-      builder.addName(name);
-    }
+    builder.addAllName(reconfigurableProperties);
     return builder.build();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d62b4a4d/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 7785260..c1646c5 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
@@ -42,6 +42,8 @@ import com.google.common.collect.Lists;
 
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.ReconfigurationException;
+import org.apache.hadoop.conf.ReconfigurationTaskStatus;
 import org.apache.hadoop.crypto.CryptoProtocolVersion;
 import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
 import org.apache.hadoop.fs.CacheFlag;
@@ -111,12 +113,15 @@ import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ClientNamenodeProtocol;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeProtocolService;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.NamenodeProtocolService;
+import org.apache.hadoop.hdfs.protocol.proto.ReconfigurationProtocolProtos.ReconfigurationProtocolService;
 import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB;
 import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB;
 import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolPB;
 import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolServerSideTranslatorPB;
 import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolPB;
 import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolServerSideTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.ReconfigurationProtocolPB;
+import org.apache.hadoop.hdfs.protocolPB.ReconfigurationProtocolServerSideTranslatorPB;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
@@ -286,6 +291,11 @@ class NameNodeRpcServer implements NamenodeProtocols {
     BlockingService haPbService = HAServiceProtocolService
         .newReflectiveBlockingService(haServiceProtocolXlator);
 
+    ReconfigurationProtocolServerSideTranslatorPB reconfigurationProtocolXlator
+        = new ReconfigurationProtocolServerSideTranslatorPB(this);
+    BlockingService reconfigurationPbService = ReconfigurationProtocolService
+        .newReflectiveBlockingService(reconfigurationProtocolXlator);
+
     TraceAdminProtocolServerSideTranslatorPB traceAdminXlator =
         new TraceAdminProtocolServerSideTranslatorPB(this);
     BlockingService traceAdminService = TraceAdminService
@@ -319,6 +329,8 @@ class NameNodeRpcServer implements NamenodeProtocols {
       // 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,
@@ -403,6 +415,8 @@ class NameNodeRpcServer implements NamenodeProtocols {
     // Add all the RPC protocols that the namenode implements
     DFSUtil.addPBProtocol(conf, HAServiceProtocolPB.class, haPbService,
         clientRpcServer);
+    DFSUtil.addPBProtocol(conf, ReconfigurationProtocolPB.class,
+        reconfigurationPbService, clientRpcServer);
     DFSUtil.addPBProtocol(conf, NamenodeProtocolPB.class, NNPbService,
         clientRpcServer);
     DFSUtil.addPBProtocol(conf, DatanodeProtocolPB.class, dnProtoPbService,
@@ -2173,4 +2187,25 @@ class NameNodeRpcServer implements NamenodeProtocols {
     checkNNStartup();
     return namesystem.getErasureCodingPolicy(src);
   }
+
+  @Override // ReconfigurationProtocol
+  public void startReconfiguration() {
+    throw new UnsupportedOperationException(
+        "Namenode startReconfiguration is not implemented.",
+        new ReconfigurationException());
+  }
+
+  @Override // ReconfigurationProtocol
+  public ReconfigurationTaskStatus getReconfigurationStatus() {
+    throw new UnsupportedOperationException(
+        " Namenode getReconfigurationStatus is not implemented.",
+        new ReconfigurationException());
+  }
+
+  @Override // ReconfigurationProtocol
+  public List<String> listReconfigurableProperties() {
+    throw new UnsupportedOperationException(
+        " Namenode listReconfigurableProperties is not implemented.",
+        new ReconfigurationException());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d62b4a4d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocols.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocols.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocols.java
index 23b6f2e..4a3d83d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocols.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocols.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hdfs.server.protocol;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.ha.HAServiceProtocol;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.ReconfigurationProtocol;
 import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
 import org.apache.hadoop.security.RefreshUserMappingsProtocol;
 import org.apache.hadoop.ipc.RefreshCallQueueProtocol;
@@ -35,6 +36,7 @@ public interface NamenodeProtocols
           DatanodeProtocol,
           NamenodeProtocol,
           RefreshAuthorizationPolicyProtocol,
+          ReconfigurationProtocol,
           RefreshUserMappingsProtocol,
           RefreshCallQueueProtocol,
           GenericRefreshProtocol,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d62b4a4d/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 5da3bc5..9c782e9 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
@@ -37,6 +37,7 @@ import java.util.TreeSet;
 
 import com.google.common.base.Joiner;
 import com.google.common.base.Optional;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -69,6 +70,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.protocol.ReconfigurationProtocol;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
@@ -414,7 +416,8 @@ public class DFSAdmin extends FsShell {
     "\t[-refreshSuperUserGroupsConfiguration]\n" +
     "\t[-refreshCallQueue]\n" +
     "\t[-refresh <host:ipc_port> <key> [arg1..argn]\n" +
-    "\t[-reconfig <datanode|...> <host:ipc_port> <start|status|properties>]\n" +
+    "\t[-reconfig <namenode|datanode> <host:ipc_port> " +
+      "<start|status|properties>]\n" +
     "\t[-printTopology]\n" +
     "\t[-refreshNamenodes datanode_host:ipc_port]\n"+
     "\t[-deleteBlockPool datanode_host:ipc_port blockpoolId [force]]\n"+
@@ -1028,12 +1031,12 @@ public class DFSAdmin extends FsShell {
 
     String refreshCallQueue = "-refreshCallQueue: Reload the call queue from config\n";
 
-    String reconfig = "-reconfig <datanode|...> <host:ipc_port> <start|status|properties>:\n" +
+    String reconfig = "-reconfig <namenode|datanode> <host:ipc_port> " +
+        "<start|status|properties>:\n" +
         "\tStarts or gets the status of a reconfiguration operation, \n" +
         "\tor gets a list of reconfigurable properties.\n" +
-        "\tThe second parameter specifies the node type.\n" +
-        "\tCurrently, only reloading DataNode's configuration is supported.\n";
 
+        "\tThe second parameter specifies the node type\n";
     String genericRefresh = "-refresh: Arguments are <hostname:port> <resource_identifier> [arg1..argn]\n" +
       "\tTriggers a runtime-refresh of the resource specified by <resource_identifier>\n" +
       "\ton <hostname:port>. All other args after are sent to the host.\n";
@@ -1494,104 +1497,186 @@ public class DFSAdmin extends FsShell {
     String nodeType = argv[i];
     String address = argv[i + 1];
     String op = argv[i + 2];
+
     if ("start".equals(op)) {
-      return startReconfiguration(nodeType, address);
+      return startReconfiguration(nodeType, address, System.out, System.err);
     } else if ("status".equals(op)) {
       return getReconfigurationStatus(nodeType, address, System.out, System.err);
     } else if ("properties".equals(op)) {
-      return getReconfigurableProperties(
-          nodeType, address, System.out, System.err);
+      return getReconfigurableProperties(nodeType, address, System.out,
+          System.err);
     }
     System.err.println("Unknown operation: " + op);
     return -1;
   }
 
-  int startReconfiguration(String nodeType, String address) throws IOException {
-    if ("datanode".equals(nodeType)) {
-      ClientDatanodeProtocol dnProxy = getDataNodeProxy(address);
-      dnProxy.startReconfiguration();
-      System.out.println("Started reconfiguration task on DataNode " + address);
+  int startReconfiguration(final String nodeThpe, final String address)
+      throws IOException {
+    return startReconfiguration(nodeThpe, address, System.out, System.err);
+  }
+
+  int startReconfiguration(final String nodeType, final String address,
+      final PrintStream out, final PrintStream err) throws IOException {
+    String outMsg = null;
+    String errMsg = null;
+    int ret = 0;
+
+    try {
+      ret = startReconfigurationDispatch(nodeType, address, out, err);
+      outMsg = String.format("Started reconfiguration task on node [%s].",
+          address);
+    } catch (IOException e) {
+      errMsg = String.format("Node [%s] reconfiguring: %s.", address,
+          e.toString());
+    }
+
+    if (errMsg != null) {
+      err.println(errMsg);
+      return 1;
+    } else {
+      out.println(outMsg);
+      return ret;
+    }
+  }
+
+  int startReconfigurationDispatch(final String nodeType,
+      final String address, final PrintStream out, final PrintStream err)
+      throws IOException {
+    if ("namenode".equals(nodeType)) {
+      ReconfigurationProtocol reconfProxy = getNameNodeProxy(address);
+      reconfProxy.startReconfiguration();
+      return 0;
+    } else if ("datanode".equals(nodeType)) {
+      ClientDatanodeProtocol reconfProxy = getDataNodeProxy(address);
+      reconfProxy.startReconfiguration();
       return 0;
     } else {
-      System.err.println("Node type " + nodeType +
-          " does not support reconfiguration.");
+      System.err.println("Node type " + nodeType
+          + " does not support reconfiguration.");
       return 1;
     }
   }
 
-  int getReconfigurationStatus(String nodeType, String address,
-      PrintStream out, PrintStream err) throws IOException {
-    if ("datanode".equals(nodeType)) {
-      ClientDatanodeProtocol dnProxy = getDataNodeProxy(address);
-      try {
-        ReconfigurationTaskStatus status = dnProxy.getReconfigurationStatus();
-        out.print("Reconfiguring status for DataNode[" + address + "]: ");
-        if (!status.hasTask()) {
-          out.println("no task was found.");
-          return 0;
-        }
-        out.print("started at " + new Date(status.getStartTime()));
-        if (!status.stopped()) {
-          out.println(" and is still running.");
-          return 0;
-        }
+  int getReconfigurationStatus(final String nodeType, final String address,
+      final PrintStream out, final PrintStream err) throws IOException {
+    String outMsg = null;
+    String errMsg = null;
+    ReconfigurationTaskStatus status = null;
 
-        out.println(" and finished at " +
-            new Date(status.getEndTime()).toString() + ".");
-        if (status.getStatus() == null) {
-          // Nothing to report.
-          return 0;
-        }
-        for (Map.Entry<PropertyChange, Optional<String>> result :
-            status.getStatus().entrySet()) {
-          if (!result.getValue().isPresent()) {
-            out.printf(
-                "SUCCESS: Changed property %s%n\tFrom: \"%s\"%n\tTo: \"%s\"%n",
-                result.getKey().prop, result.getKey().oldVal,
-                result.getKey().newVal);
-          } else {
-            final String errorMsg = result.getValue().get();
-            out.printf(
-                  "FAILED: Change property %s%n\tFrom: \"%s\"%n\tTo: \"%s\"%n",
-                  result.getKey().prop, result.getKey().oldVal,
-                  result.getKey().newVal);
-            out.println("\tError: " + errorMsg + ".");
-          }
+    try {
+      status = getReconfigurationStatusDispatch(nodeType, address, out, err);
+      outMsg = String.format("Reconfiguring status for node [%s]: ", address);
+    } catch (IOException e) {
+      errMsg = String.format("Node [%s] reloading configuration: %s.", address,
+          e.toString());
+    }
+
+    if (errMsg != null) {
+      err.println(errMsg);
+      return 1;
+    } else {
+      out.print(outMsg);
+    }
+
+    if (status != null) {
+      if (!status.hasTask()) {
+        out.println("no task was found.");
+        return 0;
+      }
+      out.print("started at " + new Date(status.getStartTime()));
+      if (!status.stopped()) {
+        out.println(" and is still running.");
+        return 0;
+      }
+
+      out.println(" and finished at "
+          + new Date(status.getEndTime()).toString() + ".");
+      if (status.getStatus() == null) {
+        // Nothing to report.
+        return 0;
+      }
+      for (Map.Entry<PropertyChange, Optional<String>> result : status
+          .getStatus().entrySet()) {
+        if (!result.getValue().isPresent()) {
+          out.printf(
+              "SUCCESS: Changed property %s%n\tFrom: \"%s\"%n\tTo: \"%s\"%n",
+              result.getKey().prop, result.getKey().oldVal,
+              result.getKey().newVal);
+        } else {
+          final String errorMsg = result.getValue().get();
+          out.printf(
+              "FAILED: Change property %s%n\tFrom: \"%s\"%n\tTo: \"%s\"%n",
+              result.getKey().prop, result.getKey().oldVal,
+              result.getKey().newVal);
+          out.println("\tError: " + errorMsg + ".");
         }
-      } catch (IOException e) {
-        err.println("DataNode reloading configuration: " + e + ".");
-        return 1;
       }
     } else {
-      err.println("Node type " + nodeType +
-          " does not support reconfiguration.");
       return 1;
     }
+
     return 0;
   }
 
-  int getReconfigurableProperties(String nodeType, String address,
-      PrintStream out, PrintStream err) throws IOException {
-    if ("datanode".equals(nodeType)) {
-      ClientDatanodeProtocol dnProxy = getDataNodeProxy(address);
-      try {
-        List<String> properties =
-            dnProxy.listReconfigurableProperties();
-        out.println(
-            "Configuration properties that are allowed to be reconfigured:");
-        for (String name : properties) {
-          out.println(name);
-        }
-      } catch (IOException e) {
-        err.println("DataNode reconfiguration: " + e + ".");
-        return 1;
-      }
+  ReconfigurationTaskStatus getReconfigurationStatusDispatch(
+      final String nodeType, final String address, final PrintStream out,
+      final PrintStream err) throws IOException {
+    if ("namenode".equals(nodeType)) {
+      ReconfigurationProtocol reconfProxy = getNameNodeProxy(address);
+      return reconfProxy.getReconfigurationStatus();
+    } else if ("datanode".equals(nodeType)) {
+      ClientDatanodeProtocol reconfProxy = getDataNodeProxy(address);
+      return reconfProxy.getReconfigurationStatus();
     } else {
-      err.println("Node type " + nodeType +
-          " does not support reconfiguration.");
+      err.println("Node type " + nodeType
+          + " does not support reconfiguration.");
+      return null;
+    }
+  }
+
+  int getReconfigurableProperties(final String nodeType, final String address,
+      final PrintStream out, final PrintStream err) throws IOException {
+    String outMsg = null;
+    String errMsg = null;
+    List<String> properties = null;
+
+    try {
+      properties = getReconfigurablePropertiesDispatch(nodeType, address, out,
+          err);
+      outMsg = String.format("Node [%s] Reconfigurable properties:", address);
+    } catch (IOException e) {
+      errMsg = String.format("Node [%s] reconfiguration: %s.", address,
+          e.toString());
+    }
+
+    if (errMsg != null) {
+      err.println(errMsg);
       return 1;
+    } else if (properties == null) {
+      return 1;
+    } else {
+      out.println(outMsg);
+      for (String name : properties) {
+        out.println(name);
+      }
+      return 0;
+    }
+  }
+
+  List<String> getReconfigurablePropertiesDispatch(final String nodeType,
+      final String address, final PrintStream out, final PrintStream err)
+      throws IOException {
+    if ("namenode".equals(nodeType)) {
+      ReconfigurationProtocol reconfProxy = getNameNodeProxy(address);
+      return reconfProxy.listReconfigurableProperties();
+    } else if ("datanode".equals(nodeType)) {
+      ClientDatanodeProtocol reconfProxy = getDataNodeProxy(address);
+      return reconfProxy.listReconfigurableProperties();
+    } else {
+      err.println("Node type " + nodeType
+          + " does not support reconfiguration.");
+      return null;
     }
-    return 0;
   }
 
   public int genericRefresh(String[] argv, int i) throws IOException {
@@ -1712,7 +1797,7 @@ public class DFSAdmin extends FsShell {
                          + " [-refreshCallQueue]");
     } else if ("-reconfig".equals(cmd)) {
       System.err.println("Usage: hdfs dfsadmin"
-                         + " [-reconfig <datanode|...> <host:port> <start|status>]");
+          + " [-reconfig <namenode|datanode> <host:port> <start|status>]");
     } else if ("-refresh".equals(cmd)) {
       System.err.println("Usage: hdfs dfsadmin"
                          + " [-refresh <hostname:port> <resource_identifier> [arg1..argn]");
@@ -2028,6 +2113,23 @@ public class DFSAdmin extends FsShell {
             NetUtils.getSocketFactory(conf, ClientDatanodeProtocol.class));
     return dnProtocol;
   }
+
+  private ReconfigurationProtocol getNameNodeProxy(String node)
+      throws IOException {
+    InetSocketAddress nodeAddr = NetUtils.createSocketAddr(node);
+    // Get the current configuration
+    Configuration conf = getConf();
+
+    // For namenode proxy the server principal should be NN's one.
+    conf.set(CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_USER_NAME_KEY,
+        conf.get(DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY, ""));
+
+    // Create the client
+    ReconfigurationProtocol reconfigProtocol = DFSUtilClient
+        .createReconfigurationProtocolProxy(nodeAddr, getUGI(), conf,
+            NetUtils.getSocketFactory(conf, ReconfigurationProtocol.class));
+    return reconfigProtocol;
+  }
   
   private int deleteBlockPool(String[] argv, int i) throws IOException {
     ClientDatanodeProtocol dnProxy = getDataNodeProxy(argv[i]);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d62b4a4d/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 3a30ccf..a3ed4f6 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
@@ -20,6 +20,9 @@ package org.apache.hadoop.hdfs.tools;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
 
 import com.google.common.collect.Lists;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.ReconfigurationUtil;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -27,6 +30,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -52,10 +56,12 @@ import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
 public class TestDFSAdmin {
+  private static final Log LOG = LogFactory.getLog(DFSAdmin.class);
   private Configuration conf = null;
   private MiniDFSCluster cluster;
   private DFSAdmin admin;
   private DataNode datanode;
+  private NameNode namenode;
 
   @Before
   public void setUp() throws Exception {
@@ -80,21 +86,64 @@ public class TestDFSAdmin {
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
     cluster.waitActive();
     datanode = cluster.getDataNodes().get(0);
+    namenode = cluster.getNameNode();
+  }
+
+  private void startReconfiguration(String nodeType, String address,
+      final List<String> outs, final List<String> errs) throws IOException {
+    reconfigurationOutErrFormatter("startReconfiguration", nodeType,
+        address, outs, errs);
+  }
+
+  private void getReconfigurableProperties(String nodeType, String address,
+      final List<String> outs, final List<String> errs) throws IOException {
+    reconfigurationOutErrFormatter("getReconfigurableProperties", nodeType,
+        address, outs, errs);
+  }
+
+  private void getReconfigurationStatus(String nodeType, String address,
+      final List<String> outs, final List<String> errs) throws IOException {
+    reconfigurationOutErrFormatter("getReconfigurationStatus", nodeType,
+        address, outs, errs);
   }
 
-  private List<String> getReconfigureStatus(String nodeType, String address)
-      throws IOException {
+  private void reconfigurationOutErrFormatter(String methodName,
+      String nodeType, String address, final List<String> outs,
+      final List<String> errs) throws IOException {
     ByteArrayOutputStream bufOut = new ByteArrayOutputStream();
     PrintStream out = new PrintStream(bufOut);
     ByteArrayOutputStream bufErr = new ByteArrayOutputStream();
     PrintStream err = new PrintStream(bufErr);
-    admin.getReconfigurationStatus(nodeType, address, out, err);
+
+    if (methodName.equals("getReconfigurableProperties")) {
+      admin.getReconfigurableProperties(nodeType, address, out, err);
+    } else if (methodName.equals("getReconfigurationStatus")) {
+      admin.getReconfigurationStatus(nodeType, address, out, err);
+    } else if (methodName.equals("startReconfiguration")) {
+      admin.startReconfiguration(nodeType, address, out, err);
+    }
+
     Scanner scanner = new Scanner(bufOut.toString());
-    List<String> outputs = Lists.newArrayList();
     while (scanner.hasNextLine()) {
-      outputs.add(scanner.nextLine());
+      outs.add(scanner.nextLine());
+    }
+    scanner.close();
+    scanner = new Scanner(bufErr.toString());
+    while (scanner.hasNextLine()) {
+      errs.add(scanner.nextLine());
     }
-    return outputs;
+    scanner.close();
+  }
+
+  @Test(timeout = 30000)
+  public void testDataNodeGetReconfigurableProperties() throws IOException {
+    final int port = datanode.getIpcPort();
+    final String address = "localhost:" + port;
+    final List<String> outs = Lists.newArrayList();
+    final List<String> errs = Lists.newArrayList();
+    getReconfigurableProperties("datanode", address, outs, errs);
+    assertEquals(3, outs.size());
+    assertEquals(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, outs.get(1));
   }
 
   /**
@@ -103,7 +152,7 @@ public class TestDFSAdmin {
    * @throws IOException
    * @throws InterruptedException
    */
-  private void testGetReconfigurationStatus(boolean expectedSuccuss)
+  private void testDataNodeGetReconfigurationStatus(boolean expectedSuccuss)
       throws IOException, InterruptedException {
     ReconfigurationUtil ru = mock(ReconfigurationUtil.class);
     datanode.setReconfigurationUtil(ru);
@@ -130,21 +179,25 @@ public class TestDFSAdmin {
 
     assertThat(admin.startReconfiguration("datanode", address), is(0));
 
-    List<String> outputs = null;
     int count = 100;
+    final List<String> outs = Lists.newArrayList();
+    final List<String> errs = Lists.newArrayList();
     while (count > 0) {
-      outputs = getReconfigureStatus("datanode", address);
-      if (!outputs.isEmpty() && outputs.get(0).contains("finished")) {
+      outs.clear();
+      errs.clear();
+      getReconfigurationStatus("datanode", address, outs, errs);
+      if (!outs.isEmpty() && outs.get(0).contains("finished")) {
         break;
       }
       count--;
       Thread.sleep(100);
     }
+    LOG.info(String.format("count=%d", count));
     assertTrue(count > 0);
     if (expectedSuccuss) {
-      assertThat(outputs.size(), is(4));
+      assertThat(outs.size(), is(4));
     } else {
-      assertThat(outputs.size(), is(6));
+      assertThat(outs.size(), is(6));
     }
 
     List<StorageLocation> locations = DataNode.getStorageLocations(
@@ -160,55 +213,78 @@ public class TestDFSAdmin {
 
     int offset = 1;
     if (expectedSuccuss) {
-      assertThat(outputs.get(offset),
+      assertThat(outs.get(offset),
           containsString("SUCCESS: Changed property " +
               DFS_DATANODE_DATA_DIR_KEY));
     } else {
-      assertThat(outputs.get(offset),
+      assertThat(outs.get(offset),
           containsString("FAILED: Change property " +
               DFS_DATANODE_DATA_DIR_KEY));
     }
-    assertThat(outputs.get(offset + 1),
+    assertThat(outs.get(offset + 1),
         is(allOf(containsString("From:"), containsString("data1"),
             containsString("data2"))));
-    assertThat(outputs.get(offset + 2),
+    assertThat(outs.get(offset + 2),
         is(not(anyOf(containsString("data1"), containsString("data2")))));
-    assertThat(outputs.get(offset + 2),
+    assertThat(outs.get(offset + 2),
         is(allOf(containsString("To"), containsString("data_new"))));
   }
 
   @Test(timeout = 30000)
-  public void testGetReconfigurationStatus()
-      throws IOException, InterruptedException {
-    testGetReconfigurationStatus(true);
+  public void testDataNodeGetReconfigurationStatus() throws IOException,
+      InterruptedException {
+    testDataNodeGetReconfigurationStatus(true);
     restartCluster();
-    testGetReconfigurationStatus(false);
+    testDataNodeGetReconfigurationStatus(false);
   }
 
-  private List<String> getReconfigurationAllowedProperties(
-      String nodeType, String address)
-      throws IOException {
-    ByteArrayOutputStream bufOut = new ByteArrayOutputStream();
-    PrintStream out = new PrintStream(bufOut);
-    ByteArrayOutputStream bufErr = new ByteArrayOutputStream();
-    PrintStream err = new PrintStream(bufErr);
-    admin.getReconfigurableProperties(nodeType, address, out, err);
-    Scanner scanner = new Scanner(bufOut.toString());
-    List<String> outputs = Lists.newArrayList();
-    while (scanner.hasNextLine()) {
-      outputs.add(scanner.nextLine());
-    }
-    return outputs;
+  @Test(timeout = 30000)
+  public void testNameNodeStartReconfiguration() throws IOException {
+    final String address = namenode.getHostAndPort();
+    final List<String> outs = Lists.newArrayList();
+    final List<String> errs = Lists.newArrayList();
+    startReconfiguration("namenode", address, outs, errs);
+    assertEquals(0, outs.size());
+    assertTrue(errs.size() > 1);
+    assertThat(
+        errs.get(0),
+        is(allOf(containsString("Namenode"), containsString("reconfiguring:"),
+            containsString("startReconfiguration"),
+            containsString("is not implemented"),
+            containsString("UnsupportedOperationException"))));
   }
 
   @Test(timeout = 30000)
-  public void testGetReconfigAllowedProperties() throws IOException {
-    final int port = datanode.getIpcPort();
-    final String address = "localhost:" + port;
-    List<String> outputs =
-        getReconfigurationAllowedProperties("datanode", address);
-    assertEquals(3, outputs.size());
-    assertEquals(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY,
-        outputs.get(1));
+  public void testNameNodeGetReconfigurableProperties() throws IOException {
+    final String address = namenode.getHostAndPort();
+    final List<String> outs = Lists.newArrayList();
+    final List<String> errs = Lists.newArrayList();
+    getReconfigurableProperties("namenode", address, outs, errs);
+    assertEquals(0, outs.size());
+    assertTrue(errs.size() > 1);
+    assertThat(
+        errs.get(0),
+        is(allOf(containsString("Namenode"),
+            containsString("reconfiguration:"),
+            containsString("listReconfigurableProperties"),
+            containsString("is not implemented"),
+            containsString("UnsupportedOperationException"))));
+  }
+
+  @Test(timeout = 30000)
+  public void testNameNodeGetReconfigurationStatus() throws IOException {
+    final String address = namenode.getHostAndPort();
+    final List<String> outs = Lists.newArrayList();
+    final List<String> errs = Lists.newArrayList();
+    getReconfigurationStatus("namenode", address, outs, errs);
+    assertEquals(0, outs.size());
+    assertTrue(errs.size() > 1);
+    assertThat(
+        errs.get(0),
+        is(allOf(containsString("Namenode"),
+            containsString("reloading configuration:"),
+            containsString("getReconfigurationStatus"),
+            containsString("is not implemented"),
+            containsString("UnsupportedOperationException"))));
   }
 }
\ No newline at end of file


[34/50] hadoop git commit: HDFS-9654. Code refactoring for HDFS-8578.

Posted by ar...@apache.org.
HDFS-9654. Code refactoring for HDFS-8578.


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

Branch: refs/heads/HDFS-1312
Commit: 662e17b46a0f41ade6a304e12925b70b5d09fc2f
Parents: dca0dc8
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Thu Jan 28 10:56:01 2016 +0800
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Thu Jan 28 10:58:03 2016 +0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   2 +
 .../hadoop/hdfs/server/common/Storage.java      |   3 +-
 .../server/datanode/BlockPoolSliceStorage.java  | 131 +++++----
 .../hdfs/server/datanode/DataStorage.java       | 282 ++++++++++---------
 .../hdfs/server/datanode/StorageLocation.java   |  15 +
 .../org/apache/hadoop/hdfs/TestReplication.java |   3 +-
 .../apache/hadoop/hdfs/UpgradeUtilities.java    |   2 +-
 .../server/datanode/SimulatedFSDataset.java     |   2 +-
 .../datanode/TestDataNodeHotSwapVolumes.java    |  48 +++-
 .../hdfs/server/datanode/TestDataStorage.java   |   7 +-
 .../fsdataset/impl/TestFsDatasetImpl.java       |   2 +-
 11 files changed, 297 insertions(+), 200 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/662e17b4/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 7e75558..a51dc15 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -2665,6 +2665,8 @@ Release 2.7.3 - UNRELEASED
     HDFS-9634. webhdfs client side exceptions don't provide enough details
     (Eric Payne via kihwal)
 
+    HDFS-9654. Code refactoring for HDFS-8578.  (szetszwo)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/662e17b4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
index 7b4b571..41719b9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
@@ -640,7 +640,8 @@ public abstract class Storage extends StorageInfo {
         rename(getLastCheckpointTmp(), curDir);
         return;
       default:
-        throw new IOException("Unexpected FS state: " + curState);
+        throw new IOException("Unexpected FS state: " + curState
+            + " for storage directory: " + rootPath);
       }
     }
     

http://git-wip-us.apache.org/repos/asf/hadoop/blob/662e17b4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java
index 1bbeeee..acf10f1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java
@@ -18,10 +18,21 @@
 
 package org.apache.hadoop.hdfs.server.datanode;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.HardLink;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
@@ -34,18 +45,9 @@ import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.util.Daemon;
 
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Properties;
-import java.util.Set;
-import java.util.Collections;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
 
 /**
  * Manages storage for the set of BlockPoolSlices which share a particular 
@@ -136,15 +138,15 @@ public class BlockPoolSliceStorage extends Storage {
   /**
    * Load one storage directory. Recover from previous transitions if required.
    *
-   * @param datanode datanode instance
    * @param nsInfo namespace information
    * @param dataDir the root path of the storage directory
    * @param startOpt startup option
    * @return the StorageDirectory successfully loaded.
    * @throws IOException
    */
-  private StorageDirectory loadStorageDirectory(DataNode datanode,
-      NamespaceInfo nsInfo, File dataDir, StartupOption startOpt) throws IOException {
+  private StorageDirectory loadStorageDirectory(NamespaceInfo nsInfo,
+      File dataDir, StartupOption startOpt, Configuration conf)
+          throws IOException {
     StorageDirectory sd = new StorageDirectory(dataDir, null, true);
     try {
       StorageState curState = sd.analyzeStorage(startOpt, this);
@@ -158,8 +160,8 @@ public class BlockPoolSliceStorage extends Storage {
             + " does not exist");
       case NOT_FORMATTED: // format
         LOG.info("Block pool storage directory " + dataDir
-            + " is not formatted for " + nsInfo.getBlockPoolID());
-        LOG.info("Formatting ...");
+            + " is not formatted for " + nsInfo.getBlockPoolID()
+            + ". Formatting ...");
         format(sd, nsInfo);
         break;
       default:  // recovery part is common
@@ -170,10 +172,13 @@ public class BlockPoolSliceStorage extends Storage {
       // Each storage directory is treated individually.
       // During startup some of them can upgrade or roll back
       // while others could be up-to-date for the regular startup.
-      doTransition(datanode, sd, nsInfo, startOpt);
+      if (doTransition(sd, nsInfo, startOpt, conf)) {
+        return sd;
+      }
+
       if (getCTime() != nsInfo.getCTime()) {
-        throw new IOException(
-            "Data-node and name-node CTimes must be the same.");
+        throw new IOException("Datanode CTime (=" + getCTime()
+            + ") is not equal to namenode CTime (=" + nsInfo.getCTime() + ")");
       }
 
       // 3. Update successfully loaded storage.
@@ -195,16 +200,15 @@ public class BlockPoolSliceStorage extends Storage {
    * Therefore, a failure on loading any block pool storage results a faulty
    * data volume.
    *
-   * @param datanode Datanode to which this storage belongs to
    * @param nsInfo namespace information
    * @param dataDirs storage directories of block pool
    * @param startOpt startup option
    * @return an array of loaded block pool directories.
    * @throws IOException on error
    */
-  List<StorageDirectory> loadBpStorageDirectories(
-      DataNode datanode, NamespaceInfo nsInfo,
-      Collection<File> dataDirs, StartupOption startOpt) throws IOException {
+  List<StorageDirectory> loadBpStorageDirectories(NamespaceInfo nsInfo,
+      Collection<File> dataDirs, StartupOption startOpt,
+      Configuration conf) throws IOException {
     List<StorageDirectory> succeedDirs = Lists.newArrayList();
     try {
       for (File dataDir : dataDirs) {
@@ -213,8 +217,8 @@ public class BlockPoolSliceStorage extends Storage {
               "BlockPoolSliceStorage.recoverTransitionRead: " +
                   "attempt to load an used block storage: " + dataDir);
         }
-        StorageDirectory sd =
-            loadStorageDirectory(datanode, nsInfo, dataDir, startOpt);
+        final StorageDirectory sd = loadStorageDirectory(
+            nsInfo, dataDir, startOpt, conf);
         succeedDirs.add(sd);
       }
     } catch (IOException e) {
@@ -232,19 +236,21 @@ public class BlockPoolSliceStorage extends Storage {
    * Therefore, a failure on loading any block pool storage results a faulty
    * data volume.
    *
-   * @param datanode Datanode to which this storage belongs to
    * @param nsInfo namespace information
    * @param dataDirs storage directories of block pool
    * @param startOpt startup option
    * @throws IOException on error
    */
-  void recoverTransitionRead(DataNode datanode, NamespaceInfo nsInfo,
-      Collection<File> dataDirs, StartupOption startOpt) throws IOException {
+  List<StorageDirectory> recoverTransitionRead(NamespaceInfo nsInfo,
+      Collection<File> dataDirs, StartupOption startOpt, Configuration conf)
+          throws IOException {
     LOG.info("Analyzing storage directories for bpid " + nsInfo.getBlockPoolID());
-    for (StorageDirectory sd : loadBpStorageDirectories(
-        datanode, nsInfo, dataDirs, startOpt)) {
+    final List<StorageDirectory> loaded = loadBpStorageDirectories(
+        nsInfo, dataDirs, startOpt, conf);
+    for (StorageDirectory sd : loaded) {
       addStorageDir(sd);
     }
+    return loaded;
   }
 
   /**
@@ -344,10 +350,10 @@ public class BlockPoolSliceStorage extends Storage {
    * @param sd storage directory <SD>/current/<bpid>
    * @param nsInfo namespace info
    * @param startOpt startup option
-   * @throws IOException
+   * @return true if the new properties has been written.
    */
-  private void doTransition(DataNode datanode, StorageDirectory sd,
-      NamespaceInfo nsInfo, StartupOption startOpt) throws IOException {
+  private boolean doTransition(StorageDirectory sd, NamespaceInfo nsInfo,
+      StartupOption startOpt, Configuration conf) throws IOException {
     if (startOpt == StartupOption.ROLLBACK && sd.getPreviousDir().exists()) {
       Preconditions.checkState(!getTrashRootDir(sd).exists(),
           sd.getPreviousDir() + " and " + getTrashRootDir(sd) + " should not " +
@@ -379,7 +385,7 @@ public class BlockPoolSliceStorage extends Storage {
     }
     if (this.layoutVersion == HdfsServerConstants.DATANODE_LAYOUT_VERSION
         && this.cTime == nsInfo.getCTime()) {
-      return; // regular startup
+      return false; // regular startup
     }
     if (this.layoutVersion > HdfsServerConstants.DATANODE_LAYOUT_VERSION) {
       int restored = restoreBlockFilesFromTrash(getTrashRootDir(sd));
@@ -389,8 +395,8 @@ public class BlockPoolSliceStorage extends Storage {
     }
     if (this.layoutVersion > HdfsServerConstants.DATANODE_LAYOUT_VERSION
         || this.cTime < nsInfo.getCTime()) {
-      doUpgrade(datanode, sd, nsInfo); // upgrade
-      return;
+      doUpgrade(sd, nsInfo, conf); // upgrade
+      return true;
     }
     // layoutVersion == LAYOUT_VERSION && this.cTime > nsInfo.cTime
     // must shutdown
@@ -418,16 +424,18 @@ public class BlockPoolSliceStorage extends Storage {
    * @param nsInfo Namespace Info from the namenode
    * @throws IOException on error
    */
-  void doUpgrade(DataNode datanode, StorageDirectory bpSd, NamespaceInfo nsInfo)
-      throws IOException {
+  private void doUpgrade(final StorageDirectory bpSd,
+      final NamespaceInfo nsInfo, final Configuration conf) throws IOException {
     // Upgrading is applicable only to release with federation or after
     if (!DataNodeLayoutVersion.supports(
         LayoutVersion.Feature.FEDERATION, layoutVersion)) {
       return;
     }
+    final int oldLV = getLayoutVersion();
     LOG.info("Upgrading block pool storage directory " + bpSd.getRoot()
-        + ".\n   old LV = " + this.getLayoutVersion() + "; old CTime = "
-        + this.getCTime() + ".\n   new LV = " + HdfsServerConstants.DATANODE_LAYOUT_VERSION
+        + ".\n   old LV = " + oldLV
+        + "; old CTime = " + this.getCTime()
+        + ".\n   new LV = " + HdfsServerConstants.DATANODE_LAYOUT_VERSION
         + "; new CTime = " + nsInfo.getCTime());
     // get <SD>/previous directory
     String dnRoot = getDataNodeStorageRoot(bpSd.getRoot().getCanonicalPath());
@@ -438,8 +446,8 @@ public class BlockPoolSliceStorage extends Storage {
     if (dnPrevDir.exists()) {
       deleteDir(dnPrevDir);
     }
-    File bpCurDir = bpSd.getCurrentDir();
-    File bpPrevDir = bpSd.getPreviousDir();
+    final File bpCurDir = bpSd.getCurrentDir();
+    final File bpPrevDir = bpSd.getPreviousDir();
     assert bpCurDir.exists() : "BP level current directory must exist.";
     cleanupDetachDir(new File(bpCurDir, DataStorage.STORAGE_DIR_DETACHED));
     
@@ -447,15 +455,23 @@ public class BlockPoolSliceStorage extends Storage {
     if (bpPrevDir.exists()) {
       deleteDir(bpPrevDir);
     }
-    File bpTmpDir = bpSd.getPreviousTmp();
+    final File bpTmpDir = bpSd.getPreviousTmp();
     assert !bpTmpDir.exists() : "previous.tmp directory must not exist.";
     
     // 2. Rename <SD>/current/<bpid>/current to
     //    <SD>/current/<bpid>/previous.tmp
     rename(bpCurDir, bpTmpDir);
     
+    final String name = "block pool " + blockpoolID + " at " + bpSd.getRoot();
+    doUgrade(name, bpSd, nsInfo, bpPrevDir, bpTmpDir, bpCurDir, oldLV, conf);
+  }
+
+  private void doUgrade(String name, final StorageDirectory bpSd,
+      NamespaceInfo nsInfo, final File bpPrevDir, final File bpTmpDir,
+      final File bpCurDir, final int oldLV, Configuration conf)
+          throws IOException {
     // 3. Create new <SD>/current with block files hardlinks and VERSION
-    linkAllBlocks(datanode, bpTmpDir, bpCurDir);
+    linkAllBlocks(bpTmpDir, bpCurDir, oldLV, conf);
     this.layoutVersion = HdfsServerConstants.DATANODE_LAYOUT_VERSION;
     assert this.namespaceID == nsInfo.getNamespaceID() 
         : "Data-node and name-node layout versions must be the same.";
@@ -465,8 +481,7 @@ public class BlockPoolSliceStorage extends Storage {
     // 4.rename <SD>/current/<bpid>/previous.tmp to
     // <SD>/current/<bpid>/previous
     rename(bpTmpDir, bpPrevDir);
-    LOG.info("Upgrade of block pool " + blockpoolID + " at " + bpSd.getRoot()
-        + " is complete");
+    LOG.info("Upgrade of " + name + " is complete");
   }
 
   /**
@@ -640,17 +655,17 @@ public class BlockPoolSliceStorage extends Storage {
    * @param toDir the current data directory
    * @throws IOException if error occurs during hardlink
    */
-  private void linkAllBlocks(DataNode datanode, File fromDir, File toDir)
-      throws IOException {
+  private static void linkAllBlocks(File fromDir, File toDir,
+      int diskLayoutVersion, Configuration conf) throws IOException {
     // do the link
-    int diskLayoutVersion = this.getLayoutVersion();
     // hardlink finalized blocks in tmpDir
     HardLink hardLink = new HardLink();
-    DataStorage.linkBlocks(datanode, new File(fromDir, DataStorage.STORAGE_DIR_FINALIZED),
-      new File(toDir,DataStorage.STORAGE_DIR_FINALIZED), diskLayoutVersion, hardLink);
-    DataStorage.linkBlocks(datanode, new File(fromDir, DataStorage.STORAGE_DIR_RBW),
-        new File(toDir, DataStorage.STORAGE_DIR_RBW), diskLayoutVersion, hardLink);
-    LOG.info( hardLink.linkStats.report() );
+    DataStorage.linkBlocks(fromDir, toDir, DataStorage.STORAGE_DIR_FINALIZED,
+        diskLayoutVersion, hardLink, conf);
+    DataStorage.linkBlocks(fromDir, toDir, DataStorage.STORAGE_DIR_RBW,
+        diskLayoutVersion, hardLink, conf);
+    LOG.info("Linked blocks from " + fromDir + " to " + toDir + ". "
+        + hardLink.linkStats.report());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/662e17b4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
index 707f1a5..57bb8b2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
@@ -18,11 +18,29 @@
 
 package org.apache.hadoop.hdfs.server.datanode;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.ComparisonChain;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.util.concurrent.Futures;
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.channels.FileLock;
+import java.nio.channels.OverlappingFileLockException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
@@ -47,28 +65,11 @@ import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.DiskChecker;
 
-import java.io.File;
-import java.io.IOException;
-import java.io.RandomAccessFile;
-import java.nio.channels.FileLock;
-import java.nio.channels.OverlappingFileLockException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ComparisonChain;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.util.concurrent.Futures;
 
 /** 
  * Data storage information file.
@@ -104,7 +105,7 @@ public class DataStorage extends Storage {
    *  upgraded from a pre-UUID version. For compatibility with prior
    *  versions of Datanodes we cannot make this field a UUID.
    */
-  private String datanodeUuid = null;
+  private volatile String datanodeUuid = null;
   
   // Maps block pool IDs to block pool storage
   private final Map<String, BlockPoolSliceStorage> bpStorageMap
@@ -125,18 +126,28 @@ public class DataStorage extends Storage {
     super(storageInfo);
   }
 
-  public synchronized String getDatanodeUuid() {
+  public String getDatanodeUuid() {
     return datanodeUuid;
   }
 
-  public synchronized void setDatanodeUuid(String newDatanodeUuid) {
+  public void setDatanodeUuid(String newDatanodeUuid) {
     this.datanodeUuid = newDatanodeUuid;
   }
 
+  private static boolean createStorageID(StorageDirectory sd, int lv) {
+    // Clusters previously upgraded from layout versions earlier than
+    // ADD_DATANODE_AND_STORAGE_UUIDS failed to correctly generate a
+    // new storage ID. We check for that and fix it now.
+    final boolean haveValidStorageId = DataNodeLayoutVersion.supports(
+        LayoutVersion.Feature.ADD_DATANODE_AND_STORAGE_UUIDS, lv)
+        && DatanodeStorage.isValidStorageId(sd.getStorageUuid());
+    return createStorageID(sd, !haveValidStorageId);
+  }
+
   /** Create an ID for this storage.
    * @return true if a new storage ID was generated.
    * */
-  public synchronized boolean createStorageID(
+  public static boolean createStorageID(
       StorageDirectory sd, boolean regenerateStorageIds) {
     final String oldStorageID = sd.getStorageUuid();
     if (oldStorageID == null || regenerateStorageIds) {
@@ -250,7 +261,7 @@ public class DataStorage extends Storage {
 
   private StorageDirectory loadStorageDirectory(DataNode datanode,
       NamespaceInfo nsInfo, File dataDir, StartupOption startOpt)
-      throws IOException {
+          throws IOException {
     StorageDirectory sd = new StorageDirectory(dataDir, null, false);
     try {
       StorageState curState = sd.analyzeStorage(startOpt, this);
@@ -263,9 +274,9 @@ public class DataStorage extends Storage {
         throw new IOException("Storage directory " + dataDir
             + " does not exist");
       case NOT_FORMATTED: // format
-        LOG.info("Storage directory " + dataDir + " is not formatted for "
-            + nsInfo.getBlockPoolID());
-        LOG.info("Formatting ...");
+        LOG.info("Storage directory " + dataDir
+            + " is not formatted for namespace " + nsInfo.getNamespaceID()
+            + ". Formatting...");
         format(sd, nsInfo, datanode.getDatanodeUuid());
         break;
       default:  // recovery part is common
@@ -276,7 +287,9 @@ public class DataStorage extends Storage {
       // Each storage directory is treated individually.
       // During startup some of them can upgrade or roll back
       // while others could be up-to-date for the regular startup.
-      doTransition(datanode, sd, nsInfo, startOpt);
+      if (doTransition(sd, nsInfo, startOpt, datanode.getConf())) {
+        return sd;
+      }
 
       // 3. Update successfully loaded storage.
       setServiceLayoutVersion(getServiceLayoutVersion());
@@ -321,20 +334,10 @@ public class DataStorage extends Storage {
           nsInfo.getBlockPoolID(), new File(volume, STORAGE_DIR_CURRENT)));
       makeBlockPoolDataDir(bpDataDirs, null);
 
-      BlockPoolSliceStorage bpStorage;
-      final String bpid = nsInfo.getBlockPoolID();
-      synchronized (this) {
-        bpStorage = this.bpStorageMap.get(bpid);
-        if (bpStorage == null) {
-          bpStorage = new BlockPoolSliceStorage(
-              nsInfo.getNamespaceID(), bpid, nsInfo.getCTime(),
-              nsInfo.getClusterID());
-          addBlockPoolStorage(bpid, bpStorage);
-        }
-      }
-      builder.addBpStorageDirectories(
-          bpid, bpStorage.loadBpStorageDirectories(
-              datanode, nsInfo, bpDataDirs, StartupOption.HOTSWAP));
+      final BlockPoolSliceStorage bpStorage = getBlockPoolSliceStorage(nsInfo);
+      final List<StorageDirectory> dirs = bpStorage.loadBpStorageDirectories(
+          nsInfo, bpDataDirs, StartupOption.HOTSWAP, datanode.getConf());
+      builder.addBpStorageDirectories(nsInfo.getBlockPoolID(), dirs);
     }
     return builder;
   }
@@ -347,53 +350,68 @@ public class DataStorage extends Storage {
    * @param nsInfo namespace information
    * @param dataDirs array of data storage directories
    * @param startOpt startup option
-   * @return a list of successfully loaded volumes.
-   * @throws IOException
+   * @return a list of successfully loaded storage directories.
    */
   @VisibleForTesting
-  synchronized List<StorageLocation> addStorageLocations(DataNode datanode,
+  synchronized List<StorageDirectory> addStorageLocations(DataNode datanode,
       NamespaceInfo nsInfo, Collection<StorageLocation> dataDirs,
       StartupOption startOpt) throws IOException {
-    final String bpid = nsInfo.getBlockPoolID();
-    List<StorageLocation> successVolumes = Lists.newArrayList();
+    final List<StorageLocation> successLocations = loadDataStorage(
+        datanode, nsInfo, dataDirs, startOpt);
+    return loadBlockPoolSliceStorage(
+        datanode, nsInfo, successLocations, startOpt);
+  }
+
+  private List<StorageLocation> loadDataStorage(DataNode datanode,
+      NamespaceInfo nsInfo, Collection<StorageLocation> dataDirs,
+      StartupOption startOpt) throws IOException {
+    final List<StorageLocation> success = Lists.newArrayList();
     for (StorageLocation dataDir : dataDirs) {
       File root = dataDir.getFile();
       if (!containsStorageDir(root)) {
         try {
           // It first ensures the datanode level format is completed.
-          StorageDirectory sd = loadStorageDirectory(
+          final StorageDirectory sd = loadStorageDirectory(
               datanode, nsInfo, root, startOpt);
           addStorageDir(sd);
         } catch (IOException e) {
-          LOG.warn(e);
+          LOG.warn("Failed to add storage directory " + dataDir, e);
           continue;
         }
       } else {
         LOG.info("Storage directory " + dataDir + " has already been used.");
       }
+      success.add(dataDir);
+    }
 
+    return success;
+  }
+
+  private List<StorageDirectory> loadBlockPoolSliceStorage(DataNode datanode,
+      NamespaceInfo nsInfo, Collection<StorageLocation> dataDirs,
+      StartupOption startOpt) throws IOException {
+    final String bpid = nsInfo.getBlockPoolID();
+    final BlockPoolSliceStorage bpStorage = getBlockPoolSliceStorage(nsInfo);
+    final List<StorageDirectory> success = Lists.newArrayList();
+    for (StorageLocation dataDir : dataDirs) {
+      final File curDir = new File(dataDir.getFile(), STORAGE_DIR_CURRENT);
       List<File> bpDataDirs = new ArrayList<File>();
-      bpDataDirs.add(BlockPoolSliceStorage.getBpRoot(bpid, new File(root,
-              STORAGE_DIR_CURRENT)));
+      bpDataDirs.add(BlockPoolSliceStorage.getBpRoot(bpid, curDir));
       try {
         makeBlockPoolDataDir(bpDataDirs, null);
-        BlockPoolSliceStorage bpStorage = this.bpStorageMap.get(bpid);
-        if (bpStorage == null) {
-          bpStorage = new BlockPoolSliceStorage(
-              nsInfo.getNamespaceID(), bpid, nsInfo.getCTime(),
-              nsInfo.getClusterID());
-        }
 
-        bpStorage.recoverTransitionRead(datanode, nsInfo, bpDataDirs, startOpt);
-        addBlockPoolStorage(bpid, bpStorage);
+        final List<StorageDirectory> dirs = bpStorage.recoverTransitionRead(
+            nsInfo, bpDataDirs, startOpt, datanode.getConf());
+        for(StorageDirectory sd : dirs) {
+          success.add(sd);
+        }
       } catch (IOException e) {
-        LOG.warn("Failed to add storage for block pool: " + bpid + " : "
-            + e.getMessage());
-        continue;
+        LOG.warn("Failed to add storage directory " + dataDir
+            + " for block pool " + bpid, e);
       }
-      successVolumes.add(dataDir);
     }
-    return successVolumes;
+
+    return success;
   }
 
   /**
@@ -635,17 +653,13 @@ public class DataStorage extends Storage {
    * Upgrade if this.LV > LAYOUT_VERSION
    * Regular startup if this.LV = LAYOUT_VERSION
    * 
-   * @param datanode Datanode to which this storage belongs to
    * @param sd  storage directory
    * @param nsInfo  namespace info
    * @param startOpt  startup option
-   * @throws IOException
+   * @return true if the new properties has been written.
    */
-  private void doTransition( DataNode datanode,
-                             StorageDirectory sd, 
-                             NamespaceInfo nsInfo, 
-                             StartupOption startOpt
-                             ) throws IOException {
+  private boolean doTransition(StorageDirectory sd, NamespaceInfo nsInfo,
+      StartupOption startOpt, Configuration conf) throws IOException {
     if (startOpt == StartupOption.ROLLBACK) {
       doRollback(sd, nsInfo); // rollback if applicable
     }
@@ -674,25 +688,16 @@ public class DataStorage extends Storage {
           + nsInfo.getClusterID() + "; datanode clusterID = " + getClusterID());
     }
 
-    // Clusters previously upgraded from layout versions earlier than
-    // ADD_DATANODE_AND_STORAGE_UUIDS failed to correctly generate a
-    // new storage ID. We check for that and fix it now.
-    boolean haveValidStorageId =
-        DataNodeLayoutVersion.supports(
-            LayoutVersion.Feature.ADD_DATANODE_AND_STORAGE_UUIDS, layoutVersion) &&
-            DatanodeStorage.isValidStorageId(sd.getStorageUuid());
-
     // regular start up.
     if (this.layoutVersion == HdfsServerConstants.DATANODE_LAYOUT_VERSION) {
-      createStorageID(sd, !haveValidStorageId);
-      return; // regular startup
+      createStorageID(sd, layoutVersion);
+      return false; // need to write properties
     }
 
     // do upgrade
     if (this.layoutVersion > HdfsServerConstants.DATANODE_LAYOUT_VERSION) {
-      doUpgrade(datanode, sd, nsInfo);  // upgrade
-      createStorageID(sd, !haveValidStorageId);
-      return;
+      doUpgrade(sd, nsInfo, conf);  // upgrade
+      return true; // doUgrade already has written properties
     }
     
     // layoutVersion < DATANODE_LAYOUT_VERSION. I.e. stored layout version is newer
@@ -726,8 +731,8 @@ public class DataStorage extends Storage {
    * @param sd  storage directory
    * @throws IOException on error
    */
-  void doUpgrade(DataNode datanode, StorageDirectory sd, NamespaceInfo nsInfo)
-      throws IOException {
+  void doUpgrade(final StorageDirectory sd, final NamespaceInfo nsInfo,
+      final Configuration conf) throws IOException {
     // If the existing on-disk layout version supportes federation, simply
     // update its layout version.
     if (DataNodeLayoutVersion.supports(
@@ -743,15 +748,16 @@ public class DataStorage extends Storage {
       return;
     }
     
+    final int oldLV = getLayoutVersion();
     LOG.info("Upgrading storage directory " + sd.getRoot()
-             + ".\n   old LV = " + this.getLayoutVersion()
+             + ".\n   old LV = " + oldLV
              + "; old CTime = " + this.getCTime()
              + ".\n   new LV = " + HdfsServerConstants.DATANODE_LAYOUT_VERSION
              + "; new CTime = " + nsInfo.getCTime());
     
-    File curDir = sd.getCurrentDir();
-    File prevDir = sd.getPreviousDir();
-    File bbwDir = new File(sd.getRoot(), Storage.STORAGE_1_BBW);
+    final File curDir = sd.getCurrentDir();
+    final File prevDir = sd.getPreviousDir();
+    final File bbwDir = new File(sd.getRoot(), Storage.STORAGE_1_BBW);
 
     assert curDir.exists() : "Data node current directory must exist.";
     // Cleanup directory "detach"
@@ -761,21 +767,29 @@ public class DataStorage extends Storage {
     if (prevDir.exists())
       deleteDir(prevDir);
     // get previous.tmp directory, <SD>/previous.tmp
-    File tmpDir = sd.getPreviousTmp();
+    final File tmpDir = sd.getPreviousTmp();
     assert !tmpDir.exists() : 
       "Data node previous.tmp directory must not exist.";
     
     // 2. Rename <SD>/current to <SD>/previous.tmp
     rename(curDir, tmpDir);
     
-    // 3. Format BP and hard link blocks from previous directory
+    // 3.1. Format BP
     File curBpDir = BlockPoolSliceStorage.getBpRoot(nsInfo.getBlockPoolID(), curDir);
-    BlockPoolSliceStorage bpStorage = new BlockPoolSliceStorage(nsInfo.getNamespaceID(), 
-        nsInfo.getBlockPoolID(), nsInfo.getCTime(), nsInfo.getClusterID());
+    BlockPoolSliceStorage bpStorage = getBlockPoolSliceStorage(nsInfo);
     bpStorage.format(curDir, nsInfo);
-    linkAllBlocks(datanode, tmpDir, bbwDir, new File(curBpDir,
-        STORAGE_DIR_CURRENT));
-    
+
+    final File toDir = new File(curBpDir, STORAGE_DIR_CURRENT);
+    doUgrade(sd, nsInfo, prevDir, tmpDir, bbwDir, toDir, oldLV, conf);
+  }
+
+  private void doUgrade(final StorageDirectory sd,
+      final NamespaceInfo nsInfo, final File prevDir,
+      final File tmpDir, final File bbwDir, final File toDir, final int oldLV,
+      Configuration conf) throws IOException {
+    // 3.2. Link block files from <SD>/previous.tmp to <SD>/current
+    linkAllBlocks(tmpDir, bbwDir, toDir, oldLV, conf);
+
     // 4. Write version file under <SD>/current
     layoutVersion = HdfsServerConstants.DATANODE_LAYOUT_VERSION;
     clusterID = nsInfo.getClusterID();
@@ -784,7 +798,8 @@ public class DataStorage extends Storage {
     // 5. Rename <SD>/previous.tmp to <SD>/previous
     rename(tmpDir, prevDir);
     LOG.info("Upgrade of " + sd.getRoot()+ " is complete");
-    addBlockPoolStorage(nsInfo.getBlockPoolID(), bpStorage);
+
+    createStorageID(sd, layoutVersion);
   }
 
   /**
@@ -952,23 +967,22 @@ public class DataStorage extends Storage {
    *
    * @throws IOException If error occurs during hardlink
    */
-  private void linkAllBlocks(DataNode datanode, File fromDir, File fromBbwDir,
-      File toDir) throws IOException {
+  private static void linkAllBlocks(File fromDir, File fromBbwDir, File toDir,
+      int diskLayoutVersion, Configuration conf) throws IOException {
     HardLink hardLink = new HardLink();
     // do the link
-    int diskLayoutVersion = this.getLayoutVersion();
     if (DataNodeLayoutVersion.supports(
         LayoutVersion.Feature.APPEND_RBW_DIR, diskLayoutVersion)) {
       // hardlink finalized blocks in tmpDir/finalized
-      linkBlocks(datanode, new File(fromDir, STORAGE_DIR_FINALIZED),
-          new File(toDir, STORAGE_DIR_FINALIZED), diskLayoutVersion, hardLink);
+      linkBlocks(fromDir, toDir, STORAGE_DIR_FINALIZED,
+          diskLayoutVersion, hardLink, conf);
       // hardlink rbw blocks in tmpDir/rbw
-      linkBlocks(datanode, new File(fromDir, STORAGE_DIR_RBW),
-          new File(toDir, STORAGE_DIR_RBW), diskLayoutVersion, hardLink);
+      linkBlocks(fromDir, toDir, STORAGE_DIR_RBW,
+          diskLayoutVersion, hardLink, conf);
     } else { // pre-RBW version
       // hardlink finalized blocks in tmpDir
-      linkBlocks(datanode, fromDir, new File(toDir, STORAGE_DIR_FINALIZED),
-          diskLayoutVersion, hardLink);      
+      linkBlocks(fromDir, new File(toDir, STORAGE_DIR_FINALIZED),
+          diskLayoutVersion, hardLink, conf);
       if (fromBbwDir.exists()) {
         /*
          * We need to put the 'blocksBeingWritten' from HDFS 1.x into the rbw
@@ -976,11 +990,12 @@ public class DataStorage extends Storage {
          * NOT underneath the 'current' directory in those releases.  See
          * HDFS-3731 for details.
          */
-        linkBlocks(datanode, fromBbwDir,
-            new File(toDir, STORAGE_DIR_RBW), diskLayoutVersion, hardLink);
+        linkBlocks(fromBbwDir, new File(toDir, STORAGE_DIR_RBW),
+            diskLayoutVersion, hardLink, conf);
       }
-    } 
-    LOG.info( hardLink.linkStats.report() );
+    }
+    LOG.info("Linked blocks from " + fromDir + " to " + toDir + ". "
+        + hardLink.linkStats.report());
   }
 
   private static class LinkArgs {
@@ -993,8 +1008,15 @@ public class DataStorage extends Storage {
     }
   }
 
-  static void linkBlocks(DataNode datanode, File from, File to, int oldLV,
-      HardLink hl) throws IOException {
+  static void linkBlocks(File fromDir, File toDir, String subdir, int oldLV,
+      HardLink hl, Configuration conf) throws IOException {
+    linkBlocks(new File(fromDir, subdir), new File(toDir, subdir),
+        oldLV, hl, conf);
+  }
+
+  private static void linkBlocks(File from, File to, int oldLV,
+      HardLink hl, Configuration conf) throws IOException {
+    LOG.info("Start linking block files from " + from + " to " + to);
     boolean upgradeToIdBasedLayout = false;
     // If we are upgrading from a version older than the one where we introduced
     // block ID-based layout AND we're working with the finalized directory,
@@ -1017,7 +1039,7 @@ public class DataStorage extends Storage {
       removeDuplicateEntries(idBasedLayoutSingleLinks, duplicates);
     }
 
-    int numLinkWorkers = datanode.getConf().getInt(
+    final int numLinkWorkers = conf.getInt(
         DFSConfigKeys.DFS_DATANODE_BLOCK_ID_LAYOUT_UPGRADE_THREADS_KEY,
         DFSConfigKeys.DFS_DATANODE_BLOCK_ID_LAYOUT_UPGRADE_THREADS);
     ExecutorService linkWorkers = Executors.newFixedThreadPool(numLinkWorkers);
@@ -1268,13 +1290,19 @@ public class DataStorage extends Storage {
   }
 
   /**
-   * Add bpStorage into bpStorageMap
+   * Get the BlockPoolSliceStorage from {@link bpStorageMap}.
+   * If the object is not found, create a new object and put it to the map.
    */
-  private void addBlockPoolStorage(String bpID, BlockPoolSliceStorage bpStorage
-      ) {
-    if (!this.bpStorageMap.containsKey(bpID)) {
-      this.bpStorageMap.put(bpID, bpStorage);
+  synchronized BlockPoolSliceStorage getBlockPoolSliceStorage(
+      final NamespaceInfo nsInfo) {
+    final String bpid = nsInfo.getBlockPoolID();
+    BlockPoolSliceStorage bpStorage = bpStorageMap.get(bpid);
+    if (bpStorage == null) {
+      bpStorage = new BlockPoolSliceStorage(nsInfo.getNamespaceID(), bpid,
+            nsInfo.getCTime(), nsInfo.getClusterID());
+      bpStorageMap.put(bpid, bpStorage);
     }
+    return bpStorage;
   }
 
   synchronized void removeBlockPoolStorage(String bpId) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/662e17b4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java
index 46e8e8a..3162c5c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java
@@ -101,4 +101,19 @@ public class StorageLocation {
   public String toString() {
     return "[" + storageType + "]" + file.toURI();
   }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj == this) {
+      return true;
+    } else if (obj == null || !(obj instanceof StorageLocation)) {
+      return false;
+    }
+    return toString().equals(obj.toString());
+  }
+
+  @Override
+  public int hashCode() {
+    return toString().hashCode();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/662e17b4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplication.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplication.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplication.java
index d9c96ab..ca1092d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplication.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplication.java
@@ -508,7 +508,8 @@ public class TestReplication {
       Configuration conf = new HdfsConfiguration();
       conf.setLong(
           DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY, 1);
-      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3)
+          .storagesPerDatanode(1).build();
       FileSystem fs = cluster.getFileSystem();
       Path filePath = new Path("/test");
       FSDataOutputStream create = fs.create(filePath);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/662e17b4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/UpgradeUtilities.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/UpgradeUtilities.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/UpgradeUtilities.java
index 9ac58ba..138cc96 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/UpgradeUtilities.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/UpgradeUtilities.java
@@ -477,7 +477,7 @@ public class UpgradeUtilities {
     for (int i = 0; i < parent.length; i++) {
       File versionFile = new File(parent[i], "VERSION");
       StorageDirectory sd = new StorageDirectory(parent[i].getParentFile());
-      storage.createStorageID(sd, false);
+      DataStorage.createStorageID(sd, false);
       storage.writeProperties(versionFile, sd);
       versionFiles[i] = versionFile;
       File bpDir = BlockPoolSliceStorage.getBpRoot(bpid, parent[i]);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/662e17b4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
index 7a10379..c843938 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
@@ -537,7 +537,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
     this.datanode = datanode;
     if (storage != null) {
       for (int i = 0; i < storage.getNumStorageDirs(); ++i) {
-        storage.createStorageID(storage.getStorageDir(i), false);
+        DataStorage.createStorageID(storage.getStorageDir(i), false);
       }
       this.datanodeUuid = storage.getDatanodeUuid();
     } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/662e17b4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java
index a396b0e..212d2e6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java
@@ -51,10 +51,12 @@ import org.junit.Test;
 
 import java.io.File;
 import java.io.IOException;
+import java.net.URI;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Comparator;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
@@ -288,15 +290,27 @@ public class TestDataNodeHotSwapVolumes {
     String[] effectiveDataDirs = conf.get(DFS_DATANODE_DATA_DIR_KEY).split(",");
     String[] expectDataDirs = newDataDir.split(",");
     assertEquals(expectDataDirs.length, effectiveDataDirs.length);
+    List<StorageLocation> expectedStorageLocations = new ArrayList<>();
+    List<StorageLocation> effectiveStorageLocations = new ArrayList<>();
     for (int i = 0; i < expectDataDirs.length; i++) {
       StorageLocation expectLocation = StorageLocation.parse(expectDataDirs[i]);
-      StorageLocation effectiveLocation =
-          StorageLocation.parse(effectiveDataDirs[i]);
-      assertEquals(expectLocation.getStorageType(),
-          effectiveLocation.getStorageType());
-      assertEquals(expectLocation.getFile().getCanonicalFile(),
-          effectiveLocation.getFile().getCanonicalFile());
+      StorageLocation effectiveLocation = StorageLocation
+          .parse(effectiveDataDirs[i]);
+      expectedStorageLocations.add(expectLocation);
+      effectiveStorageLocations.add(effectiveLocation);
     }
+    Comparator<StorageLocation> comparator = new Comparator<StorageLocation>() {
+
+      @Override
+      public int compare(StorageLocation o1, StorageLocation o2) {
+        return o1.toString().compareTo(o2.toString());
+      }
+
+    };
+    Collections.sort(expectedStorageLocations, comparator);
+    Collections.sort(effectiveStorageLocations, comparator);
+    assertEquals("Effective volumes doesnt match expected",
+        expectedStorageLocations, effectiveStorageLocations);
 
     // Check that all newly created volumes are appropriately formatted.
     for (File volumeDir : newVolumeDirs) {
@@ -473,11 +487,27 @@ public class TestDataNodeHotSwapVolumes {
 
     DataNode dn = cluster.getDataNodes().get(0);
     Collection<String> oldDirs = getDataDirs(dn);
-    String newDirs = oldDirs.iterator().next();  // Keep the first volume.
+    // Findout the storage with block and remove it
+    ExtendedBlock block =
+        DFSTestUtil.getAllBlocks(fs, testFile).get(1).getBlock();
+    FsVolumeSpi volumeWithBlock = dn.getFSDataset().getVolume(block);
+    String basePath = volumeWithBlock.getBasePath();
+    File storageDir = new File(basePath);
+    URI fileUri = storageDir.toURI();
+    String dirWithBlock =
+        "[" + volumeWithBlock.getStorageType() + "]" + fileUri;
+    String newDirs = dirWithBlock;
+    for (String dir : oldDirs) {
+      if (dirWithBlock.startsWith(dir)) {
+        continue;
+      }
+      newDirs = dir;
+      break;
+    }
     dn.reconfigurePropertyImpl(
         DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, newDirs);
-    assertFileLocksReleased(
-        new ArrayList<String>(oldDirs).subList(1, oldDirs.size()));
+    oldDirs.remove(newDirs);
+    assertFileLocksReleased(oldDirs);
 
     triggerDeleteReport(dn);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/662e17b4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataStorage.java
index c90b8e5..c55dbae 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataStorage.java
@@ -18,9 +18,12 @@
 
 package org.apache.hadoop.hdfs.server.datanode;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
@@ -54,11 +57,13 @@ public class TestDataStorage {
 
   @Before
   public void setUp() throws IOException {
+    Configuration conf = new HdfsConfiguration();
     storage = new DataStorage();
     nsInfo = new NamespaceInfo(0, CLUSTER_ID, DEFAULT_BPID, CTIME,
         BUILD_VERSION, SOFTWARE_VERSION);
     FileUtil.fullyDelete(TEST_DIR);
     assertTrue("Failed to make test dir.", TEST_DIR.mkdirs());
+    Mockito.when(mockDN.getConf()).thenReturn(conf);
   }
 
   @After
@@ -146,7 +151,7 @@ public class TestDataStorage {
     assertEquals(numLocations, storage.getNumStorageDirs());
 
     locations = createStorageLocations(numLocations);
-    List<StorageLocation> addedLocation =
+    List<StorageDirectory> addedLocation =
         storage.addStorageLocations(mockDN, namespaceInfos.get(0),
             locations, START_OPT);
     assertTrue(addedLocation.isEmpty());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/662e17b4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java
index cdc1d61..261a8b0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java
@@ -110,7 +110,7 @@ public class TestFsDatasetImpl {
 
   private static Storage.StorageDirectory createStorageDirectory(File root) {
     Storage.StorageDirectory sd = new Storage.StorageDirectory(root);
-    dsForStorageUuid.createStorageID(sd, false);
+    DataStorage.createStorageID(sd, false);
     return sd;
   }
 


[07/50] hadoop git commit: YARN-4613. Fix test failure in TestClientRMService#testGetClusterNodes. (Takashi Ohnishi via rohithsharmaks)

Posted by ar...@apache.org.
YARN-4613. Fix test failure in TestClientRMService#testGetClusterNodes. (Takashi Ohnishi via rohithsharmaks)


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

Branch: refs/heads/HDFS-1312
Commit: 10dc2c049304671f0ed9fd737a30cd843427b53e
Parents: 99829eb
Author: rohithsharmaks <ro...@apache.org>
Authored: Sun Jan 24 23:36:15 2016 +0530
Committer: rohithsharmaks <ro...@apache.org>
Committed: Sun Jan 24 23:36:15 2016 +0530

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                   | 3 +++
 .../hadoop/yarn/server/resourcemanager/TestClientRMService.java   | 1 +
 2 files changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/10dc2c04/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 1e9f83c..6674194 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -152,6 +152,9 @@ Release 2.9.0 - UNRELEASED
     YARN-4574. Fix random failure in TestAMRMClientOnRMRestart.
     (Takashi Ohnishi via rohithsharmaks)
 
+    YARN-4613. Fix test failure in TestClientRMService#testGetClusterNodes.
+    (Takashi Ohnishi via rohithsharmaks)
+
 Release 2.8.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10dc2c04/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.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/TestClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
index 0f892cd..df95957 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
@@ -209,6 +209,7 @@ public class TestClientRMService {
 
     // Now make the node unhealthy.
     node.nodeHeartbeat(false);
+    rm.NMwaitForState(node.getNodeId(), NodeState.UNHEALTHY);
 
     // Call again
     nodeReports = client.getClusterNodes(request).getNodeReports();


[31/50] hadoop git commit: Revert "HDFS-9677. Rename generationStampV1/generationStampV2 to legacyGenerationStamp/generationStamp. Contributed by Mingliang Liu."

Posted by ar...@apache.org.
Revert "HDFS-9677. Rename generationStampV1/generationStampV2 to legacyGenerationStamp/generationStamp. Contributed by Mingliang Liu."

This reverts commit 8a91109d16394310f2568717f103e6fff7cbddb0.


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

Branch: refs/heads/HDFS-1312
Commit: 3a9571308e99cc374681bbc451a517d41a150aa0
Parents: 8a91109
Author: Jing Zhao <ji...@apache.org>
Authored: Wed Jan 27 16:31:19 2016 -0800
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Jan 27 16:31:19 2016 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 -
 .../server/blockmanagement/BlockIdManager.java  | 83 ++++++++++----------
 .../server/blockmanagement/BlockManager.java    |  8 +-
 .../OutOfLegacyGenerationStampsException.java   | 38 ---------
 .../OutOfV1GenerationStampsException.java       | 38 +++++++++
 .../hdfs/server/common/HdfsServerConstants.java |  3 +-
 .../hadoop/hdfs/server/namenode/FSEditLog.java  |  4 +-
 .../hdfs/server/namenode/FSEditLogLoader.java   |  4 +-
 .../hdfs/server/namenode/FSImageFormat.java     | 12 +--
 .../server/namenode/FSImageFormatProtobuf.java  | 12 +--
 .../hdfs/server/namenode/FSNamesystem.java      |  8 +-
 .../hadoop/hdfs/server/namenode/Namesystem.java | 13 ++-
 .../hadoop-hdfs/src/main/proto/fsimage.proto    |  4 +-
 .../blockmanagement/TestSequentialBlockId.java  | 18 ++---
 .../hdfs/server/namenode/TestEditLog.java       |  6 +-
 .../hdfs/server/namenode/TestFileTruncate.java  |  4 +-
 .../hdfs/server/namenode/TestSaveNamespace.java |  2 +-
 17 files changed, 134 insertions(+), 126 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a957130/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 7e75558..097c051 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -959,9 +959,6 @@ Release 2.9.0 - UNRELEASED
     HDFS-9541. Add hdfsStreamBuilder API to libhdfs to support defaultBlockSizes
     greater than 2 GB. (cmccabe via zhz)
 
-    HDFS-9677. Rename generationStampV1/generationStampV2 to
-    legacyGenerationStamp/generationStamp. (Mingliang Liu via jing9)
-
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a957130/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
index 3f21d9b..9c71287 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
@@ -36,11 +36,11 @@ public class BlockIdManager {
    * The global generation stamp for legacy blocks with randomly
    * generated block IDs.
    */
-  private final GenerationStamp legacyGenerationStamp = new GenerationStamp();
+  private final GenerationStamp generationStampV1 = new GenerationStamp();
   /**
    * The global generation stamp for this file system.
    */
-  private final GenerationStamp generationStamp = new GenerationStamp();
+  private final GenerationStamp generationStampV2 = new GenerationStamp();
   /**
    * The value of the generation stamp when the first switch to sequential
    * block IDs was made. Blocks with generation stamps below this value
@@ -49,7 +49,7 @@ public class BlockIdManager {
    * (or initialized as an offset from the V1 (legacy) generation stamp on
    * upgrade).
    */
-  private long legacyGenerationStampLimit;
+  private long generationStampV1Limit;
   /**
    * The global block ID space for this file system.
    */
@@ -57,8 +57,7 @@ public class BlockIdManager {
   private final SequentialBlockGroupIdGenerator blockGroupIdGenerator;
 
   public BlockIdManager(BlockManager blockManager) {
-    this.legacyGenerationStampLimit =
-        HdfsConstants.GRANDFATHER_GENERATION_STAMP;
+    this.generationStampV1Limit = HdfsConstants.GRANDFATHER_GENERATION_STAMP;
     this.blockIdGenerator = new SequentialBlockIdGenerator(blockManager);
     this.blockGroupIdGenerator = new SequentialBlockGroupIdGenerator(blockManager);
   }
@@ -69,14 +68,14 @@ public class BlockIdManager {
    * Should be invoked only during the first upgrade to
    * sequential block IDs.
    */
-  public long upgradeLegacyGenerationStamp() {
-    Preconditions.checkState(generationStamp.getCurrentValue() ==
+  public long upgradeGenerationStampToV2() {
+    Preconditions.checkState(generationStampV2.getCurrentValue() ==
       GenerationStamp.LAST_RESERVED_STAMP);
-    generationStamp.skipTo(legacyGenerationStamp.getCurrentValue() +
-      HdfsServerConstants.RESERVED_LEGACY_GENERATION_STAMPS);
+    generationStampV2.skipTo(generationStampV1.getCurrentValue() +
+      HdfsServerConstants.RESERVED_GENERATION_STAMPS_V1);
 
-    legacyGenerationStampLimit = generationStamp.getCurrentValue();
-    return generationStamp.getCurrentValue();
+    generationStampV1Limit = generationStampV2.getCurrentValue();
+    return generationStampV2.getCurrentValue();
   }
 
   /**
@@ -85,10 +84,10 @@ public class BlockIdManager {
    *
    * @param stamp set generation stamp limit to this value
    */
-  public void setLegacyGenerationStampLimit(long stamp) {
-    Preconditions.checkState(legacyGenerationStampLimit ==
-        HdfsConstants.GRANDFATHER_GENERATION_STAMP);
-    legacyGenerationStampLimit = stamp;
+  public void setGenerationStampV1Limit(long stamp) {
+    Preconditions.checkState(generationStampV1Limit == HdfsConstants
+      .GRANDFATHER_GENERATION_STAMP);
+    generationStampV1Limit = stamp;
   }
 
   /**
@@ -96,7 +95,7 @@ public class BlockIdManager {
    * and random block IDs.
    */
   public long getGenerationStampAtblockIdSwitch() {
-    return legacyGenerationStampLimit;
+    return generationStampV1Limit;
   }
 
   @VisibleForTesting
@@ -139,58 +138,58 @@ public class BlockIdManager {
   /**
    * Sets the current generation stamp for legacy blocks
    */
-  public void setLegacyGenerationStamp(long stamp) {
-    legacyGenerationStamp.setCurrentValue(stamp);
+  public void setGenerationStampV1(long stamp) {
+    generationStampV1.setCurrentValue(stamp);
   }
 
   /**
    * Gets the current generation stamp for legacy blocks
    */
-  public long getLegacyGenerationStamp() {
-    return legacyGenerationStamp.getCurrentValue();
+  public long getGenerationStampV1() {
+    return generationStampV1.getCurrentValue();
   }
 
   /**
    * Gets the current generation stamp for this filesystem
    */
-  public void setGenerationStamp(long stamp) {
-    generationStamp.setCurrentValue(stamp);
+  public void setGenerationStampV2(long stamp) {
+    generationStampV2.setCurrentValue(stamp);
   }
 
-  public long getGenerationStamp() {
-    return generationStamp.getCurrentValue();
+  public long getGenerationStampV2() {
+    return generationStampV2.getCurrentValue();
   }
 
   /**
    * Increments, logs and then returns the stamp
    */
   long nextGenerationStamp(boolean legacyBlock) throws IOException {
-    return legacyBlock ? getNextLegacyGenerationStamp() :
-        getNextGenerationStamp();
+    return legacyBlock ? getNextGenerationStampV1() :
+      getNextGenerationStampV2();
   }
 
   @VisibleForTesting
-  long getNextLegacyGenerationStamp() throws IOException {
-    long legacyGenStamp = legacyGenerationStamp.nextValue();
+  long getNextGenerationStampV1() throws IOException {
+    long genStampV1 = generationStampV1.nextValue();
 
-    if (legacyGenStamp >= legacyGenerationStampLimit) {
+    if (genStampV1 >= generationStampV1Limit) {
       // We ran out of generation stamps for legacy blocks. In practice, it
-      // is extremely unlikely as we reserved 1T legacy generation stamps. The
+      // is extremely unlikely as we reserved 1T v1 generation stamps. The
       // result is that we can no longer append to the legacy blocks that
       // were created before the upgrade to sequential block IDs.
-      throw new OutOfLegacyGenerationStampsException();
+      throw new OutOfV1GenerationStampsException();
     }
 
-    return legacyGenStamp;
+    return genStampV1;
   }
 
   @VisibleForTesting
-  long getNextGenerationStamp() {
-    return generationStamp.nextValue();
+  long getNextGenerationStampV2() {
+    return generationStampV2.nextValue();
   }
 
-  public long getLegacyGenerationStampLimit() {
-    return legacyGenerationStampLimit;
+  public long getGenerationStampV1Limit() {
+    return generationStampV1Limit;
   }
 
   /**
@@ -201,7 +200,7 @@ public class BlockIdManager {
    * @return true if the block ID was randomly generated, false otherwise.
    */
   boolean isLegacyBlock(Block block) {
-    return block.getGenerationStamp() < getLegacyGenerationStampLimit();
+    return block.getGenerationStamp() < getGenerationStampV1Limit();
   }
 
   /**
@@ -214,18 +213,18 @@ public class BlockIdManager {
 
   boolean isGenStampInFuture(Block block) {
     if (isLegacyBlock(block)) {
-      return block.getGenerationStamp() > getLegacyGenerationStamp();
+      return block.getGenerationStamp() > getGenerationStampV1();
     } else {
-      return block.getGenerationStamp() > getGenerationStamp();
+      return block.getGenerationStamp() > getGenerationStampV2();
     }
   }
 
   void clear() {
-    legacyGenerationStamp.setCurrentValue(GenerationStamp.LAST_RESERVED_STAMP);
-    generationStamp.setCurrentValue(GenerationStamp.LAST_RESERVED_STAMP);
+    generationStampV1.setCurrentValue(GenerationStamp.LAST_RESERVED_STAMP);
+    generationStampV2.setCurrentValue(GenerationStamp.LAST_RESERVED_STAMP);
     getBlockIdGenerator().setCurrentValue(SequentialBlockIdGenerator
       .LAST_RESERVED_BLOCK_ID);
-    legacyGenerationStampLimit = HdfsConstants.GRANDFATHER_GENERATION_STAMP;
+    generationStampV1Limit = HdfsConstants.GRANDFATHER_GENERATION_STAMP;
   }
 
   public static boolean isStripedBlockID(long id) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a957130/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 6215437..a76429e 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
@@ -927,13 +927,13 @@ public class BlockManager implements BlockStatsMXBean {
       final BlockUnderConstructionFeature uc = blk.getUnderConstructionFeature();
       if (blk.isStriped()) {
         final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
-        final ExtendedBlock eb = new ExtendedBlock(getBlockPoolId(),
+        final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(),
             blk);
         return newLocatedStripedBlock(eb, storages, uc.getBlockIndices(), pos,
             false);
       } else {
         final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
-        final ExtendedBlock eb = new ExtendedBlock(getBlockPoolId(),
+        final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(),
             blk);
         return newLocatedBlock(eb, storages, pos, false);
       }
@@ -976,7 +976,7 @@ public class BlockManager implements BlockStatsMXBean {
       " numNodes: " + numNodes +
       " numCorrupt: " + numCorruptNodes +
       " numCorruptRepls: " + numCorruptReplicas;
-    final ExtendedBlock eb = new ExtendedBlock(getBlockPoolId(), blk);
+    final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk);
     return blockIndices == null ?
         newLocatedBlock(eb, machines, pos, isCorrupt) :
         newLocatedStripedBlock(eb, machines, blockIndices, pos, isCorrupt);
@@ -1653,7 +1653,7 @@ public class BlockManager implements BlockStatsMXBean {
         return false;
       }
       rw.getTargets()[0].getDatanodeDescriptor().addBlockToBeErasureCoded(
-          new ExtendedBlock(getBlockPoolId(), block),
+          new ExtendedBlock(namesystem.getBlockPoolId(), block),
           rw.getSrcNodes(), rw.getTargets(),
           ((ErasureCodingWork) rw).getLiveBlockIndicies(), ecPolicy);
     } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a957130/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/OutOfLegacyGenerationStampsException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/OutOfLegacyGenerationStampsException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/OutOfLegacyGenerationStampsException.java
deleted file mode 100644
index f81392f..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/OutOfLegacyGenerationStampsException.java
+++ /dev/null
@@ -1,38 +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.server.blockmanagement;
-
-import java.io.IOException;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * This exception is thrown when the name node runs out of V1 (legacy)
- * generation stamps.
- *
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class OutOfLegacyGenerationStampsException extends IOException {
-  private static final long serialVersionUID = 1L;
-
-  public OutOfLegacyGenerationStampsException() {
-    super("Out of V1 (legacy) generation stamps\n");
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a957130/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/OutOfV1GenerationStampsException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/OutOfV1GenerationStampsException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/OutOfV1GenerationStampsException.java
new file mode 100644
index 0000000..127ad54
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/OutOfV1GenerationStampsException.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.server.blockmanagement;
+
+import java.io.IOException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * This exception is thrown when the name node runs out of V1 generation
+ * stamps.
+ *
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class OutOfV1GenerationStampsException extends IOException {
+  private static final long serialVersionUID = 1L;
+
+  public OutOfV1GenerationStampsException() {
+    super("Out of V1 (legacy) generation stamps\n");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a957130/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
index b2dda3c..13c9137 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
@@ -61,7 +61,8 @@ public interface HdfsServerConstants {
   // An invalid transaction ID that will never be seen in a real namesystem.
   long INVALID_TXID = -12345;
   // Number of generation stamps reserved for legacy blocks.
-  long RESERVED_LEGACY_GENERATION_STAMPS = 1024L * 1024 * 1024 * 1024;
+  long RESERVED_GENERATION_STAMPS_V1 =
+      1024L * 1024 * 1024 * 1024;
   /**
    * Current layout version for NameNode.
    * Please see {@link NameNodeLayoutVersion.Feature} on adding new layout version.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a957130/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
index c8986dc..e255cff 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
@@ -960,7 +960,7 @@ public class FSEditLog implements LogsPurgeable {
   /**
    * Add legacy block generation stamp record to edit log
    */
-  void logLegacyGenerationStamp(long genstamp) {
+  void logGenerationStampV1(long genstamp) {
     SetGenstampV1Op op = SetGenstampV1Op.getInstance(cache.get())
         .setGenerationStamp(genstamp);
     logEdit(op);
@@ -969,7 +969,7 @@ public class FSEditLog implements LogsPurgeable {
   /**
    * Add generation stamp record to edit log
    */
-  void logGenerationStamp(long genstamp) {
+  void logGenerationStampV2(long genstamp) {
     SetGenstampV2Op op = SetGenstampV2Op.getInstance(cache.get())
         .setGenerationStamp(genstamp);
     logEdit(op);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a957130/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
index 25f5a4f..1a9d7a9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
@@ -589,7 +589,7 @@ public class FSEditLogLoader {
     }
     case OP_SET_GENSTAMP_V1: {
       SetGenstampV1Op setGenstampV1Op = (SetGenstampV1Op)op;
-      blockManager.getBlockIdManager().setLegacyGenerationStamp(
+      blockManager.getBlockIdManager().setGenerationStampV1(
           setGenstampV1Op.genStampV1);
       break;
     }
@@ -797,7 +797,7 @@ public class FSEditLogLoader {
     }
     case OP_SET_GENSTAMP_V2: {
       SetGenstampV2Op setGenstampV2Op = (SetGenstampV2Op) op;
-      blockManager.getBlockIdManager().setGenerationStamp(
+      blockManager.getBlockIdManager().setGenerationStampV2(
           setGenstampV2Op.genStampV2);
       break;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a957130/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
index e4263bd..a815864 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
@@ -346,24 +346,24 @@ public class FSImageFormat {
         long genstamp = in.readLong();
         final BlockIdManager blockIdManager = namesystem.getBlockManager()
             .getBlockIdManager();
-        blockIdManager.setLegacyGenerationStamp(genstamp);
+        blockIdManager.setGenerationStampV1(genstamp);
 
         if (NameNodeLayoutVersion.supports(
             LayoutVersion.Feature.SEQUENTIAL_BLOCK_ID, imgVersion)) {
           // read the starting generation stamp for sequential block IDs
           genstamp = in.readLong();
-          blockIdManager.setGenerationStamp(genstamp);
+          blockIdManager.setGenerationStampV2(genstamp);
 
           // read the last generation stamp for blocks created after
           // the switch to sequential block IDs.
           long stampAtIdSwitch = in.readLong();
-          blockIdManager.setLegacyGenerationStampLimit(stampAtIdSwitch);
+          blockIdManager.setGenerationStampV1Limit(stampAtIdSwitch);
 
           // read the max sequential block ID.
           long maxSequentialBlockId = in.readLong();
           blockIdManager.setLastAllocatedContiguousBlockId(maxSequentialBlockId);
         } else {
-          long startingGenStamp = blockIdManager.upgradeLegacyGenerationStamp();
+          long startingGenStamp = blockIdManager.upgradeGenerationStampToV2();
           // This is an upgrade.
           LOG.info("Upgrading to sequential block IDs. Generation stamp " +
                    "for new blocks set to " + startingGenStamp);
@@ -1270,8 +1270,8 @@ public class FSImageFormat {
         out.writeLong(numINodes);
         final BlockIdManager blockIdManager = sourceNamesystem.getBlockManager()
             .getBlockIdManager();
-        out.writeLong(blockIdManager.getLegacyGenerationStamp());
-        out.writeLong(blockIdManager.getGenerationStamp());
+        out.writeLong(blockIdManager.getGenerationStampV1());
+        out.writeLong(blockIdManager.getGenerationStampV2());
         out.writeLong(blockIdManager.getGenerationStampAtblockIdSwitch());
         out.writeLong(blockIdManager.getLastAllocatedContiguousBlockId());
         out.writeLong(context.getTxId());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a957130/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
index e9bd8c5..635dc34 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
@@ -294,9 +294,9 @@ public final class FSImageFormatProtobuf {
     private void loadNameSystemSection(InputStream in) throws IOException {
       NameSystemSection s = NameSystemSection.parseDelimitedFrom(in);
       BlockIdManager blockIdManager = fsn.getBlockManager().getBlockIdManager();
-      blockIdManager.setLegacyGenerationStamp(s.getGenstampV1());
-      blockIdManager.setGenerationStamp(s.getGenstampV2());
-      blockIdManager.setLegacyGenerationStampLimit(s.getGenstampV1Limit());
+      blockIdManager.setGenerationStampV1(s.getGenstampV1());
+      blockIdManager.setGenerationStampV2(s.getGenstampV2());
+      blockIdManager.setGenerationStampV1Limit(s.getGenstampV1Limit());
       blockIdManager.setLastAllocatedContiguousBlockId(s.getLastAllocatedBlockId());
       if (s.hasLastAllocatedStripedBlockId()) {
         blockIdManager.setLastAllocatedStripedBlockId(
@@ -550,9 +550,9 @@ public final class FSImageFormatProtobuf {
       OutputStream out = sectionOutputStream;
       BlockIdManager blockIdManager = fsn.getBlockManager().getBlockIdManager();
       NameSystemSection.Builder b = NameSystemSection.newBuilder()
-          .setGenstampV1(blockIdManager.getLegacyGenerationStamp())
-          .setGenstampV1Limit(blockIdManager.getLegacyGenerationStampLimit())
-          .setGenstampV2(blockIdManager.getGenerationStamp())
+          .setGenstampV1(blockIdManager.getGenerationStampV1())
+          .setGenstampV1Limit(blockIdManager.getGenerationStampV1Limit())
+          .setGenstampV2(blockIdManager.getGenerationStampV2())
           .setLastAllocatedBlockId(blockIdManager.getLastAllocatedContiguousBlockId())
           .setLastAllocatedStripedBlockId(blockIdManager.getLastAllocatedStripedBlockId())
           .setTransactionId(context.getTxId());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a957130/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 166526b..1693958 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
@@ -4307,7 +4307,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return new PermissionStatus(fsOwner.getShortUserName(), supergroup, permission);
   }
 
-  void checkSuperuserPrivilege() throws AccessControlException {
+  @Override
+  public void checkSuperuserPrivilege()
+      throws AccessControlException {
     if (isPermissionEnabled) {
       FSPermissionChecker pc = getPermissionChecker();
       pc.checkSuperuserPrivilege();
@@ -4573,9 +4575,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
     long gs = blockManager.nextGenerationStamp(legacyBlock);
     if (legacyBlock) {
-      getEditLog().logLegacyGenerationStamp(gs);
+      getEditLog().logGenerationStampV1(gs);
     } else {
-      getEditLog().logGenerationStamp(gs);
+      getEditLog().logGenerationStampV2(gs);
     }
 
     // NB: callers sync the log

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a957130/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
index 95d7c20..5a9e69b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
@@ -35,13 +35,22 @@ public interface Namesystem extends RwLock, SafeMode {
   /** Is this name system running? */
   boolean isRunning();
 
+  /** Check if the user has superuser privilege. */
+  void checkSuperuserPrivilege() throws AccessControlException;
+
+  /** @return the block pool ID */
+  String getBlockPoolId();
+
   BlockCollection getBlockCollection(long id);
 
   void startSecretManagerIfNecessary();
 
   /**
-   * @param src file/directory path
-   * @return The {@link ErasureCodingPolicy} for the given file/directory path
+   * Gets the erasure coding policy for the path
+   * @param src
+   *          - path
+   * @return {@link ErasureCodingPolicy}
+   * @throws IOException
    */
   ErasureCodingPolicy getErasureCodingPolicyForPath(String src)
       throws IOException;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a957130/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
index e57efee..43a1fc5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
@@ -67,8 +67,8 @@ message FileSummary {
  */
 message NameSystemSection {
   optional uint32 namespaceId = 1;
-  optional uint64 genstampV1 = 2; // legacy generation stamp
-  optional uint64 genstampV2 = 3; // generation stamp of latest version
+  optional uint64 genstampV1 = 2;
+  optional uint64 genstampV2 = 3;
   optional uint64 genstampV1Limit = 4;
   optional uint64 lastAllocatedBlockId = 5;
   optional uint64 transactionId = 6;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a957130/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockId.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockId.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockId.java
index bfda393..e612ea9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockId.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockId.java
@@ -151,7 +151,7 @@ public class TestSequentialBlockId {
     BlockIdManager bid = mock(BlockIdManager.class);
     final long maxGenStampForLegacyBlocks = 10000;
 
-    when(bid.getLegacyGenerationStampLimit())
+    when(bid.getGenerationStampV1Limit())
         .thenReturn(maxGenStampForLegacyBlocks);
 
     Block legacyBlock = spy(new Block());
@@ -180,18 +180,18 @@ public class TestSequentialBlockId {
     // Setup a mock object and stub out a few routines to
     // retrieve the generation stamp counters.
     BlockIdManager bid = mock(BlockIdManager.class);
-    final long nextLegacyGenerationStamp = 5000;
-    final long nextGenerationStamp = 20000;
+    final long nextGenerationStampV1 = 5000;
+    final long nextGenerationStampV2 = 20000;
 
-    when(bid.getNextLegacyGenerationStamp())
-        .thenReturn(nextLegacyGenerationStamp);
-    when(bid.getNextGenerationStamp())
-        .thenReturn(nextGenerationStamp);
+    when(bid.getNextGenerationStampV1())
+        .thenReturn(nextGenerationStampV1);
+    when(bid.getNextGenerationStampV2())
+        .thenReturn(nextGenerationStampV2);
 
     // Make sure that the generation stamp is set correctly for both
     // kinds of blocks.
     when(bid.nextGenerationStamp(anyBoolean())).thenCallRealMethod();
-    assertThat(bid.nextGenerationStamp(true), is(nextLegacyGenerationStamp));
-    assertThat(bid.nextGenerationStamp(false), is(nextGenerationStamp));
+    assertThat(bid.nextGenerationStamp(true), is(nextGenerationStampV1));
+    assertThat(bid.nextGenerationStamp(false), is(nextGenerationStampV2));
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a957130/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
index 14240e0..4b72112 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
@@ -1145,7 +1145,7 @@ public class TestEditLog {
     editlog.initJournalsForWrite();
     editlog.openForWrite(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
     for (int i = 2; i < TXNS_PER_ROLL; i++) {
-      editlog.logGenerationStamp((long) 0);
+      editlog.logGenerationStampV2((long) 0);
     }
     editlog.logSync();
     
@@ -1157,7 +1157,7 @@ public class TestEditLog {
     for (int i = 0; i < numrolls; i++) {
       editlog.rollEditLog(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
       
-      editlog.logGenerationStamp((long) i);
+      editlog.logGenerationStampV2((long) i);
       editlog.logSync();
 
       while (aborts.size() > 0 
@@ -1167,7 +1167,7 @@ public class TestEditLog {
       } 
       
       for (int j = 3; j < TXNS_PER_ROLL; j++) {
-        editlog.logGenerationStamp((long) i);
+        editlog.logGenerationStampV2((long) i);
       }
       editlog.logSync();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a957130/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
index d723525..f2c0330 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
@@ -1020,7 +1020,7 @@ public class TestFileTruncate {
       assertThat(truncateBlock.getNumBytes(),
           is(oldBlock.getNumBytes()));
       assertThat(truncateBlock.getGenerationStamp(),
-          is(fsn.getBlockManager().getBlockIdManager().getGenerationStamp()));
+          is(fsn.getBlockManager().getBlockIdManager().getGenerationStampV2()));
       assertThat(file.getLastBlock().getBlockUCState(),
           is(HdfsServerConstants.BlockUCState.UNDER_RECOVERY));
       long blockRecoveryId = file.getLastBlock().getUnderConstructionFeature()
@@ -1054,7 +1054,7 @@ public class TestFileTruncate {
       assertThat(truncateBlock.getNumBytes() < oldBlock.getNumBytes(),
           is(true));
       assertThat(truncateBlock.getGenerationStamp(),
-          is(fsn.getBlockManager().getBlockIdManager().getGenerationStamp()));
+          is(fsn.getBlockManager().getBlockIdManager().getGenerationStampV2()));
       assertThat(file.getLastBlock().getBlockUCState(),
           is(HdfsServerConstants.BlockUCState.UNDER_RECOVERY));
       long blockRecoveryId = file.getLastBlock().getUnderConstructionFeature()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a957130/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java
index a03e900..4d37d61 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java
@@ -519,7 +519,7 @@ public class TestSaveNamespace {
     DelayAnswer delayer = new GenericTestUtils.DelayAnswer(LOG);
     BlockIdManager bid = spy(spyFsn.getBlockManager().getBlockIdManager());
     Whitebox.setInternalState(finalFsn.getBlockManager(), "blockIdManager", bid);
-    doAnswer(delayer).when(bid).getGenerationStamp();
+    doAnswer(delayer).when(bid).getGenerationStampV2();
 
     ExecutorService pool = Executors.newFixedThreadPool(2);
     


[47/50] hadoop git commit: YARN-4411. RMAppAttemptImpl#createApplicationAttemptReport throws IllegalArgumentException. Contributed by Bibin A Chundatt and yarntime.

Posted by ar...@apache.org.
YARN-4411. RMAppAttemptImpl#createApplicationAttemptReport throws
IllegalArgumentException. Contributed by Bibin A Chundatt and yarntime.


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

Branch: refs/heads/HDFS-1312
Commit: a277bdc9edc66bef419fcd063b832073e512f234
Parents: df99ea8
Author: Devaraj K <de...@apache.org>
Authored: Fri Jan 29 13:51:37 2016 +0530
Committer: Devaraj K <de...@apache.org>
Committed: Fri Jan 29 13:51:37 2016 +0530

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                        |  3 +++
 .../rmapp/attempt/RMAppAttemptImpl.java                | 10 +++++-----
 .../rmapp/attempt/TestRMAppAttemptTransitions.java     | 13 +++++++++++++
 3 files changed, 21 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a277bdc9/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index dd61f2a..138e581 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -392,6 +392,9 @@ Release 2.8.0 - UNRELEASED
 
     YARN-4219. New levelDB cache storage for timeline v1.5. (Li Lu via xgong)
 
+    YARN-4411. RMAppAttemptImpl#createApplicationAttemptReport throws
+    IllegalArgumentException. (Bibin A Chundatt, yarntime via devaraj)
+
   IMPROVEMENTS
 
     YARN-644. Basic null check is not performed on passed in arguments before

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a277bdc9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.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/attempt/RMAppAttemptImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
index 3f45cb4..99f1b21 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
@@ -2067,11 +2067,11 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
       // am container.
       ContainerId amId =
           masterContainer == null ? null : masterContainer.getId();
-      attemptReport = ApplicationAttemptReport.newInstance(this
-          .getAppAttemptId(), this.getHost(), this.getRpcPort(), this
-          .getTrackingUrl(), this.getOriginalTrackingUrl(), this.getDiagnostics(),
-              YarnApplicationAttemptState.valueOf(this.getState().toString()),
-              amId, this.startTime, this.finishTime);
+      attemptReport = ApplicationAttemptReport.newInstance(
+          this.getAppAttemptId(), this.getHost(), this.getRpcPort(),
+          this.getTrackingUrl(), this.getOriginalTrackingUrl(),
+          this.getDiagnostics(), createApplicationAttemptState(), amId,
+          this.startTime, this.finishTime);
     } finally {
       this.readLock.unlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a277bdc9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.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/attempt/TestRMAppAttemptTransitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java
index 7f9610f..e7985fa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java
@@ -918,6 +918,19 @@ public class TestRMAppAttemptTransitions {
   }
   
   @Test(timeout = 10000)
+  public void testCreateAppAttemptReport() {
+    RMAppAttemptState[] attemptStates = RMAppAttemptState.values();
+    applicationAttempt.handle(new RMAppAttemptEvent(
+        applicationAttempt.getAppAttemptId(), RMAppAttemptEventType.KILL));
+    // ALL RMAppAttemptState TO BE CHECK
+    RMAppAttempt attempt = spy(applicationAttempt);
+    for (RMAppAttemptState rmAppAttemptState : attemptStates) {
+      when(attempt.getState()).thenReturn(rmAppAttemptState);
+      attempt.createApplicationAttemptReport();
+    }
+  }
+
+  @Test(timeout = 10000)
   public void testLaunchedAtFinalSaving() {
     Container amContainer = allocateApplicationAttempt();
 


[46/50] hadoop git commit: MAPREDUCE-6616. Fail to create jobhistory file if there are some multibyte characters in the job name. Contributed by Kousuke Saruta.

Posted by ar...@apache.org.
MAPREDUCE-6616. Fail to create jobhistory file if there are some multibyte characters in the job name. Contributed by Kousuke Saruta.


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

Branch: refs/heads/HDFS-1312
Commit: df99ea8a92d600e669606d41d3887bd004e7a3cc
Parents: 8ee0603
Author: Akira Ajisaka <aa...@apache.org>
Authored: Fri Jan 29 16:19:28 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Fri Jan 29 16:20:29 2016 +0900

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt            |   3 +
 .../v2/jobhistory/FileNameIndexUtils.java       | 171 +++++++++++-----
 .../v2/jobhistory/TestFileNameIndexUtils.java   | 199 ++++++++++++++++---
 3 files changed, 296 insertions(+), 77 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/df99ea8a/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 08cd1d3..8261b34 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -709,6 +709,9 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6563. Streaming documentation contains a stray '%' character.
     (cnauroth)
 
+    MAPREDUCE-6616. Fail to create jobhistory file if there are some multibyte
+    characters in the job name. (Kousuke Saruta via aajisaka)
+
 Release 2.7.3 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df99ea8a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/FileNameIndexUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/FileNameIndexUtils.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/FileNameIndexUtils.java
index eb0c54c..284fe80 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/FileNameIndexUtils.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/FileNameIndexUtils.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.io.UnsupportedEncodingException;
 import java.net.URLDecoder;
 import java.net.URLEncoder;
+import static java.nio.charset.StandardCharsets.UTF_8;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -35,7 +36,7 @@ public class FileNameIndexUtils {
   // Sanitize job history file for predictable parsing
   static final String DELIMITER = "-";
   static final String DELIMITER_ESCAPE = "%2D";
-  
+
   private static final Log LOG = LogFactory.getLog(FileNameIndexUtils.class);
 
   // Job history file names need to be backwards compatible
@@ -57,7 +58,8 @@ public class FileNameIndexUtils {
    * @param indexInfo the index info.
    * @return the done job history filename.
    */
-  public static String getDoneFileName(JobIndexInfo indexInfo) throws IOException {
+  public static String getDoneFileName(JobIndexInfo indexInfo)
+      throws IOException {
     return getDoneFileName(indexInfo,
         JHAdminConfig.DEFAULT_MR_HS_JOBNAME_LIMIT);
   }
@@ -66,49 +68,58 @@ public class FileNameIndexUtils {
       int jobNameLimit) throws IOException {
     StringBuilder sb = new StringBuilder();
     //JobId
-    sb.append(escapeDelimiters(TypeConverter.fromYarn(indexInfo.getJobId()).toString()));
+    sb.append(encodeJobHistoryFileName(escapeDelimiters(
+        TypeConverter.fromYarn(indexInfo.getJobId()).toString())));
     sb.append(DELIMITER);
-    
+
     //SubmitTime
-    sb.append(indexInfo.getSubmitTime());
+    sb.append(encodeJobHistoryFileName(String.valueOf(
+        indexInfo.getSubmitTime())));
     sb.append(DELIMITER);
-    
+
     //UserName
-    sb.append(escapeDelimiters(getUserName(indexInfo)));
+    sb.append(encodeJobHistoryFileName(escapeDelimiters(
+        getUserName(indexInfo))));
     sb.append(DELIMITER);
-    
+
     //JobName
-    sb.append(escapeDelimiters(trimJobName(
-        getJobName(indexInfo), jobNameLimit)));
+    sb.append(trimURLEncodedString(encodeJobHistoryFileName(escapeDelimiters(
+        getJobName(indexInfo))), jobNameLimit));
     sb.append(DELIMITER);
-    
+
     //FinishTime
-    sb.append(indexInfo.getFinishTime());
+    sb.append(encodeJobHistoryFileName(
+        String.valueOf(indexInfo.getFinishTime())));
     sb.append(DELIMITER);
-    
+
     //NumMaps
-    sb.append(indexInfo.getNumMaps());
+    sb.append(encodeJobHistoryFileName(
+        String.valueOf(indexInfo.getNumMaps())));
     sb.append(DELIMITER);
-    
+
     //NumReduces
-    sb.append(indexInfo.getNumReduces());
+    sb.append(encodeJobHistoryFileName(
+        String.valueOf(indexInfo.getNumReduces())));
     sb.append(DELIMITER);
-    
+
     //JobStatus
-    sb.append(indexInfo.getJobStatus());
+    sb.append(encodeJobHistoryFileName(indexInfo.getJobStatus()));
     sb.append(DELIMITER);
-    
+
     //QueueName
-    sb.append(escapeDelimiters(getQueueName(indexInfo)));
+    sb.append(escapeDelimiters(encodeJobHistoryFileName(
+        getQueueName(indexInfo))));
     sb.append(DELIMITER);
 
     //JobStartTime
-    sb.append(indexInfo.getJobStartTime());
+    sb.append(encodeJobHistoryFileName(
+        String.valueOf(indexInfo.getJobStartTime())));
 
-    sb.append(JobHistoryUtils.JOB_HISTORY_FILE_EXTENSION);
-    return encodeJobHistoryFileName(sb.toString());
+    sb.append(encodeJobHistoryFileName(
+        JobHistoryUtils.JOB_HISTORY_FILE_EXTENSION));
+    return sb.toString();
   }
-  
+
   /**
    * Parses the provided job history file name to construct a
    * JobIndexInfo object which is returned.
@@ -116,21 +127,24 @@ public class FileNameIndexUtils {
    * @param jhFileName the job history filename.
    * @return a JobIndexInfo object built from the filename.
    */
-  public static JobIndexInfo getIndexInfo(String jhFileName) throws IOException {
-    String fileName = jhFileName.substring(0, jhFileName.indexOf(JobHistoryUtils.JOB_HISTORY_FILE_EXTENSION));
+  public static JobIndexInfo getIndexInfo(String jhFileName)
+      throws IOException {
+    String fileName = jhFileName.substring(0,
+        jhFileName.indexOf(JobHistoryUtils.JOB_HISTORY_FILE_EXTENSION));
     JobIndexInfo indexInfo = new JobIndexInfo();
-    
+
     String[] jobDetails = fileName.split(DELIMITER);
-    
-    JobID oldJobId = JobID.forName(decodeJobHistoryFileName(jobDetails[JOB_ID_INDEX]));
+
+    JobID oldJobId =
+        JobID.forName(decodeJobHistoryFileName(jobDetails[JOB_ID_INDEX]));
     JobId jobId = TypeConverter.toYarn(oldJobId);
     indexInfo.setJobId(jobId);
 
     // Do not fail if there are some minor parse errors
     try {
       try {
-        indexInfo.setSubmitTime(
-            Long.parseLong(decodeJobHistoryFileName(jobDetails[SUBMIT_TIME_INDEX])));
+        indexInfo.setSubmitTime(Long.parseLong(
+            decodeJobHistoryFileName(jobDetails[SUBMIT_TIME_INDEX])));
       } catch (NumberFormatException e) {
         LOG.warn("Unable to parse submit time from job history file "
             + jhFileName + " : " + e);
@@ -143,24 +157,24 @@ public class FileNameIndexUtils {
           decodeJobHistoryFileName(jobDetails[JOB_NAME_INDEX]));
 
       try {
-        indexInfo.setFinishTime(
-            Long.parseLong(decodeJobHistoryFileName(jobDetails[FINISH_TIME_INDEX])));
+        indexInfo.setFinishTime(Long.parseLong(
+            decodeJobHistoryFileName(jobDetails[FINISH_TIME_INDEX])));
       } catch (NumberFormatException e) {
         LOG.warn("Unable to parse finish time from job history file "
             + jhFileName + " : " + e);
       }
 
       try {
-        indexInfo.setNumMaps(
-            Integer.parseInt(decodeJobHistoryFileName(jobDetails[NUM_MAPS_INDEX])));
+        indexInfo.setNumMaps(Integer.parseInt(
+            decodeJobHistoryFileName(jobDetails[NUM_MAPS_INDEX])));
       } catch (NumberFormatException e) {
         LOG.warn("Unable to parse num maps from job history file "
             + jhFileName + " : " + e);
       }
 
       try {
-        indexInfo.setNumReduces(
-            Integer.parseInt(decodeJobHistoryFileName(jobDetails[NUM_REDUCES_INDEX])));
+        indexInfo.setNumReduces(Integer.parseInt(
+            decodeJobHistoryFileName(jobDetails[NUM_REDUCES_INDEX])));
       } catch (NumberFormatException e) {
         LOG.warn("Unable to parse num reduces from job history file "
             + jhFileName + " : " + e);
@@ -176,8 +190,8 @@ public class FileNameIndexUtils {
         if (jobDetails.length <= JOB_START_TIME_INDEX) {
           indexInfo.setJobStartTime(indexInfo.getSubmitTime());
         } else {
-          indexInfo.setJobStartTime(
-              Long.parseLong(decodeJobHistoryFileName(jobDetails[JOB_START_TIME_INDEX])));
+          indexInfo.setJobStartTime(Long.parseLong(
+              decodeJobHistoryFileName(jobDetails[JOB_START_TIME_INDEX])));
         }
       } catch (NumberFormatException e){
         LOG.warn("Unable to parse start time from job history file "
@@ -187,13 +201,13 @@ public class FileNameIndexUtils {
       LOG.warn("Parsing job history file with partial data encoded into name: "
           + jhFileName);
     }
-    
+
     return indexInfo;
   }
 
   
   /**
-   * Helper function to encode the URL of the filename of the job-history 
+   * Helper function to encode the URL of the filename of the job-history
    * log file.
    * 
    * @param logFileName file name of the job-history file
@@ -208,7 +222,8 @@ public class FileNameIndexUtils {
     if (logFileName.contains(DELIMITER_ESCAPE)) {
       replacementDelimiterEscape = nonOccursString(logFileName);
 
-      logFileName = logFileName.replaceAll(DELIMITER_ESCAPE, replacementDelimiterEscape);
+      logFileName = logFileName.replaceAll(
+          DELIMITER_ESCAPE, replacementDelimiterEscape);
     }
 
     String encodedFileName = null;
@@ -223,14 +238,15 @@ public class FileNameIndexUtils {
 
     // Restore protected escape delimiters after encoding
     if (replacementDelimiterEscape != null) {
-      encodedFileName = encodedFileName.replaceAll(replacementDelimiterEscape, DELIMITER_ESCAPE);
+      encodedFileName = encodedFileName.replaceAll(
+          replacementDelimiterEscape, DELIMITER_ESCAPE);
     }
 
     return encodedFileName;
   }
-  
+
   /**
-   * Helper function to decode the URL of the filename of the job-history 
+   * Helper function to decode the URL of the filename of the job-history
    * log file.
    * 
    * @param logFileName file name of the job-history file
@@ -250,7 +266,7 @@ public class FileNameIndexUtils {
     }
     return decodedFileName;
   }
-  
+
   static String nonOccursString(String logFileName) {
     int adHocIndex = 0;
 
@@ -262,11 +278,11 @@ public class FileNameIndexUtils {
 
     return unfoundString + "q";
   }
-  
+
   private static String getUserName(JobIndexInfo indexInfo) {
     return getNonEmptyString(indexInfo.getUser());
   }
-  
+
   private static String getJobName(JobIndexInfo indexInfo) {
     return getNonEmptyString(indexInfo.getJobName());
   }
@@ -283,18 +299,65 @@ public class FileNameIndexUtils {
     }
     return in;
   }
-  
+
   private static String escapeDelimiters(String escapee) {
     return escapee.replaceAll(DELIMITER, DELIMITER_ESCAPE);
   }
 
   /**
-   * Trims the job-name if required
+   * Trims the url-encoded string if required
    */
-  private static String trimJobName(String jobName, int jobNameLimit) {
-    if (jobName.length() > jobNameLimit) {
-      jobName = jobName.substring(0, jobNameLimit);
+  private static String trimURLEncodedString(
+      String encodedString, int limitLength) {
+    assert(limitLength >= 0) : "limitLength should be positive integer";
+
+    if (encodedString.length() < limitLength) {
+      return encodedString;
+    }
+
+    int index = 0;
+    int increase = 0;
+    byte[] strBytes = encodedString.getBytes(UTF_8);
+
+    // calculate effective character length based on UTF-8 specification.
+    // The size of a character coded in UTF-8 should be 4-byte at most.
+    // See RFC3629
+    while (true) {
+      byte b = strBytes[index];
+      if (b == '%') {
+        byte minuend1 = strBytes[index + 1];
+        byte subtrahend1 = (byte)(Character.isDigit(
+            minuend1) ? '0' : 'A' - 10);
+        byte minuend2 = strBytes[index + 2];
+        byte subtrahend2 = (byte)(Character.isDigit(
+            minuend2) ? '0' : 'A' - 10);
+        int initialHex =
+            ((Character.toUpperCase(minuend1) - subtrahend1) << 4) +
+            (Character.toUpperCase(minuend2) - subtrahend2);
+
+        if (0x00 <= initialHex && initialHex <= 0x7F) {
+          // For 1-byte UTF-8 characters
+          increase = 3;
+        } else if (0xC2 <= initialHex && initialHex <= 0xDF) {
+          // For 2-byte UTF-8 characters
+          increase = 6;
+        } else if (0xE0 <= initialHex && initialHex <= 0xEF) {
+          // For 3-byte UTF-8 characters
+          increase = 9;
+        } else {
+          // For 4-byte UTF-8 characters
+          increase = 12;
+        }
+      } else {
+        increase = 1;
+      }
+      if (index + increase > limitLength) {
+        break;
+      } else {
+        index += increase;
+      }
     }
-    return jobName;
+
+    return encodedString.substring(0, index);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df99ea8a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/jobhistory/TestFileNameIndexUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/jobhistory/TestFileNameIndexUtils.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/jobhistory/TestFileNameIndexUtils.java
index 3d722e0..6fa7dbc 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/jobhistory/TestFileNameIndexUtils.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/jobhistory/TestFileNameIndexUtils.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.mapreduce.v2.jobhistory;
 
 import java.io.IOException;
+import static java.nio.charset.StandardCharsets.UTF_8;
 
 import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.mapreduce.TypeConverter;
@@ -30,14 +31,14 @@ import org.junit.Test;
 public class TestFileNameIndexUtils {
 
   private static final String OLD_JOB_HISTORY_FILE_FORMATTER = "%s"
-    + FileNameIndexUtils.DELIMITER + "%s"
-    + FileNameIndexUtils.DELIMITER + "%s"
-    + FileNameIndexUtils.DELIMITER + "%s"
-    + FileNameIndexUtils.DELIMITER + "%s"
-    + FileNameIndexUtils.DELIMITER + "%s"
-    + FileNameIndexUtils.DELIMITER + "%s"
-    + FileNameIndexUtils.DELIMITER + "%s"
-    + JobHistoryUtils.JOB_HISTORY_FILE_EXTENSION;
+      + FileNameIndexUtils.DELIMITER + "%s"
+      + FileNameIndexUtils.DELIMITER + "%s"
+      + FileNameIndexUtils.DELIMITER + "%s"
+      + FileNameIndexUtils.DELIMITER + "%s"
+      + FileNameIndexUtils.DELIMITER + "%s"
+      + FileNameIndexUtils.DELIMITER + "%s"
+      + FileNameIndexUtils.DELIMITER + "%s"
+      + JobHistoryUtils.JOB_HISTORY_FILE_EXTENSION;
 
   private static final String OLD_FORMAT_BEFORE_ADD_START_TIME = "%s"
       + FileNameIndexUtils.DELIMITER + "%s"
@@ -51,29 +52,29 @@ public class TestFileNameIndexUtils {
       + JobHistoryUtils.JOB_HISTORY_FILE_EXTENSION;
 
   private static final String JOB_HISTORY_FILE_FORMATTER = "%s"
-    + FileNameIndexUtils.DELIMITER + "%s"
-    + FileNameIndexUtils.DELIMITER + "%s"
-    + FileNameIndexUtils.DELIMITER + "%s"
-    + FileNameIndexUtils.DELIMITER + "%s"
-    + FileNameIndexUtils.DELIMITER + "%s"
-    + FileNameIndexUtils.DELIMITER + "%s"
-    + FileNameIndexUtils.DELIMITER + "%s"
-    + FileNameIndexUtils.DELIMITER + "%s"
-    + FileNameIndexUtils.DELIMITER + "%s"
-    + JobHistoryUtils.JOB_HISTORY_FILE_EXTENSION;
+      + FileNameIndexUtils.DELIMITER + "%s"
+      + FileNameIndexUtils.DELIMITER + "%s"
+      + FileNameIndexUtils.DELIMITER + "%s"
+      + FileNameIndexUtils.DELIMITER + "%s"
+      + FileNameIndexUtils.DELIMITER + "%s"
+      + FileNameIndexUtils.DELIMITER + "%s"
+      + FileNameIndexUtils.DELIMITER + "%s"
+      + FileNameIndexUtils.DELIMITER + "%s"
+      + FileNameIndexUtils.DELIMITER + "%s"
+      + JobHistoryUtils.JOB_HISTORY_FILE_EXTENSION;
 
   private static final String JOB_ID = "job_1317928501754_0001";
   private static final String SUBMIT_TIME = "1317928742025";
   private static final String USER_NAME = "username";
   private static final String USER_NAME_WITH_DELIMITER = "user"
-    + FileNameIndexUtils.DELIMITER + "name";
+      + FileNameIndexUtils.DELIMITER + "name";
   private static final String USER_NAME_WITH_DELIMITER_ESCAPE = "user"
-    + FileNameIndexUtils.DELIMITER_ESCAPE + "name";
+      + FileNameIndexUtils.DELIMITER_ESCAPE + "name";
   private static final String JOB_NAME = "mapreduce";
   private static final String JOB_NAME_WITH_DELIMITER = "map"
-    + FileNameIndexUtils.DELIMITER + "reduce";
+      + FileNameIndexUtils.DELIMITER + "reduce";
   private static final String JOB_NAME_WITH_DELIMITER_ESCAPE = "map"
-    + FileNameIndexUtils.DELIMITER_ESCAPE + "reduce";
+      + FileNameIndexUtils.DELIMITER_ESCAPE + "reduce";
   private static final String FINISH_TIME = "1317928754958";
   private static final String NUM_MAPS = "1";
   private static final String NUM_REDUCES = "1";
@@ -123,7 +124,7 @@ public class TestFileNameIndexUtils {
     Assert.assertEquals("Queue name different after encoding and decoding",
         info.getQueueName(), parsedInfo.getQueueName());
     Assert.assertEquals("Job start time different after encoding and decoding",
-              info.getJobStartTime(), parsedInfo.getJobStartTime());
+        info.getJobStartTime(), parsedInfo.getJobStartTime());
   }
 
   @Test
@@ -173,6 +174,158 @@ public class TestFileNameIndexUtils {
         parsedInfo.getJobName());
   }
 
+  /**
+   * Verify the name of jobhistory file is not greater than 255 bytes
+   * even if there are some multibyte characters in the job name.
+   */
+  @Test
+  public void testJobNameWithMultibyteChars() throws IOException {
+    JobIndexInfo info = new JobIndexInfo();
+    JobID oldJobId = JobID.forName(JOB_ID);
+    JobId jobId = TypeConverter.toYarn(oldJobId);
+    info.setJobId(jobId);
+    info.setSubmitTime(Long.parseLong(SUBMIT_TIME));
+    info.setUser(USER_NAME);
+
+    StringBuilder sb = new StringBuilder();
+    info.setFinishTime(Long.parseLong(FINISH_TIME));
+    info.setNumMaps(Integer.parseInt(NUM_MAPS));
+    info.setNumReduces(Integer.parseInt(NUM_REDUCES));
+    info.setJobStatus(JOB_STATUS);
+    info.setQueueName(QUEUE_NAME);
+    info.setJobStartTime(Long.parseLong(JOB_START_TIME));
+
+    // Test for 1 byte UTF-8 character
+    // which is encoded into 1 x 3 = 3 characters by URL encode.
+    for (int i = 0; i < 100; i++) {
+      sb.append('%');
+    }
+    String longJobName = sb.toString();
+    info.setJobName(longJobName);
+
+    String jobHistoryFile =
+        FileNameIndexUtils.getDoneFileName(info, 50);
+
+    Assert.assertTrue(jobHistoryFile.length() <= 255);
+    String trimedJobName = jobHistoryFile.split(
+        FileNameIndexUtils.DELIMITER)[3]; // 3 is index of job name
+
+    // 3 x 16 < 50 < 3 x 17 so the length of trimedJobName should be 48
+    Assert.assertEquals(48, trimedJobName.getBytes(UTF_8).length);
+
+    // validate whether trimmedJobName by testing reversibility
+    byte[] trimedJobNameInByte = trimedJobName.getBytes(UTF_8);
+    String reEncodedTrimedJobName = new String(trimedJobNameInByte, UTF_8);
+    Assert.assertArrayEquals(trimedJobNameInByte,
+        reEncodedTrimedJobName.getBytes(UTF_8));
+    sb.setLength(0);
+
+    // Test for 2 bytes UTF-8 character
+    // which is encoded into 2 x 3 = 6 characters by URL encode.
+    for (int i = 0; i < 100; i++) {
+      sb.append('\u03A9'); // large omega
+    }
+    longJobName = sb.toString();
+    info.setJobName(longJobName);
+
+    jobHistoryFile =
+        FileNameIndexUtils.getDoneFileName(info, 27);
+
+    Assert.assertTrue(jobHistoryFile.length() <= 255);
+    trimedJobName = jobHistoryFile.split(
+        FileNameIndexUtils.DELIMITER)[3]; // 3 is index of job name
+
+    // 6 x 4 < 27 < 6 x 5 so the length of trimedJobName should be 24
+    Assert.assertEquals(24, trimedJobName.getBytes(UTF_8).length);
+
+    // validate whether trimmedJobName by testing reversibility
+    trimedJobNameInByte = trimedJobName.getBytes(UTF_8);
+    reEncodedTrimedJobName = new String(trimedJobNameInByte, UTF_8);
+    Assert.assertArrayEquals(trimedJobNameInByte,
+        reEncodedTrimedJobName.getBytes(UTF_8));
+    sb.setLength(0);
+
+    // Test for 3 bytes UTF-8 character
+    // which is encoded into 3 x 3 = 9 characters by URL encode.
+    for (int i = 0; i < 100; i++) {
+      sb.append('\u2192'); // rightwards arrow
+    }
+    longJobName = sb.toString();
+    info.setJobName(longJobName);
+
+    jobHistoryFile =
+        FileNameIndexUtils.getDoneFileName(info, 40);
+
+    Assert.assertTrue(jobHistoryFile.length() <= 255);
+    trimedJobName = jobHistoryFile.split(
+        FileNameIndexUtils.DELIMITER)[3]; // 3 is index of job name
+
+    // 9 x 4 < 40 < 9 x 5 so the length of trimedJobName should be 36
+    Assert.assertEquals(36, trimedJobName.getBytes(UTF_8).length);
+
+    // validate whether trimmedJobName by testing reversibility
+    trimedJobNameInByte = trimedJobName.getBytes(UTF_8);
+    reEncodedTrimedJobName = new String(trimedJobNameInByte, UTF_8);
+    Assert.assertArrayEquals(trimedJobNameInByte,
+        reEncodedTrimedJobName.getBytes(UTF_8));
+    sb.setLength(0);
+
+    // Test for 4 bytes UTF-8 character
+    // which is encoded into 4 x 3 = 12 characters by URL encode.
+    for (int i = 0; i < 100; i++) {
+      sb.append("\uD867\uDE3D"); // Mugil cephalus in Kanji.
+    }
+    longJobName = sb.toString();
+    info.setJobName(longJobName);
+
+    jobHistoryFile =
+        FileNameIndexUtils.getDoneFileName(info, 49);
+
+    Assert.assertTrue(jobHistoryFile.length() <= 255);
+    trimedJobName = jobHistoryFile.split(
+        FileNameIndexUtils.DELIMITER)[3]; // 3 is index of job name
+
+    // 12 x 4 < 49 < 12 x 5 so the length of trimedJobName should be 48
+    Assert.assertEquals(48, trimedJobName.getBytes(UTF_8).length);
+
+    // validate whether trimmedJobName by testing reversibility
+    trimedJobNameInByte = trimedJobName.getBytes(UTF_8);
+    reEncodedTrimedJobName = new String(trimedJobNameInByte, UTF_8);
+    Assert.assertArrayEquals(trimedJobNameInByte,
+        reEncodedTrimedJobName.getBytes(UTF_8));
+    sb.setLength(0);
+
+    // Test for the combination of 1 to 4 bytes UTF-8 characters
+    sb.append('\u732B') // cat in Kanji (encoded into 3 bytes x 3 characters)
+      .append("[") // (encoded into 1 byte x 3 characters)
+      .append('\u03BB') // small lambda (encoded into 2 bytes x 3 characters)
+      .append('/') // (encoded into 1 byte x 3 characters)
+      .append('A') // not url-encoded (1 byte x 1 character)
+      .append("\ud867\ude49") // flying fish in
+                              // Kanji (encoded into 4 bytes x 3 characters)
+      .append('\u72AC'); // dog in Kanji (encoded into 3 bytes x 3 characters)
+
+    longJobName = sb.toString();
+    info.setJobName(longJobName);
+
+    jobHistoryFile =
+        FileNameIndexUtils.getDoneFileName(info, 23);
+
+    Assert.assertTrue(jobHistoryFile.length() <= 255);
+    trimedJobName = jobHistoryFile.split(
+        FileNameIndexUtils.DELIMITER)[3]; // 3 is index of job name
+
+    // total size of the first 5 characters = 22
+    // 23 < total size of the first 6 characters
+    Assert.assertEquals(22, trimedJobName.getBytes(UTF_8).length);
+
+    // validate whether trimmedJobName by testing reversibility
+    trimedJobNameInByte = trimedJobName.getBytes(UTF_8);
+    reEncodedTrimedJobName = new String(trimedJobNameInByte, UTF_8);
+    Assert.assertArrayEquals(trimedJobNameInByte,
+        reEncodedTrimedJobName.getBytes(UTF_8));
+  }
+
   @Test
   public void testUserNamePercentDecoding() throws IOException {
     String jobHistoryFile = String.format(JOB_HISTORY_FILE_FORMATTER,


[18/50] hadoop git commit: YARN-3542. Refactored existing CPU cgroups support to use the newer and integrated ResourceHandler mechanism, and also deprecated the old LCEResourceHandler inteface hierarchy. Contributed by Varun Vasudev.

Posted by ar...@apache.org.
YARN-3542. Refactored existing CPU cgroups support to use the newer and integrated ResourceHandler mechanism, and also deprecated the old LCEResourceHandler inteface hierarchy. Contributed by Varun Vasudev.


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

Branch: refs/heads/HDFS-1312
Commit: 2085e60a9655b59aa2ba8917acdc511ab71ff6e4
Parents: e8650fe
Author: Vinod Kumar Vavilapalli (I am also known as @tshooter.) <vi...@apache.org>
Authored: Mon Jan 25 16:19:03 2016 -0800
Committer: Vinod Kumar Vavilapalli (I am also known as @tshooter.) <vi...@apache.org>
Committed: Mon Jan 25 16:19:36 2016 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   4 +
 .../hadoop/yarn/conf/YarnConfiguration.java     |  12 +
 .../yarn/conf/TestYarnConfigurationFields.java  |   2 +
 .../nodemanager/LinuxContainerExecutor.java     |  28 +-
 .../CGroupsCpuResourceHandlerImpl.java          | 235 +++++++++++++++
 .../linux/resources/CGroupsHandler.java         |   4 +
 .../linux/resources/CpuResourceHandler.java     |  32 ++
 .../linux/resources/ResourceHandlerModule.java  |  34 +++
 .../util/CgroupsLCEResourcesHandler.java        |  68 +----
 .../util/DefaultLCEResourcesHandler.java        |   1 +
 .../nodemanager/util/LCEResourcesHandler.java   |   1 +
 .../TestCGroupsCpuResourceHandlerImpl.java      | 297 +++++++++++++++++++
 .../util/TestCgroupsLCEResourcesHandler.java    |   1 +
 13 files changed, 661 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2085e60a/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 41802ae..c2f16d5 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -107,6 +107,10 @@ Release 2.9.0 - UNRELEASED
     YARN-4496. Improve HA ResourceManager Failover detection on the client.
     (Jian He via xgong)
 
+    YARN-3542. Refactored existing CPU cgroups support to use the newer and
+    integrated ResourceHandler mechanism, and also deprecated the old
+    LCEResourceHandler inteface hierarchy. (Varun Vasudev via vinodkv)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2085e60a/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 23c2969..e214a86 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
@@ -938,6 +938,18 @@ public class YarnConfiguration extends Configuration {
       DEFAULT_NM_MEMORY_RESOURCE_CGROUPS_SOFT_LIMIT_PERCENTAGE =
       90.0f;
 
+  @Private
+  public static final String NM_CPU_RESOURCE_PREFIX = NM_PREFIX
+      + "resource.cpu.";
+
+  /** Enable cpu isolation. */
+  @Private
+  public static final String NM_CPU_RESOURCE_ENABLED =
+      NM_CPU_RESOURCE_PREFIX + "enabled";
+
+  @Private
+  public static final boolean DEFAULT_NM_CPU_RESOURCE_ENABLED = false;
+
   /**
    * Prefix for disk configurations. Work in progress: This configuration
    * parameter may be changed/removed in the future.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2085e60a/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 0e508ed..529d63b 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
@@ -111,6 +111,8 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
         .add(YarnConfiguration.NM_DISK_RESOURCE_ENABLED);
     configurationPrefixToSkipCompare
         .add(YarnConfiguration.NM_MEMORY_RESOURCE_PREFIX);
+    configurationPrefixToSkipCompare
+        .add(YarnConfiguration.NM_CPU_RESOURCE_ENABLED);
 
     // Set by container-executor.cfg
     configurationPrefixToSkipCompare.add(YarnConfiguration.NM_USER_HOME_DIR);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2085e60a/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 ff82e97..1367b13 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
@@ -49,6 +49,7 @@ import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerSignalContext
 import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerStartContext;
 import org.apache.hadoop.yarn.server.nodemanager.executor.DeletionAsUserContext;
 import org.apache.hadoop.yarn.server.nodemanager.executor.LocalizerStartContext;
+import org.apache.hadoop.yarn.server.nodemanager.util.CgroupsLCEResourcesHandler;
 import org.apache.hadoop.yarn.server.nodemanager.util.DefaultLCEResourcesHandler;
 import org.apache.hadoop.yarn.server.nodemanager.util.LCEResourcesHandler;
 import org.apache.hadoop.yarn.util.ConverterUtils;
@@ -95,10 +96,7 @@ public class LinuxContainerExecutor extends ContainerExecutor {
   public void setConf(Configuration conf) {
     super.setConf(conf);
 
-    resourcesHandler = ReflectionUtils.newInstance(
-        conf.getClass(YarnConfiguration.NM_LINUX_CONTAINER_RESOURCES_HANDLER,
-            DefaultLCEResourcesHandler.class, LCEResourcesHandler.class), conf);
-    resourcesHandler.setConf(conf);
+    resourcesHandler = getResourcesHandler(conf);
 
     if (conf.get(YarnConfiguration.NM_CONTAINER_EXECUTOR_SCHED_PRIORITY)
         != null) {
@@ -122,6 +120,23 @@ public class LinuxContainerExecutor extends ContainerExecutor {
     }
   }
 
+  private LCEResourcesHandler getResourcesHandler(Configuration conf) {
+    LCEResourcesHandler handler = ReflectionUtils.newInstance(
+        conf.getClass(YarnConfiguration.NM_LINUX_CONTAINER_RESOURCES_HANDLER,
+            DefaultLCEResourcesHandler.class, LCEResourcesHandler.class), conf);
+
+    // Stop using CgroupsLCEResourcesHandler
+    // use the resource handler chain instead
+    // ResourceHandlerModule will create the cgroup cpu module if
+    // CgroupsLCEResourcesHandler is set
+    if (handler instanceof CgroupsLCEResourcesHandler) {
+      handler =
+          ReflectionUtils.newInstance(DefaultLCEResourcesHandler.class, conf);
+    }
+    handler.setConf(conf);
+    return handler;
+  }
+
   void verifyUsernamePattern(String user) {
     if (!UserGroupInformation.isSecurityEnabled() &&
         !nonsecureLocalUserPattern.matcher(user).matches()) {
@@ -184,7 +199,12 @@ public class LinuxContainerExecutor extends ContainerExecutor {
     try {
       resourceHandlerChain = ResourceHandlerModule
           .getConfiguredResourceHandlerChain(conf);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Resource handler chain enabled = " + (resourceHandlerChain
+            == null));
+      }
       if (resourceHandlerChain != null) {
+        LOG.debug("Bootstrapping resource handler chain");
         resourceHandlerChain.bootstrap(conf);
       }
     } catch (ResourceHandlerException e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2085e60a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsCpuResourceHandlerImpl.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/CGroupsCpuResourceHandlerImpl.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/CGroupsCpuResourceHandlerImpl.java
new file mode 100644
index 0000000..0071c7e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsCpuResourceHandlerImpl.java
@@ -0,0 +1,235 @@
+/**
+ * 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
+ * <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.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Resource;
+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.PrivilegedOperation;
+import org.apache.hadoop.yarn.server.nodemanager.util.NodeManagerHardwareUtils;
+import org.apache.hadoop.yarn.util.ResourceCalculatorPlugin;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * An implementation for using CGroups to restrict CPU usage on Linux. The
+ * implementation supports 3 different controls - restrict usage of all YARN
+ * containers, restrict relative usage of individual YARN containers and
+ * restrict usage of individual YARN containers. Admins can set the overall CPU
+ * to be used by all YARN containers - this is implemented by setting
+ * cpu.cfs_period_us and cpu.cfs_quota_us to the ratio desired. If strict
+ * resource usage mode is not enabled, cpu.shares is set for individual
+ * containers - this prevents containers from exceeding the overall limit for
+ * YARN containers but individual containers can use as much of the CPU as
+ * available(under the YARN limit). If strict resource usage is enabled, then
+ * container can only use the percentage of CPU allocated to them and this is
+ * again implemented using cpu.cfs_period_us and cpu.cfs_quota_us.
+ *
+ */
+@InterfaceStability.Unstable
+@InterfaceAudience.Private
+public class CGroupsCpuResourceHandlerImpl implements CpuResourceHandler {
+
+  static final Log LOG = LogFactory.getLog(CGroupsCpuResourceHandlerImpl.class);
+
+  private CGroupsHandler cGroupsHandler;
+  private boolean strictResourceUsageMode = false;
+  private float yarnProcessors;
+  private int nodeVCores;
+  private static final CGroupsHandler.CGroupController CPU =
+      CGroupsHandler.CGroupController.CPU;
+
+  @VisibleForTesting
+  static final int MAX_QUOTA_US = 1000 * 1000;
+  @VisibleForTesting
+  static final int MIN_PERIOD_US = 1000;
+  @VisibleForTesting
+  static final int CPU_DEFAULT_WEIGHT = 1024; // set by kernel
+
+  CGroupsCpuResourceHandlerImpl(CGroupsHandler cGroupsHandler) {
+    this.cGroupsHandler = cGroupsHandler;
+  }
+
+  @Override
+  public List<PrivilegedOperation> bootstrap(Configuration conf)
+      throws ResourceHandlerException {
+    return bootstrap(
+        ResourceCalculatorPlugin.getResourceCalculatorPlugin(null, conf), conf);
+  }
+
+  @VisibleForTesting
+  List<PrivilegedOperation> bootstrap(
+      ResourceCalculatorPlugin plugin, Configuration conf)
+      throws ResourceHandlerException {
+    this.strictResourceUsageMode = conf.getBoolean(
+        YarnConfiguration.NM_LINUX_CONTAINER_CGROUPS_STRICT_RESOURCE_USAGE,
+        YarnConfiguration.DEFAULT_NM_LINUX_CONTAINER_CGROUPS_STRICT_RESOURCE_USAGE);
+    this.cGroupsHandler.mountCGroupController(CPU);
+    nodeVCores = NodeManagerHardwareUtils.getVCores(plugin, conf);
+
+    // cap overall usage to the number of cores allocated to YARN
+    yarnProcessors = NodeManagerHardwareUtils.getContainersCPUs(plugin, conf);
+    int systemProcessors = NodeManagerHardwareUtils.getNodeCPUs(plugin, conf);
+    boolean existingCpuLimits;
+    try {
+      existingCpuLimits =
+          cpuLimitsExist(cGroupsHandler.getPathForCGroup(CPU, ""));
+    } catch (IOException ie) {
+      throw new ResourceHandlerException(ie);
+    }
+    if (systemProcessors != (int) yarnProcessors) {
+      LOG.info("YARN containers restricted to " + yarnProcessors + " cores");
+      int[] limits = getOverallLimits(yarnProcessors);
+      cGroupsHandler
+          .updateCGroupParam(CPU, "", CGroupsHandler.CGROUP_CPU_PERIOD_US,
+              String.valueOf(limits[0]));
+      cGroupsHandler
+          .updateCGroupParam(CPU, "", CGroupsHandler.CGROUP_CPU_QUOTA_US,
+              String.valueOf(limits[1]));
+    } else if (existingCpuLimits) {
+      LOG.info("Removing CPU constraints for YARN containers.");
+      cGroupsHandler
+          .updateCGroupParam(CPU, "", CGroupsHandler.CGROUP_CPU_QUOTA_US,
+              String.valueOf(-1));
+    }
+    return null;
+  }
+
+  @InterfaceAudience.Private
+  public static boolean cpuLimitsExist(String path)
+      throws IOException {
+    File quotaFile = new File(path,
+        CPU.getName() + "." + CGroupsHandler.CGROUP_CPU_QUOTA_US);
+    if (quotaFile.exists()) {
+      String contents = FileUtils.readFileToString(quotaFile, "UTF-8");
+      int quotaUS = Integer.parseInt(contents.trim());
+      if (quotaUS != -1) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  @VisibleForTesting
+  @InterfaceAudience.Private
+  public static int[] getOverallLimits(float yarnProcessors) {
+
+    int[] ret = new int[2];
+
+    if (yarnProcessors < 0.01f) {
+      throw new IllegalArgumentException("Number of processors can't be <= 0.");
+    }
+
+    int quotaUS = MAX_QUOTA_US;
+    int periodUS = (int) (MAX_QUOTA_US / yarnProcessors);
+    if (yarnProcessors < 1.0f) {
+      periodUS = MAX_QUOTA_US;
+      quotaUS = (int) (periodUS * yarnProcessors);
+      if (quotaUS < MIN_PERIOD_US) {
+        LOG.warn("The quota calculated for the cgroup was too low."
+            + " The minimum value is " + MIN_PERIOD_US
+            + ", calculated value is " + quotaUS
+            + ". Setting quota to minimum value.");
+        quotaUS = MIN_PERIOD_US;
+      }
+    }
+
+    // cfs_period_us can't be less than 1000 microseconds
+    // if the value of periodUS is less than 1000, we can't really use cgroups
+    // to limit cpu
+    if (periodUS < MIN_PERIOD_US) {
+      LOG.warn("The period calculated for the cgroup was too low."
+          + " The minimum value is " + MIN_PERIOD_US
+          + ", calculated value is " + periodUS
+          + ". Using all available CPU.");
+      periodUS = MAX_QUOTA_US;
+      quotaUS = -1;
+    }
+
+    ret[0] = periodUS;
+    ret[1] = quotaUS;
+    return ret;
+  }
+
+  @Override
+  public List<PrivilegedOperation> preStart(Container container)
+      throws ResourceHandlerException {
+
+    String cgroupId = container.getContainerId().toString();
+    Resource containerResource = container.getResource();
+    cGroupsHandler.createCGroup(CPU, cgroupId);
+    try {
+      int containerVCores = containerResource.getVirtualCores();
+      int cpuShares = CPU_DEFAULT_WEIGHT * containerVCores;
+      cGroupsHandler
+          .updateCGroupParam(CPU, cgroupId, CGroupsHandler.CGROUP_CPU_SHARES,
+              String.valueOf(cpuShares));
+      if (strictResourceUsageMode) {
+        if (nodeVCores != containerVCores) {
+          float containerCPU =
+              (containerVCores * yarnProcessors) / (float) nodeVCores;
+          int[] limits = getOverallLimits(containerCPU);
+          cGroupsHandler.updateCGroupParam(CPU, cgroupId,
+              CGroupsHandler.CGROUP_CPU_PERIOD_US, String.valueOf(limits[0]));
+          cGroupsHandler.updateCGroupParam(CPU, cgroupId,
+              CGroupsHandler.CGROUP_CPU_QUOTA_US, String.valueOf(limits[1]));
+        }
+      }
+    } catch (ResourceHandlerException re) {
+      cGroupsHandler.deleteCGroup(CPU, cgroupId);
+      LOG.warn("Could not update cgroup for container", re);
+      throw re;
+    }
+    List<PrivilegedOperation> ret = new ArrayList<>();
+    ret.add(new PrivilegedOperation(
+        PrivilegedOperation.OperationType.ADD_PID_TO_CGROUP,
+        PrivilegedOperation.CGROUP_ARG_PREFIX + cGroupsHandler
+            .getPathForCGroupTasks(CPU, cgroupId)));
+    return ret;
+  }
+
+  @Override
+  public List<PrivilegedOperation> reacquireContainer(ContainerId containerId)
+      throws ResourceHandlerException {
+    return null;
+  }
+
+  @Override
+  public List<PrivilegedOperation> postComplete(ContainerId containerId)
+      throws ResourceHandlerException {
+    cGroupsHandler.deleteCGroup(CPU, containerId.toString());
+    return null;
+  }
+
+  @Override public List<PrivilegedOperation> teardown()
+      throws ResourceHandlerException {
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2085e60a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandler.java
index d86c5f2..eefaabd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandler.java
@@ -58,6 +58,10 @@ public interface CGroupsHandler {
   String CGROUP_PARAM_MEMORY_SWAPPINESS = "swappiness";
 
 
+  String CGROUP_CPU_PERIOD_US = "cfs_period_us";
+  String CGROUP_CPU_QUOTA_US = "cfs_quota_us";
+  String CGROUP_CPU_SHARES = "shares";
+
   /**
    * Mounts a cgroup controller
    * @param controller - the controller being mounted

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2085e60a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CpuResourceHandler.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/CpuResourceHandler.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/CpuResourceHandler.java
new file mode 100644
index 0000000..b253ce5
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CpuResourceHandler.java
@@ -0,0 +1,32 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Resource handler for cpu resources.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public interface CpuResourceHandler extends ResourceHandler {
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2085e60a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/ResourceHandlerModule.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/ResourceHandlerModule.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/ResourceHandlerModule.java
index da70536..7507a82 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/ResourceHandlerModule.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/ResourceHandlerModule.java
@@ -21,11 +21,15 @@
 package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.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.linux.privileged.PrivilegedOperationExecutor;
+import org.apache.hadoop.yarn.server.nodemanager.util.CgroupsLCEResourcesHandler;
+import org.apache.hadoop.yarn.server.nodemanager.util.DefaultLCEResourcesHandler;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -38,6 +42,7 @@ import java.util.List;
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
 public class ResourceHandlerModule {
+  static final Log LOG = LogFactory.getLog(ResourceHandlerModule.class);
   private static volatile ResourceHandlerChain resourceHandlerChain;
 
   /**
@@ -52,6 +57,8 @@ public class ResourceHandlerModule {
       cGroupsBlkioResourceHandler;
   private static volatile CGroupsMemoryResourceHandlerImpl
       cGroupsMemoryResourceHandler;
+  private static volatile CGroupsCpuResourceHandlerImpl
+      cGroupsCpuResourceHandler;
 
   /**
    * Returns an initialized, thread-safe CGroupsHandler instance.
@@ -70,6 +77,30 @@ public class ResourceHandlerModule {
     return cGroupsHandler;
   }
 
+  private static CGroupsCpuResourceHandlerImpl getcGroupsCpuResourceHandler(
+      Configuration conf) throws ResourceHandlerException {
+    boolean cgroupsCpuEnabled =
+        conf.getBoolean(YarnConfiguration.NM_CPU_RESOURCE_ENABLED,
+            YarnConfiguration.DEFAULT_NM_CPU_RESOURCE_ENABLED);
+    boolean cgroupsLCEResourcesHandlerEnabled =
+        conf.getClass(YarnConfiguration.NM_LINUX_CONTAINER_RESOURCES_HANDLER,
+            DefaultLCEResourcesHandler.class)
+            .equals(CgroupsLCEResourcesHandler.class);
+    if (cgroupsCpuEnabled || cgroupsLCEResourcesHandlerEnabled) {
+      if (cGroupsCpuResourceHandler == null) {
+        synchronized (CpuResourceHandler.class) {
+          if (cGroupsCpuResourceHandler == null) {
+            LOG.debug("Creating new cgroups cpu handler");
+            cGroupsCpuResourceHandler =
+                new CGroupsCpuResourceHandlerImpl(getCGroupsHandler(conf));
+            return cGroupsCpuResourceHandler;
+          }
+        }
+      }
+    }
+    return null;
+  }
+
   private static TrafficControlBandwidthHandlerImpl
     getTrafficControlBandwidthHandler(Configuration conf)
       throws ResourceHandlerException {
@@ -78,6 +109,7 @@ public class ResourceHandlerModule {
       if (trafficControlBandwidthHandler == null) {
         synchronized (OutboundBandwidthResourceHandler.class) {
           if (trafficControlBandwidthHandler == null) {
+            LOG.debug("Creating new traffic control bandwidth handler");
             trafficControlBandwidthHandler = new
                 TrafficControlBandwidthHandlerImpl(PrivilegedOperationExecutor
                 .getInstance(conf), getCGroupsHandler(conf),
@@ -113,6 +145,7 @@ public class ResourceHandlerModule {
     if (cGroupsBlkioResourceHandler == null) {
       synchronized (DiskResourceHandler.class) {
         if (cGroupsBlkioResourceHandler == null) {
+          LOG.debug("Creating new cgroups blkio handler");
           cGroupsBlkioResourceHandler =
               new CGroupsBlkioResourceHandlerImpl(getCGroupsHandler(conf));
         }
@@ -158,6 +191,7 @@ public class ResourceHandlerModule {
     addHandlerIfNotNull(handlerList, getOutboundBandwidthResourceHandler(conf));
     addHandlerIfNotNull(handlerList, getDiskResourceHandler(conf));
     addHandlerIfNotNull(handlerList, getMemoryResourceHandler(conf));
+    addHandlerIfNotNull(handlerList, getcGroupsCpuResourceHandler(conf));
     resourceHandlerChain = new ResourceHandlerChain(handlerList);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2085e60a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/CgroupsLCEResourcesHandler.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/util/CgroupsLCEResourcesHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/CgroupsLCEResourcesHandler.java
index da39fa2..f04fcd2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/CgroupsLCEResourcesHandler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/CgroupsLCEResourcesHandler.java
@@ -39,7 +39,6 @@ import java.util.regex.Pattern;
 
 import com.google.common.annotations.VisibleForTesting;
 
-import org.apache.commons.io.FileUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -50,10 +49,20 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.nodemanager.LinuxContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperation;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.CGroupsCpuResourceHandlerImpl;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.ResourceCalculatorPlugin;
 import org.apache.hadoop.yarn.util.SystemClock;
 
+/**
+ * Resource handler that lets you setup cgroups
+ * to to handle cpu isolation. Please look at the ResourceHandlerModule
+ * and CGroupsCpuResourceHandlerImpl classes which let you isolate multiple
+ * resources using cgroups.
+ * Deprecated - please look at ResourceHandlerModule and
+ * CGroupsCpuResourceHandlerImpl
+ */
+@Deprecated
 public class CgroupsLCEResourcesHandler implements LCEResourcesHandler {
 
   final static Log LOG = LogFactory
@@ -73,8 +82,6 @@ public class CgroupsLCEResourcesHandler implements LCEResourcesHandler {
   private final String CPU_PERIOD_US = "cfs_period_us";
   private final String CPU_QUOTA_US = "cfs_quota_us";
   private final int CPU_DEFAULT_WEIGHT = 1024; // set by kernel
-  private final int MAX_QUOTA_US = 1000 * 1000;
-  private final int MIN_PERIOD_US = 1000;
   private final Map<String, String> controllerPaths; // Controller -> path
 
   private long deleteCgroupTimeout;
@@ -163,65 +170,18 @@ public class CgroupsLCEResourcesHandler implements LCEResourcesHandler {
       int[] limits = getOverallLimits(yarnProcessors);
       updateCgroup(CONTROLLER_CPU, "", CPU_PERIOD_US, String.valueOf(limits[0]));
       updateCgroup(CONTROLLER_CPU, "", CPU_QUOTA_US, String.valueOf(limits[1]));
-    } else if (cpuLimitsExist()) {
+    } else if (CGroupsCpuResourceHandlerImpl.cpuLimitsExist(
+        pathForCgroup(CONTROLLER_CPU, ""))) {
       LOG.info("Removing CPU constraints for YARN containers.");
       updateCgroup(CONTROLLER_CPU, "", CPU_QUOTA_US, String.valueOf(-1));
     }
   }
 
-  boolean cpuLimitsExist() throws IOException {
-    String path = pathForCgroup(CONTROLLER_CPU, "");
-    File quotaFile = new File(path, CONTROLLER_CPU + "." + CPU_QUOTA_US);
-    if (quotaFile.exists()) {
-      String contents = FileUtils.readFileToString(quotaFile, "UTF-8");
-      int quotaUS = Integer.parseInt(contents.trim());
-      if (quotaUS != -1) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  @VisibleForTesting
   int[] getOverallLimits(float yarnProcessors) {
-
-    int[] ret = new int[2];
-
-    if (yarnProcessors < 0.01f) {
-      throw new IllegalArgumentException("Number of processors can't be <= 0.");
-    }
-
-    int quotaUS = MAX_QUOTA_US;
-    int periodUS = (int) (MAX_QUOTA_US / yarnProcessors);
-    if (yarnProcessors < 1.0f) {
-      periodUS = MAX_QUOTA_US;
-      quotaUS = (int) (periodUS * yarnProcessors);
-      if (quotaUS < MIN_PERIOD_US) {
-        LOG
-          .warn("The quota calculated for the cgroup was too low. The minimum value is "
-              + MIN_PERIOD_US + ", calculated value is " + quotaUS
-              + ". Setting quota to minimum value.");
-        quotaUS = MIN_PERIOD_US;
-      }
-    }
-
-    // cfs_period_us can't be less than 1000 microseconds
-    // if the value of periodUS is less than 1000, we can't really use cgroups
-    // to limit cpu
-    if (periodUS < MIN_PERIOD_US) {
-      LOG
-        .warn("The period calculated for the cgroup was too low. The minimum value is "
-            + MIN_PERIOD_US + ", calculated value is " + periodUS
-            + ". Using all available CPU.");
-      periodUS = MAX_QUOTA_US;
-      quotaUS = -1;
-    }
-
-    ret[0] = periodUS;
-    ret[1] = quotaUS;
-    return ret;
+    return CGroupsCpuResourceHandlerImpl.getOverallLimits(yarnProcessors);
   }
 
+
   boolean isCpuWeightEnabled() {
     return this.cpuWeightEnabled;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2085e60a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/DefaultLCEResourcesHandler.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/util/DefaultLCEResourcesHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/DefaultLCEResourcesHandler.java
index 9fb8707..df2cc52 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/DefaultLCEResourcesHandler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/DefaultLCEResourcesHandler.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.nodemanager.LinuxContainerExecutor;
 
+@Deprecated
 public class DefaultLCEResourcesHandler implements LCEResourcesHandler {
 
   final static Log LOG = LogFactory

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2085e60a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/LCEResourcesHandler.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/util/LCEResourcesHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/LCEResourcesHandler.java
index 34f7f31..b2d219d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/LCEResourcesHandler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/LCEResourcesHandler.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.nodemanager.LinuxContainerExecutor;
 
+@Deprecated
 public interface LCEResourcesHandler extends Configurable {
 
   void init(LinuxContainerExecutor lce) throws IOException;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2085e60a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsCpuResourceHandlerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsCpuResourceHandlerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsCpuResourceHandlerImpl.java
new file mode 100644
index 0000000..119235c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsCpuResourceHandlerImpl.java
@@ -0,0 +1,297 @@
+/**
+ * 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
+ * <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.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Resource;
+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.PrivilegedOperation;
+import org.apache.hadoop.yarn.util.ResourceCalculatorPlugin;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.io.File;
+import java.util.List;
+
+import static org.mockito.Mockito.*;
+import static org.mockito.Mockito.times;
+
+public class TestCGroupsCpuResourceHandlerImpl {
+
+  private CGroupsHandler mockCGroupsHandler;
+  private CGroupsCpuResourceHandlerImpl cGroupsCpuResourceHandler;
+  private ResourceCalculatorPlugin plugin;
+  final int numProcessors = 4;
+
+  @Before
+  public void setup() {
+    mockCGroupsHandler = mock(CGroupsHandler.class);
+    cGroupsCpuResourceHandler =
+        new CGroupsCpuResourceHandlerImpl(mockCGroupsHandler);
+
+    plugin = mock(ResourceCalculatorPlugin.class);
+    Mockito.doReturn(numProcessors).when(plugin).getNumProcessors();
+    Mockito.doReturn(numProcessors).when(plugin).getNumCores();
+  }
+
+  @Test
+  public void testBootstrap() throws Exception {
+    Configuration conf = new YarnConfiguration();
+
+    List<PrivilegedOperation> ret =
+        cGroupsCpuResourceHandler.bootstrap(plugin, conf);
+    verify(mockCGroupsHandler, times(1))
+        .mountCGroupController(CGroupsHandler.CGroupController.CPU);
+    verify(mockCGroupsHandler, times(0))
+        .updateCGroupParam(CGroupsHandler.CGroupController.CPU, "",
+            CGroupsHandler.CGROUP_CPU_PERIOD_US, "");
+    verify(mockCGroupsHandler, times(0))
+        .updateCGroupParam(CGroupsHandler.CGroupController.CPU, "",
+            CGroupsHandler.CGROUP_CPU_QUOTA_US, "");
+    Assert.assertNull(ret);
+  }
+
+  @Test
+  public void testBootstrapLimits() throws Exception {
+    Configuration conf = new YarnConfiguration();
+
+    int cpuPerc = 80;
+    conf.setInt(YarnConfiguration.NM_RESOURCE_PERCENTAGE_PHYSICAL_CPU_LIMIT,
+        cpuPerc);
+    int period = (CGroupsCpuResourceHandlerImpl.MAX_QUOTA_US * 100) / (cpuPerc
+        * numProcessors);
+    List<PrivilegedOperation> ret =
+        cGroupsCpuResourceHandler.bootstrap(plugin, conf);
+    verify(mockCGroupsHandler, times(1))
+        .mountCGroupController(CGroupsHandler.CGroupController.CPU);
+    verify(mockCGroupsHandler, times(1))
+        .updateCGroupParam(CGroupsHandler.CGroupController.CPU, "",
+            CGroupsHandler.CGROUP_CPU_PERIOD_US, String.valueOf(period));
+    verify(mockCGroupsHandler, times(1))
+        .updateCGroupParam(CGroupsHandler.CGroupController.CPU, "",
+            CGroupsHandler.CGROUP_CPU_QUOTA_US,
+            String.valueOf(CGroupsCpuResourceHandlerImpl.MAX_QUOTA_US));
+    Assert.assertNull(ret);
+  }
+
+  @Test
+  public void testBootstrapExistingLimits() throws Exception {
+    File existingLimit = new File(CGroupsHandler.CGroupController.CPU.getName()
+        + "." + CGroupsHandler.CGROUP_CPU_QUOTA_US);
+    try {
+      FileUtils.write(existingLimit, "10000"); // value doesn't matter
+      when(mockCGroupsHandler
+          .getPathForCGroup(CGroupsHandler.CGroupController.CPU, ""))
+          .thenReturn(".");
+      Configuration conf = new YarnConfiguration();
+
+      List<PrivilegedOperation> ret =
+          cGroupsCpuResourceHandler.bootstrap(plugin, conf);
+      verify(mockCGroupsHandler, times(1))
+          .mountCGroupController(CGroupsHandler.CGroupController.CPU);
+      verify(mockCGroupsHandler, times(1))
+          .updateCGroupParam(CGroupsHandler.CGroupController.CPU, "",
+              CGroupsHandler.CGROUP_CPU_QUOTA_US, "-1");
+      Assert.assertNull(ret);
+    } finally {
+      FileUtils.deleteQuietly(existingLimit);
+    }
+  }
+
+  @Test
+  public void testPreStart() throws Exception {
+    String id = "container_01_01";
+    String path = "test-path/" + id;
+    ContainerId mockContainerId = mock(ContainerId.class);
+    when(mockContainerId.toString()).thenReturn(id);
+    Container mockContainer = mock(Container.class);
+    when(mockContainer.getContainerId()).thenReturn(mockContainerId);
+    when(mockCGroupsHandler
+        .getPathForCGroupTasks(CGroupsHandler.CGroupController.CPU, id))
+        .thenReturn(path);
+    when(mockContainer.getResource()).thenReturn(Resource.newInstance(1024, 2));
+
+    List<PrivilegedOperation> ret =
+        cGroupsCpuResourceHandler.preStart(mockContainer);
+    verify(mockCGroupsHandler, times(1))
+        .createCGroup(CGroupsHandler.CGroupController.CPU, id);
+    verify(mockCGroupsHandler, times(1))
+        .updateCGroupParam(CGroupsHandler.CGroupController.CPU, id,
+            CGroupsHandler.CGROUP_CPU_SHARES, String
+                .valueOf(CGroupsCpuResourceHandlerImpl.CPU_DEFAULT_WEIGHT * 2));
+
+    // don't set quota or period
+    verify(mockCGroupsHandler, never())
+        .updateCGroupParam(eq(CGroupsHandler.CGroupController.CPU), eq(id),
+            eq(CGroupsHandler.CGROUP_CPU_PERIOD_US), anyString());
+    verify(mockCGroupsHandler, never())
+        .updateCGroupParam(eq(CGroupsHandler.CGroupController.CPU), eq(id),
+            eq(CGroupsHandler.CGROUP_CPU_QUOTA_US), anyString());
+    Assert.assertNotNull(ret);
+    Assert.assertEquals(1, ret.size());
+    PrivilegedOperation op = ret.get(0);
+    Assert.assertEquals(PrivilegedOperation.OperationType.ADD_PID_TO_CGROUP,
+        op.getOperationType());
+    List<String> args = op.getArguments();
+    Assert.assertEquals(1, args.size());
+    Assert.assertEquals(PrivilegedOperation.CGROUP_ARG_PREFIX + path,
+        args.get(0));
+  }
+
+  @Test
+  public void testPreStartStrictUsage() throws Exception {
+    String id = "container_01_01";
+    String path = "test-path/" + id;
+    ContainerId mockContainerId = mock(ContainerId.class);
+    when(mockContainerId.toString()).thenReturn(id);
+    Container mockContainer = mock(Container.class);
+    when(mockContainer.getContainerId()).thenReturn(mockContainerId);
+    when(mockCGroupsHandler
+        .getPathForCGroupTasks(CGroupsHandler.CGroupController.CPU, id))
+        .thenReturn(path);
+    when(mockContainer.getResource()).thenReturn(Resource.newInstance(1024, 1));
+    Configuration conf = new YarnConfiguration();
+    conf.setBoolean(
+        YarnConfiguration.NM_LINUX_CONTAINER_CGROUPS_STRICT_RESOURCE_USAGE,
+        true);
+    cGroupsCpuResourceHandler.bootstrap(plugin, conf);
+    int defaultVCores = 8;
+    float share = (float) numProcessors / (float) defaultVCores;
+    List<PrivilegedOperation> ret =
+        cGroupsCpuResourceHandler.preStart(mockContainer);
+    verify(mockCGroupsHandler, times(1))
+        .createCGroup(CGroupsHandler.CGroupController.CPU, id);
+    verify(mockCGroupsHandler, times(1))
+        .updateCGroupParam(CGroupsHandler.CGroupController.CPU, id,
+            CGroupsHandler.CGROUP_CPU_SHARES,
+            String.valueOf(CGroupsCpuResourceHandlerImpl.CPU_DEFAULT_WEIGHT));
+    // set quota and period
+    verify(mockCGroupsHandler, times(1))
+        .updateCGroupParam(CGroupsHandler.CGroupController.CPU, id,
+            CGroupsHandler.CGROUP_CPU_PERIOD_US,
+            String.valueOf(CGroupsCpuResourceHandlerImpl.MAX_QUOTA_US));
+    verify(mockCGroupsHandler, times(1))
+        .updateCGroupParam(CGroupsHandler.CGroupController.CPU, id,
+            CGroupsHandler.CGROUP_CPU_QUOTA_US, String.valueOf(
+                (int) (CGroupsCpuResourceHandlerImpl.MAX_QUOTA_US * share)));
+    Assert.assertNotNull(ret);
+    Assert.assertEquals(1, ret.size());
+    PrivilegedOperation op = ret.get(0);
+    Assert.assertEquals(PrivilegedOperation.OperationType.ADD_PID_TO_CGROUP,
+        op.getOperationType());
+    List<String> args = op.getArguments();
+    Assert.assertEquals(1, args.size());
+    Assert.assertEquals(PrivilegedOperation.CGROUP_ARG_PREFIX + path,
+        args.get(0));
+  }
+
+  @Test
+  public void testPreStartRestrictedContainers() throws Exception {
+    String id = "container_01_01";
+    String path = "test-path/" + id;
+    int defaultVCores = 8;
+    Configuration conf = new YarnConfiguration();
+    conf.setBoolean(
+        YarnConfiguration.NM_LINUX_CONTAINER_CGROUPS_STRICT_RESOURCE_USAGE,
+        true);
+    int cpuPerc = 75;
+    conf.setInt(YarnConfiguration.NM_RESOURCE_PERCENTAGE_PHYSICAL_CPU_LIMIT,
+        cpuPerc);
+    cGroupsCpuResourceHandler.bootstrap(plugin, conf);
+    verify(mockCGroupsHandler, times(1))
+        .updateCGroupParam(CGroupsHandler.CGroupController.CPU, "",
+            CGroupsHandler.CGROUP_CPU_PERIOD_US, String.valueOf("333333"));
+    verify(mockCGroupsHandler, times(1))
+        .updateCGroupParam(CGroupsHandler.CGroupController.CPU, "",
+            CGroupsHandler.CGROUP_CPU_QUOTA_US,
+            String.valueOf(CGroupsCpuResourceHandlerImpl.MAX_QUOTA_US));
+    float yarnCores = (cpuPerc * numProcessors) / 100;
+    int[] containerVCores = { 2, 4 };
+    for (int cVcores : containerVCores) {
+      ContainerId mockContainerId = mock(ContainerId.class);
+      when(mockContainerId.toString()).thenReturn(id);
+      Container mockContainer = mock(Container.class);
+      when(mockContainer.getContainerId()).thenReturn(mockContainerId);
+      when(mockCGroupsHandler
+          .getPathForCGroupTasks(CGroupsHandler.CGroupController.CPU, id))
+          .thenReturn(path);
+      when(mockContainer.getResource())
+          .thenReturn(Resource.newInstance(1024, cVcores));
+      when(mockCGroupsHandler
+          .getPathForCGroupTasks(CGroupsHandler.CGroupController.CPU, id))
+          .thenReturn(path);
+
+      float share = (cVcores * yarnCores) / defaultVCores;
+      int quotaUS;
+      int periodUS;
+      if (share > 1.0f) {
+        quotaUS = CGroupsCpuResourceHandlerImpl.MAX_QUOTA_US;
+        periodUS =
+            (int) ((float) CGroupsCpuResourceHandlerImpl.MAX_QUOTA_US / share);
+      } else {
+        quotaUS = (int) (CGroupsCpuResourceHandlerImpl.MAX_QUOTA_US * share);
+        periodUS = CGroupsCpuResourceHandlerImpl.MAX_QUOTA_US;
+      }
+      cGroupsCpuResourceHandler.preStart(mockContainer);
+      verify(mockCGroupsHandler, times(1))
+          .updateCGroupParam(CGroupsHandler.CGroupController.CPU, id,
+              CGroupsHandler.CGROUP_CPU_SHARES, String.valueOf(
+              CGroupsCpuResourceHandlerImpl.CPU_DEFAULT_WEIGHT * cVcores));
+      // set quota and period
+      verify(mockCGroupsHandler, times(1))
+          .updateCGroupParam(CGroupsHandler.CGroupController.CPU, id,
+              CGroupsHandler.CGROUP_CPU_PERIOD_US, String.valueOf(periodUS));
+      verify(mockCGroupsHandler, times(1))
+          .updateCGroupParam(CGroupsHandler.CGroupController.CPU, id,
+              CGroupsHandler.CGROUP_CPU_QUOTA_US, String.valueOf(quotaUS));
+    }
+  }
+
+  @Test
+  public void testReacquireContainer() throws Exception {
+    ContainerId containerIdMock = mock(ContainerId.class);
+    Assert.assertNull(
+        cGroupsCpuResourceHandler.reacquireContainer(containerIdMock));
+  }
+
+  @Test
+  public void testPostComplete() throws Exception {
+    String id = "container_01_01";
+    ContainerId mockContainerId = mock(ContainerId.class);
+    when(mockContainerId.toString()).thenReturn(id);
+    Assert.assertNull(cGroupsCpuResourceHandler.postComplete(mockContainerId));
+    verify(mockCGroupsHandler, times(1))
+        .deleteCGroup(CGroupsHandler.CGroupController.CPU, id);
+  }
+
+  @Test
+  public void testTeardown() throws Exception {
+    Assert.assertNull(cGroupsCpuResourceHandler.teardown());
+  }
+
+  @Test
+  public void testStrictResourceUsage() throws Exception {
+    Assert.assertNull(cGroupsCpuResourceHandler.teardown());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2085e60a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/util/TestCgroupsLCEResourcesHandler.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/util/TestCgroupsLCEResourcesHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/util/TestCgroupsLCEResourcesHandler.java
index cfab65c..34c9ad1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/util/TestCgroupsLCEResourcesHandler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/util/TestCgroupsLCEResourcesHandler.java
@@ -37,6 +37,7 @@ import java.util.List;
 import java.util.Scanner;
 import java.util.concurrent.CountDownLatch;
 
+@Deprecated
 public class TestCgroupsLCEResourcesHandler {
   static File cgroupDir = null;
 


[16/50] hadoop git commit: Release process for 2.7.2: Set the release date for 2.7.2

Posted by ar...@apache.org.
Release process for 2.7.2: Set the release date for 2.7.2


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

Branch: refs/heads/HDFS-1312
Commit: ec4d2d9f40c1cb52ca4561b3d010ffc046a73495
Parents: 992dd2f
Author: Vinod Kumar Vavilapalli (I am also known as @tshooter.) <vi...@apache.org>
Authored: Mon Jan 25 15:45:12 2016 -0800
Committer: Vinod Kumar Vavilapalli (I am also known as @tshooter.) <vi...@apache.org>
Committed: Mon Jan 25 15:45:57 2016 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt | 2 +-
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     | 2 +-
 hadoop-mapreduce-project/CHANGES.txt            | 2 +-
 hadoop-yarn-project/CHANGES.txt                 | 2 +-
 4 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec4d2d9f/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 9606296..5121a83 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -1693,7 +1693,7 @@ Release 2.7.3 - UNRELEASED
     HADOOP-12706. TestLocalFsFCStatistics#testStatisticsThreadLocalDataCleanUp
     times out occasionally (Sangjin Lee and Colin Patrick McCabe via jlowe)
 
-Release 2.7.2 - UNRELEASED
+Release 2.7.2 - 2016-01-25
 
   INCOMPATIBLE CHANGES
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec4d2d9f/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index e5285b6..f35ae3d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -2690,7 +2690,7 @@ Release 2.7.3 - UNRELEASED
     HDFS-9625. set replication for empty file failed when set storage policy
     (DENG FEI via vinayakumarb)
 
-Release 2.7.2 - UNRELEASED
+Release 2.7.2 - 2016-01-25
 
   INCOMPATIBLE CHANGES
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec4d2d9f/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index ba392c3..8f35c6f 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -722,7 +722,7 @@ Release 2.7.3 - UNRELEASED
     MAPREDUCE-6554. MRAppMaster servicestart failing with NPE in
     MRAppMaster#parsePreviousJobHistory (Bibin A Chundatt via jlowe)
 
-Release 2.7.2 - UNRELEASED
+Release 2.7.2 - 2016-01-25
 
   INCOMPATIBLE CHANGES
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec4d2d9f/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index e5049d9..41802ae 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -1406,7 +1406,7 @@ Release 2.7.3 - UNRELEASED
     YARN-4598. Invalid event: RESOURCE_FAILED at
     CONTAINER_CLEANEDUP_AFTER_KILL (tangshangwen via jlowe)
 
-Release 2.7.2 - UNRELEASED
+Release 2.7.2 - 2016-01-25
 
   INCOMPATIBLE CHANGES
 


[23/50] hadoop git commit: HADOOP-12743. Fix git environment check during test-patch (aw)

Posted by ar...@apache.org.
HADOOP-12743. Fix git environment check during test-patch (aw)


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

Branch: refs/heads/HDFS-1312
Commit: d323639686eab28f1510031e52e4390f82d78989
Parents: cf8af7b
Author: Allen Wittenauer <aw...@apache.org>
Authored: Tue Jan 26 15:46:57 2016 -0800
Committer: Allen Wittenauer <aw...@apache.org>
Committed: Tue Jan 26 15:47:07 2016 -0800

----------------------------------------------------------------------
 dev-support/bin/yetus-wrapper | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3236396/dev-support/bin/yetus-wrapper
----------------------------------------------------------------------
diff --git a/dev-support/bin/yetus-wrapper b/dev-support/bin/yetus-wrapper
index 37082d8..ac3e121 100755
--- a/dev-support/bin/yetus-wrapper
+++ b/dev-support/bin/yetus-wrapper
@@ -165,6 +165,7 @@ if [[ $? != 0 ]]; then
 fi
 
 if [[ -x "${HADOOP_PATCHPROCESS}/yetus-${HADOOP_YETUS_VERSION}/bin/${WANTED}" ]]; then
+  popd >/dev/null
   exec "${HADOOP_PATCHPROCESS}/yetus-${HADOOP_YETUS_VERSION}/bin/${WANTED}" "${ARGV[@]}"
 fi
 


[39/50] hadoop git commit: MAPREDUCE-6563. Streaming documentation contains a stray % character. Contributed by Chris Nauroth.

Posted by ar...@apache.org.
MAPREDUCE-6563. Streaming documentation contains a stray % character. Contributed by Chris Nauroth.


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

Branch: refs/heads/HDFS-1312
Commit: 41da9a07738c6d019e2467f139750b7e2f50a914
Parents: 9fab22b
Author: cnauroth <cn...@apache.org>
Authored: Thu Jan 28 14:26:52 2016 -0800
Committer: cnauroth <cn...@apache.org>
Committed: Thu Jan 28 14:46:01 2016 -0800

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                              | 3 +++
 .../hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm      | 2 +-
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/41da9a07/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index a8abdb4..08cd1d3 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -706,6 +706,9 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6610. JobHistoryEventHandler should not swallow timeline response
     (Li Lu via jianhe)
 
+    MAPREDUCE-6563. Streaming documentation contains a stray '%' character.
+    (cnauroth)
+
 Release 2.7.3 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/41da9a07/hadoop-tools/hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm b/hadoop-tools/hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm
index 2223352..cc8ed69 100644
--- a/hadoop-tools/hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm
+++ b/hadoop-tools/hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm
@@ -1,4 +1,4 @@
-%<!---
+<!---
   Licensed 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


[50/50] hadoop git commit: Merge branch 'trunk' into HDFS-1312

Posted by ar...@apache.org.
Merge branch 'trunk' into HDFS-1312


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

Branch: refs/heads/HDFS-1312
Commit: a0efd5328eb4dfe2580de764ed5becceed0b225a
Parents: e4e585a c9a09d6
Author: Arpit Agarwal <ar...@apache.org>
Authored: Fri Jan 29 11:06:05 2016 -0800
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Fri Jan 29 11:06:05 2016 -0800

----------------------------------------------------------------------
 BUILDING.txt                                    |    2 +-
 dev-support/README.md                           |   57 +
 dev-support/bin/releasedocmaker                 |   18 +
 dev-support/bin/shelldocs                       |   18 +
 dev-support/bin/smart-apply-patch               |   18 +
 dev-support/bin/test-patch                      |   18 +
 dev-support/bin/yetus-wrapper                   |  176 ++
 dev-support/releasedocmaker.py                  |  580 ----
 dev-support/shelldocs.py                        |  271 --
 dev-support/smart-apply-patch.sh                |  187 --
 dev-support/test-patch.d/checkstyle.sh          |  205 --
 dev-support/test-patch.d/shellcheck.sh          |  178 --
 dev-support/test-patch.d/whitespace.sh          |   46 -
 dev-support/test-patch.sh                       | 2814 ------------------
 .../util/TestZKSignerSecretProvider.java        |    2 +-
 hadoop-common-project/hadoop-common/CHANGES.txt |   37 +-
 hadoop-common-project/hadoop-common/pom.xml     |   12 +-
 .../fs/CommonConfigurationKeysPublic.java       |    3 +
 .../org/apache/hadoop/fs/ContentSummary.java    |  241 +-
 .../java/org/apache/hadoop/fs/FileContext.java  |    4 +-
 .../java/org/apache/hadoop/fs/FileSystem.java   |    7 +
 .../java/org/apache/hadoop/fs/QuotaUsage.java   |  359 +++
 .../apache/hadoop/fs/RawLocalFileSystem.java    |    5 +
 .../java/org/apache/hadoop/fs/shell/Count.java  |   37 +-
 .../hadoop/fs/viewfs/ChRootedFileSystem.java    |    7 +-
 .../apache/hadoop/fs/viewfs/ViewFileSystem.java |   15 +-
 .../org/apache/hadoop/ha/SshFenceByTcpPort.java |    4 +-
 .../apache/hadoop/io/AbstractMapWritable.java   |   54 +-
 .../coder/AbstractErasureDecoder.java           |    2 +-
 .../coder/AbstractHHErasureCodingStep.java      |   49 +
 .../erasurecode/coder/HHXORErasureDecoder.java  |   95 +
 .../coder/HHXORErasureDecodingStep.java         |  349 +++
 .../erasurecode/coder/HHXORErasureEncoder.java  |   92 +
 .../coder/HHXORErasureEncodingStep.java         |  146 +
 .../io/erasurecode/coder/util/HHUtil.java       |  216 ++
 .../metrics2/sink/RollingFileSystemSink.java    |  420 +++
 .../hadoop/security/ShellBasedIdMapping.java    |    9 +-
 .../hadoop/security/UserGroupInformation.java   |   22 +
 .../ZKDelegationTokenSecretManager.java         |    2 +-
 .../delegation/web/DelegationTokenManager.java  |    4 +-
 .../org/apache/hadoop/util/JvmPauseMonitor.java |    1 +
 .../apache/hadoop/util/ShutdownHookManager.java |   31 +-
 .../java/org/apache/hadoop/util/bloom/Key.java  |    4 +-
 .../src/main/resources/core-default.xml         |    8 +-
 .../src/site/markdown/FileSystemShell.md        |   11 +-
 .../org/apache/hadoop/cli/CLITestHelper.java    |    2 +-
 .../hadoop/crypto/key/TestValueQueue.java       |   65 +-
 .../apache/hadoop/fs/TestFilterFileSystem.java  |    3 +-
 .../org/apache/hadoop/fs/TestHarFileSystem.java |    1 +
 .../org/apache/hadoop/fs/TestQuotaUsage.java    |  146 +
 .../org/apache/hadoop/fs/shell/TestCount.java   |  109 +-
 .../apache/hadoop/ha/TestSshFenceByTcpPort.java |    8 +-
 .../erasurecode/coder/TestErasureCoderBase.java |    4 +-
 .../coder/TestHHErasureCoderBase.java           |   61 +
 .../coder/TestHHXORErasureCoder.java            |  120 +
 .../java/org/apache/hadoop/ipc/TestIPC.java     |   18 +-
 .../sink/RollingFileSystemSinkTestBase.java     |  506 ++++
 .../sink/TestRollingFileSystemSink.java         |  156 +
 .../security/TestUserGroupInformation.java      |   48 +-
 .../apache/hadoop/test/GenericTestUtils.java    |    2 +-
 .../hadoop/test/TestTimedOutTestsListener.java  |    2 +-
 .../src/test/resources/testConf.xml             |    2 +-
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  122 +-
 .../org/apache/hadoop/hdfs/DFSInputStream.java  |   82 +-
 .../hadoop/hdfs/DFSStripedInputStream.java      |    2 +-
 .../org/apache/hadoop/hdfs/DFSUtilClient.java   |    8 +
 .../org/apache/hadoop/hdfs/DataStreamer.java    |    3 +-
 .../hadoop/hdfs/DistributedFileSystem.java      |   28 +
 .../hadoop/hdfs/protocol/ClientProtocol.java    |   18 +-
 .../hdfs/protocol/ReconfigurationProtocol.java  |    4 +
 .../ClientNamenodeProtocolTranslatorPB.java     |   14 +
 .../hadoop/hdfs/protocolPB/PBHelperClient.java  |   86 +-
 .../hadoop/hdfs/util/StripedBlockUtil.java      |   13 +-
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      |   31 +-
 .../hdfs/web/resources/DelegationParam.java     |    5 +-
 .../src/main/proto/ClientNamenodeProtocol.proto |   11 +
 .../src/main/proto/hdfs.proto                   |   11 +
 .../src/site/markdown/index.md                  |    2 +-
 .../src/main/native/libhdfs/hdfs.c              |   98 +-
 .../src/main/native/libhdfs/include/hdfs/hdfs.h |   89 +-
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   66 +-
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |    4 +
 ...tNamenodeProtocolServerSideTranslatorPB.java |   16 +
 .../ReconfigurationProtocolServerSideUtils.java |    4 +-
 .../server/blockmanagement/BlockIdManager.java  |  102 +-
 .../server/blockmanagement/BlockManager.java    |  109 +-
 .../blockmanagement/BlockManagerSafeMode.java   |    6 +-
 .../blockmanagement/HeartbeatManager.java       |    1 +
 .../OutOfLegacyGenerationStampsException.java   |   38 +
 .../OutOfV1GenerationStampsException.java       |   38 -
 .../hdfs/server/common/HdfsServerConstants.java |    3 +-
 .../hadoop/hdfs/server/common/Storage.java      |    3 +-
 .../server/datanode/BlockPoolSliceStorage.java  |  131 +-
 .../hdfs/server/datanode/BlockReceiver.java     |   29 +-
 .../hdfs/server/datanode/DataStorage.java       |  282 +-
 .../hdfs/server/datanode/DirectoryScanner.java  |   72 +-
 .../hdfs/server/datanode/StorageLocation.java   |   15 +
 .../erasurecode/ErasureCodingWorker.java        |  146 +-
 .../hdfs/server/namenode/FSDirAppendOp.java     |   15 +-
 .../server/namenode/FSDirStatAndListingOp.java  |   53 +
 .../hdfs/server/namenode/FSDirTruncateOp.java   |    2 +-
 .../hdfs/server/namenode/FSDirWriteFileOp.java  |   15 +-
 .../hdfs/server/namenode/FSDirectory.java       |    4 +
 .../hadoop/hdfs/server/namenode/FSEditLog.java  |    4 +-
 .../hdfs/server/namenode/FSEditLogLoader.java   |   19 +-
 .../hdfs/server/namenode/FSImageFormat.java     |   27 +-
 .../server/namenode/FSImageFormatProtobuf.java  |   16 +-
 .../hdfs/server/namenode/FSNamesystem.java      |  168 +-
 .../hadoop/hdfs/server/namenode/INodeFile.java  |   56 +-
 .../hdfs/server/namenode/LeaseManager.java      |   17 +-
 .../hadoop/hdfs/server/namenode/NameNode.java   |   12 +-
 .../hdfs/server/namenode/NameNodeRpcServer.java |   44 +-
 .../hadoop/hdfs/server/namenode/Namesystem.java |   11 -
 .../server/protocol/BlockECRecoveryCommand.java |    2 +-
 .../hdfs/server/protocol/NamenodeProtocols.java |    2 +
 .../org/apache/hadoop/hdfs/tools/DFSAdmin.java  |  256 +-
 .../hadoop-hdfs/src/main/proto/fsimage.proto    |    4 +-
 .../fs/viewfs/TestViewFsDefaultValue.java       |   53 +-
 .../hadoop/hdfs/TestDFSClientRetries.java       |   36 +-
 .../org/apache/hadoop/hdfs/TestFileAppend.java  |   56 +-
 .../apache/hadoop/hdfs/TestLeaseRecovery2.java  |   48 +-
 .../java/org/apache/hadoop/hdfs/TestQuota.java  |   66 +-
 .../hdfs/TestReadStripedFileWithDecoding.java   |   17 +-
 .../hadoop/hdfs/TestRecoverStripedFile.java     |  172 +-
 .../org/apache/hadoop/hdfs/TestReplication.java |    3 +-
 .../hadoop/hdfs/TestSetrepIncreasing.java       |   19 +
 .../apache/hadoop/hdfs/UpgradeUtilities.java    |    2 +-
 .../blockmanagement/TestBlockManager.java       |    2 +-
 .../TestBlockManagerSafeMode.java               |    6 +-
 .../TestPendingInvalidateBlock.java             |   19 +
 .../blockmanagement/TestReplicationPolicy.java  |    6 +-
 .../TestSequentialBlockGroupId.java             |   12 +-
 .../blockmanagement/TestSequentialBlockId.java  |   22 +-
 .../server/datanode/SimulatedFSDataset.java     |    2 +-
 .../datanode/TestDataNodeHotSwapVolumes.java    |   48 +-
 .../hdfs/server/datanode/TestDataStorage.java   |    7 +-
 .../server/datanode/TestDirectoryScanner.java   |   50 +
 .../fsdataset/impl/TestFsDatasetImpl.java       |    2 +-
 .../server/namenode/NNThroughputBenchmark.java  |   34 +-
 .../hdfs/server/namenode/TestEditLog.java       |    6 +-
 .../server/namenode/TestFSEditLogLoader.java    |   23 +-
 .../hdfs/server/namenode/TestFSImage.java       |    4 +-
 .../hdfs/server/namenode/TestFileTruncate.java  |    4 +-
 .../hdfs/server/namenode/TestINodeFile.java     |    6 +-
 .../hdfs/server/namenode/TestLeaseManager.java  |    4 +-
 .../hdfs/server/namenode/TestSaveNamespace.java |    6 +-
 .../server/namenode/ha/TestQuotasWithHA.java    |   14 +-
 .../apache/hadoop/hdfs/tools/TestDFSAdmin.java  |  162 +-
 .../hadoop/hdfs/web/TestWebHdfsTimeouts.java    |   14 +-
 .../apache/hadoop/hdfs/web/TestWebHdfsUrl.java  |    5 +-
 hadoop-mapreduce-project/CHANGES.txt            |   23 +-
 .../dev-support/findbugs-exclude.xml            |    2 +
 .../jobhistory/JobHistoryEventHandler.java      |   18 +-
 .../v2/jobhistory/FileNameIndexUtils.java       |  171 +-
 .../org/apache/hadoop/mapred/TestJobClient.java |   10 +
 .../v2/jobhistory/TestFileNameIndexUtils.java   |  199 +-
 .../org/apache/hadoop/mapred/JobClient.java     |    3 +-
 .../src/main/resources/mapred-default.xml       |   24 +
 hadoop-project/pom.xml                          |    5 -
 .../gridmix/DistributedCacheEmulator.java       |    4 +-
 .../hadoop/mapred/gridmix/CommonJobTest.java    |    2 +-
 .../mapred/gridmix/TestPseudoLocalFs.java       |    2 +-
 .../apache/hadoop/tools/rumen/JobBuilder.java   |   11 +-
 .../src/main/data/2jobs2min-rumen-jh.json       |  606 ++++
 .../org/apache/hadoop/yarn/sls/SLSRunner.java   |    6 +
 .../apache/hadoop/yarn/sls/utils/SLSUtils.java  |   17 +-
 .../yarn/sls/nodemanager/TestNMSimulator.java   |    2 +-
 .../hadoop/yarn/sls/utils/TestSLSUtils.java     |    5 +
 .../src/site/markdown/HadoopStreaming.md.vm     |    6 +-
 .../apache/hadoop/streaming/DelayEchoApp.java   |    2 +-
 hadoop-yarn-project/CHANGES.txt                 |   92 +-
 .../dev-support/findbugs-exclude.xml            |    5 +
 .../hadoop/yarn/conf/YarnConfiguration.java     |   19 +
 .../src/main/proto/yarn_service_protos.proto    |    7 -
 .../yarn/conf/TestYarnConfigurationFields.java  |    2 +
 .../hadoop/yarn/client/cli/ApplicationCLI.java  |   61 +-
 ...stHedgingRequestRMFailoverProxyProvider.java |   98 +
 .../api/impl/TestAMRMClientOnRMRestart.java     |    1 +
 .../hadoop/yarn/client/cli/TestYarnCLI.java     |  143 +-
 .../ConfiguredRMFailoverProxyProvider.java      |    6 +-
 .../org/apache/hadoop/yarn/client/RMProxy.java  |   33 +-
 .../RequestHedgingRMFailoverProxyProvider.java  |  194 ++
 .../src/main/resources/yarn-default.xml         |    8 +
 .../timeline/KeyValueBasedTimelineStore.java    |  574 ++++
 .../server/timeline/MemoryTimelineStore.java    |  491 +--
 .../timeline/TimelineStoreMapAdapter.java       |   60 +
 .../yarn/server/timeline/util/LeveldbUtils.java |    7 +
 .../pb/RegisterNodeManagerRequestPBImpl.java    |   75 +-
 .../nodemanager/LinuxContainerExecutor.java     |   32 +-
 .../containermanager/ContainerManagerImpl.java  |    6 +
 .../container/ContainerImpl.java                |    1 +
 .../launcher/ContainerLaunch.java               |   11 +
 .../launcher/RecoveredContainerLaunch.java      |    7 +-
 .../CGroupsCpuResourceHandlerImpl.java          |  235 ++
 .../linux/resources/CGroupsHandler.java         |    4 +
 .../linux/resources/CpuResourceHandler.java     |   32 +
 .../linux/resources/ResourceHandlerModule.java  |   34 +
 .../runtime/DockerLinuxContainerRuntime.java    |   10 +-
 .../runtime/LinuxContainerRuntimeConstants.java |    4 +
 .../executor/ContainerStartContext.java         |   24 +
 .../util/CgroupsLCEResourcesHandler.java        |   68 +-
 .../util/DefaultLCEResourcesHandler.java        |    1 +
 .../nodemanager/util/LCEResourcesHandler.java   |    1 +
 .../nodemanager/TestLinuxContainerExecutor.java |    9 +
 .../nodemanager/TestNodeStatusUpdater.java      |   16 +-
 .../TestCGroupsCpuResourceHandlerImpl.java      |  297 ++
 .../runtime/TestDockerContainerRuntime.java     |   12 +-
 .../util/TestCgroupsLCEResourcesHandler.java    |    1 +
 .../server/resourcemanager/ClientRMService.java |    6 +-
 .../resourcemanager/LeaderElectorService.java   |   23 +-
 .../server/resourcemanager/RMServerUtils.java   |   76 +-
 .../server/resourcemanager/ResourceManager.java |   98 +-
 .../resourcemanager/recovery/RMStateStore.java  |    2 +-
 .../recovery/ZKRMStateStore.java                |  111 +-
 .../server/resourcemanager/rmapp/RMAppImpl.java |   49 +-
 .../rmapp/attempt/RMAppAttemptImpl.java         |   18 +-
 .../scheduler/AbstractYarnScheduler.java        |   64 +-
 .../scheduler/AppSchedulingInfo.java            |   32 +-
 .../scheduler/SchedContainerChangeRequest.java  |   33 +-
 .../scheduler/capacity/CSQueue.java             |    3 +-
 .../scheduler/capacity/CapacityScheduler.java   |  119 +-
 .../scheduler/capacity/LeafQueue.java           |   89 +-
 .../scheduler/capacity/ParentQueue.java         |    4 +-
 .../capacity/PartitionedQueueComparator.java    |   10 +-
 .../scheduler/fair/AllocationConfiguration.java |   11 +-
 .../fair/AllocationFileLoaderService.java       |   16 +-
 .../scheduler/fair/FSParentQueue.java           |    8 +
 .../resourcemanager/scheduler/fair/FSQueue.java |   11 +-
 .../scheduler/fair/FairScheduler.java           |    3 +-
 .../webapp/FairSchedulerPage.java               |    1 +
 .../webapp/dao/FairSchedulerQueueInfo.java      |    7 +
 .../resourcemanager/TestClientRMService.java    |    7 +-
 .../server/resourcemanager/TestRMRestart.java   |  120 +
 .../recovery/TestZKRMStateStore.java            |    1 +
 .../recovery/TestZKRMStateStorePerf.java        |    2 +
 .../TestZKRMStateStoreZKClientConnections.java  |   22 +-
 .../rmapp/TestRMAppTransitions.java             |    9 +-
 .../attempt/TestRMAppAttemptTransitions.java    |   13 +
 .../capacity/TestApplicationPriority.java       |    1 -
 .../capacity/TestContainerResizing.java         |   87 +
 .../TestNodeLabelContainerAllocation.java       |   77 +-
 .../scheduler/capacity/TestReservations.java    |  147 +-
 .../scheduler/fair/TestFairScheduler.java       |  327 ++
 .../hadoop/yarn/server/MiniYARNCluster.java     |    1 +
 .../pom.xml                                     |    4 +
 .../yarn/server/timeline/EntityCacheItem.java   |    3 +-
 .../timeline/LevelDBCacheTimelineStore.java     |  316 ++
 .../server/timeline/PluginStoreTestUtils.java   |    2 +-
 .../timeline/TestLevelDBCacheTimelineStore.java |   94 +
 .../src/site/markdown/CapacityScheduler.md      |   71 +-
 .../site/markdown/WritingYarnApplications.md    |    8 +-
 251 files changed, 10732 insertions(+), 6611 deletions(-)
----------------------------------------------------------------------



[40/50] hadoop git commit: YARN-4519. Potential deadlock of CapacityScheduler between decrease container and assign containers. Contributed by Meng Ding

Posted by ar...@apache.org.
YARN-4519. Potential deadlock of CapacityScheduler between decrease container and assign containers. Contributed by Meng Ding


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

Branch: refs/heads/HDFS-1312
Commit: 7f46636495e23693d588b0915f464fa7afd9102e
Parents: 41da9a0
Author: Jian He <ji...@apache.org>
Authored: Wed Jan 27 15:38:32 2016 -0800
Committer: Jian He <ji...@apache.org>
Committed: Thu Jan 28 14:51:00 2016 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../server/resourcemanager/RMServerUtils.java   |  76 ++++++------
 .../scheduler/AbstractYarnScheduler.java        |  64 ++++------
 .../scheduler/AppSchedulingInfo.java            |  30 +++--
 .../scheduler/SchedContainerChangeRequest.java  |  33 +++--
 .../scheduler/capacity/CSQueue.java             |   3 +-
 .../scheduler/capacity/CapacityScheduler.java   | 119 ++++++++++---------
 .../scheduler/capacity/LeafQueue.java           |  83 ++++++++++---
 .../scheduler/capacity/ParentQueue.java         |   4 +-
 .../capacity/TestContainerResizing.java         |  87 ++++++++++++++
 10 files changed, 322 insertions(+), 180 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f466364/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 8eaed42..ee57e4b 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -170,6 +170,9 @@ Release 2.9.0 - UNRELEASED
     YARN-4633. Fix random test failure in TestRMRestart#testRMRestartAfterPreemption
     (Bibin A Chundatt via rohithsharmaks)
 
+    YARN-4519. Potential deadlock of CapacityScheduler between decrease container
+    and assign containers. (Meng Ding via jianhe)
+
 Release 2.8.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f466364/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.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/RMServerUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java
index cc30593..e19d55e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java
@@ -53,9 +53,10 @@ import org.apache.hadoop.yarn.security.YarnAuthorizationProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
-import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler
+    .SchedContainerChangeRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
@@ -114,43 +115,25 @@ public class RMServerUtils {
           queueName, scheduler, rmContext, queueInfo);
     }
   }
-  
+
   /**
-   * Normalize container increase/decrease request, it will normalize and update
-   * ContainerResourceChangeRequest.targetResource
+   * Validate increase/decrease request. This function must be called under
+   * the queue lock to make sure that the access to container resource is
+   * atomic. Refer to LeafQueue.decreaseContainer() and
+   * CapacityScheduelr.updateIncreaseRequests()
+   *
    * 
    * <pre>
    * - Throw exception when any other error happens
    * </pre>
    */
-  public static void checkAndNormalizeContainerChangeRequest(
-      RMContext rmContext, ContainerResourceChangeRequest request,
-      boolean increase) throws InvalidResourceRequestException {
+  public static void checkSchedContainerChangeRequest(
+      SchedContainerChangeRequest request, boolean increase)
+      throws InvalidResourceRequestException {
+    RMContext rmContext = request.getRmContext();
     ContainerId containerId = request.getContainerId();
-    ResourceScheduler scheduler = rmContext.getScheduler();
-    RMContainer rmContainer = scheduler.getRMContainer(containerId);
-    ResourceCalculator rc = scheduler.getResourceCalculator();
-    
-    if (null == rmContainer) {
-      String msg =
-          "Failed to get rmContainer for "
-              + (increase ? "increase" : "decrease")
-              + " request, with container-id=" + containerId;
-      throw new InvalidResourceRequestException(msg);
-    }
-
-    if (rmContainer.getState() != RMContainerState.RUNNING) {
-      String msg =
-          "rmContainer's state is not RUNNING, for "
-              + (increase ? "increase" : "decrease")
-              + " request, with container-id=" + containerId;
-      throw new InvalidResourceRequestException(msg);
-    }
-
-    Resource targetResource = Resources.normalize(rc, request.getCapability(),
-        scheduler.getMinimumResourceCapability(),
-        scheduler.getMaximumResourceCapability(),
-        scheduler.getMinimumResourceCapability());
+    RMContainer rmContainer = request.getRMContainer();
+    Resource targetResource = request.getTargetCapacity();
 
     // Compare targetResource and original resource
     Resource originalResource = rmContainer.getAllocatedResource();
@@ -181,10 +164,10 @@ public class RMServerUtils {
         throw new InvalidResourceRequestException(msg);
       }
     }
-    
-    RMNode rmNode = rmContext.getRMNodes().get(rmContainer.getAllocatedNode());
-    
+
     // Target resource of the increase request is more than NM can offer
+    ResourceScheduler scheduler = rmContext.getScheduler();
+    RMNode rmNode = request.getSchedulerNode().getRMNode();
     if (!Resources.fitsIn(scheduler.getResourceCalculator(),
         scheduler.getClusterResource(), targetResource,
         rmNode.getTotalCapability())) {
@@ -193,9 +176,6 @@ public class RMServerUtils {
           + rmNode.getTotalCapability();
       throw new InvalidResourceRequestException(msg);
     }
-
-    // Update normalized target resource
-    request.setCapability(targetResource);
   }
 
   /*
@@ -253,7 +233,8 @@ public class RMServerUtils {
       }
     }
   }
-  
+
+  // Sanity check and normalize target resource
   private static void validateIncreaseDecreaseRequest(RMContext rmContext,
       List<ContainerResourceChangeRequest> requests, Resource maximumAllocation,
       boolean increase)
@@ -283,8 +264,23 @@ public class RMServerUtils {
             + request.getCapability().getVirtualCores() + ", maxVirtualCores="
             + maximumAllocation.getVirtualCores());
       }
-      
-      checkAndNormalizeContainerChangeRequest(rmContext, request, increase);
+      ContainerId containerId = request.getContainerId();
+      ResourceScheduler scheduler = rmContext.getScheduler();
+      RMContainer rmContainer = scheduler.getRMContainer(containerId);
+      if (null == rmContainer) {
+        String msg =
+            "Failed to get rmContainer for "
+                + (increase ? "increase" : "decrease")
+                + " request, with container-id=" + containerId;
+        throw new InvalidResourceRequestException(msg);
+      }
+      ResourceCalculator rc = scheduler.getResourceCalculator();
+      Resource targetResource = Resources.normalize(rc, request.getCapability(),
+          scheduler.getMinimumResourceCapability(),
+          scheduler.getMaximumResourceCapability(),
+          scheduler.getMinimumResourceCapability());
+      // Update normalized target resource
+      request.setCapability(targetResource);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f466364/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 41a04f2..27d4f91 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
@@ -22,7 +22,6 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
@@ -55,13 +54,13 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceOption;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
-import org.apache.hadoop.yarn.server.resourcemanager.RMServerUtils;
 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.RMAppEvent;
@@ -74,6 +73,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEven
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerFinishedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerRecoverEvent;
+
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeDecreaseContainerEvent;
@@ -618,28 +618,20 @@ public abstract class AbstractYarnScheduler
           SchedulerUtils.RELEASED_CONTAINER), RMContainerEventType.RELEASED);
     }
   }
-  
+
   protected void decreaseContainers(
-      List<SchedContainerChangeRequest> decreaseRequests,
+      List<ContainerResourceChangeRequest> decreaseRequests,
       SchedulerApplicationAttempt attempt) {
-    for (SchedContainerChangeRequest request : decreaseRequests) {
+    if (null == decreaseRequests || decreaseRequests.isEmpty()) {
+      return;
+    }
+    // Pre-process decrease requests
+    List<SchedContainerChangeRequest> schedDecreaseRequests =
+        createSchedContainerChangeRequests(decreaseRequests, false);
+    for (SchedContainerChangeRequest request : schedDecreaseRequests) {
       if (LOG.isDebugEnabled()) {
         LOG.debug("Processing decrease request:" + request);
       }
-      
-      boolean hasIncreaseRequest =
-          attempt.removeIncreaseRequest(request.getNodeId(),
-              request.getPriority(), request.getContainerId());
-      
-      if (hasIncreaseRequest) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("While processing decrease request, found a increase request "
-              + "for the same container "
-              + request.getContainerId()
-              + ", removed the increase request");
-        }
-      }
-      
       // handle decrease request
       decreaseContainer(request, attempt);
     }
@@ -877,7 +869,7 @@ public abstract class AbstractYarnScheduler
   }
   
   /**
-   * Normalize container increase/decrease request, and return
+   * Sanity check increase/decrease request, and return
    * SchedulerContainerResourceChangeRequest according to given
    * ContainerResourceChangeRequest.
    * 
@@ -886,37 +878,34 @@ public abstract class AbstractYarnScheduler
    * - Throw exception when any other error happens
    * </pre>
    */
-  private SchedContainerChangeRequest
-      checkAndNormalizeContainerChangeRequest(
-          ContainerResourceChangeRequest request, boolean increase)
-          throws YarnException {
-    // We have done a check in ApplicationMasterService, but RMContainer status
-    // / Node resource could change since AMS won't acquire lock of scheduler.
-    RMServerUtils.checkAndNormalizeContainerChangeRequest(rmContext, request,
-        increase);
+  private SchedContainerChangeRequest createSchedContainerChangeRequest(
+      ContainerResourceChangeRequest request, boolean increase)
+      throws YarnException {
     ContainerId containerId = request.getContainerId();
     RMContainer rmContainer = getRMContainer(containerId);
+    if (null == rmContainer) {
+      String msg =
+          "Failed to get rmContainer for "
+              + (increase ? "increase" : "decrease")
+              + " request, with container-id=" + containerId;
+      throw new InvalidResourceRequestException(msg);
+    }
     SchedulerNode schedulerNode =
         getSchedulerNode(rmContainer.getAllocatedNode());
-    
-    return new SchedContainerChangeRequest(schedulerNode, rmContainer,
-        request.getCapability());
+    return new SchedContainerChangeRequest(
+        this.rmContext, schedulerNode, rmContainer, request.getCapability());
   }
 
   protected List<SchedContainerChangeRequest>
-      checkAndNormalizeContainerChangeRequests(
+      createSchedContainerChangeRequests(
           List<ContainerResourceChangeRequest> changeRequests,
           boolean increase) {
-    if (null == changeRequests || changeRequests.isEmpty()) {
-      return Collections.EMPTY_LIST;
-    }
-    
     List<SchedContainerChangeRequest> schedulerChangeRequests =
         new ArrayList<SchedContainerChangeRequest>();
     for (ContainerResourceChangeRequest r : changeRequests) {
       SchedContainerChangeRequest sr = null;
       try {
-        sr = checkAndNormalizeContainerChangeRequest(r, increase);
+        sr = createSchedContainerChangeRequest(r, increase);
       } catch (YarnException e) {
         LOG.warn("Error happens when checking increase request, Ignoring.."
             + " exception=", e);
@@ -924,7 +913,6 @@ public abstract class AbstractYarnScheduler
       }
       schedulerChangeRequests.add(sr);
     }
-
     return schedulerChangeRequests;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f466364/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 07f3d8b..a61001e 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
@@ -44,6 +44,8 @@ import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.resourcemanager.RMServerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
 import org.apache.hadoop.yarn.util.resource.Resources;
@@ -148,6 +150,18 @@ public class AppSchedulingInfo {
     boolean resourceUpdated = false;
 
     for (SchedContainerChangeRequest r : increaseRequests) {
+      if (r.getRMContainer().getState() != RMContainerState.RUNNING) {
+        LOG.warn("rmContainer's state is not RUNNING, for increase request with"
+            + " container-id=" + r.getContainerId());
+        continue;
+      }
+      try {
+        RMServerUtils.checkSchedContainerChangeRequest(r, true);
+      } catch (YarnException e) {
+        LOG.warn("Error happens when checking increase request, Ignoring.."
+            + " exception=", e);
+        continue;
+      }
       NodeId nodeId = r.getRMContainer().getAllocatedNode();
 
       Map<Priority, Map<ContainerId, SchedContainerChangeRequest>> requestsOnNode =
@@ -221,7 +235,7 @@ public class AppSchedulingInfo {
     
     if (LOG.isDebugEnabled()) {
       LOG.debug("Added increase request:" + request.getContainerId()
-          + " delta=" + request.getDeltaCapacity());
+          + " delta=" + delta);
     }
     
     // update priorities
@@ -520,24 +534,20 @@ public class AppSchedulingInfo {
     NodeId nodeId = increaseRequest.getNodeId();
     Priority priority = increaseRequest.getPriority();
     ContainerId containerId = increaseRequest.getContainerId();
-    
+    Resource deltaCapacity = increaseRequest.getDeltaCapacity();
+
     if (LOG.isDebugEnabled()) {
       LOG.debug("allocated increase request : applicationId=" + applicationId
           + " container=" + containerId + " host="
           + increaseRequest.getNodeId() + " user=" + user + " resource="
-          + increaseRequest.getDeltaCapacity());
+          + deltaCapacity);
     }
-    
     // Set queue metrics
-    queue.getMetrics().allocateResources(user,
-        increaseRequest.getDeltaCapacity());
-    
+    queue.getMetrics().allocateResources(user, deltaCapacity);
     // remove the increase request from pending increase request map
     removeIncreaseRequest(nodeId, priority, containerId);
-    
     // update usage
-    appResourceUsage.incUsed(increaseRequest.getNodePartition(),
-        increaseRequest.getDeltaCapacity());
+    appResourceUsage.incUsed(increaseRequest.getNodePartition(), deltaCapacity);
   }
   
   public synchronized void decreaseContainer(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f466364/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedContainerChangeRequest.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/SchedContainerChangeRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedContainerChangeRequest.java
index ea109fd..e4ab3a2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedContainerChangeRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedContainerChangeRequest.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
@@ -32,18 +33,19 @@ import org.apache.hadoop.yarn.util.resource.Resources;
  */
 public class SchedContainerChangeRequest implements
     Comparable<SchedContainerChangeRequest> {
-  RMContainer rmContainer;
-  Resource targetCapacity;
-  SchedulerNode schedulerNode;
-  Resource deltaCapacity;
+  private RMContext rmContext;
+  private RMContainer rmContainer;
+  private Resource targetCapacity;
+  private SchedulerNode schedulerNode;
+  private Resource deltaCapacity;
 
-  public SchedContainerChangeRequest(SchedulerNode schedulerNode,
+  public SchedContainerChangeRequest(
+      RMContext rmContext, SchedulerNode schedulerNode,
       RMContainer rmContainer, Resource targetCapacity) {
+    this.rmContext = rmContext;
     this.rmContainer = rmContainer;
     this.targetCapacity = targetCapacity;
     this.schedulerNode = schedulerNode;
-    deltaCapacity = Resources.subtract(targetCapacity,
-        rmContainer.getAllocatedResource());
   }
   
   public NodeId getNodeId() {
@@ -58,11 +60,19 @@ public class SchedContainerChangeRequest implements
     return this.targetCapacity;
   }
 
+  public RMContext getRmContext() {
+    return this.rmContext;
+  }
   /**
-   * Delta capacity = before - target, so if it is a decrease request, delta
+   * Delta capacity = target - before, so if it is a decrease request, delta
    * capacity will be negative
    */
-  public Resource getDeltaCapacity() {
+  public synchronized Resource getDeltaCapacity() {
+    // Only calculate deltaCapacity once
+    if (deltaCapacity == null) {
+      deltaCapacity = Resources.subtract(
+          targetCapacity, rmContainer.getAllocatedResource());
+    }
     return deltaCapacity;
   }
   
@@ -81,7 +91,7 @@ public class SchedContainerChangeRequest implements
   public SchedulerNode getSchedulerNode() {
     return schedulerNode;
   }
-  
+
   @Override
   public int hashCode() {
     return (getContainerId().hashCode() << 16) + targetCapacity.hashCode();
@@ -112,7 +122,6 @@ public class SchedContainerChangeRequest implements
   @Override
   public String toString() {
     return "<container=" + getContainerId() + ", targetCapacity="
-        + targetCapacity + ", delta=" + deltaCapacity + ", node="
-        + getNodeId().toString() + ">";
+        + targetCapacity + ", node=" + getNodeId().toString() + ">";
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f466364/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.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/CSQueue.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/CSQueue.java
index 6ffba02..daf7790 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/CSQueue.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/CSQueue.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueState;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
 import org.apache.hadoop.yarn.security.PrivilegedEntity;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
@@ -332,7 +333,7 @@ extends org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue {
    */
   public void decreaseContainer(Resource clusterResource,
       SchedContainerChangeRequest decreaseRequest,
-      FiCaSchedulerApp app);
+      FiCaSchedulerApp app) throws InvalidResourceRequestException;
 
   /**
    * Get valid Node Labels for this queue

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f466364/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 e773384..dcb60fc 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
@@ -21,8 +21,8 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 import java.io.IOException;
 import java.io.InputStream;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.Comparator;
 import java.util.EnumSet;
 import java.util.HashMap;
@@ -65,6 +65,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceOption;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
@@ -895,9 +896,36 @@ public class CapacityScheduler extends
     }
   }
 
+  // It is crucial to acquire leaf queue lock first to prevent:
+  // 1. Race condition when calculating the delta resource in
+  //    SchedContainerChangeRequest
+  // 2. Deadlock with the scheduling thread.
+  private LeafQueue updateIncreaseRequests(
+      List<ContainerResourceChangeRequest> increaseRequests,
+      FiCaSchedulerApp app) {
+    if (null == increaseRequests || increaseRequests.isEmpty()) {
+      return null;
+    }
+    // Pre-process increase requests
+    List<SchedContainerChangeRequest> schedIncreaseRequests =
+        createSchedContainerChangeRequests(increaseRequests, true);
+    LeafQueue leafQueue = (LeafQueue) app.getQueue();
+    synchronized(leafQueue) {
+      // make sure we aren't stopping/removing the application
+      // when the allocate comes in
+      if (app.isStopped()) {
+        return null;
+      }
+      // Process increase resource requests
+      if (app.updateIncreaseRequests(schedIncreaseRequests)) {
+        return leafQueue;
+      }
+      return null;
+    }
+  }
+
   @Override
-  // Note: when AM asks to decrease container or release container, we will
-  // acquire scheduler lock
+  // Note: when AM asks to release container, we will acquire scheduler lock
   @Lock(Lock.NoLock.class)
   public Allocation allocate(ApplicationAttemptId applicationAttemptId,
       List<ResourceRequest> ask, List<ContainerId> release,
@@ -909,26 +937,23 @@ public class CapacityScheduler extends
     if (application == null) {
       return EMPTY_ALLOCATION;
     }
-    
-    // Sanity check
-    SchedulerUtils.normalizeRequests(
-        ask, getResourceCalculator(), getClusterResource(),
-        getMinimumResourceCapability(), getMaximumResourceCapability());
-    
-    // Pre-process increase requests
-    List<SchedContainerChangeRequest> normalizedIncreaseRequests =
-        checkAndNormalizeContainerChangeRequests(increaseRequests, true);
-    
-    // Pre-process decrease requests
-    List<SchedContainerChangeRequest> normalizedDecreaseRequests =
-        checkAndNormalizeContainerChangeRequests(decreaseRequests, false);
 
     // Release containers
     releaseContainers(release, application);
 
-    Allocation allocation;
+    // update increase requests
+    LeafQueue updateDemandForQueue =
+        updateIncreaseRequests(increaseRequests, application);
+
+    // Decrease containers
+    decreaseContainers(decreaseRequests, application);
 
-    LeafQueue updateDemandForQueue = null;
+    // Sanity check for new allocation requests
+    SchedulerUtils.normalizeRequests(
+        ask, getResourceCalculator(), getClusterResource(),
+        getMinimumResourceCapability(), getMaximumResourceCapability());
+
+    Allocation allocation;
 
     synchronized (application) {
 
@@ -947,7 +972,8 @@ public class CapacityScheduler extends
         }
 
         // Update application requests
-        if (application.updateResourceRequests(ask)) {
+        if (application.updateResourceRequests(ask)
+            && (updateDemandForQueue == null)) {
           updateDemandForQueue = (LeafQueue) application.getQueue();
         }
 
@@ -957,12 +983,6 @@ public class CapacityScheduler extends
         }
       }
       
-      // Process increase resource requests
-      if (application.updateIncreaseRequests(normalizedIncreaseRequests)
-          && (updateDemandForQueue == null)) {
-        updateDemandForQueue = (LeafQueue) application.getQueue();
-      }
-
       if (application.isWaitingForAMContainer()) {
         // Allocate is for AM and update AM blacklist for this
         application.updateAMBlacklist(
@@ -971,8 +991,6 @@ public class CapacityScheduler extends
         application.updateBlacklist(blacklistAdditions, blacklistRemovals);
       }
       
-      // Decrease containers
-      decreaseContainers(normalizedDecreaseRequests, application);
 
       allocation = application.getAllocation(getResourceCalculator(),
                    clusterResource, getMinimumResourceCapability());
@@ -1167,7 +1185,8 @@ public class CapacityScheduler extends
       .getAssignmentInformation().getReserved());
  }
 
-  private synchronized void allocateContainersToNode(FiCaSchedulerNode node) {
+  @VisibleForTesting
+  protected synchronized void allocateContainersToNode(FiCaSchedulerNode node) {
     if (rmContext.isWorkPreservingRecoveryEnabled()
         && !rmContext.isSchedulerReadyForAllocatingContainers()) {
       return;
@@ -1517,48 +1536,30 @@ public class CapacityScheduler extends
     }
   }
   
-  @Lock(CapacityScheduler.class)
   @Override
-  protected synchronized void decreaseContainer(
-      SchedContainerChangeRequest decreaseRequest,
+  protected void decreaseContainer(SchedContainerChangeRequest decreaseRequest,
       SchedulerApplicationAttempt attempt) {
     RMContainer rmContainer = decreaseRequest.getRMContainer();
-
     // Check container status before doing decrease
     if (rmContainer.getState() != RMContainerState.RUNNING) {
       LOG.info("Trying to decrease a container not in RUNNING state, container="
           + rmContainer + " state=" + rmContainer.getState().name());
       return;
     }
-    
-    // Delta capacity of this decrease request is 0, this decrease request may
-    // just to cancel increase request
-    if (Resources.equals(decreaseRequest.getDeltaCapacity(), Resources.none())) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Decrease target resource equals to existing resource for container:"
-            + decreaseRequest.getContainerId()
-            + " ignore this decrease request.");
-      }
-      return;
-    }
-
-    // Save resource before decrease
-    Resource resourceBeforeDecrease =
-        Resources.clone(rmContainer.getContainer().getResource());
-
     FiCaSchedulerApp app = (FiCaSchedulerApp)attempt;
     LeafQueue queue = (LeafQueue) attempt.getQueue();
-    queue.decreaseContainer(clusterResource, decreaseRequest, app);
-    
-    // Notify RMNode the container will be decreased
-    this.rmContext.getDispatcher().getEventHandler()
-        .handle(new RMNodeDecreaseContainerEvent(decreaseRequest.getNodeId(),
-            Arrays.asList(rmContainer.getContainer())));
-    
-    LOG.info("Application attempt " + app.getApplicationAttemptId()
-        + " decreased container:" + decreaseRequest.getContainerId() + " from "
-        + resourceBeforeDecrease + " to "
-        + decreaseRequest.getTargetCapacity());
+    try {
+      queue.decreaseContainer(clusterResource, decreaseRequest, app);
+      // Notify RMNode that the container can be pulled by NodeManager in the
+      // next heartbeat
+      this.rmContext.getDispatcher().getEventHandler()
+          .handle(new RMNodeDecreaseContainerEvent(
+              decreaseRequest.getNodeId(),
+              Collections.singletonList(rmContainer.getContainer())));
+    } catch (InvalidResourceRequestException e) {
+      LOG.warn("Error happens when checking decrease request, Ignoring.."
+          + " exception=", e);
+    }
   }
 
   @Lock(Lock.NoLock.class)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f466364/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 9e64b42..56e4502 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
@@ -47,10 +47,12 @@ 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.InvalidResourceRequestException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 import org.apache.hadoop.yarn.security.AccessType;
+import org.apache.hadoop.yarn.server.resourcemanager.RMServerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
@@ -1676,11 +1678,17 @@ public class LeafQueue extends AbstractCSQueue {
   public Priority getDefaultApplicationPriority() {
     return defaultAppPriorityPerQueue;
   }
-  
+
+  /**
+   *
+   * @param clusterResource Total cluster resource
+   * @param decreaseRequest The decrease request
+   * @param app The application of interest
+   */
   @Override
   public void decreaseContainer(Resource clusterResource,
       SchedContainerChangeRequest decreaseRequest,
-      FiCaSchedulerApp app) {
+      FiCaSchedulerApp app) throws InvalidResourceRequestException {
     // If the container being decreased is reserved, we need to unreserve it
     // first.
     RMContainer rmContainer = decreaseRequest.getRMContainer();
@@ -1688,25 +1696,62 @@ public class LeafQueue extends AbstractCSQueue {
       unreserveIncreasedContainer(clusterResource, app,
           (FiCaSchedulerNode)decreaseRequest.getSchedulerNode(), rmContainer);
     }
-    
-    // Delta capacity is negative when it's a decrease request
-    Resource absDelta = Resources.negate(decreaseRequest.getDeltaCapacity());
-    
+    boolean resourceDecreased = false;
+    Resource resourceBeforeDecrease;
+    // Grab queue lock to avoid race condition when getting container resource
     synchronized (this) {
-      // Delta is negative when it's a decrease request
-      releaseResource(clusterResource, app, absDelta,
-          decreaseRequest.getNodePartition(), decreaseRequest.getRMContainer(),
-          true);
-      // Notify application
-      app.decreaseContainer(decreaseRequest);
-      // Notify node
-      decreaseRequest.getSchedulerNode()
-          .decreaseContainer(decreaseRequest.getContainerId(), absDelta);
-    }
-
-    // Notify parent
-    if (getParent() != null) {
+      // Make sure the decrease request is valid in terms of current resource
+      // and target resource. This must be done under the leaf queue lock.
+      // Throws exception if the check fails.
+      RMServerUtils.checkSchedContainerChangeRequest(decreaseRequest, false);
+      // Save resource before decrease for debug log
+      resourceBeforeDecrease =
+          Resources.clone(rmContainer.getAllocatedResource());
+      // Do we have increase request for the same container? If so, remove it
+      boolean hasIncreaseRequest =
+          app.removeIncreaseRequest(decreaseRequest.getNodeId(),
+              decreaseRequest.getPriority(), decreaseRequest.getContainerId());
+      if (hasIncreaseRequest) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("While processing decrease requests, found an increase"
+              + " request for the same container "
+              + decreaseRequest.getContainerId()
+              + ", removed the increase request");
+        }
+      }
+      // Delta capacity is negative when it's a decrease request
+      Resource absDelta = Resources.negate(decreaseRequest.getDeltaCapacity());
+      if (Resources.equals(absDelta, Resources.none())) {
+        // If delta capacity of this decrease request is 0, this decrease
+        // request serves the purpose of cancelling an existing increase request
+        // if any
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Decrease target resource equals to existing resource for"
+              + " container:" + decreaseRequest.getContainerId()
+              + " ignore this decrease request.");
+        }
+      } else {
+        // Release the delta resource
+        releaseResource(clusterResource, app, absDelta,
+            decreaseRequest.getNodePartition(),
+            decreaseRequest.getRMContainer(),
+            true);
+        // Notify application
+        app.decreaseContainer(decreaseRequest);
+        // Notify node
+        decreaseRequest.getSchedulerNode()
+            .decreaseContainer(decreaseRequest.getContainerId(), absDelta);
+        resourceDecreased = true;
+      }
+    }
+
+    if (resourceDecreased) {
+      // Notify parent queue outside of leaf queue lock
       getParent().decreaseContainer(clusterResource, decreaseRequest, app);
+      LOG.info("Application attempt " + app.getApplicationAttemptId()
+          + " decreased container:" + decreaseRequest.getContainerId()
+          + " from " + resourceBeforeDecrease + " to "
+          + decreaseRequest.getTargetCapacity());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f466364/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 badab72..a7d8796 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
@@ -46,6 +46,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.InvalidResourceRequestException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.security.AccessType;
@@ -656,7 +657,8 @@ public class ParentQueue extends AbstractCSQueue {
   
   @Override
   public void decreaseContainer(Resource clusterResource,
-      SchedContainerChangeRequest decreaseRequest, FiCaSchedulerApp app) {
+      SchedContainerChangeRequest decreaseRequest, FiCaSchedulerApp app)
+      throws InvalidResourceRequestException {
     // delta capacity is negative when it's a decrease request
     Resource absDeltaCapacity =
         Resources.negate(decreaseRequest.getDeltaCapacity());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f466364/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerResizing.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/TestContainerResizing.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/TestContainerResizing.java
index 672af64..c08af9d 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/TestContainerResizing.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/TestContainerResizing.java
@@ -21,8 +21,11 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -47,8 +50,12 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerStat
 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.scheduler.ResourceScheduler;
+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.common.fica.FiCaSchedulerApp;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica
+    .FiCaSchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
 import org.apache.hadoop.yarn.util.resource.Resources;
@@ -57,12 +64,48 @@ import org.junit.Before;
 import org.junit.Test;
 
 public class TestContainerResizing {
+  private static final Log LOG = LogFactory.getLog(TestContainerResizing.class);
   private final int GB = 1024;
 
   private YarnConfiguration conf;
 
   RMNodeLabelsManager mgr;
 
+  class MyScheduler extends CapacityScheduler {
+    /*
+     * A Mock Scheduler to simulate the potential effect of deadlock between:
+     * 1. The AbstractYarnScheduler.decreaseContainers() call (from
+     *    ApplicationMasterService thread)
+     * 2. The CapacityScheduler.allocateContainersToNode() call (from the
+     *    scheduler thread)
+     */
+    MyScheduler() {
+      super();
+    }
+
+    @Override
+    protected void decreaseContainers(
+        List<ContainerResourceChangeRequest> decreaseRequests,
+        SchedulerApplicationAttempt attempt) {
+      try {
+        Thread.sleep(1000);
+      } catch(InterruptedException e) {
+        LOG.debug("Thread interrupted.");
+      }
+      super.decreaseContainers(decreaseRequests, attempt);
+    }
+
+    @Override
+    public synchronized void allocateContainersToNode(FiCaSchedulerNode node) {
+      try {
+        Thread.sleep(1000);
+      } catch(InterruptedException e) {
+        LOG.debug("Thread interrupted.");
+      }
+      super.allocateContainersToNode(node);
+    }
+  }
+
   @Before
   public void setUp() throws Exception {
     conf = new YarnConfiguration();
@@ -958,6 +1001,50 @@ public class TestContainerResizing {
     rm1.close();
   }
 
+  @Test (timeout = 60000)
+  public void testDecreaseContainerWillNotDeadlockContainerAllocation()
+      throws Exception {
+    // create and start MockRM with our MyScheduler
+    MockRM rm = new MockRM() {
+      @Override
+      public ResourceScheduler createScheduler() {
+        CapacityScheduler cs = new MyScheduler();
+        cs.setConf(conf);
+        return cs;
+      }
+    };
+    rm.start();
+    // register a node
+    MockNM nm = rm.registerNode("h1:1234", 20 * GB);
+    // submit an application -> app1
+    RMApp app1 = rm.submitApp(3 * GB, "app", "user", null, "default");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm);
+    // making sure resource is allocated
+    checkUsedResource(rm, "default", 3 * GB, null);
+    FiCaSchedulerApp app = getFiCaSchedulerApp(rm, app1.getApplicationId());
+    Assert.assertEquals(3 * GB,
+        app.getAppAttemptResourceUsage().getUsed().getMemory());
+    // making sure container is launched
+    ContainerId containerId1 =
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 1);
+    sentRMContainerLaunched(rm, containerId1);
+    // submit allocation request for a new container
+    am1.allocate(Collections.singletonList(ResourceRequest.newInstance(
+        Priority.newInstance(1), "*", Resources.createResource(2 * GB), 1)),
+        null);
+    // nm reports status update and triggers container allocation
+    nm.nodeHeartbeat(true);
+    // *In the mean time*, am1 asks to decrease its AM container resource from
+    // 3GB to 1GB
+    AllocateResponse response = am1.sendContainerResizingRequest(null,
+        Collections.singletonList(ContainerResourceChangeRequest
+            .newInstance(containerId1, Resources.createResource(GB))));
+    // verify that the containe resource is decreased
+    verifyContainerDecreased(response, containerId1, GB);
+
+    rm.close();
+  }
+
   private void checkPendingResource(MockRM rm, String queueName, int memory,
       String label) {
     CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();


[36/50] hadoop git commit: YARN-4633. Fix random test failure in TestRMRestart#testRMRestartAfterPreemption. (Bibin A Chundatt via rohithsharmaks)

Posted by ar...@apache.org.
YARN-4633. Fix random test failure in TestRMRestart#testRMRestartAfterPreemption. (Bibin A Chundatt via rohithsharmaks)


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

Branch: refs/heads/HDFS-1312
Commit: ef343be82b4268ebd52f6a11e1a1ce53a5d232a9
Parents: 86560a4
Author: Rohith Sharma K S <ro...@apache.org>
Authored: Thu Jan 28 21:53:45 2016 +0530
Committer: Rohith Sharma K S <ro...@apache.org>
Committed: Thu Jan 28 21:53:45 2016 +0530

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                   | 3 +++
 .../apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java  | 1 +
 2 files changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef343be8/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 2fae034..c8a8c06 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -167,6 +167,9 @@ Release 2.9.0 - UNRELEASED
     YARN-4573. Fix test failure in TestRMAppTransitions#testAppRunningKill and
     testAppKilledKilled. (Takashi Ohnishi via rohithsharmaks)
 
+    YARN-4633. Fix random test failure in TestRMRestart#testRMRestartAfterPreemption
+    (Bibin A Chundatt via rohithsharmaks)
+
 Release 2.8.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef343be8/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 3bab88a..e999e6b 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
@@ -2359,6 +2359,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
       // kill app0-attempt
       cs.killPreemptedContainer(schedulerAppAttempt.getRMContainer(
           app0.getCurrentAppAttempt().getMasterContainer().getId()));
+      am0.waitForState(RMAppAttemptState.FAILED);
     }
     am0 = MockRM.launchAM(app0, rm1, nm1);
     am0.registerAppAttempt();


[45/50] hadoop git commit: HDFS-9706. Log more details in debug logs in BlockReceiver's constructor. (Xiao Chen via Yongjun Zhang)

Posted by ar...@apache.org.
HDFS-9706. Log more details in debug logs in BlockReceiver's constructor. (Xiao Chen via Yongjun Zhang)


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

Branch: refs/heads/HDFS-1312
Commit: 8ee060311c89b7faa71dd039481a97ba15e2413d
Parents: ac68666
Author: Yongjun Zhang <yz...@cloudera.com>
Authored: Thu Jan 28 22:53:26 2016 -0800
Committer: Yongjun Zhang <yz...@cloudera.com>
Committed: Thu Jan 28 23:04:03 2016 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt       |  3 +++
 .../hdfs/server/datanode/BlockReceiver.java       | 18 ++++++++++++------
 2 files changed, 15 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8ee06031/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 9b80aa1..570caa5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1860,6 +1860,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8898. Create API and command-line argument to get quota and quota
     usage without detailed content summary. (Ming Ma via kihwal)
 
+    HDFS-9706. Log more details in debug logs in BlockReceiver's constructor.
+    (Xiao Chen via Yongjun Zhang)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8ee06031/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
index e7908a5..0dc8cab 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
@@ -187,12 +187,18 @@ class BlockReceiver implements Closeable {
       this.maxSendIdleTime = (long) (readTimeout * 0.9);
       if (LOG.isDebugEnabled()) {
         LOG.debug(getClass().getSimpleName() + ": " + block
-            + "\n  isClient  =" + isClient + ", clientname=" + clientname
-            + "\n  isDatanode=" + isDatanode + ", srcDataNode=" + srcDataNode
-            + "\n  inAddr=" + inAddr + ", myAddr=" + myAddr
-            + "\n  cachingStrategy = " + cachingStrategy
-            + "\n  pinning=" + pinning
-            );
+            + "\n storageType=" + storageType + ", inAddr=" + inAddr
+            + ", myAddr=" + myAddr + "\n stage=" + stage + ", newGs=" + newGs
+            + ", minBytesRcvd=" + minBytesRcvd
+            + ", maxBytesRcvd=" + maxBytesRcvd + "\n clientname=" + clientname
+            + ", srcDataNode=" + srcDataNode
+            + ", datanode=" + datanode.getDisplayName()
+            + "\n requestedChecksum=" + requestedChecksum
+            + "\n cachingStrategy=" + cachingStrategy
+            + "\n allowLazyPersist=" + allowLazyPersist + ", pinning=" + pinning
+            + ", isClient=" + isClient + ", isDatanode=" + isDatanode
+            + ", responseInterval=" + responseInterval
+        );
       }
 
       //


[24/50] hadoop git commit: YARN-4612. Fix rumen and scheduler load simulator handle killed tasks properly. Contributed by Ming Ma.

Posted by ar...@apache.org.
YARN-4612. Fix rumen and scheduler load simulator handle killed tasks
properly. Contributed by Ming Ma.


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

Branch: refs/heads/HDFS-1312
Commit: 4efdf3a979c361348612f817a3253be6d0de58f7
Parents: d323639
Author: Xuan <xg...@apache.org>
Authored: Tue Jan 26 18:17:12 2016 -0800
Committer: Xuan <xg...@apache.org>
Committed: Tue Jan 26 18:17:12 2016 -0800

----------------------------------------------------------------------
 .../apache/hadoop/tools/rumen/JobBuilder.java   |  11 +-
 .../src/main/data/2jobs2min-rumen-jh.json       | 606 +++++++++++++++++++
 .../org/apache/hadoop/yarn/sls/SLSRunner.java   |   6 +
 .../apache/hadoop/yarn/sls/utils/SLSUtils.java  |   6 +
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 5 files changed, 628 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4efdf3a9/hadoop-tools/hadoop-rumen/src/main/java/org/apache/hadoop/tools/rumen/JobBuilder.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-rumen/src/main/java/org/apache/hadoop/tools/rumen/JobBuilder.java b/hadoop-tools/hadoop-rumen/src/main/java/org/apache/hadoop/tools/rumen/JobBuilder.java
index c5ae2fc..890f388 100644
--- a/hadoop-tools/hadoop-rumen/src/main/java/org/apache/hadoop/tools/rumen/JobBuilder.java
+++ b/hadoop-tools/hadoop-rumen/src/main/java/org/apache/hadoop/tools/rumen/JobBuilder.java
@@ -473,9 +473,12 @@ public class JobBuilder {
     task.setTaskStatus(getPre21Value(event.getTaskStatus()));
     TaskFailed t = (TaskFailed)(event.getDatum());
     task.putDiagnosticInfo(t.error.toString());
-    task.putFailedDueToAttemptId(t.failedDueToAttempt.toString());
+    // killed task wouldn't have failed attempt.
+    if (t.getFailedDueToAttempt() != null) {
+      task.putFailedDueToAttemptId(t.getFailedDueToAttempt().toString());
+    }
     org.apache.hadoop.mapreduce.jobhistory.JhCounters counters =
-        ((TaskFailed) event.getDatum()).counters;
+        ((TaskFailed) event.getDatum()).getCounters();
     task.incorporateCounters(
         counters == null ? EMPTY_COUNTERS : counters);
   }
@@ -500,7 +503,7 @@ public class JobBuilder {
 
     attempt.setFinishTime(event.getFinishTime());
     org.apache.hadoop.mapreduce.jobhistory.JhCounters counters =
-        ((TaskAttemptUnsuccessfulCompletion) event.getDatum()).counters;
+        ((TaskAttemptUnsuccessfulCompletion) event.getDatum()).getCounters();
     attempt.incorporateCounters(
         counters == null ? EMPTY_COUNTERS : counters);
     attempt.arraySetClockSplits(event.getClockSplits());
@@ -509,7 +512,7 @@ public class JobBuilder {
     attempt.arraySetPhysMemKbytes(event.getPhysMemKbytes());
     TaskAttemptUnsuccessfulCompletion t =
         (TaskAttemptUnsuccessfulCompletion) (event.getDatum());
-    attempt.putDiagnosticInfo(t.error.toString());
+    attempt.putDiagnosticInfo(t.getError().toString());
   }
 
   private void processTaskAttemptStartedEvent(TaskAttemptStartedEvent event) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4efdf3a9/hadoop-tools/hadoop-sls/src/main/data/2jobs2min-rumen-jh.json
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/data/2jobs2min-rumen-jh.json b/hadoop-tools/hadoop-sls/src/main/data/2jobs2min-rumen-jh.json
index 83629ed..9d90deb 100644
--- a/hadoop-tools/hadoop-sls/src/main/data/2jobs2min-rumen-jh.json
+++ b/hadoop-tools/hadoop-sls/src/main/data/2jobs2min-rumen-jh.json
@@ -10208,4 +10208,610 @@
   "clusterReduceMB" : -1,
   "jobMapMB" : 200,
   "jobReduceMB" : 200
+} {
+"priority" : "NORMAL",
+"jobID" : "job_1369942127770_1207",
+"user" : "jenkins",
+"jobName" : "TeraGen",
+"submitTime" : 1371223054499,
+"finishTime" : 1371223153874,
+"queue" : "sls_queue_1",
+"mapTasks" : [ {
+"startTime" : 1371223059053,
+"taskID" : "task_1369942127770_1207_m_000000",
+"taskType" : "MAP",
+"finishTime" : 1371223078206,
+"attempts" : [ ],
+"preferredLocations" : [ ],
+"taskStatus" : "KILLED",
+"inputBytes" : -1,
+"inputRecords" : -1,
+"outputBytes" : -1,
+"outputRecords" : -1
+} ],
+"reduceTasks" : [ ],
+"launchTime" : 1371223058937,
+"totalMaps" : 1,
+"totalReduces" : 0,
+"otherTasks" : [ ],
+"jobProperties" : {
+"mapreduce.job.ubertask.enable" : "false",
+"yarn.resourcemanager.max-completed-applications" : "10000",
+"yarn.resourcemanager.delayed.delegation-token.removal-interval-ms" : "30000",
+"mapreduce.client.submit.file.replication" : "2",
+"yarn.nodemanager.container-manager.thread-count" : "20",
+"mapred.queue.default.acl-administer-jobs" : "*",
+"dfs.image.transfer.bandwidthPerSec" : "0",
+"mapreduce.tasktracker.healthchecker.interval" : "60000",
+"mapreduce.jobtracker.staging.root.dir" : "/user",
+"yarn.resourcemanager.recovery.enabled" : "false",
+"yarn.resourcemanager.am.max-retries" : "1",
+"dfs.block.access.token.lifetime" : "600",
+"fs.AbstractFileSystem.file.impl" : "org.apache.hadoop.fs.local.LocalFs",
+"mapreduce.client.completion.pollinterval" : "5000",
+"mapreduce.job.ubertask.maxreduces" : "1",
+"mapreduce.reduce.shuffle.memory.limit.percent" : "0.25",
+"dfs.domain.socket.path" : "/var/run/hdfs-sockets/dn",
+"hadoop.ssl.keystores.factory.class" : "org.apache.hadoop.security.ssl.FileBasedKeyStoresFactory",
+"hadoop.http.authentication.kerberos.keytab" : "${user.home}/hadoop.keytab",
+"yarn.nodemanager.keytab" : "/etc/krb5.keytab",
+"io.seqfile.sorter.recordlimit" : "1000000",
+"s3.blocksize" : "67108864",
+"mapreduce.task.io.sort.factor" : "10",
+"yarn.nodemanager.disk-health-checker.interval-ms" : "120000",
+"mapreduce.job.working.dir" : "hdfs://a2115.smile.com:8020/user/jenkins",
+"yarn.admin.acl" : "*",
+"mapreduce.job.speculative.speculativecap" : "0.1",
+"dfs.namenode.num.checkpoints.retained" : "2",
+"dfs.namenode.delegation.token.renew-interval" : "86400000",
+"yarn.nodemanager.resource.memory-mb" : "8192",
+"io.map.index.interval" : "128",
+"s3.client-write-packet-size" : "65536",
+"mapreduce.task.files.preserve.failedtasks" : "false",
+"dfs.namenode.http-address" : "a2115.smile.com:20101",
+"ha.zookeeper.session-timeout.ms" : "5000",
+"hadoop.hdfs.configuration.version" : "1",
+"s3.replication" : "3",
+"dfs.datanode.balance.bandwidthPerSec" : "1048576",
+"mapreduce.reduce.shuffle.connect.timeout" : "180000",
+"hadoop.ssl.enabled" : "false",
+"dfs.journalnode.rpc-address" : "0.0.0.0:8485",
+"yarn.nodemanager.aux-services" : "mapreduce.shuffle",
+"mapreduce.job.counters.max" : "120",
+"dfs.datanode.readahead.bytes" : "4193404",
+"ipc.client.connect.max.retries.on.timeouts" : "45",
+"mapreduce.job.complete.cancel.delegation.tokens" : "true",
+"dfs.client.failover.max.attempts" : "15",
+"dfs.namenode.checkpoint.dir" : "file://${hadoop.tmp.dir}/dfs/namesecondary",
+"dfs.namenode.replication.work.multiplier.per.iteration" : "2",
+"fs.trash.interval" : "1",
+"yarn.resourcemanager.admin.address" : "a2115.smile.com:8033",
+"ha.health-monitor.check-interval.ms" : "1000",
+"mapreduce.job.outputformat.class" : "org.apache.hadoop.examples.terasort.TeraOutputFormat",
+"hadoop.jetty.logs.serve.aliases" : "true",
+"hadoop.http.authentication.kerberos.principal" : "HTTP/_HOST@LOCALHOST",
+"mapreduce.job.reduce.shuffle.consumer.plugin.class" : "org.apache.hadoop.mapreduce.task.reduce.Shuffle",
+"s3native.blocksize" : "67108864",
+"dfs.namenode.edits.dir" : "${dfs.namenode.name.dir}",
+"ha.health-monitor.sleep-after-disconnect.ms" : "1000",
+"dfs.encrypt.data.transfer" : "false",
+"dfs.datanode.http.address" : "0.0.0.0:50075",
+"mapreduce.terasort.num-rows" : "400000000",
+"mapreduce.job.map.class" : "org.apache.hadoop.examples.terasort.TeraGen$SortGenMapper",
+"mapreduce.jobtracker.jobhistory.task.numberprogresssplits" : "12",
+"dfs.namenode.write.stale.datanode.ratio" : "0.5f",
+"dfs.client.use.datanode.hostname" : "false",
+"yarn.acl.enable" : "true",
+"hadoop.security.instrumentation.requires.admin" : "false",
+"yarn.nodemanager.localizer.fetch.thread-count" : "4",
+"hadoop.security.authorization" : "false",
+"user.name" : "jenkins",
+"dfs.namenode.fs-limits.min-block-size" : "1048576",
+"dfs.client.failover.connection.retries.on.timeouts" : "0",
+"hadoop.security.group.mapping.ldap.search.filter.group" : "(objectClass=group)",
+"mapreduce.output.fileoutputformat.compress.codec" : "org.apache.hadoop.io.compress.DefaultCodec",
+"dfs.namenode.safemode.extension" : "30000",
+"mapreduce.shuffle.port" : "8080",
+"mapreduce.reduce.log.level" : "INFO",
+"yarn.log-aggregation-enable" : "false",
+"dfs.datanode.sync.behind.writes" : "false",
+"mapreduce.jobtracker.instrumentation" : "org.apache.hadoop.mapred.JobTrackerMetricsInst",
+"dfs.https.server.keystore.resource" : "ssl-server.xml",
+"hadoop.security.group.mapping.ldap.search.attr.group.name" : "cn",
+"dfs.namenode.replication.min" : "1",
+"mapreduce.map.java.opts" : " -Xmx825955249",
+"yarn.scheduler.fair.allocation.file" : "/etc/yarn/fair-scheduler.xml",
+"s3native.bytes-per-checksum" : "512",
+"mapreduce.tasktracker.tasks.sleeptimebeforesigkill" : "5000",
+"tfile.fs.output.buffer.size" : "262144",
+"yarn.nodemanager.local-dirs" : "${hadoop.tmp.dir}/nm-local-dir",
+"mapreduce.jobtracker.persist.jobstatus.active" : "false",
+"fs.AbstractFileSystem.hdfs.impl" : "org.apache.hadoop.fs.Hdfs",
+"mapreduce.job.map.output.collector.class" : "org.apache.hadoop.mapred.MapTask$MapOutputBuffer",
+"mapreduce.tasktracker.local.dir.minspacestart" : "0",
+"dfs.namenode.safemode.min.datanodes" : "0",
+"hadoop.security.uid.cache.secs" : "14400",
+"dfs.client.https.need-auth" : "false",
+"dfs.client.write.exclude.nodes.cache.expiry.interval.millis" : "600000",
+"dfs.client.https.keystore.resource" : "ssl-client.xml",
+"dfs.namenode.max.objects" : "0",
+"hadoop.ssl.client.conf" : "ssl-client.xml",
+"dfs.namenode.safemode.threshold-pct" : "0.999f",
+"mapreduce.tasktracker.local.dir.minspacekill" : "0",
+"mapreduce.jobtracker.retiredjobs.cache.size" : "1000",
+"dfs.blocksize" : "134217728",
+"yarn.resourcemanager.scheduler.class" : "org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler",
+"mapreduce.job.reduce.slowstart.completedmaps" : "0.8",
+"mapreduce.job.end-notification.retry.attempts" : "5",
+"mapreduce.job.inputformat.class" : "org.apache.hadoop.examples.terasort.TeraGen$RangeInputFormat",
+"mapreduce.map.memory.mb" : "1024",
+"mapreduce.job.user.name" : "jenkins",
+"mapreduce.tasktracker.outofband.heartbeat" : "false",
+"io.native.lib.available" : "true",
+"mapreduce.jobtracker.persist.jobstatus.hours" : "0",
+"dfs.client-write-packet-size" : "65536",
+"mapreduce.client.progressmonitor.pollinterval" : "1000",
+"dfs.namenode.name.dir" : "file://${hadoop.tmp.dir}/dfs/name",
+"dfs.ha.log-roll.period" : "120",
+"mapreduce.reduce.input.buffer.percent" : "0.0",
+"mapreduce.map.output.compress.codec" : "org.apache.hadoop.io.compress.SnappyCodec",
+"dfs.client.failover.sleep.base.millis" : "500",
+"dfs.datanode.directoryscan.threads" : "1",
+"mapreduce.jobtracker.address" : "neededForHive:999999",
+"mapreduce.cluster.local.dir" : "${hadoop.tmp.dir}/mapred/local",
+"yarn.scheduler.fair.user-as-default-queue" : "true",
+"mapreduce.job.application.attempt.id" : "1",
+"dfs.permissions.enabled" : "true",
+"mapreduce.tasktracker.taskcontroller" : "org.apache.hadoop.mapred.DefaultTaskController",
+"yarn.scheduler.fair.preemption" : "true",
+"mapreduce.reduce.shuffle.parallelcopies" : "5",
+"dfs.support.append" : "true",
+"yarn.nodemanager.env-whitelist" : "JAVA_HOME,HADOOP_COMMON_HOME,HADOOP_HDFS_HOME,HADOOP_CONF_DIR,YARN_HOME",
+"mapreduce.jobtracker.heartbeats.in.second" : "100",
+"mapreduce.job.maxtaskfailures.per.tracker" : "3",
+"ipc.client.connection.maxidletime" : "10000",
+"mapreduce.shuffle.ssl.enabled" : "false",
+"dfs.namenode.invalidate.work.pct.per.iteration" : "0.32f",
+"dfs.blockreport.intervalMsec" : "21600000",
+"fs.s3.sleepTimeSeconds" : "10",
+"dfs.namenode.replication.considerLoad" : "true",
+"dfs.client.block.write.retries" : "3",
+"hadoop.ssl.server.conf" : "ssl-server.xml",
+"dfs.namenode.name.dir.restore" : "false",
+"rpc.engine.org.apache.hadoop.mapreduce.v2.api.MRClientProtocolPB" : "org.apache.hadoop.ipc.ProtobufRpcEngine",
+"dfs.datanode.hdfs-blocks-metadata.enabled" : "true",
+"ha.zookeeper.parent-znode" : "/hadoop-ha",
+"io.seqfile.lazydecompress" : "true",
+"mapreduce.reduce.merge.inmem.threshold" : "1000",
+"mapreduce.input.fileinputformat.split.minsize" : "0",
+"dfs.replication" : "3",
+"ipc.client.tcpnodelay" : "false",
+"dfs.namenode.accesstime.precision" : "3600000",
+"s3.stream-buffer-size" : "4096",
+"mapreduce.jobtracker.tasktracker.maxblacklists" : "4",
+"dfs.client.read.shortcircuit.skip.checksum" : "false",
+"mapreduce.job.jvm.numtasks" : "1",
+"mapreduce.task.io.sort.mb" : "100",
+"io.file.buffer.size" : "65536",
+"dfs.namenode.audit.loggers" : "default",
+"dfs.namenode.checkpoint.txns" : "1000000",
+"yarn.nodemanager.admin-env" : "MALLOC_ARENA_MAX=$MALLOC_ARENA_MAX",
+"mapreduce.job.jar" : "/user/jenkins/.staging/job_1369942127770_1207/job.jar",
+"mapreduce.job.split.metainfo.maxsize" : "10000000",
+"kfs.replication" : "3",
+"rpc.engine.org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB" : "org.apache.hadoop.ipc.ProtobufRpcEngine",
+"yarn.app.mapreduce.am.scheduler.heartbeat.interval-ms" : "1000",
+"mapreduce.reduce.maxattempts" : "4",
+"kfs.stream-buffer-size" : "4096",
+"dfs.ha.tail-edits.period" : "60",
+"hadoop.security.authentication" : "simple",
+"fs.s3.buffer.dir" : "${hadoop.tmp.dir}/s3",
+"rpc.engine.org.apache.hadoop.yarn.api.AMRMProtocolPB" : "org.apache.hadoop.ipc.ProtobufRpcEngine",
+"mapreduce.jobtracker.taskscheduler" : "org.apache.hadoop.mapred.JobQueueTaskScheduler",
+"yarn.app.mapreduce.am.job.task.listener.thread-count" : "30",
+"dfs.namenode.avoid.read.stale.datanode" : "false",
+"mapreduce.job.reduces" : "0",
+"mapreduce.map.sort.spill.percent" : "0.8",
+"dfs.client.file-block-storage-locations.timeout" : "60",
+"dfs.datanode.drop.cache.behind.writes" : "false",
+"mapreduce.job.end-notification.retry.interval" : "1",
+"mapreduce.job.maps" : "96",
+"mapreduce.job.speculative.slownodethreshold" : "1.0",
+"tfile.fs.input.buffer.size" : "262144",
+"mapreduce.map.speculative" : "false",
+"dfs.block.access.token.enable" : "false",
+"dfs.journalnode.http-address" : "0.0.0.0:8480",
+"mapreduce.job.acl-view-job" : " ",
+"mapreduce.reduce.shuffle.retry-delay.max.ms" : "60000",
+"mapreduce.job.end-notification.max.retry.interval" : "5",
+"ftp.blocksize" : "67108864",
+"mapreduce.tasktracker.http.threads" : "80",
+"mapreduce.reduce.java.opts" : " -Xmx825955249",
+"dfs.datanode.data.dir" : "file://${hadoop.tmp.dir}/dfs/data",
+"ha.failover-controller.cli-check.rpc-timeout.ms" : "20000",
+"dfs.namenode.max.extra.edits.segments.retained" : "10000",
+"dfs.https.port" : "20102",
+"dfs.namenode.replication.interval" : "3",
+"mapreduce.task.skip.start.attempts" : "2",
+"dfs.namenode.https-address" : "a2115.smile.com:20102",
+"mapreduce.jobtracker.persist.jobstatus.dir" : "/jobtracker/jobsInfo",
+"ipc.client.kill.max" : "10",
+"dfs.ha.automatic-failover.enabled" : "false",
+"mapreduce.jobhistory.keytab" : "/etc/security/keytab/jhs.service.keytab",
+"dfs.image.transfer.timeout" : "600000",
+"dfs.client.failover.sleep.max.millis" : "15000",
+"mapreduce.job.end-notification.max.attempts" : "5",
+"mapreduce.task.tmp.dir" : "./tmp",
+"dfs.default.chunk.view.size" : "32768",
+"kfs.bytes-per-checksum" : "512",
+"mapreduce.reduce.memory.mb" : "1024",
+"hadoop.http.filter.initializers" : "org.apache.hadoop.yarn.server.webproxy.amfilter.AmFilterInitializer",
+"dfs.datanode.failed.volumes.tolerated" : "0",
+"hadoop.http.authentication.type" : "simple",
+"dfs.datanode.data.dir.perm" : "700",
+"yarn.resourcemanager.client.thread-count" : "50",
+"ipc.server.listen.queue.size" : "128",
+"mapreduce.reduce.skip.maxgroups" : "0",
+"file.stream-buffer-size" : "4096",
+"dfs.namenode.fs-limits.max-directory-items" : "0",
+"io.mapfile.bloom.size" : "1048576",
+"yarn.nodemanager.container-executor.class" : "org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor",
+"mapreduce.map.maxattempts" : "4",
+"mapreduce.jobtracker.jobhistory.block.size" : "3145728",
+"yarn.log-aggregation.retain-seconds" : "-1",
+"yarn.app.mapreduce.am.job.committer.cancel-timeout" : "60000",
+"ftp.replication" : "3",
+"mapreduce.jobtracker.http.address" : "0.0.0.0:50030",
+"yarn.nodemanager.health-checker.script.timeout-ms" : "1200000",
+"mapreduce.jobhistory.address" : "a2115.smile.com:10020",
+"mapreduce.jobtracker.taskcache.levels" : "2",
+"dfs.datanode.dns.nameserver" : "default",
+"mapreduce.application.classpath" : "$HADOOP_MAPRED_HOME/share/hadoop/mapreduce/*,$HADOOP_MAPRED_HOME/share/hadoop/mapreduce/lib/*",
+"yarn.nodemanager.log.retain-seconds" : "10800",
+"mapred.child.java.opts" : "-Xmx200m",
+"dfs.replication.max" : "512",
+"map.sort.class" : "org.apache.hadoop.util.QuickSort",
+"dfs.stream-buffer-size" : "4096",
+"dfs.namenode.backup.address" : "0.0.0.0:50100",
+"hadoop.util.hash.type" : "murmur",
+"dfs.block.access.key.update.interval" : "600",
+"dfs.datanode.dns.interface" : "default",
+"dfs.datanode.use.datanode.hostname" : "false",
+"mapreduce.job.output.key.class" : "org.apache.hadoop.io.Text",
+"dfs.client.read.shortcircuit" : "false",
+"dfs.namenode.backup.http-address" : "0.0.0.0:50105",
+"yarn.nodemanager.container-monitor.interval-ms" : "3000",
+"yarn.nodemanager.disk-health-checker.min-healthy-disks" : "0.25",
+"kfs.client-write-packet-size" : "65536",
+"ha.zookeeper.acl" : "world:anyone:rwcda",
+"yarn.nodemanager.sleep-delay-before-sigkill.ms" : "250",
+"mapreduce.job.dir" : "/user/jenkins/.staging/job_1369942127770_1207",
+"io.map.index.skip" : "0",
+"net.topology.node.switch.mapping.impl" : "org.apache.hadoop.net.ScriptBasedMapping",
+"fs.s3.maxRetries" : "4",
+"ha.failover-controller.new-active.rpc-timeout.ms" : "60000",
+"s3native.client-write-packet-size" : "65536",
+"yarn.resourcemanager.amliveliness-monitor.interval-ms" : "1000",
+"hadoop.http.staticuser.user" : "dr.who",
+"mapreduce.reduce.speculative" : "false",
+"mapreduce.client.output.filter" : "FAILED",
+"mapreduce.ifile.readahead.bytes" : "4194304",
+"mapreduce.tasktracker.report.address" : "127.0.0.1:0",
+"mapreduce.task.userlog.limit.kb" : "0",
+"mapreduce.tasktracker.map.tasks.maximum" : "2",
+"hadoop.http.authentication.simple.anonymous.allowed" : "true",
+"hadoop.fuse.timer.period" : "5",
+"dfs.namenode.num.extra.edits.retained" : "1000000",
+"hadoop.rpc.socket.factory.class.default" : "org.apache.hadoop.net.StandardSocketFactory",
+"mapreduce.job.submithostname" : "a2115.smile.com",
+"dfs.namenode.handler.count" : "10",
+"fs.automatic.close" : "false",
+"mapreduce.job.submithostaddress" : "10.20.206.115",
+"mapreduce.tasktracker.healthchecker.script.timeout" : "600000",
+"dfs.datanode.directoryscan.interval" : "21600",
+"yarn.resourcemanager.address" : "a2115.smile.com:8032",
+"yarn.nodemanager.health-checker.interval-ms" : "600000",
+"dfs.client.file-block-storage-locations.num-threads" : "10",
+"yarn.resourcemanager.container-tokens.master-key-rolling-interval-secs" : "86400",
+"mapreduce.reduce.markreset.buffer.percent" : "0.0",
+"hadoop.security.group.mapping.ldap.directory.search.timeout" : "10000",
+"mapreduce.map.log.level" : "INFO",
+"dfs.bytes-per-checksum" : "512",
+"yarn.nodemanager.localizer.address" : "0.0.0.0:8040",
+"dfs.namenode.checkpoint.max-retries" : "3",
+"ha.health-monitor.rpc-timeout.ms" : "45000",
+"yarn.resourcemanager.keytab" : "/etc/krb5.keytab",
+"ftp.stream-buffer-size" : "4096",
+"dfs.namenode.avoid.write.stale.datanode" : "false",
+"hadoop.security.group.mapping.ldap.search.attr.member" : "member",
+"mapreduce.output.fileoutputformat.outputdir" : "hdfs://a2115.smile.com:8020/user/jenkins/tera-gen-1",
+"dfs.blockreport.initialDelay" : "0",
+"yarn.nm.liveness-monitor.expiry-interval-ms" : "600000",
+"hadoop.http.authentication.token.validity" : "36000",
+"dfs.namenode.delegation.token.max-lifetime" : "604800000",
+"mapreduce.job.hdfs-servers" : "${fs.defaultFS}",
+"s3native.replication" : "3",
+"yarn.nodemanager.localizer.client.thread-count" : "5",
+"dfs.heartbeat.interval" : "3",
+"rpc.engine.org.apache.hadoop.ipc.ProtocolMetaInfoPB" : "org.apache.hadoop.ipc.ProtobufRpcEngine",
+"dfs.ha.fencing.ssh.connect-timeout" : "30000",
+"yarn.resourcemanager.container.liveness-monitor.interval-ms" : "600000",
+"yarn.am.liveness-monitor.expiry-interval-ms" : "600000",
+"mapreduce.task.profile" : "false",
+"mapreduce.tasktracker.http.address" : "0.0.0.0:50060",
+"mapreduce.tasktracker.instrumentation" : "org.apache.hadoop.mapred.TaskTrackerMetricsInst",
+"mapreduce.jobhistory.webapp.address" : "a2115.smile.com:19888",
+"ha.failover-controller.graceful-fence.rpc-timeout.ms" : "5000",
+"yarn.ipc.rpc.class" : "org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC",
+"mapreduce.job.name" : "TeraGen",
+"kfs.blocksize" : "67108864",
+"yarn.resourcemanager.am-rm-tokens.master-key-rolling-interval-secs" : "86400",
+"mapreduce.job.ubertask.maxmaps" : "9",
+"yarn.scheduler.maximum-allocation-mb" : "8192",
+"yarn.nodemanager.heartbeat.interval-ms" : "1000",
+"mapreduce.job.userlog.retain.hours" : "24",
+"dfs.namenode.secondary.http-address" : "0.0.0.0:50090",
+"mapreduce.task.timeout" : "600000",
+"mapreduce.framework.name" : "yarn",
+"ipc.client.idlethreshold" : "4000",
+"ftp.bytes-per-checksum" : "512",
+"ipc.server.tcpnodelay" : "false",
+"dfs.namenode.stale.datanode.interval" : "30000",
+"s3.bytes-per-checksum" : "512",
+"mapreduce.job.speculative.slowtaskthreshold" : "1.0",
+"yarn.nodemanager.localizer.cache.target-size-mb" : "10240",
+"yarn.nodemanager.remote-app-log-dir" : "/tmp/logs",
+"fs.s3.block.size" : "67108864",
+"mapreduce.job.queuename" : "sls_queue_1",
+"dfs.client.failover.connection.retries" : "0",
+"hadoop.rpc.protection" : "authentication",
+"yarn.scheduler.minimum-allocation-mb" : "1024",
+"yarn.app.mapreduce.client-am.ipc.max-retries" : "1",
+"hadoop.security.auth_to_local" : "DEFAULT",
+"dfs.secondary.namenode.kerberos.internal.spnego.principal" : "${dfs.web.authentication.kerberos.principal}",
+"ftp.client-write-packet-size" : "65536",
+"fs.defaultFS" : "hdfs://a2115.smile.com:8020",
+"yarn.nodemanager.address" : "0.0.0.0:0",
+"yarn.scheduler.fair.assignmultiple" : "true",
+"yarn.resourcemanager.scheduler.client.thread-count" : "50",
+"mapreduce.task.merge.progress.records" : "10000",
+"file.client-write-packet-size" : "65536",
+"yarn.nodemanager.delete.thread-count" : "4",
+"yarn.resourcemanager.scheduler.address" : "a2115.smile.com:8030",
+"fs.trash.checkpoint.interval" : "0",
+"hadoop.http.authentication.signature.secret.file" : "${user.home}/hadoop-http-auth-signature-secret",
+"s3native.stream-buffer-size" : "4096",
+"mapreduce.reduce.shuffle.read.timeout" : "180000",
+"mapreduce.admin.user.env" : "LD_LIBRARY_PATH=$HADOOP_COMMON_HOME/lib/native",
+"yarn.app.mapreduce.am.command-opts" : " -Xmx1238932873",
+"dfs.namenode.checkpoint.edits.dir" : "${dfs.namenode.checkpoint.dir}",
+"fs.permissions.umask-mode" : "022",
+"dfs.client.domain.socket.data.traffic" : "false",
+"hadoop.common.configuration.version" : "0.23.0",
+"mapreduce.tasktracker.dns.interface" : "default",
+"mapreduce.output.fileoutputformat.compress.type" : "BLOCK",
+"mapreduce.ifile.readahead" : "true",
+"hadoop.security.group.mapping.ldap.ssl" : "false",
+"io.serializations" : "org.apache.hadoop.io.serializer.WritableSerialization,org.apache.hadoop.io.serializer.avro.AvroSpecificSerialization,org.apache.hadoop.io.serializer.avro.AvroReflectSerialization",
+"yarn.nodemanager.aux-services.mapreduce.shuffle.class" : "org.apache.hadoop.mapred.ShuffleHandler",
+"fs.df.interval" : "60000",
+"mapreduce.reduce.shuffle.input.buffer.percent" : "0.70",
+"io.seqfile.compress.blocksize" : "1000000",
+"hadoop.security.groups.cache.secs" : "300",
+"ipc.client.connect.max.retries" : "10",
+"dfs.namenode.delegation.key.update-interval" : "86400000",
+"yarn.nodemanager.process-kill-wait.ms" : "2000",
+"yarn.application.classpath" : "$HADOOP_CONF_DIR,$HADOOP_COMMON_HOME/*,$HADOOP_COMMON_HOME/lib/*,$HADOOP_HDFS_HOME/*,$HADOOP_HDFS_HOME/lib/*,$HADOOP_MAPRED_HOME/*,$HADOOP_MAPRED_HOME/lib/*,$YARN_HOME/*,$YARN_HOME/lib/*",
+"yarn.app.mapreduce.client.max-retries" : "3",
+"dfs.datanode.available-space-volume-choosing-policy.balanced-space-preference-fraction" : "0.75f",
+"yarn.nodemanager.log-aggregation.compression-type" : "none",
+"hadoop.security.group.mapping.ldap.search.filter.user" : "(&(objectClass=user)(sAMAccountName={0}))",
+"yarn.nodemanager.localizer.cache.cleanup.interval-ms" : "600000",
+"dfs.image.compress" : "false",
+"mapred.mapper.new-api" : "true",
+"yarn.nodemanager.log-dirs" : "${yarn.log.dir}/userlogs",
+"dfs.namenode.kerberos.internal.spnego.principal" : "${dfs.web.authentication.kerberos.principal}",
+"fs.s3n.block.size" : "67108864",
+"fs.ftp.host" : "0.0.0.0",
+"hadoop.security.group.mapping" : "org.apache.hadoop.security.JniBasedUnixGroupsMappingWithFallback",
+"dfs.datanode.address" : "0.0.0.0:50010",
+"mapreduce.map.skip.maxrecords" : "0",
+"dfs.datanode.https.address" : "0.0.0.0:50475",
+"file.replication" : "1",
+"yarn.resourcemanager.resource-tracker.address" : "a2115.smile.com:8031",
+"dfs.datanode.drop.cache.behind.reads" : "false",
+"hadoop.fuse.connection.timeout" : "300",
+"hadoop.work.around.non.threadsafe.getpwuid" : "false",
+"mapreduce.jobtracker.restart.recover" : "false",
+"hadoop.tmp.dir" : "/tmp/hadoop-${user.name}",
+"mapreduce.output.fileoutputformat.compress" : "false",
+"mapreduce.tasktracker.indexcache.mb" : "10",
+"mapreduce.client.genericoptionsparser.used" : "true",
+"dfs.client.block.write.replace-datanode-on-failure.policy" : "DEFAULT",
+"mapreduce.job.committer.setup.cleanup.needed" : "true",
+"hadoop.kerberos.kinit.command" : "kinit",
+"dfs.datanode.du.reserved" : "0",
+"dfs.namenode.fs-limits.max-blocks-per-file" : "1048576",
+"file.bytes-per-checksum" : "512",
+"mapreduce.task.profile.reduces" : "0-2",
+"mapreduce.jobtracker.handler.count" : "10",
+"dfs.client.block.write.replace-datanode-on-failure.enable" : "true",
+"mapreduce.job.output.value.class" : "org.apache.hadoop.io.Text",
+"yarn.dispatcher.exit-on-error" : "true",
+"net.topology.script.number.args" : "100",
+"mapreduce.task.profile.maps" : "0-2",
+"dfs.namenode.decommission.interval" : "30",
+"dfs.image.compression.codec" : "org.apache.hadoop.io.compress.DefaultCodec",
+"yarn.resourcemanager.webapp.address" : "a2115.smile.com:8088",
+"mapreduce.jobtracker.system.dir" : "${hadoop.tmp.dir}/mapred/system",
+"hadoop.ssl.hostname.verifier" : "DEFAULT",
+"yarn.nodemanager.vmem-pmem-ratio" : "2.1",
+"dfs.namenode.support.allow.format" : "true",
+"mapreduce.jobhistory.principal" : "jhs/_HOST@REALM.TLD",
+"io.mapfile.bloom.error.rate" : "0.005",
+"mapreduce.shuffle.ssl.file.buffer.size" : "65536",
+"dfs.permissions.superusergroup" : "supergroup",
+"dfs.datanode.available-space-volume-choosing-policy.balanced-space-threshold" : "10737418240",
+"mapreduce.jobtracker.expire.trackers.interval" : "600000",
+"mapreduce.cluster.acls.enabled" : "false",
+"yarn.nodemanager.remote-app-log-dir-suffix" : "logs",
+"ha.failover-controller.graceful-fence.connection.retries" : "1",
+"ha.health-monitor.connect-retry-interval.ms" : "1000",
+"mapreduce.reduce.shuffle.merge.percent" : "0.66",
+"yarn.app.mapreduce.am.resource.mb" : "1536",
+"io.seqfile.local.dir" : "${hadoop.tmp.dir}/io/local",
+"dfs.namenode.checkpoint.check.period" : "60",
+"yarn.resourcemanager.nm.liveness-monitor.interval-ms" : "1000",
+"mapreduce.jobtracker.maxtasks.perjob" : "-1",
+"mapreduce.jobtracker.jobhistory.lru.cache.size" : "5",
+"file.blocksize" : "67108864",
+"tfile.io.chunk.size" : "1048576",
+"mapreduce.job.acl-modify-job" : " ",
+"yarn.nodemanager.webapp.address" : "0.0.0.0:8042",
+"mapreduce.tasktracker.reduce.tasks.maximum" : "2",
+"io.skip.checksum.errors" : "false",
+"mapreduce.cluster.temp.dir" : "${hadoop.tmp.dir}/mapred/temp",
+"yarn.app.mapreduce.am.staging-dir" : "/user",
+"dfs.namenode.edits.journal-plugin.qjournal" : "org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager",
+"dfs.datanode.handler.count" : "10",
+"fs.ftp.host.port" : "21",
+"dfs.namenode.decommission.nodes.per.interval" : "5",
+"yarn.resourcemanager.admin.client.thread-count" : "1",
+"dfs.namenode.fs-limits.max-component-length" : "0",
+"dfs.namenode.checkpoint.period" : "3600",
+"fs.AbstractFileSystem.viewfs.impl" : "org.apache.hadoop.fs.viewfs.ViewFs",
+"yarn.resourcemanager.resource-tracker.client.thread-count" : "50",
+"mapreduce.tasktracker.dns.nameserver" : "default",
+"mapreduce.map.output.compress" : "true",
+"dfs.datanode.ipc.address" : "0.0.0.0:50020",
+"hadoop.ssl.require.client.cert" : "false",
+"yarn.nodemanager.delete.debug-delay-sec" : "0",
+"dfs.datanode.max.transfer.threads" : "4096"
+},
+"computonsPerMapInputByte" : -1,
+"computonsPerMapOutputByte" : -1,
+"computonsPerReduceInputByte" : -1,
+"computonsPerReduceOutputByte" : -1,
+"heapMegabytes" : 200,
+"outcome" : "SUCCESS",
+"jobtype" : "JAVA",
+"directDependantJobs" : [ ],
+"successfulMapAttemptCDFs" : [ {
+"maximum" : 9223372036854775807,
+"minimum" : -9223372036854775808,
+"rankings" : [ ],
+"numberValues" : 0
+}, {
+"maximum" : 9223372036854775807,
+"minimum" : -9223372036854775808,
+"rankings" : [ ],
+"numberValues" : 0
+}, {
+"maximum" : 9223372036854775807,
+"minimum" : -9223372036854775808,
+"rankings" : [ ],
+"numberValues" : 0
+}, {
+"maximum" : 47021,
+"minimum" : 11143,
+"rankings" : [ {
+"datum" : 13354,
+"relativeRanking" : 0.05
+}, {
+"datum" : 14101,
+"relativeRanking" : 0.1
+}, {
+"datum" : 15609,
+"relativeRanking" : 0.15
+}, {
+"datum" : 15919,
+"relativeRanking" : 0.2
+}, {
+"datum" : 17003,
+"relativeRanking" : 0.25
+}, {
+"datum" : 17109,
+"relativeRanking" : 0.3
+}, {
+"datum" : 18342,
+"relativeRanking" : 0.35
+}, {
+"datum" : 18870,
+"relativeRanking" : 0.4
+}, {
+"datum" : 19127,
+"relativeRanking" : 0.45
+}, {
+"datum" : 19221,
+"relativeRanking" : 0.5
+}, {
+"datum" : 19481,
+"relativeRanking" : 0.55
+}, {
+"datum" : 19896,
+"relativeRanking" : 0.6
+}, {
+"datum" : 20585,
+"relativeRanking" : 0.65
+}, {
+"datum" : 20784,
+"relativeRanking" : 0.7
+}, {
+"datum" : 21452,
+"relativeRanking" : 0.75
+}, {
+"datum" : 21853,
+"relativeRanking" : 0.8
+}, {
+"datum" : 22436,
+"relativeRanking" : 0.85
+}, {
+"datum" : 32646,
+"relativeRanking" : 0.9
+}, {
+"datum" : 41553,
+"relativeRanking" : 0.95
+} ],
+"numberValues" : 96
+} ],
+"failedMapAttemptCDFs" : [ {
+"maximum" : 9223372036854775807,
+"minimum" : -9223372036854775808,
+"rankings" : [ ],
+"numberValues" : 0
+}, {
+"maximum" : 9223372036854775807,
+"minimum" : -9223372036854775808,
+"rankings" : [ ],
+"numberValues" : 0
+}, {
+"maximum" : 9223372036854775807,
+"minimum" : -9223372036854775808,
+"rankings" : [ ],
+"numberValues" : 0
+}, {
+"maximum" : 9223372036854775807,
+"minimum" : -9223372036854775808,
+"rankings" : [ ],
+"numberValues" : 0
+} ],
+"successfulReduceAttemptCDF" : {
+"maximum" : 9223372036854775807,
+"minimum" : -9223372036854775808,
+"rankings" : [ ],
+"numberValues" : 0
+},
+"failedReduceAttemptCDF" : {
+"maximum" : 9223372036854775807,
+"minimum" : -9223372036854775808,
+"rankings" : [ ],
+"numberValues" : 0
+},
+"mapperTriesToSucceed" : [ 1.0 ],
+"failedMapperFraction" : 0.0,
+"relativeTime" : 0,
+"clusterMapMB" : -1,
+"clusterReduceMB" : -1,
+"jobMapMB" : 200,
+"jobReduceMB" : 200
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4efdf3a9/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java
index b36edc9..c792335 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java
@@ -389,6 +389,9 @@ public class SLSRunner {
                   new ArrayList<ContainerSimulator>();
           // map tasks
           for(LoggedTask mapTask : job.getMapTasks()) {
+            if (mapTask.getAttempts().size() == 0) {
+              continue;
+            }
             LoggedTaskAttempt taskAttempt = mapTask.getAttempts()
                     .get(mapTask.getAttempts().size() - 1);
             String hostname = taskAttempt.getHostName().getValue();
@@ -400,6 +403,9 @@ public class SLSRunner {
 
           // reduce tasks
           for(LoggedTask reduceTask : job.getReduceTasks()) {
+            if (reduceTask.getAttempts().size() == 0) {
+              continue;
+            }
             LoggedTaskAttempt taskAttempt = reduceTask.getAttempts()
                     .get(reduceTask.getAttempts().size() - 1);
             String hostname = taskAttempt.getHostName().getValue();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4efdf3a9/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/utils/SLSUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/utils/SLSUtils.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/utils/SLSUtils.java
index d950aee..f1b4f07 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/utils/SLSUtils.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/utils/SLSUtils.java
@@ -72,11 +72,17 @@ public class SLSUtils {
       while ((job = reader.getNext()) != null) {
         for(LoggedTask mapTask : job.getMapTasks()) {
           // select the last attempt
+          if (mapTask.getAttempts().size() == 0) {
+            continue;
+          }
           LoggedTaskAttempt taskAttempt = mapTask.getAttempts()
                   .get(mapTask.getAttempts().size() - 1);
           nodeSet.add(taskAttempt.getHostName().getValue());
         }
         for(LoggedTask reduceTask : job.getReduceTasks()) {
+          if (reduceTask.getAttempts().size() == 0) {
+            continue;
+          }
           LoggedTaskAttempt taskAttempt = reduceTask.getAttempts()
                   .get(reduceTask.getAttempts().size() - 1);
           nodeSet.add(taskAttempt.getHostName().getValue());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4efdf3a9/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index c2f16d5..435eb68 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -159,6 +159,9 @@ Release 2.9.0 - UNRELEASED
     YARN-4613. Fix test failure in TestClientRMService#testGetClusterNodes.
     (Takashi Ohnishi via rohithsharmaks)
 
+    YARN-4612. Fix rumen and scheduler load simulator handle killed tasks properly.
+    (Ming Ma via xgong)
+
 Release 2.8.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES


[08/50] hadoop git commit: HDFS-9653. Added blocks pending deletion report to dfsadmin. (Weiwei Yang via eyang)

Posted by ar...@apache.org.
HDFS-9653.  Added blocks pending deletion report to dfsadmin.
(Weiwei Yang via eyang)


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

Branch: refs/heads/HDFS-1312
Commit: 10a2bc0dffaece216eb9a6bac3236a086b9ece31
Parents: 10dc2c0
Author: Eric Yang <ey...@apache.org>
Authored: Sun Jan 24 14:19:49 2016 -0800
Committer: Eric Yang <ey...@apache.org>
Committed: Sun Jan 24 14:19:49 2016 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/DFSClient.java  | 34 ++++++++++++++------
 .../hadoop/hdfs/DistributedFileSystem.java      |  9 ++++++
 .../hadoop/hdfs/protocol/ClientProtocol.java    |  4 ++-
 .../hadoop/hdfs/protocolPB/PBHelperClient.java  |  7 ++++
 .../src/main/proto/ClientNamenodeProtocol.proto |  1 +
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../blockmanagement/HeartbeatManager.java       |  1 +
 .../hdfs/server/namenode/FSNamesystem.java      |  2 ++
 .../org/apache/hadoop/hdfs/tools/DFSAdmin.java  |  2 ++
 .../TestPendingInvalidateBlock.java             | 19 +++++++++++
 10 files changed, 71 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/10a2bc0d/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index 277990f..02ef47e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -1957,10 +1957,11 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     }
   }
 
-  private long[] callGetStats() throws IOException {
+  private long getStateByIndex(int stateIndex) throws IOException {
     checkOpen();
     try (TraceScope ignored = tracer.newScope("getStats")) {
-      return namenode.getStats();
+      long[] states =  namenode.getStats();
+      return states.length > stateIndex ? states[stateIndex] : -1;
     }
   }
 
@@ -1968,8 +1969,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @see ClientProtocol#getStats()
    */
   public FsStatus getDiskStatus() throws IOException {
-    long rawNums[] = callGetStats();
-    return new FsStatus(rawNums[0], rawNums[1], rawNums[2]);
+    return new FsStatus(getStateByIndex(0),
+        getStateByIndex(1), getStateByIndex(2));
   }
 
   /**
@@ -1978,7 +1979,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @throws IOException
    */
   public long getMissingBlocksCount() throws IOException {
-    return callGetStats()[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX];
+    return getStateByIndex(ClientProtocol.
+        GET_STATS_MISSING_BLOCKS_IDX);
   }
 
   /**
@@ -1987,8 +1989,17 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @throws IOException
    */
   public long getMissingReplOneBlocksCount() throws IOException {
-    return callGetStats()[ClientProtocol.
-        GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX];
+    return getStateByIndex(ClientProtocol.
+        GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX);
+  }
+
+  /**
+   * Returns count of blocks pending on deletion.
+   * @throws IOException
+   */
+  public long getPendingDeletionBlocksCount() throws IOException {
+    return getStateByIndex(ClientProtocol.
+        GET_STATS_PENDING_DELETION_BLOCKS_IDX);
   }
 
   /**
@@ -1996,7 +2007,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @throws IOException
    */
   public long getUnderReplicatedBlocksCount() throws IOException {
-    return callGetStats()[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX];
+    return getStateByIndex(ClientProtocol.
+        GET_STATS_UNDER_REPLICATED_IDX);
   }
 
   /**
@@ -2004,7 +2016,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @throws IOException
    */
   public long getCorruptBlocksCount() throws IOException {
-    return callGetStats()[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX];
+    return getStateByIndex(ClientProtocol.
+        GET_STATS_CORRUPT_BLOCKS_IDX);
   }
 
   /**
@@ -2014,7 +2027,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @throws IOException
    */
   public long getBytesInFutureBlocks() throws IOException {
-    return callGetStats()[ClientProtocol.GET_STATS_BYTES_IN_FUTURE_BLOCKS_IDX];
+    return getStateByIndex(ClientProtocol.
+        GET_STATS_BYTES_IN_FUTURE_BLOCKS_IDX);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10a2bc0d/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
index 551e2e2..d042a53 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
@@ -1115,6 +1115,15 @@ public class DistributedFileSystem extends FileSystem {
   }
 
   /**
+   * Returns count of blocks pending on deletion.
+   *
+   * @throws IOException
+   */
+  public long getPendingDeletionBlocksCount() throws IOException {
+    return dfs.getPendingDeletionBlocksCount();
+  }
+
+  /**
    * Returns count of blocks with replication factor 1 and have
    * lost the only replica.
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10a2bc0d/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 e6a99fe..3e82eb3 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
@@ -714,7 +714,8 @@ public interface ClientProtocol {
   int GET_STATS_MISSING_BLOCKS_IDX = 5;
   int GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX = 6;
   int GET_STATS_BYTES_IN_FUTURE_BLOCKS_IDX = 7;
-  int STATS_ARRAY_LENGTH = 8;
+  int GET_STATS_PENDING_DELETION_BLOCKS_IDX = 8;
+  int STATS_ARRAY_LENGTH = 9;
 
   /**
    * Get a set of statistics about the filesystem.
@@ -729,6 +730,7 @@ public interface ClientProtocol {
    * <li> [6] contains number of blocks which have replication factor
    *          1 and have lost the only replica. </li>
    * <li> [7] contains number of bytes  that are at risk for deletion. </li>
+   * <li> [8] contains number of pending deletion blocks. </li>
    * </ul>
    * Use public constants like {@link #GET_STATS_CAPACITY_IDX} in place of
    * actual numbers to index into the array.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10a2bc0d/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 1e48a69..6989cb9 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
@@ -1562,6 +1562,8 @@ public class PBHelperClient {
         res.getMissingReplOneBlocks();
     result[ClientProtocol.GET_STATS_BYTES_IN_FUTURE_BLOCKS_IDX] =
         res.hasBlocksInFuture() ? res.getBlocksInFuture() : 0;
+    result[ClientProtocol.GET_STATS_PENDING_DELETION_BLOCKS_IDX] =
+        res.getPendingDeletionBlocks();
     return result;
   }
 
@@ -1931,6 +1933,11 @@ public class PBHelperClient {
       result.setBlocksInFuture(
           fsStats[ClientProtocol.GET_STATS_BYTES_IN_FUTURE_BLOCKS_IDX]);
     }
+    if (fsStats.length >=
+        ClientProtocol.GET_STATS_PENDING_DELETION_BLOCKS_IDX + 1) {
+      result.setPendingDeletionBlocks(
+          fsStats[ClientProtocol.GET_STATS_PENDING_DELETION_BLOCKS_IDX]);
+    }
     return result.build();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10a2bc0d/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 6a1f777..88f63d0 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
@@ -307,6 +307,7 @@ message GetFsStatsResponseProto {
   required uint64 missing_blocks = 6;
   optional uint64 missing_repl_one_blocks = 7;
   optional uint64 blocks_in_future = 8;
+  optional uint64 pending_deletion_blocks = 9;
 }
 
 enum DatanodeReportTypeProto {  // type of the datanode report

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10a2bc0d/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 74b8aac..49b4d8a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -60,6 +60,9 @@ Trunk (Unreleased)
 
   IMPROVEMENTS
 
+    HDFS-9653.  Added blocks pending deletion report to dfsadmin.  
+    (Weiwei Yang via eyang)
+
     HDFS-4665. Move TestNetworkTopologyWithNodeGroup to common.
     (Junping Du via llu)
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10a2bc0d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java
index 7546b1a..b8d3043 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java
@@ -184,6 +184,7 @@ class HeartbeatManager implements DatanodeStatistics {
                        -1L,
                        -1L,
                        -1L,
+                       -1L,
                        -1L};
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10a2bc0d/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 abc0614..afa41c1 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
@@ -3809,6 +3809,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         getMissingReplOneBlocksCount();
     stats[ClientProtocol.GET_STATS_BYTES_IN_FUTURE_BLOCKS_IDX] =
         blockManager.getBytesInFuture();
+    stats[ClientProtocol.GET_STATS_PENDING_DELETION_BLOCKS_IDX] =
+        blockManager.getPendingDeletionBlocksCount();
     return stats;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10a2bc0d/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 3b40608..5da3bc5 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
@@ -502,6 +502,8 @@ public class DFSAdmin extends FsShell {
                        dfs.getMissingBlocksCount());
     System.out.println("Missing blocks (with replication factor 1): " +
                       dfs.getMissingReplOneBlocksCount());
+    System.out.println("Pending deletion blocks: " +
+        dfs.getPendingDeletionBlocksCount());
 
     System.out.println();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10a2bc0d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingInvalidateBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingInvalidateBlock.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingInvalidateBlock.java
index a588a73..7500544 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingInvalidateBlock.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingInvalidateBlock.java
@@ -17,16 +17,19 @@
  */
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
+import java.lang.reflect.Method;
 import java.text.SimpleDateFormat;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.log4j.Level;
@@ -88,15 +91,31 @@ public class TestPendingInvalidateBlock {
     Assert.assertEquals(0, cluster.getNamesystem().getBlocksTotal());
     Assert.assertEquals(REPLICATION, cluster.getNamesystem()
         .getPendingDeletionBlocks());
+    Assert.assertEquals(REPLICATION,
+        dfs.getPendingDeletionBlocksCount());
     Thread.sleep(6000);
     Assert.assertEquals(0, cluster.getNamesystem().getBlocksTotal());
     Assert.assertEquals(0, cluster.getNamesystem().getPendingDeletionBlocks());
+    Assert.assertEquals(0, dfs.getPendingDeletionBlocksCount());
     long nnStarted = cluster.getNamesystem().getNNStartedTimeInMillis();
     long blockDeletionStartTime = cluster.getNamesystem()
         .getBlockDeletionStartTime();
     Assert.assertTrue(String.format(
         "Expect blockDeletionStartTime = %d > nnStarted = %d.",
         blockDeletionStartTime, nnStarted), blockDeletionStartTime > nnStarted);
+
+    // test client protocol compatibility
+    Method method = DFSClient.class.
+        getDeclaredMethod("getStateByIndex", int.class);
+    method.setAccessible(true);
+    // get number of pending deletion blocks by its index
+    long validState = (Long) method.invoke(dfs.getClient(),
+        ClientProtocol.GET_STATS_PENDING_DELETION_BLOCKS_IDX);
+    // get an out of index value
+    long invalidState = (Long) method.invoke(dfs.getClient(),
+        ClientProtocol.STATS_ARRAY_LENGTH);
+    Assert.assertEquals(0, validState);
+    Assert.assertEquals(-1, invalidState);
   }
 
   /**


[35/50] hadoop git commit: MAPREDUCE-6595. Fix findbugs warnings in OutputCommitter and FileOutputCommitter. Contributed by Akira AJISAKA.

Posted by ar...@apache.org.
MAPREDUCE-6595. Fix findbugs warnings in OutputCommitter and FileOutputCommitter. Contributed by Akira AJISAKA.


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

Branch: refs/heads/HDFS-1312
Commit: 86560a4978c498bd5e892c509a466807a812b8a7
Parents: 662e17b
Author: Junping Du <ju...@apache.org>
Authored: Thu Jan 28 06:14:21 2016 -0800
Committer: Junping Du <ju...@apache.org>
Committed: Thu Jan 28 06:14:21 2016 -0800

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                      | 3 +++
 hadoop-mapreduce-project/dev-support/findbugs-exclude.xml | 2 ++
 2 files changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/86560a49/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 3f85a9b..a8abdb4 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -462,6 +462,9 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6584. Remove trailing whitespaces from mapred-default.xml.
     (aajisaka)
 
+    MAPREDUCE-6595. Fix findbugs warnings in OutputCommitter and 
+    FileOutputCommitter. (Akira AJISAKA via junping_du)
+
   OPTIMIZATIONS
 
     MAPREDUCE-6376. Add avro binary support for jhist files (Ray Chiang via

http://git-wip-us.apache.org/repos/asf/hadoop/blob/86560a49/hadoop-mapreduce-project/dev-support/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/dev-support/findbugs-exclude.xml b/hadoop-mapreduce-project/dev-support/findbugs-exclude.xml
index f1ef2b8..9b4d8c9 100644
--- a/hadoop-mapreduce-project/dev-support/findbugs-exclude.xml
+++ b/hadoop-mapreduce-project/dev-support/findbugs-exclude.xml
@@ -159,6 +159,7 @@
        <Method name="commitJob" />
        <Method name="recoverTask" />
        <Method name="isRecoverySupported" />
+       <Method name="isCommitJobRepeatable" />
        </Or>
        <Bug pattern="NM_WRONG_PACKAGE" />
      </Match>
@@ -170,6 +171,7 @@
        <Method name="cleanupJob" />
        <Method name="recoverTask" />
        <Method name="isRecoverySupported" />
+       <Method name="isCommitJobRepeatable" />
        </Or>
        <Bug pattern="NM_WRONG_PACKAGE_INTENTIONAL" />
      </Match>


[27/50] hadoop git commit: HADOOP-12492. maven install triggers bats test (aw)

Posted by ar...@apache.org.
HADOOP-12492. maven install triggers bats test (aw)


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

Branch: refs/heads/HDFS-1312
Commit: 79d7949fbb19928e0cae4f6b5dd9f1af82242f53
Parents: 97056c3
Author: Allen Wittenauer <aw...@apache.org>
Authored: Wed Jan 27 11:27:27 2016 -0800
Committer: Allen Wittenauer <aw...@apache.org>
Committed: Wed Jan 27 11:27:27 2016 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/pom.xml | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/79d7949f/hadoop-common-project/hadoop-common/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/pom.xml b/hadoop-common-project/hadoop-common/pom.xml
index 68ad350..7e4d090 100644
--- a/hadoop-common-project/hadoop-common/pom.xml
+++ b/hadoop-common-project/hadoop-common/pom.xml
@@ -1028,7 +1028,9 @@
     <profile>
       <id>shelltest</id>
       <activation>
-        <activeByDefault>true</activeByDefault>
+        <property>
+          <name>!skipTests</name>
+        </property>
       </activation>
       <build>
         <plugins>
@@ -1037,7 +1039,7 @@
             <executions>
                 <execution>
                     <id>common-test-bats-driver</id>
-                    <phase>process-test-classes</phase>
+                    <phase>test</phase>
                     <goals>
                         <goal>run</goal>
                     </goals>


[28/50] hadoop git commit: HADOOP-12735. core-default.xml misspells hadoop.workaround.non.threadsafe.getpwuid (Ray Chiang via cmccabe)

Posted by ar...@apache.org.
HADOOP-12735. core-default.xml misspells hadoop.workaround.non.threadsafe.getpwuid (Ray Chiang via cmccabe)


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

Branch: refs/heads/HDFS-1312
Commit: 2e8ab3d46568162af6aa90b612ed61d487e7c7b0
Parents: 79d7949
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Wed Jan 27 11:39:55 2016 -0800
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Wed Jan 27 11:39:55 2016 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt                   | 3 +++
 .../hadoop-common/src/main/resources/core-default.xml             | 2 +-
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e8ab3d4/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 3b8376f..4da20e0 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -1651,6 +1651,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-12718. Incorrect error message by fs -put local dir without
     permission. (John Zhuge via Yongjun Zhang)
 
+    HADOOP-12735. core-default.xml misspells
+    hadoop.workaround.non.threadsafe.getpwuid (Ray Chiang via cmccabe)
+
 Release 2.7.3 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e8ab3d4/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 c25f49e..ed3802f 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
@@ -428,7 +428,7 @@ for ldap providers in the same way as above does.
 </property>
 
 <property>
-  <name>hadoop.work.around.non.threadsafe.getpwuid</name>
+  <name>hadoop.workaround.non.threadsafe.getpwuid</name>
   <value>false</value>
   <description>Some operating systems or authentication modules are known to
   have broken implementations of getpwuid_r and getpwgid_r, such that these


[30/50] hadoop git commit: HDFS-9677. Rename generationStampV1/generationStampV2 to legacyGenerationStamp/generationStamp. Contributed by Mingliang Liu.

Posted by ar...@apache.org.
HDFS-9677. Rename generationStampV1/generationStampV2 to legacyGenerationStamp/generationStamp. Contributed by Mingliang Liu.


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

Branch: refs/heads/HDFS-1312
Commit: 8a91109d16394310f2568717f103e6fff7cbddb0
Parents: fb238d7
Author: Jing Zhao <ji...@apache.org>
Authored: Wed Jan 27 15:48:47 2016 -0800
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Jan 27 15:48:47 2016 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +
 .../server/blockmanagement/BlockIdManager.java  | 83 ++++++++++----------
 .../server/blockmanagement/BlockManager.java    |  8 +-
 .../OutOfLegacyGenerationStampsException.java   | 38 +++++++++
 .../OutOfV1GenerationStampsException.java       | 38 ---------
 .../hdfs/server/common/HdfsServerConstants.java |  3 +-
 .../hadoop/hdfs/server/namenode/FSEditLog.java  |  4 +-
 .../hdfs/server/namenode/FSEditLogLoader.java   |  4 +-
 .../hdfs/server/namenode/FSImageFormat.java     | 12 +--
 .../server/namenode/FSImageFormatProtobuf.java  | 12 +--
 .../hdfs/server/namenode/FSNamesystem.java      |  8 +-
 .../hadoop/hdfs/server/namenode/Namesystem.java | 13 +--
 .../hadoop-hdfs/src/main/proto/fsimage.proto    |  4 +-
 .../blockmanagement/TestSequentialBlockId.java  | 18 ++---
 .../hdfs/server/namenode/TestEditLog.java       |  6 +-
 .../hdfs/server/namenode/TestFileTruncate.java  |  4 +-
 .../hdfs/server/namenode/TestSaveNamespace.java |  2 +-
 17 files changed, 126 insertions(+), 134 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a91109d/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 097c051..7e75558 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -959,6 +959,9 @@ Release 2.9.0 - UNRELEASED
     HDFS-9541. Add hdfsStreamBuilder API to libhdfs to support defaultBlockSizes
     greater than 2 GB. (cmccabe via zhz)
 
+    HDFS-9677. Rename generationStampV1/generationStampV2 to
+    legacyGenerationStamp/generationStamp. (Mingliang Liu via jing9)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a91109d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
index 9c71287..3f21d9b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
@@ -36,11 +36,11 @@ public class BlockIdManager {
    * The global generation stamp for legacy blocks with randomly
    * generated block IDs.
    */
-  private final GenerationStamp generationStampV1 = new GenerationStamp();
+  private final GenerationStamp legacyGenerationStamp = new GenerationStamp();
   /**
    * The global generation stamp for this file system.
    */
-  private final GenerationStamp generationStampV2 = new GenerationStamp();
+  private final GenerationStamp generationStamp = new GenerationStamp();
   /**
    * The value of the generation stamp when the first switch to sequential
    * block IDs was made. Blocks with generation stamps below this value
@@ -49,7 +49,7 @@ public class BlockIdManager {
    * (or initialized as an offset from the V1 (legacy) generation stamp on
    * upgrade).
    */
-  private long generationStampV1Limit;
+  private long legacyGenerationStampLimit;
   /**
    * The global block ID space for this file system.
    */
@@ -57,7 +57,8 @@ public class BlockIdManager {
   private final SequentialBlockGroupIdGenerator blockGroupIdGenerator;
 
   public BlockIdManager(BlockManager blockManager) {
-    this.generationStampV1Limit = HdfsConstants.GRANDFATHER_GENERATION_STAMP;
+    this.legacyGenerationStampLimit =
+        HdfsConstants.GRANDFATHER_GENERATION_STAMP;
     this.blockIdGenerator = new SequentialBlockIdGenerator(blockManager);
     this.blockGroupIdGenerator = new SequentialBlockGroupIdGenerator(blockManager);
   }
@@ -68,14 +69,14 @@ public class BlockIdManager {
    * Should be invoked only during the first upgrade to
    * sequential block IDs.
    */
-  public long upgradeGenerationStampToV2() {
-    Preconditions.checkState(generationStampV2.getCurrentValue() ==
+  public long upgradeLegacyGenerationStamp() {
+    Preconditions.checkState(generationStamp.getCurrentValue() ==
       GenerationStamp.LAST_RESERVED_STAMP);
-    generationStampV2.skipTo(generationStampV1.getCurrentValue() +
-      HdfsServerConstants.RESERVED_GENERATION_STAMPS_V1);
+    generationStamp.skipTo(legacyGenerationStamp.getCurrentValue() +
+      HdfsServerConstants.RESERVED_LEGACY_GENERATION_STAMPS);
 
-    generationStampV1Limit = generationStampV2.getCurrentValue();
-    return generationStampV2.getCurrentValue();
+    legacyGenerationStampLimit = generationStamp.getCurrentValue();
+    return generationStamp.getCurrentValue();
   }
 
   /**
@@ -84,10 +85,10 @@ public class BlockIdManager {
    *
    * @param stamp set generation stamp limit to this value
    */
-  public void setGenerationStampV1Limit(long stamp) {
-    Preconditions.checkState(generationStampV1Limit == HdfsConstants
-      .GRANDFATHER_GENERATION_STAMP);
-    generationStampV1Limit = stamp;
+  public void setLegacyGenerationStampLimit(long stamp) {
+    Preconditions.checkState(legacyGenerationStampLimit ==
+        HdfsConstants.GRANDFATHER_GENERATION_STAMP);
+    legacyGenerationStampLimit = stamp;
   }
 
   /**
@@ -95,7 +96,7 @@ public class BlockIdManager {
    * and random block IDs.
    */
   public long getGenerationStampAtblockIdSwitch() {
-    return generationStampV1Limit;
+    return legacyGenerationStampLimit;
   }
 
   @VisibleForTesting
@@ -138,58 +139,58 @@ public class BlockIdManager {
   /**
    * Sets the current generation stamp for legacy blocks
    */
-  public void setGenerationStampV1(long stamp) {
-    generationStampV1.setCurrentValue(stamp);
+  public void setLegacyGenerationStamp(long stamp) {
+    legacyGenerationStamp.setCurrentValue(stamp);
   }
 
   /**
    * Gets the current generation stamp for legacy blocks
    */
-  public long getGenerationStampV1() {
-    return generationStampV1.getCurrentValue();
+  public long getLegacyGenerationStamp() {
+    return legacyGenerationStamp.getCurrentValue();
   }
 
   /**
    * Gets the current generation stamp for this filesystem
    */
-  public void setGenerationStampV2(long stamp) {
-    generationStampV2.setCurrentValue(stamp);
+  public void setGenerationStamp(long stamp) {
+    generationStamp.setCurrentValue(stamp);
   }
 
-  public long getGenerationStampV2() {
-    return generationStampV2.getCurrentValue();
+  public long getGenerationStamp() {
+    return generationStamp.getCurrentValue();
   }
 
   /**
    * Increments, logs and then returns the stamp
    */
   long nextGenerationStamp(boolean legacyBlock) throws IOException {
-    return legacyBlock ? getNextGenerationStampV1() :
-      getNextGenerationStampV2();
+    return legacyBlock ? getNextLegacyGenerationStamp() :
+        getNextGenerationStamp();
   }
 
   @VisibleForTesting
-  long getNextGenerationStampV1() throws IOException {
-    long genStampV1 = generationStampV1.nextValue();
+  long getNextLegacyGenerationStamp() throws IOException {
+    long legacyGenStamp = legacyGenerationStamp.nextValue();
 
-    if (genStampV1 >= generationStampV1Limit) {
+    if (legacyGenStamp >= legacyGenerationStampLimit) {
       // We ran out of generation stamps for legacy blocks. In practice, it
-      // is extremely unlikely as we reserved 1T v1 generation stamps. The
+      // is extremely unlikely as we reserved 1T legacy generation stamps. The
       // result is that we can no longer append to the legacy blocks that
       // were created before the upgrade to sequential block IDs.
-      throw new OutOfV1GenerationStampsException();
+      throw new OutOfLegacyGenerationStampsException();
     }
 
-    return genStampV1;
+    return legacyGenStamp;
   }
 
   @VisibleForTesting
-  long getNextGenerationStampV2() {
-    return generationStampV2.nextValue();
+  long getNextGenerationStamp() {
+    return generationStamp.nextValue();
   }
 
-  public long getGenerationStampV1Limit() {
-    return generationStampV1Limit;
+  public long getLegacyGenerationStampLimit() {
+    return legacyGenerationStampLimit;
   }
 
   /**
@@ -200,7 +201,7 @@ public class BlockIdManager {
    * @return true if the block ID was randomly generated, false otherwise.
    */
   boolean isLegacyBlock(Block block) {
-    return block.getGenerationStamp() < getGenerationStampV1Limit();
+    return block.getGenerationStamp() < getLegacyGenerationStampLimit();
   }
 
   /**
@@ -213,18 +214,18 @@ public class BlockIdManager {
 
   boolean isGenStampInFuture(Block block) {
     if (isLegacyBlock(block)) {
-      return block.getGenerationStamp() > getGenerationStampV1();
+      return block.getGenerationStamp() > getLegacyGenerationStamp();
     } else {
-      return block.getGenerationStamp() > getGenerationStampV2();
+      return block.getGenerationStamp() > getGenerationStamp();
     }
   }
 
   void clear() {
-    generationStampV1.setCurrentValue(GenerationStamp.LAST_RESERVED_STAMP);
-    generationStampV2.setCurrentValue(GenerationStamp.LAST_RESERVED_STAMP);
+    legacyGenerationStamp.setCurrentValue(GenerationStamp.LAST_RESERVED_STAMP);
+    generationStamp.setCurrentValue(GenerationStamp.LAST_RESERVED_STAMP);
     getBlockIdGenerator().setCurrentValue(SequentialBlockIdGenerator
       .LAST_RESERVED_BLOCK_ID);
-    generationStampV1Limit = HdfsConstants.GRANDFATHER_GENERATION_STAMP;
+    legacyGenerationStampLimit = HdfsConstants.GRANDFATHER_GENERATION_STAMP;
   }
 
   public static boolean isStripedBlockID(long id) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a91109d/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 a76429e..6215437 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
@@ -927,13 +927,13 @@ public class BlockManager implements BlockStatsMXBean {
       final BlockUnderConstructionFeature uc = blk.getUnderConstructionFeature();
       if (blk.isStriped()) {
         final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
-        final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(),
+        final ExtendedBlock eb = new ExtendedBlock(getBlockPoolId(),
             blk);
         return newLocatedStripedBlock(eb, storages, uc.getBlockIndices(), pos,
             false);
       } else {
         final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
-        final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(),
+        final ExtendedBlock eb = new ExtendedBlock(getBlockPoolId(),
             blk);
         return newLocatedBlock(eb, storages, pos, false);
       }
@@ -976,7 +976,7 @@ public class BlockManager implements BlockStatsMXBean {
       " numNodes: " + numNodes +
       " numCorrupt: " + numCorruptNodes +
       " numCorruptRepls: " + numCorruptReplicas;
-    final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk);
+    final ExtendedBlock eb = new ExtendedBlock(getBlockPoolId(), blk);
     return blockIndices == null ?
         newLocatedBlock(eb, machines, pos, isCorrupt) :
         newLocatedStripedBlock(eb, machines, blockIndices, pos, isCorrupt);
@@ -1653,7 +1653,7 @@ public class BlockManager implements BlockStatsMXBean {
         return false;
       }
       rw.getTargets()[0].getDatanodeDescriptor().addBlockToBeErasureCoded(
-          new ExtendedBlock(namesystem.getBlockPoolId(), block),
+          new ExtendedBlock(getBlockPoolId(), block),
           rw.getSrcNodes(), rw.getTargets(),
           ((ErasureCodingWork) rw).getLiveBlockIndicies(), ecPolicy);
     } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a91109d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/OutOfLegacyGenerationStampsException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/OutOfLegacyGenerationStampsException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/OutOfLegacyGenerationStampsException.java
new file mode 100644
index 0000000..f81392f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/OutOfLegacyGenerationStampsException.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.server.blockmanagement;
+
+import java.io.IOException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * This exception is thrown when the name node runs out of V1 (legacy)
+ * generation stamps.
+ *
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class OutOfLegacyGenerationStampsException extends IOException {
+  private static final long serialVersionUID = 1L;
+
+  public OutOfLegacyGenerationStampsException() {
+    super("Out of V1 (legacy) generation stamps\n");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a91109d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/OutOfV1GenerationStampsException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/OutOfV1GenerationStampsException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/OutOfV1GenerationStampsException.java
deleted file mode 100644
index 127ad54..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/OutOfV1GenerationStampsException.java
+++ /dev/null
@@ -1,38 +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.server.blockmanagement;
-
-import java.io.IOException;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * This exception is thrown when the name node runs out of V1 generation
- * stamps.
- *
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class OutOfV1GenerationStampsException extends IOException {
-  private static final long serialVersionUID = 1L;
-
-  public OutOfV1GenerationStampsException() {
-    super("Out of V1 (legacy) generation stamps\n");
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a91109d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
index 13c9137..b2dda3c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
@@ -61,8 +61,7 @@ public interface HdfsServerConstants {
   // An invalid transaction ID that will never be seen in a real namesystem.
   long INVALID_TXID = -12345;
   // Number of generation stamps reserved for legacy blocks.
-  long RESERVED_GENERATION_STAMPS_V1 =
-      1024L * 1024 * 1024 * 1024;
+  long RESERVED_LEGACY_GENERATION_STAMPS = 1024L * 1024 * 1024 * 1024;
   /**
    * Current layout version for NameNode.
    * Please see {@link NameNodeLayoutVersion.Feature} on adding new layout version.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a91109d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
index e255cff..c8986dc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
@@ -960,7 +960,7 @@ public class FSEditLog implements LogsPurgeable {
   /**
    * Add legacy block generation stamp record to edit log
    */
-  void logGenerationStampV1(long genstamp) {
+  void logLegacyGenerationStamp(long genstamp) {
     SetGenstampV1Op op = SetGenstampV1Op.getInstance(cache.get())
         .setGenerationStamp(genstamp);
     logEdit(op);
@@ -969,7 +969,7 @@ public class FSEditLog implements LogsPurgeable {
   /**
    * Add generation stamp record to edit log
    */
-  void logGenerationStampV2(long genstamp) {
+  void logGenerationStamp(long genstamp) {
     SetGenstampV2Op op = SetGenstampV2Op.getInstance(cache.get())
         .setGenerationStamp(genstamp);
     logEdit(op);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a91109d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
index 1a9d7a9..25f5a4f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
@@ -589,7 +589,7 @@ public class FSEditLogLoader {
     }
     case OP_SET_GENSTAMP_V1: {
       SetGenstampV1Op setGenstampV1Op = (SetGenstampV1Op)op;
-      blockManager.getBlockIdManager().setGenerationStampV1(
+      blockManager.getBlockIdManager().setLegacyGenerationStamp(
           setGenstampV1Op.genStampV1);
       break;
     }
@@ -797,7 +797,7 @@ public class FSEditLogLoader {
     }
     case OP_SET_GENSTAMP_V2: {
       SetGenstampV2Op setGenstampV2Op = (SetGenstampV2Op) op;
-      blockManager.getBlockIdManager().setGenerationStampV2(
+      blockManager.getBlockIdManager().setGenerationStamp(
           setGenstampV2Op.genStampV2);
       break;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a91109d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
index a815864..e4263bd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
@@ -346,24 +346,24 @@ public class FSImageFormat {
         long genstamp = in.readLong();
         final BlockIdManager blockIdManager = namesystem.getBlockManager()
             .getBlockIdManager();
-        blockIdManager.setGenerationStampV1(genstamp);
+        blockIdManager.setLegacyGenerationStamp(genstamp);
 
         if (NameNodeLayoutVersion.supports(
             LayoutVersion.Feature.SEQUENTIAL_BLOCK_ID, imgVersion)) {
           // read the starting generation stamp for sequential block IDs
           genstamp = in.readLong();
-          blockIdManager.setGenerationStampV2(genstamp);
+          blockIdManager.setGenerationStamp(genstamp);
 
           // read the last generation stamp for blocks created after
           // the switch to sequential block IDs.
           long stampAtIdSwitch = in.readLong();
-          blockIdManager.setGenerationStampV1Limit(stampAtIdSwitch);
+          blockIdManager.setLegacyGenerationStampLimit(stampAtIdSwitch);
 
           // read the max sequential block ID.
           long maxSequentialBlockId = in.readLong();
           blockIdManager.setLastAllocatedContiguousBlockId(maxSequentialBlockId);
         } else {
-          long startingGenStamp = blockIdManager.upgradeGenerationStampToV2();
+          long startingGenStamp = blockIdManager.upgradeLegacyGenerationStamp();
           // This is an upgrade.
           LOG.info("Upgrading to sequential block IDs. Generation stamp " +
                    "for new blocks set to " + startingGenStamp);
@@ -1270,8 +1270,8 @@ public class FSImageFormat {
         out.writeLong(numINodes);
         final BlockIdManager blockIdManager = sourceNamesystem.getBlockManager()
             .getBlockIdManager();
-        out.writeLong(blockIdManager.getGenerationStampV1());
-        out.writeLong(blockIdManager.getGenerationStampV2());
+        out.writeLong(blockIdManager.getLegacyGenerationStamp());
+        out.writeLong(blockIdManager.getGenerationStamp());
         out.writeLong(blockIdManager.getGenerationStampAtblockIdSwitch());
         out.writeLong(blockIdManager.getLastAllocatedContiguousBlockId());
         out.writeLong(context.getTxId());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a91109d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
index 635dc34..e9bd8c5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
@@ -294,9 +294,9 @@ public final class FSImageFormatProtobuf {
     private void loadNameSystemSection(InputStream in) throws IOException {
       NameSystemSection s = NameSystemSection.parseDelimitedFrom(in);
       BlockIdManager blockIdManager = fsn.getBlockManager().getBlockIdManager();
-      blockIdManager.setGenerationStampV1(s.getGenstampV1());
-      blockIdManager.setGenerationStampV2(s.getGenstampV2());
-      blockIdManager.setGenerationStampV1Limit(s.getGenstampV1Limit());
+      blockIdManager.setLegacyGenerationStamp(s.getGenstampV1());
+      blockIdManager.setGenerationStamp(s.getGenstampV2());
+      blockIdManager.setLegacyGenerationStampLimit(s.getGenstampV1Limit());
       blockIdManager.setLastAllocatedContiguousBlockId(s.getLastAllocatedBlockId());
       if (s.hasLastAllocatedStripedBlockId()) {
         blockIdManager.setLastAllocatedStripedBlockId(
@@ -550,9 +550,9 @@ public final class FSImageFormatProtobuf {
       OutputStream out = sectionOutputStream;
       BlockIdManager blockIdManager = fsn.getBlockManager().getBlockIdManager();
       NameSystemSection.Builder b = NameSystemSection.newBuilder()
-          .setGenstampV1(blockIdManager.getGenerationStampV1())
-          .setGenstampV1Limit(blockIdManager.getGenerationStampV1Limit())
-          .setGenstampV2(blockIdManager.getGenerationStampV2())
+          .setGenstampV1(blockIdManager.getLegacyGenerationStamp())
+          .setGenstampV1Limit(blockIdManager.getLegacyGenerationStampLimit())
+          .setGenstampV2(blockIdManager.getGenerationStamp())
           .setLastAllocatedBlockId(blockIdManager.getLastAllocatedContiguousBlockId())
           .setLastAllocatedStripedBlockId(blockIdManager.getLastAllocatedStripedBlockId())
           .setTransactionId(context.getTxId());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a91109d/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 1693958..166526b 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
@@ -4307,9 +4307,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return new PermissionStatus(fsOwner.getShortUserName(), supergroup, permission);
   }
 
-  @Override
-  public void checkSuperuserPrivilege()
-      throws AccessControlException {
+  void checkSuperuserPrivilege() throws AccessControlException {
     if (isPermissionEnabled) {
       FSPermissionChecker pc = getPermissionChecker();
       pc.checkSuperuserPrivilege();
@@ -4575,9 +4573,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
     long gs = blockManager.nextGenerationStamp(legacyBlock);
     if (legacyBlock) {
-      getEditLog().logGenerationStampV1(gs);
+      getEditLog().logLegacyGenerationStamp(gs);
     } else {
-      getEditLog().logGenerationStampV2(gs);
+      getEditLog().logGenerationStamp(gs);
     }
 
     // NB: callers sync the log

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a91109d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
index 5a9e69b..95d7c20 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
@@ -35,22 +35,13 @@ public interface Namesystem extends RwLock, SafeMode {
   /** Is this name system running? */
   boolean isRunning();
 
-  /** Check if the user has superuser privilege. */
-  void checkSuperuserPrivilege() throws AccessControlException;
-
-  /** @return the block pool ID */
-  String getBlockPoolId();
-
   BlockCollection getBlockCollection(long id);
 
   void startSecretManagerIfNecessary();
 
   /**
-   * Gets the erasure coding policy for the path
-   * @param src
-   *          - path
-   * @return {@link ErasureCodingPolicy}
-   * @throws IOException
+   * @param src file/directory path
+   * @return The {@link ErasureCodingPolicy} for the given file/directory path
    */
   ErasureCodingPolicy getErasureCodingPolicyForPath(String src)
       throws IOException;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a91109d/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
index 43a1fc5..e57efee 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
@@ -67,8 +67,8 @@ message FileSummary {
  */
 message NameSystemSection {
   optional uint32 namespaceId = 1;
-  optional uint64 genstampV1 = 2;
-  optional uint64 genstampV2 = 3;
+  optional uint64 genstampV1 = 2; // legacy generation stamp
+  optional uint64 genstampV2 = 3; // generation stamp of latest version
   optional uint64 genstampV1Limit = 4;
   optional uint64 lastAllocatedBlockId = 5;
   optional uint64 transactionId = 6;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a91109d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockId.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockId.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockId.java
index e612ea9..bfda393 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockId.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockId.java
@@ -151,7 +151,7 @@ public class TestSequentialBlockId {
     BlockIdManager bid = mock(BlockIdManager.class);
     final long maxGenStampForLegacyBlocks = 10000;
 
-    when(bid.getGenerationStampV1Limit())
+    when(bid.getLegacyGenerationStampLimit())
         .thenReturn(maxGenStampForLegacyBlocks);
 
     Block legacyBlock = spy(new Block());
@@ -180,18 +180,18 @@ public class TestSequentialBlockId {
     // Setup a mock object and stub out a few routines to
     // retrieve the generation stamp counters.
     BlockIdManager bid = mock(BlockIdManager.class);
-    final long nextGenerationStampV1 = 5000;
-    final long nextGenerationStampV2 = 20000;
+    final long nextLegacyGenerationStamp = 5000;
+    final long nextGenerationStamp = 20000;
 
-    when(bid.getNextGenerationStampV1())
-        .thenReturn(nextGenerationStampV1);
-    when(bid.getNextGenerationStampV2())
-        .thenReturn(nextGenerationStampV2);
+    when(bid.getNextLegacyGenerationStamp())
+        .thenReturn(nextLegacyGenerationStamp);
+    when(bid.getNextGenerationStamp())
+        .thenReturn(nextGenerationStamp);
 
     // Make sure that the generation stamp is set correctly for both
     // kinds of blocks.
     when(bid.nextGenerationStamp(anyBoolean())).thenCallRealMethod();
-    assertThat(bid.nextGenerationStamp(true), is(nextGenerationStampV1));
-    assertThat(bid.nextGenerationStamp(false), is(nextGenerationStampV2));
+    assertThat(bid.nextGenerationStamp(true), is(nextLegacyGenerationStamp));
+    assertThat(bid.nextGenerationStamp(false), is(nextGenerationStamp));
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a91109d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
index 4b72112..14240e0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
@@ -1145,7 +1145,7 @@ public class TestEditLog {
     editlog.initJournalsForWrite();
     editlog.openForWrite(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
     for (int i = 2; i < TXNS_PER_ROLL; i++) {
-      editlog.logGenerationStampV2((long) 0);
+      editlog.logGenerationStamp((long) 0);
     }
     editlog.logSync();
     
@@ -1157,7 +1157,7 @@ public class TestEditLog {
     for (int i = 0; i < numrolls; i++) {
       editlog.rollEditLog(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
       
-      editlog.logGenerationStampV2((long) i);
+      editlog.logGenerationStamp((long) i);
       editlog.logSync();
 
       while (aborts.size() > 0 
@@ -1167,7 +1167,7 @@ public class TestEditLog {
       } 
       
       for (int j = 3; j < TXNS_PER_ROLL; j++) {
-        editlog.logGenerationStampV2((long) i);
+        editlog.logGenerationStamp((long) i);
       }
       editlog.logSync();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a91109d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
index f2c0330..d723525 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
@@ -1020,7 +1020,7 @@ public class TestFileTruncate {
       assertThat(truncateBlock.getNumBytes(),
           is(oldBlock.getNumBytes()));
       assertThat(truncateBlock.getGenerationStamp(),
-          is(fsn.getBlockManager().getBlockIdManager().getGenerationStampV2()));
+          is(fsn.getBlockManager().getBlockIdManager().getGenerationStamp()));
       assertThat(file.getLastBlock().getBlockUCState(),
           is(HdfsServerConstants.BlockUCState.UNDER_RECOVERY));
       long blockRecoveryId = file.getLastBlock().getUnderConstructionFeature()
@@ -1054,7 +1054,7 @@ public class TestFileTruncate {
       assertThat(truncateBlock.getNumBytes() < oldBlock.getNumBytes(),
           is(true));
       assertThat(truncateBlock.getGenerationStamp(),
-          is(fsn.getBlockManager().getBlockIdManager().getGenerationStampV2()));
+          is(fsn.getBlockManager().getBlockIdManager().getGenerationStamp()));
       assertThat(file.getLastBlock().getBlockUCState(),
           is(HdfsServerConstants.BlockUCState.UNDER_RECOVERY));
       long blockRecoveryId = file.getLastBlock().getUnderConstructionFeature()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a91109d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java
index 4d37d61..a03e900 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java
@@ -519,7 +519,7 @@ public class TestSaveNamespace {
     DelayAnswer delayer = new GenericTestUtils.DelayAnswer(LOG);
     BlockIdManager bid = spy(spyFsn.getBlockManager().getBlockIdManager());
     Whitebox.setInternalState(finalFsn.getBlockManager(), "blockIdManager", bid);
-    doAnswer(delayer).when(bid).getGenerationStampV2();
+    doAnswer(delayer).when(bid).getGenerationStamp();
 
     ExecutorService pool = Executors.newFixedThreadPool(2);