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 zh...@apache.org on 2015/01/26 18:44:16 UTC

[01/50] [abbrv] hadoop git commit: HDFS-7643. Test case to ensure lazy persist files cannot be truncated. (Contributed by Yi Liu)

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-EC 26da8acda -> 7ab153e4a


HDFS-7643. Test case to ensure lazy persist files cannot be truncated. (Contributed by Yi Liu)


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

Branch: refs/heads/HDFS-EC
Commit: cab6f9608e43727408a4efe51fb2f0522610ef68
Parents: 4995923
Author: Arpit Agarwal <ar...@apache.org>
Authored: Tue Jan 20 21:58:15 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:25 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +++
 .../fsdataset/impl/TestLazyPersistFiles.java    | 26 +++++++++++++++-----
 2 files changed, 23 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cab6f960/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 2c7fbc7..1801d2f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -281,6 +281,9 @@ Trunk (Unreleased)
     HDFS-7634. Disallow truncation of Lazy persist files. (Yi Liu via
     Arpit Agarwal)
 
+    HDFS-7643. Test case to ensure lazy persist files cannot be truncated.
+    (Yi Liu via Arpit Agarwal)
+
 Release 2.7.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cab6f960/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistFiles.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistFiles.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistFiles.java
index ad2197a..84ac2a5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistFiles.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistFiles.java
@@ -23,13 +23,10 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Assert;
 import org.junit.Test;
 
-import java.io.File;
 import java.io.IOException;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutorService;
@@ -37,9 +34,6 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
 import static org.apache.hadoop.hdfs.StorageType.DEFAULT;
@@ -248,6 +242,26 @@ public class TestLazyPersistFiles extends LazyPersistTestCase {
   }
 
   /**
+   * Truncate to lazy persist file is denied.
+   * @throws IOException
+   */
+  @Test
+  public void testTruncateIsDenied() throws IOException {
+    startUpCluster(true, -1);
+    final String METHOD_NAME = GenericTestUtils.getMethodName();
+    Path path = new Path("/" + METHOD_NAME + ".dat");
+
+    makeTestFile(path, BLOCK_SIZE, true);
+
+    try {
+      client.truncate(path.toString(), BLOCK_SIZE/2);
+      fail("Truncate to LazyPersist file did not fail as expected");
+    } catch (Throwable t) {
+      LOG.info("Got expected exception ", t);
+    }
+  }
+
+  /**
    * If one or more replicas of a lazyPersist file are lost, then the file
    * must be discarded by the NN, instead of being kept around as a
    * 'corrupt' file.


[41/50] [abbrv] hadoop git commit: HDFS-7644. minor typo in HttpFS doc (Charles Lamb via aw)

Posted by zh...@apache.org.
HDFS-7644. minor typo in HttpFS doc (Charles Lamb via aw)


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

Branch: refs/heads/HDFS-EC
Commit: f3c4294bce71116d81e31c808e3e0782c74b455f
Parents: d56e750
Author: Allen Wittenauer <aw...@apache.org>
Authored: Fri Jan 23 13:46:31 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:30 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/apt/index.apt.vm | 2 +-
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                      | 2 ++
 2 files changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f3c4294b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/apt/index.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/apt/index.apt.vm b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/apt/index.apt.vm
index 2920cd9..f51e743 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/apt/index.apt.vm
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/apt/index.apt.vm
@@ -64,7 +64,7 @@ Hadoop HDFS over HTTP - Documentation Sets ${project.version}
 
   HttpFS was inspired by Hadoop HDFS proxy.
 
-  HttpFS can be seening as a full rewrite of Hadoop HDFS proxy.
+  HttpFS can be seen as a full rewrite of Hadoop HDFS proxy.
 
   Hadoop HDFS proxy provides a subset of file system operations (read only),
   HttpFS provides support for all file system operations.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f3c4294b/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 c9bee1a..6849229 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -775,6 +775,8 @@ Release 2.7.0 - UNRELEASED
     causes the fsvolume reference being released incorrectly. (Lei Xu via
     yliu)
 
+    HDFS-7644. minor typo in HttpFS doc (Charles Lamb via aw)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES


[35/50] [abbrv] hadoop git commit: HDFS-7660. BlockReceiver#close() might be called multiple times, which causes the fsvolume reference being released incorrectly. (Lei Xu via yliu)

Posted by zh...@apache.org.
HDFS-7660. BlockReceiver#close() might be called multiple times, which causes the fsvolume reference being released incorrectly. (Lei Xu via yliu)


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

Branch: refs/heads/HDFS-EC
Commit: 8a96c6dee1a68969d31799a30d6f48894a4d13c1
Parents: c265a0e
Author: yliu <yl...@apache.org>
Authored: Fri Jan 23 02:37:44 2015 +0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:29 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                      | 4 ++++
 .../org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java    | 3 ++-
 2 files changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a96c6de/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 74eb160..9176ec7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -768,6 +768,10 @@ Release 2.7.0 - UNRELEASED
     HDFS-3519. Checkpoint upload may interfere with a concurrent saveNamespace.
     (Ming Ma via cnauroth)
 
+    HDFS-7660. BlockReceiver#close() might be called multiple times, which
+    causes the fsvolume reference being released incorrectly. (Lei Xu via
+    yliu)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a96c6de/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 12041a6..3d37df5 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
@@ -123,7 +123,7 @@ class BlockReceiver implements Closeable {
   private boolean syncOnClose;
   private long restartBudget;
   /** the reference of the volume where the block receiver writes to */
-  private final ReplicaHandler replicaHandler;
+  private ReplicaHandler replicaHandler;
 
   /**
    * for replaceBlock response
@@ -334,6 +334,7 @@ class BlockReceiver implements Closeable {
     }
     if (replicaHandler != null) {
       IOUtils.cleanup(null, replicaHandler);
+      replicaHandler = null;
     }
     if (measuredFlushTime) {
       datanode.metrics.addFlushNanos(flushTotalNanos);


[21/50] [abbrv] hadoop git commit: HDFS-7430. Refactor the BlockScanner to use O(1) memory and use multiple threads (cmccabe)

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/df4edd9a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/VolumeScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/VolumeScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/VolumeScanner.java
new file mode 100644
index 0000000..781b4d3
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/VolumeScanner.java
@@ -0,0 +1,652 @@
+/**
+ * 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.datanode;
+
+import java.io.DataOutputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.server.datanode.BlockScanner.Conf;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi.BlockIterator;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.hdfs.util.DataTransferThrottler;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * VolumeScanner scans a single volume.  Each VolumeScanner has its own thread.<p/>
+ * They are all managed by the DataNode's BlockScanner.
+ */
+public class VolumeScanner extends Thread {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(VolumeScanner.class);
+
+  /**
+   * Number of seconds in a minute.
+   */
+  private final static int SECONDS_PER_MINUTE = 60;
+
+  /**
+   * Number of minutes in an hour.
+   */
+  private final static int MINUTES_PER_HOUR = 60;
+
+  /**
+   * Name of the block iterator used by this scanner.
+   */
+  private final static String BLOCK_ITERATOR_NAME = "scanner";
+
+  /**
+   * The configuration.
+   */
+  private final Conf conf;
+
+  /**
+   * The DataNode this VolumEscanner is associated with.
+   */
+  private final DataNode datanode;
+
+  /**
+   * A reference to the volume that we're scanning.
+   */
+  private final FsVolumeReference ref;
+
+  /**
+   * The volume that we're scanning.
+   */
+  final FsVolumeSpi volume;
+
+  /**
+   * The number of scanned bytes in each minute of the last hour.<p/>
+   *
+   * This array is managed as a circular buffer.  We take the monotonic time and
+   * divide it up into one-minute periods.  Each entry in the array represents
+   * how many bytes were scanned during that period.
+   */
+  private final long scannedBytes[] = new long[MINUTES_PER_HOUR];
+
+  /**
+   * The sum of all the values of scannedBytes.
+   */
+  private long scannedBytesSum = 0;
+
+  /**
+   * The throttler to use with BlockSender objects.
+   */
+  private final DataTransferThrottler throttler = new DataTransferThrottler(1);
+
+  /**
+   * The null output stream to use with BlockSender objects.
+   */
+  private final DataOutputStream nullStream =
+      new DataOutputStream(new IOUtils.NullOutputStream());
+
+  /**
+   * The block iterators associated with this VolumeScanner.<p/>
+   *
+   * Each block pool has its own BlockIterator.
+   */
+  private final List<BlockIterator> blockIters =
+      new LinkedList<BlockIterator>();
+
+  /**
+   * The current block iterator, or null if there is none.
+   */
+  private BlockIterator curBlockIter = null;
+
+  /**
+   * True if the thread is stopping.<p/>
+   * Protected by this object's lock.
+   */
+  private boolean stopping = false;
+
+  /**
+   * The current minute, in monotonic terms.
+   */
+  private long curMinute = 0;
+
+  /**
+   * Handles scan results.
+   */
+  private final ScanResultHandler resultHandler;
+
+  private final Statistics stats = new Statistics();
+
+  static class Statistics {
+    long bytesScannedInPastHour = 0;
+    long blocksScannedInCurrentPeriod = 0;
+    long blocksScannedSinceRestart = 0;
+    long scansSinceRestart = 0;
+    long scanErrorsSinceRestart = 0;
+    long nextBlockPoolScanStartMs = -1;
+    long blockPoolPeriodEndsMs = -1;
+    ExtendedBlock lastBlockScanned = null;
+    boolean eof = false;
+
+    Statistics() {
+    }
+
+    Statistics(Statistics other) {
+      this.bytesScannedInPastHour = other.bytesScannedInPastHour;
+      this.blocksScannedInCurrentPeriod = other.blocksScannedInCurrentPeriod;
+      this.blocksScannedSinceRestart = other.blocksScannedSinceRestart;
+      this.scansSinceRestart = other.scansSinceRestart;
+      this.scanErrorsSinceRestart = other.scanErrorsSinceRestart;
+      this.nextBlockPoolScanStartMs = other.nextBlockPoolScanStartMs;
+      this.blockPoolPeriodEndsMs = other.blockPoolPeriodEndsMs;
+      this.lastBlockScanned = other.lastBlockScanned;
+      this.eof = other.eof;
+    }
+
+    @Override
+    public String toString() {
+      return new StringBuilder().
+          append("Statistics{").
+          append("bytesScannedInPastHour=").append(bytesScannedInPastHour).
+          append(", blocksScannedInCurrentPeriod=").
+              append(blocksScannedInCurrentPeriod).
+          append(", blocksScannedSinceRestart=").
+              append(blocksScannedSinceRestart).
+          append(", scansSinceRestart=").append(scansSinceRestart).
+          append(", scanErrorsSinceRestart=").append(scanErrorsSinceRestart).
+          append(", nextBlockPoolScanStartMs=").append(nextBlockPoolScanStartMs).
+          append(", blockPoolPeriodEndsMs=").append(blockPoolPeriodEndsMs).
+          append(", lastBlockScanned=").append(lastBlockScanned).
+          append(", eof=").append(eof).
+          append("}").toString();
+    }
+  }
+
+  private static double positiveMsToHours(long ms) {
+    if (ms <= 0) {
+      return 0;
+    } else {
+      return TimeUnit.HOURS.convert(ms, TimeUnit.MILLISECONDS);
+    }
+  }
+
+  public void printStats(StringBuilder p) {
+    p.append("Block scanner information for volume " +
+        volume.getStorageID() + " with base path " + volume.getBasePath() +
+        "%n");
+    synchronized (stats) {
+      p.append(String.format("Bytes verified in last hour       : %57d%n",
+          stats.bytesScannedInPastHour));
+      p.append(String.format("Blocks scanned in current period  : %57d%n",
+          stats.blocksScannedInCurrentPeriod));
+      p.append(String.format("Blocks scanned since restart      : %57d%n",
+          stats.blocksScannedSinceRestart));
+      p.append(String.format("Block pool scans since restart    : %57d%n",
+          stats.scansSinceRestart));
+      p.append(String.format("Block scan errors since restart   : %57d%n",
+          stats.scanErrorsSinceRestart));
+      if (stats.nextBlockPoolScanStartMs > 0) {
+        p.append(String.format("Hours until next block pool scan  : %57.3f%n",
+            positiveMsToHours(stats.nextBlockPoolScanStartMs -
+                Time.monotonicNow())));
+      }
+      if (stats.blockPoolPeriodEndsMs > 0) {
+        p.append(String.format("Hours until possible pool rescan  : %57.3f%n",
+            positiveMsToHours(stats.blockPoolPeriodEndsMs -
+                Time.now())));
+      }
+      p.append(String.format("Last block scanned                : %57s%n",
+          ((stats.lastBlockScanned == null) ? "none" :
+          stats.lastBlockScanned.toString())));
+      p.append(String.format("More blocks to scan in period     : %57s%n",
+          !stats.eof));
+      p.append("%n");
+    }
+  }
+
+  static class ScanResultHandler {
+    private VolumeScanner scanner;
+
+    public void setup(VolumeScanner scanner) {
+      LOG.trace("Starting VolumeScanner {}",
+          scanner.volume.getBasePath());
+      this.scanner = scanner;
+    }
+
+    public void handle(ExtendedBlock block, IOException e) {
+      FsVolumeSpi volume = scanner.volume;
+      if (e == null) {
+        LOG.trace("Successfully scanned {} on {}", block, volume.getBasePath());
+        return;
+      }
+      // If the block does not exist anymore, then it's not an error.
+      if (!volume.getDataset().contains(block)) {
+        LOG.debug("Volume {}: block {} is no longer in the dataset.",
+            volume.getBasePath(), block);
+        return;
+      }
+      // If the block exists, the exception may due to a race with write:
+      // The BlockSender got an old block path in rbw. BlockReceiver removed
+      // the rbw block from rbw to finalized but BlockSender tried to open the
+      // file before BlockReceiver updated the VolumeMap. The state of the
+      // block can be changed again now, so ignore this error here. If there
+      // is a block really deleted by mistake, DirectoryScan should catch it.
+      if (e instanceof FileNotFoundException ) {
+        LOG.info("Volume {}: verification failed for {} because of " +
+                "FileNotFoundException.  This may be due to a race with write.",
+            volume.getBasePath(), block);
+        return;
+      }
+      LOG.warn("Reporting bad {} on {}", block, volume.getBasePath());
+      try {
+        scanner.datanode.reportBadBlocks(block);
+      } catch (IOException ie) {
+        // This is bad, but not bad enough to shut down the scanner.
+        LOG.warn("Cannot report bad " + block.getBlockId(), e);
+      }
+    }
+  }
+
+  VolumeScanner(Conf conf, DataNode datanode, FsVolumeReference ref) {
+    this.conf = conf;
+    this.datanode = datanode;
+    this.ref = ref;
+    this.volume = ref.getVolume();
+    ScanResultHandler handler;
+    try {
+      handler = conf.resultHandler.newInstance();
+    } catch (Throwable e) {
+      LOG.error("unable to instantiate {}", conf.resultHandler, e);
+      handler = new ScanResultHandler();
+    }
+    this.resultHandler = handler;
+    setName("VolumeScannerThread(" + volume.getBasePath() + ")");
+    setDaemon(true);
+  }
+
+  private void saveBlockIterator(BlockIterator iter) {
+    try {
+      iter.save();
+    } catch (IOException e) {
+      LOG.warn("{}: error saving {}.", this, iter, e);
+    }
+  }
+
+  private void expireOldScannedBytesRecords(long monotonicMs) {
+    long newMinute =
+        TimeUnit.MINUTES.convert(monotonicMs, TimeUnit.MILLISECONDS);
+    newMinute = newMinute % MINUTES_PER_HOUR;
+    if (curMinute == newMinute) {
+      return;
+    }
+    // If a minute or more has gone past since we last updated the scannedBytes
+    // array, zero out the slots corresponding to those minutes.
+    for (long m = curMinute + 1; m <= newMinute; m++) {
+      LOG.trace("{}: updateScannedBytes is zeroing out slot {}.  " +
+              "curMinute = {}; newMinute = {}", this, m % MINUTES_PER_HOUR,
+          curMinute, newMinute);
+      scannedBytesSum -= scannedBytes[(int)(m % MINUTES_PER_HOUR)];
+      scannedBytes[(int)(m % MINUTES_PER_HOUR)] = 0;
+    }
+    curMinute = newMinute;
+  }
+
+  /**
+   * Find a usable block iterator.<p/>
+   *
+   * We will consider available block iterators in order.  This property is
+   * important so that we don't keep rescanning the same block pool id over
+   * and over, while other block pools stay unscanned.<p/>
+   *
+   * A block pool is always ready to scan if the iterator is not at EOF.  If
+   * the iterator is at EOF, the block pool will be ready to scan when
+   * conf.scanPeriodMs milliseconds have elapsed since the iterator was last
+   * rewound.<p/>
+   *
+   * @return                     0 if we found a usable block iterator; the
+   *                               length of time we should delay before
+   *                               checking again otherwise.
+   */
+  private synchronized long findNextUsableBlockIter() {
+    int numBlockIters = blockIters.size();
+    if (numBlockIters == 0) {
+      LOG.debug("{}: no block pools are registered.", this);
+      return Long.MAX_VALUE;
+    }
+    int curIdx;
+    if (curBlockIter == null) {
+      curIdx = 0;
+    } else {
+      curIdx = blockIters.indexOf(curBlockIter);
+      Preconditions.checkState(curIdx >= 0);
+    }
+    // Note that this has to be wall-clock time, not monotonic time.  This is
+    // because the time saved in the cursor file is a wall-clock time.  We do
+    // not want to save a monotonic time in the cursor file, because it resets
+    // every time the machine reboots (on most platforms).
+    long nowMs = Time.now();
+    long minTimeoutMs = Long.MAX_VALUE;
+    for (int i = 0; i < numBlockIters; i++) {
+      int idx = (curIdx + i + 1) % numBlockIters;
+      BlockIterator iter = blockIters.get(idx);
+      if (!iter.atEnd()) {
+        LOG.info("Now scanning bpid {} on volume {}",
+            iter.getBlockPoolId(), volume.getBasePath());
+        curBlockIter = iter;
+        return 0L;
+      }
+      long iterStartMs = iter.getIterStartMs();
+      long waitMs = (iterStartMs + conf.scanPeriodMs) - nowMs;
+      if (waitMs <= 0) {
+        iter.rewind();
+        LOG.info("Now rescanning bpid {} on volume {}, after more than " +
+            "{} hour(s)", iter.getBlockPoolId(), volume.getBasePath(),
+            TimeUnit.HOURS.convert(conf.scanPeriodMs, TimeUnit.MILLISECONDS));
+        curBlockIter = iter;
+        return 0L;
+      }
+      minTimeoutMs = Math.min(minTimeoutMs, waitMs);
+    }
+    LOG.info("{}: no suitable block pools found to scan.  Waiting {} ms.",
+        this, minTimeoutMs);
+    return minTimeoutMs;
+  }
+
+  /**
+   * Scan a block.
+   *
+   * @param cblock               The block to scan.
+   * @param bytesPerSec          The bytes per second to scan at.
+   *
+   * @return                     The length of the block that was scanned, or
+   *                               -1 if the block could not be scanned.
+   */
+  private long scanBlock(ExtendedBlock cblock, long bytesPerSec) {
+    // 'cblock' has a valid blockId and block pool id, but we don't yet know the
+    // genstamp the block is supposed to have.  Ask the FsDatasetImpl for this
+    // information.
+    ExtendedBlock block = null;
+    try {
+      Block b = volume.getDataset().getStoredBlock(
+          cblock.getBlockPoolId(), cblock.getBlockId());
+      if (b == null) {
+        LOG.info("FileNotFound while finding block {} on volume {}",
+            cblock, volume.getBasePath());
+      } else {
+        block = new ExtendedBlock(cblock.getBlockPoolId(), b);
+      }
+    } catch (FileNotFoundException e) {
+      LOG.info("FileNotFoundException while finding block {} on volume {}",
+          cblock, volume.getBasePath());
+    } catch (IOException e) {
+      LOG.warn("I/O error while finding block {} on volume {}",
+            cblock, volume.getBasePath());
+    }
+    if (block == null) {
+      return -1; // block not found.
+    }
+    BlockSender blockSender = null;
+    try {
+      blockSender = new BlockSender(block, 0, -1,
+          false, true, true, datanode, null,
+          CachingStrategy.newDropBehind());
+      throttler.setBandwidth(bytesPerSec);
+      long bytesRead = blockSender.sendBlock(nullStream, null, throttler);
+      resultHandler.handle(block, null);
+      return bytesRead;
+    } catch (IOException e) {
+      resultHandler.handle(block, e);
+    } finally {
+      IOUtils.cleanup(null, blockSender);
+    }
+    return -1;
+  }
+
+  @VisibleForTesting
+  static boolean calculateShouldScan(long targetBytesPerSec,
+                                     long scannedBytesSum) {
+    long effectiveBytesPerSec =
+        scannedBytesSum / (SECONDS_PER_MINUTE * MINUTES_PER_HOUR);
+    boolean shouldScan = effectiveBytesPerSec <= targetBytesPerSec;
+    LOG.trace("calculateShouldScan: effectiveBytesPerSec = {}, and " +
+        "targetBytesPerSec = {}.  shouldScan = {}",
+        effectiveBytesPerSec, targetBytesPerSec, shouldScan);
+    return shouldScan;
+  }
+
+  /**
+   * Run an iteration of the VolumeScanner loop.
+   *
+   * @return     The number of milliseconds to delay before running the loop
+   *               again, or 0 to re-run the loop immediately.
+   */
+  private long runLoop() {
+    long bytesScanned = -1;
+    boolean scanError = false;
+    ExtendedBlock block = null;
+    try {
+      long monotonicMs = Time.monotonicNow();
+      expireOldScannedBytesRecords(monotonicMs);
+
+      if (!calculateShouldScan(conf.targetBytesPerSec, scannedBytesSum)) {
+        // If neededBytesPerSec is too low, then wait few seconds for some old
+        // scannedBytes records to expire.
+        return 30000L;
+      }
+
+      // Find a usable block pool to scan.
+      if ((curBlockIter == null) || curBlockIter.atEnd()) {
+        long timeout = findNextUsableBlockIter();
+        if (timeout > 0) {
+          LOG.trace("{}: no block pools are ready to scan yet.  Waiting " +
+              "{} ms.", this, timeout);
+          synchronized (stats) {
+            stats.nextBlockPoolScanStartMs = Time.monotonicNow() + timeout;
+          }
+          return timeout;
+        }
+        synchronized (stats) {
+          stats.scansSinceRestart++;
+          stats.blocksScannedInCurrentPeriod = 0;
+          stats.nextBlockPoolScanStartMs = -1;
+        }
+        return 0L;
+      }
+
+      try {
+        block = curBlockIter.nextBlock();
+      } catch (IOException e) {
+        // There was an error listing the next block in the volume.  This is a
+        // serious issue.
+        LOG.warn("{}: nextBlock error on {}", this, curBlockIter);
+        // On the next loop iteration, curBlockIter#eof will be set to true, and
+        // we will pick a different block iterator.
+        return 0L;
+      }
+      if (block == null) {
+        // The BlockIterator is at EOF.
+        LOG.info("{}: finished scanning block pool {}",
+            this, curBlockIter.getBlockPoolId());
+        saveBlockIterator(curBlockIter);
+        return 0;
+      }
+      long saveDelta = monotonicMs - curBlockIter.getLastSavedMs();
+      if (saveDelta >= conf.cursorSaveMs) {
+        LOG.debug("{}: saving block iterator {} after {} ms.",
+            this, curBlockIter, saveDelta);
+        saveBlockIterator(curBlockIter);
+      }
+      bytesScanned = scanBlock(block, conf.targetBytesPerSec);
+      if (bytesScanned >= 0) {
+        scannedBytesSum += bytesScanned;
+        scannedBytes[(int)(curMinute % MINUTES_PER_HOUR)] += bytesScanned;
+      } else {
+        scanError = true;
+      }
+      return 0L;
+    } finally {
+      synchronized (stats) {
+        stats.bytesScannedInPastHour = scannedBytesSum;
+        if (bytesScanned >= 0) {
+          stats.blocksScannedInCurrentPeriod++;
+          stats.blocksScannedSinceRestart++;
+        }
+        if (scanError) {
+          stats.scanErrorsSinceRestart++;
+        }
+        if (block != null) {
+          stats.lastBlockScanned = block;
+        }
+        if (curBlockIter == null) {
+          stats.eof = true;
+          stats.blockPoolPeriodEndsMs = -1;
+        } else {
+          stats.eof = curBlockIter.atEnd();
+          stats.blockPoolPeriodEndsMs =
+              curBlockIter.getIterStartMs() + conf.scanPeriodMs;
+        }
+      }
+    }
+  }
+
+  @Override
+  public void run() {
+    try {
+      LOG.trace("{}: thread starting.", this);
+      resultHandler.setup(this);
+      try {
+        long timeout = 0;
+        while (true) {
+          // Take the lock to check if we should stop.
+          synchronized (this) {
+            if (stopping) {
+              break;
+            }
+            if (timeout > 0) {
+              wait(timeout);
+              if (stopping) {
+                break;
+              }
+            }
+          }
+          timeout = runLoop();
+        }
+      } catch (InterruptedException e) {
+        // We are exiting because of an InterruptedException,
+        // probably sent by VolumeScanner#shutdown.
+        LOG.trace("{} exiting because of InterruptedException.", this);
+      } catch (Throwable e) {
+        LOG.error("{} exiting because of exception ", this, e);
+      }
+      LOG.info("{} exiting.", this);
+      // Save the current position of all block iterators and close them.
+      for (BlockIterator iter : blockIters) {
+        saveBlockIterator(iter);
+        IOUtils.cleanup(null, iter);
+      }
+    } finally {
+      // When the VolumeScanner exits, release the reference we were holding
+      // on the volume.  This will allow the volume to be removed later.
+      IOUtils.cleanup(null, ref);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "VolumeScanner(" + volume.getBasePath() +
+        ", " + volume.getStorageID() + ")";
+  }
+
+  /**
+   * Shut down this scanner.
+   */
+  public synchronized void shutdown() {
+    stopping = true;
+    notify();
+    this.interrupt();
+  }
+
+  /**
+   * Allow the scanner to scan the given block pool.
+   *
+   * @param bpid       The block pool id.
+   */
+  public synchronized void enableBlockPoolId(String bpid) {
+    for (BlockIterator iter : blockIters) {
+      if (iter.getBlockPoolId().equals(bpid)) {
+        LOG.warn("{}: already enabled scanning on block pool {}", this, bpid);
+        return;
+      }
+    }
+    BlockIterator iter = null;
+    try {
+      // Load a block iterator for the next block pool on the volume.
+      iter = volume.loadBlockIterator(bpid, BLOCK_ITERATOR_NAME);
+      LOG.trace("{}: loaded block iterator for {}.", this, bpid);
+    } catch (FileNotFoundException e) {
+      LOG.debug("{}: failed to load block iterator: " + e.getMessage(), this);
+    } catch (IOException e) {
+      LOG.warn("{}: failed to load block iterator.", this, e);
+    }
+    if (iter == null) {
+      iter = volume.newBlockIterator(bpid, BLOCK_ITERATOR_NAME);
+      LOG.trace("{}: created new block iterator for {}.", this, bpid);
+    }
+    iter.setMaxStalenessMs(conf.maxStalenessMs);
+    blockIters.add(iter);
+    notify();
+  }
+
+  /**
+   * Disallow the scanner from scanning the given block pool.
+   *
+   * @param bpid       The block pool id.
+   */
+  public synchronized void disableBlockPoolId(String bpid) {
+    Iterator<BlockIterator> i = blockIters.iterator();
+    while (i.hasNext()) {
+      BlockIterator iter = i.next();
+      if (iter.getBlockPoolId().equals(bpid)) {
+        LOG.trace("{}: disabling scanning on block pool {}", this, bpid);
+        i.remove();
+        IOUtils.cleanup(null, iter);
+        if (curBlockIter == iter) {
+          curBlockIter = null;
+        }
+        notify();
+        return;
+      }
+    }
+    LOG.warn("{}: can't remove block pool {}, because it was never " +
+        "added.", this, bpid);
+  }
+
+  @VisibleForTesting
+  Statistics getStatistics() {
+    synchronized (stats) {
+      return new Statistics(stats);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df4edd9a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
index 0d5de81..162e306 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
@@ -90,24 +90,30 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
     }
   }
 
-  /**
-   * Create rolling logs.
-   *
-   * @param prefix the prefix of the log names.
-   * @return rolling logs
-   */
-  public RollingLogs createRollingLogs(String bpid, String prefix
-      ) throws IOException;
-
   /** @return a list of volumes. */
   public List<V> getVolumes();
 
-  /** Add an array of StorageLocation to FsDataset. */
+  /**
+   * Add a new volume to the FsDataset.<p/>
+   *
+   * If the FSDataset supports block scanning, this function registers
+   * the new volume with the block scanner.
+   *
+   * @param location      The storage location for the new volume.
+   * @param nsInfos       Namespace information for the new volume.
+   */
   public void addVolume(
       final StorageLocation location,
       final List<NamespaceInfo> nsInfos) throws IOException;
 
-  /** Removes a collection of volumes from FsDataset. */
+  /**
+   * Removes a collection of volumes from FsDataset.
+   *
+   * If the FSDataset supports block scanning, this function removes
+   * the volumes from the block scanner.
+   *
+   * @param volumes      The storage locations of the volumes to remove.
+   */
   public void removeVolumes(Collection<StorageLocation> volumes);
 
   /** @return a storage with the given storage ID */
@@ -514,6 +520,6 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
     /**
      * Move block from one storage to another storage
      */
-    public ReplicaInfo moveBlockAcrossStorage(final ExtendedBlock block,
+   public ReplicaInfo moveBlockAcrossStorage(final ExtendedBlock block,
         StorageType targetStorageType) throws IOException;
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df4edd9a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
index 3a635b7..1355e31 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
@@ -17,11 +17,13 @@
  */
 package org.apache.hadoop.hdfs.server.datanode.fsdataset;
 
+import java.io.Closeable;
 import java.io.File;
 import java.io.IOException;
 import java.nio.channels.ClosedChannelException;
 
 import org.apache.hadoop.hdfs.StorageType;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 
 /**
  * This is an interface for the underlying volume.
@@ -69,4 +71,112 @@ public interface FsVolumeSpi {
    * Release disk space previously reserved for RBW block.
    */
   public void releaseReservedSpace(long bytesToRelease);
+
+  /**
+   * BlockIterator will return ExtendedBlock entries from a block pool in
+   * this volume.  The entries will be returned in sorted order.<p/>
+   *
+   * BlockIterator objects themselves do not always have internal
+   * synchronization, so they can only safely be used by a single thread at a
+   * time.<p/>
+   *
+   * Closing the iterator does not save it.  You must call save to save it.
+   */
+  public interface BlockIterator extends Closeable {
+    /**
+     * Get the next block.<p/>
+     *
+     * Note that this block may be removed in between the time we list it,
+     * and the time the caller tries to use it, or it may represent a stale
+     * entry.  Callers should handle the case where the returned block no
+     * longer exists.
+     *
+     * @return               The next block, or null if there are no
+     *                         more blocks.  Null if there was an error
+     *                         determining the next block.
+     *
+     * @throws IOException   If there was an error getting the next block in
+     *                         this volume.  In this case, EOF will be set on
+     *                         the iterator.
+     */
+    public ExtendedBlock nextBlock() throws IOException;
+
+    /**
+     * Returns true if we got to the end of the block pool.
+     */
+    public boolean atEnd();
+
+    /**
+     * Repositions the iterator at the beginning of the block pool.
+     */
+    public void rewind();
+
+    /**
+     * Save this block iterator to the underlying volume.
+     * Any existing saved block iterator with this name will be overwritten.
+     * maxStalenessMs will not be saved.
+     *
+     * @throws IOException   If there was an error when saving the block
+     *                         iterator.
+     */
+    public void save() throws IOException;
+
+    /**
+     * Set the maximum staleness of entries that we will return.<p/>
+     *
+     * A maximum staleness of 0 means we will never return stale entries; a
+     * larger value will allow us to reduce resource consumption in exchange
+     * for returning more potentially stale entries.  Even with staleness set
+     * to 0, consumers of this API must handle race conditions where block
+     * disappear before they can be processed.
+     */
+    public void setMaxStalenessMs(long maxStalenessMs);
+
+    /**
+     * Get the wall-clock time, measured in milliseconds since the Epoch,
+     * when this iterator was created.
+     */
+    public long getIterStartMs();
+
+    /**
+     * Get the wall-clock time, measured in milliseconds since the Epoch,
+     * when this iterator was last saved.  Returns iterStartMs if the
+     * iterator was never saved.
+     */
+    public long getLastSavedMs();
+
+    /**
+     * Get the id of the block pool which this iterator traverses.
+     */
+    public String getBlockPoolId();
+  }
+
+  /**
+   * Create a new block iterator.  It will start at the beginning of the
+   * block set.
+   *
+   * @param bpid             The block pool id to iterate over.
+   * @param name             The name of the block iterator to create.
+   *
+   * @return                 The new block iterator.
+   */
+  public BlockIterator newBlockIterator(String bpid, String name);
+
+  /**
+   * Load a saved block iterator.
+   *
+   * @param bpid             The block pool id to iterate over.
+   * @param name             The name of the block iterator to load.
+   *
+   * @return                 The saved block iterator.
+   * @throws IOException     If there was an IO error loading the saved
+   *                           block iterator.
+   */
+  public BlockIterator loadBlockIterator(String bpid, String name)
+      throws IOException;
+
+  /**
+   * Get the FSDatasetSpi which this volume is a part of.
+   */
+  public FsDatasetSpi getDataset();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df4edd9a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/RollingLogs.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/RollingLogs.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/RollingLogs.java
deleted file mode 100644
index 5d54770..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/RollingLogs.java
+++ /dev/null
@@ -1,73 +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.datanode.fsdataset;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.Iterator;
-
-/**
- * Rolling logs consist of a current log and a set of previous logs.
- *
- * The implementation should support a single appender and multiple readers.
- */
-public interface RollingLogs {
-  /**
-   * To iterate the lines of the logs.
-   */
-  public interface LineIterator extends Iterator<String>, Closeable {
-    /** Is the iterator iterating the previous? */
-    public boolean isPrevious();
-
-    /**
-     * Is the last read entry from previous? This should be called after
-     * reading.
-     */
-    public boolean isLastReadFromPrevious();
-  }
-
-  /**
-   * To append text to the logs.
-   */
-  public interface Appender extends Appendable, Closeable {
-  }
-
-  /**
-   * Create an iterator to iterate the lines in the logs.
-   * 
-   * @param skipPrevious Should it skip reading the previous log? 
-   * @return a new iterator.
-   */
-  public LineIterator iterator(boolean skipPrevious) throws IOException;
-
-  /**
-   * @return the only appender to append text to the logs.
-   *   The same object is returned if it is invoked multiple times.
-   */
-  public Appender appender();
-
-  /**
-   * Roll current to previous.
-   *
-   * @return true if the rolling succeeded.
-   *   When it returns false, it is not equivalent to an error. 
-   *   It means that the rolling cannot be performed at the moment,
-   *   e.g. the logs are being read.
-   */
-  public boolean roll() throws IOException;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df4edd9a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
index f990faf..c00d467 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
@@ -67,7 +67,7 @@ import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
-import org.apache.hadoop.hdfs.server.datanode.DataBlockScanner;
+import org.apache.hadoop.hdfs.server.datanode.BlockScanner;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataStorage;
 import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
@@ -89,7 +89,6 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaInputStreams;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.RollingLogs;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.RoundRobinVolumeChoosingPolicy;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.VolumeChoosingPolicy;
 import static org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.RamDiskReplicaTracker.RamDiskReplica;
@@ -284,7 +283,8 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
             DFSConfigKeys.DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_POLICY_KEY,
             RoundRobinVolumeChoosingPolicy.class,
             VolumeChoosingPolicy.class), conf);
-    volumes = new FsVolumeList(volsFailed, blockChooserImpl);
+    volumes = new FsVolumeList(volsFailed, datanode.getBlockScanner(),
+        blockChooserImpl);
     asyncDiskService = new FsDatasetAsyncDiskService(datanode);
     asyncLazyPersistService = new RamDiskAsyncLazyPersistService(datanode);
 
@@ -312,6 +312,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     // storageMap and asyncDiskService, consistent.
     FsVolumeImpl fsVolume = new FsVolumeImpl(
         this, sd.getStorageUuid(), dir, this.conf, storageType);
+    FsVolumeReference ref = fsVolume.obtainReference();
     ReplicaMap tempVolumeMap = new ReplicaMap(this);
     fsVolume.getVolumeMap(tempVolumeMap, ramDiskReplicaTracker);
 
@@ -322,7 +323,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
               DatanodeStorage.State.NORMAL,
               storageType));
       asyncDiskService.addVolume(sd.getCurrentDir());
-      volumes.addVolume(fsVolume);
+      volumes.addVolume(ref);
     }
 
     LOG.info("Added volume - " + dir + ", StorageType: " + storageType);
@@ -361,6 +362,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
       throw MultipleIOException.createIOException(exceptions);
     }
 
+    final FsVolumeReference ref = fsVolume.obtainReference();
     setupAsyncLazyPersistThread(fsVolume);
 
     builder.build();
@@ -371,7 +373,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
               DatanodeStorage.State.NORMAL,
               storageType));
       asyncDiskService.addVolume(sd.getCurrentDir());
-      volumes.addVolume(fsVolume);
+      volumes.addVolume(ref);
     }
     LOG.info("Added volume - " + dir + ", StorageType: " + storageType);
   }
@@ -415,9 +417,6 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
               it.remove();
             }
           }
-          // Delete blocks from the block scanner in batch.
-          datanode.getBlockScanner().deleteBlocks(bpid,
-              blocks.toArray(new Block[blocks.size()]));
         }
 
         storageMap.remove(sd.getStorageUuid());
@@ -771,7 +770,6 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     }
 
     // Replace the old block if any to reschedule the scanning.
-    datanode.getBlockScanner().addBlock(block, false);
     return replicaInfo;
   }
 
@@ -2006,10 +2004,6 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
           // Block is in memory and not on the disk
           // Remove the block from volumeMap
           volumeMap.remove(bpid, blockId);
-          final DataBlockScanner blockScanner = datanode.getBlockScanner();
-          if (blockScanner != null) {
-            blockScanner.deleteBlock(bpid, new Block(blockId));
-          }
           if (vol.isTransientStorage()) {
             ramDiskReplicaTracker.discardReplica(bpid, blockId, true);
           }
@@ -2032,12 +2026,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
         ReplicaInfo diskBlockInfo = new FinalizedReplica(blockId, 
             diskFile.length(), diskGS, vol, diskFile.getParentFile());
         volumeMap.add(bpid, diskBlockInfo);
-        final DataBlockScanner blockScanner = datanode.getBlockScanner();
-        if (!vol.isTransientStorage()) {
-          if (blockScanner != null) {
-            blockScanner.addBlock(new ExtendedBlock(bpid, diskBlockInfo), false);
-          }
-        } else {
+        if (vol.isTransientStorage()) {
           ramDiskReplicaTracker.addReplica(bpid, blockId, (FsVolumeImpl) vol);
         }
         LOG.warn("Added missing block to memory " + diskBlockInfo);
@@ -2540,23 +2529,6 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     dataStorage.clearRollingUpgradeMarker(bpid);
   }
 
-  @Override
-  public RollingLogs createRollingLogs(String bpid, String prefix
-      ) throws IOException {
-    String dir = null;
-    final List<FsVolumeImpl> volumes = getVolumes();
-    for (FsVolumeImpl vol : volumes) {
-      String bpDir = vol.getPath(bpid);
-      if (RollingLogsImpl.isFilePresent(bpDir, prefix)) {
-        dir = bpDir;
-        break;
-      }
-    }
-    if (dir == null) {
-      dir = volumes.get(0).getPath(bpid);
-    }
-    return new RollingLogsImpl(dir, prefix);
-  }
 
   @Override
   public void onCompleteLazyPersist(String bpId, long blockId,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df4edd9a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
index 7c8384d..5ce2710 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
@@ -17,9 +17,18 @@
  */
 package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
 
+import java.io.BufferedWriter;
 import java.io.File;
+import java.io.FileOutputStream;
+import java.io.FilenameFilter;
 import java.io.IOException;
 import java.nio.channels.ClosedChannelException;
+import java.io.OutputStreamWriter;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.Collections;
+import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
@@ -41,15 +50,24 @@ import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.datanode.DataStorage;
 import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.util.CloseableReferenceCount;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.util.Time;
+import org.codehaus.jackson.annotate.JsonProperty;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.map.ObjectReader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * The underlying volume used to store replica.
@@ -59,6 +77,9 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
 @InterfaceAudience.Private
 @VisibleForTesting
 public class FsVolumeImpl implements FsVolumeSpi {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(FsVolumeImpl.class);
+
   private final FsDatasetImpl dataset;
   private final String storageID;
   private final StorageType storageType;
@@ -395,6 +416,332 @@ public class FsVolumeImpl implements FsVolumeSpi {
     }
   }
 
+  private enum SubdirFilter implements FilenameFilter {
+    INSTANCE;
+
+    @Override
+    public boolean accept(File dir, String name) {
+      return name.startsWith("subdir");
+    }
+  }
+
+  private enum BlockFileFilter implements FilenameFilter {
+    INSTANCE;
+
+    @Override
+    public boolean accept(File dir, String name) {
+      return !name.endsWith(".meta") && name.startsWith("blk_");
+    }
+  }
+
+  @VisibleForTesting
+  public static String nextSorted(List<String> arr, String prev) {
+    int res = 0;
+    if (prev != null) {
+      res = Collections.binarySearch(arr, prev);
+      if (res < 0) {
+        res = -1 - res;
+      } else {
+        res++;
+      }
+    }
+    if (res >= arr.size()) {
+      return null;
+    }
+    return arr.get(res);
+  }
+
+  private static class BlockIteratorState {
+    BlockIteratorState() {
+      lastSavedMs = iterStartMs = Time.now();
+      curFinalizedDir = null;
+      curFinalizedSubDir = null;
+      curEntry = null;
+      atEnd = false;
+    }
+
+    // The wall-clock ms since the epoch at which this iterator was last saved.
+    @JsonProperty
+    private long lastSavedMs;
+
+    // The wall-clock ms since the epoch at which this iterator was created.
+    @JsonProperty
+    private long iterStartMs;
+
+    @JsonProperty
+    private String curFinalizedDir;
+
+    @JsonProperty
+    private String curFinalizedSubDir;
+
+    @JsonProperty
+    private String curEntry;
+
+    @JsonProperty
+    private boolean atEnd;
+  }
+
+  /**
+   * A BlockIterator implementation for FsVolumeImpl.
+   */
+  private class BlockIteratorImpl implements FsVolumeSpi.BlockIterator {
+    private final File bpidDir;
+    private final String name;
+    private final String bpid;
+    private long maxStalenessMs = 0;
+
+    private List<String> cache;
+    private long cacheMs;
+
+    private BlockIteratorState state;
+
+    BlockIteratorImpl(String bpid, String name) {
+      this.bpidDir = new File(currentDir, bpid);
+      this.name = name;
+      this.bpid = bpid;
+      rewind();
+    }
+
+    /**
+     * Get the next subdirectory within the block pool slice.
+     *
+     * @return         The next subdirectory within the block pool slice, or
+     *                   null if there are no more.
+     */
+    private String getNextSubDir(String prev, File dir)
+          throws IOException {
+      List<String> children =
+          IOUtils.listDirectory(dir, SubdirFilter.INSTANCE);
+      cache = null;
+      cacheMs = 0;
+      if (children.size() == 0) {
+        LOG.trace("getNextSubDir({}, {}): no subdirectories found in {}",
+            storageID, bpid, dir.getAbsolutePath());
+        return null;
+      }
+      Collections.sort(children);
+      String nextSubDir = nextSorted(children, prev);
+      if (nextSubDir == null) {
+        LOG.trace("getNextSubDir({}, {}): no more subdirectories found in {}",
+            storageID, bpid, dir.getAbsolutePath());
+      } else {
+        LOG.trace("getNextSubDir({}, {}): picking next subdirectory {} " +
+            "within {}", storageID, bpid, nextSubDir, dir.getAbsolutePath());
+      }
+      return nextSubDir;
+    }
+
+    private String getNextFinalizedDir() throws IOException {
+      File dir = Paths.get(
+          bpidDir.getAbsolutePath(), "current", "finalized").toFile();
+      return getNextSubDir(state.curFinalizedDir, dir);
+    }
+
+    private String getNextFinalizedSubDir() throws IOException {
+      if (state.curFinalizedDir == null) {
+        return null;
+      }
+      File dir = Paths.get(
+          bpidDir.getAbsolutePath(), "current", "finalized",
+              state.curFinalizedDir).toFile();
+      return getNextSubDir(state.curFinalizedSubDir, dir);
+    }
+
+    private List<String> getSubdirEntries() throws IOException {
+      if (state.curFinalizedSubDir == null) {
+        return null; // There are no entries in the null subdir.
+      }
+      long now = Time.monotonicNow();
+      if (cache != null) {
+        long delta = now - cacheMs;
+        if (delta < maxStalenessMs) {
+          return cache;
+        } else {
+          LOG.trace("getSubdirEntries({}, {}): purging entries cache for {} " +
+            "after {} ms.", storageID, bpid, state.curFinalizedSubDir, delta);
+          cache = null;
+        }
+      }
+      File dir = Paths.get(bpidDir.getAbsolutePath(), "current", "finalized",
+                    state.curFinalizedDir, state.curFinalizedSubDir).toFile();
+      List<String> entries =
+          IOUtils.listDirectory(dir, BlockFileFilter.INSTANCE);
+      if (entries.size() == 0) {
+        entries = null;
+      } else {
+        Collections.sort(entries);
+      }
+      if (entries == null) {
+        LOG.trace("getSubdirEntries({}, {}): no entries found in {}",
+            storageID, bpid, dir.getAbsolutePath());
+      } else {
+        LOG.trace("getSubdirEntries({}, {}): listed {} entries in {}", 
+            storageID, bpid, entries.size(), dir.getAbsolutePath());
+      }
+      cache = entries;
+      cacheMs = now;
+      return cache;
+    }
+
+    /**
+     * Get the next block.<p/>
+     *
+     * Each volume has a hierarchical structure.<p/>
+     *
+     * <code>
+     * BPID B0
+     *   finalized/
+     *     subdir0
+     *       subdir0
+     *         blk_000
+     *         blk_001
+     *       ...
+     *     subdir1
+     *       subdir0
+     *         ...
+     *   rbw/
+     * </code>
+     *
+     * When we run out of entries at one level of the structure, we search
+     * progressively higher levels.  For example, when we run out of blk_
+     * entries in a subdirectory, we search for the next subdirectory.
+     * And so on.
+     */
+    @Override
+    public ExtendedBlock nextBlock() throws IOException {
+      if (state.atEnd) {
+        return null;
+      }
+      try {
+        while (true) {
+          List<String> entries = getSubdirEntries();
+          if (entries != null) {
+            state.curEntry = nextSorted(entries, state.curEntry);
+            if (state.curEntry == null) {
+              LOG.trace("nextBlock({}, {}): advancing from {} to next " +
+                  "subdirectory.", storageID, bpid, state.curFinalizedSubDir);
+            } else {
+              ExtendedBlock block =
+                  new ExtendedBlock(bpid, Block.filename2id(state.curEntry));
+              LOG.trace("nextBlock({}, {}): advancing to {}",
+                  storageID, bpid, block);
+              return block;
+            }
+          }
+          state.curFinalizedSubDir = getNextFinalizedSubDir();
+          if (state.curFinalizedSubDir == null) {
+            state.curFinalizedDir = getNextFinalizedDir();
+            if (state.curFinalizedDir == null) {
+              state.atEnd = true;
+              return null;
+            }
+          }
+        }
+      } catch (IOException e) {
+        state.atEnd = true;
+        LOG.error("nextBlock({}, {}): I/O error", storageID, bpid, e);
+        throw e;
+      }
+    }
+
+    @Override
+    public boolean atEnd() {
+      return state.atEnd;
+    }
+
+    @Override
+    public void rewind() {
+      cache = null;
+      cacheMs = 0;
+      state = new BlockIteratorState();
+    }
+
+    @Override
+    public void save() throws IOException {
+      state.lastSavedMs = Time.now();
+      boolean success = false;
+      ObjectMapper mapper = new ObjectMapper();
+      try (BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(
+                new FileOutputStream(getTempSaveFile(), false), "UTF-8"))) {
+        mapper.writerWithDefaultPrettyPrinter().writeValue(writer, state);
+        success = true;
+      } finally {
+        if (!success) {
+          if (getTempSaveFile().delete()) {
+            LOG.debug("save({}, {}): error deleting temporary file.",
+                storageID, bpid);
+          }
+        }
+      }
+      Files.move(getTempSaveFile().toPath(), getSaveFile().toPath(),
+          StandardCopyOption.ATOMIC_MOVE);
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("save({}, {}): saved {}", storageID, bpid,
+            mapper.writerWithDefaultPrettyPrinter().writeValueAsString(state));
+      }
+    }
+
+    public void load() throws IOException {
+      ObjectMapper mapper = new ObjectMapper();
+      File file = getSaveFile();
+      this.state = mapper.reader(BlockIteratorState.class).readValue(file);
+      LOG.trace("load({}, {}): loaded iterator {} from {}: {}", storageID,
+          bpid, name, file.getAbsoluteFile(),
+          mapper.writerWithDefaultPrettyPrinter().writeValueAsString(state));
+    }
+
+    File getSaveFile() {
+      return new File(bpidDir, name + ".cursor");
+    }
+
+    File getTempSaveFile() {
+      return new File(bpidDir, name + ".cursor.tmp");
+    }
+
+    @Override
+    public void setMaxStalenessMs(long maxStalenessMs) {
+      this.maxStalenessMs = maxStalenessMs;
+    }
+
+    @Override
+    public void close() throws IOException {
+      // No action needed for this volume implementation.
+    }
+
+    @Override
+    public long getIterStartMs() {
+      return state.iterStartMs;
+    }
+
+    @Override
+    public long getLastSavedMs() {
+      return state.lastSavedMs;
+    }
+
+    @Override
+    public String getBlockPoolId() {
+      return bpid;
+    }
+  }
+
+  @Override
+  public BlockIterator newBlockIterator(String bpid, String name) {
+    return new BlockIteratorImpl(bpid, name);
+  }
+
+  @Override
+  public BlockIterator loadBlockIterator(String bpid, String name)
+      throws IOException {
+    BlockIteratorImpl iter = new BlockIteratorImpl(bpid, name);
+    iter.load();
+    return iter;
+  }
+
+  @Override
+  public FsDatasetSpi getDataset() {
+    return dataset;
+  }
+
   /**
    * RBW files. They get moved to the finalized block directory when
    * the block is finalized.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df4edd9a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java
index c837593..ae2f5b4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.hdfs.StorageType;
 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.VolumeChoosingPolicy;
+import org.apache.hadoop.hdfs.server.datanode.BlockScanner;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 import org.apache.hadoop.util.Time;
 
@@ -42,11 +43,13 @@ class FsVolumeList {
   private Object checkDirsMutex = new Object();
 
   private final VolumeChoosingPolicy<FsVolumeImpl> blockChooser;
+  private final BlockScanner blockScanner;
   private volatile int numFailedVolumes;
 
-  FsVolumeList(int failedVols,
+  FsVolumeList(int failedVols, BlockScanner blockScanner,
       VolumeChoosingPolicy<FsVolumeImpl> blockChooser) {
     this.blockChooser = blockChooser;
+    this.blockScanner = blockScanner;
     this.numFailedVolumes = failedVols;
   }
   
@@ -260,13 +263,14 @@ class FsVolumeList {
 
   /**
    * Dynamically add new volumes to the existing volumes that this DN manages.
-   * @param newVolume the instance of new FsVolumeImpl.
+   *
+   * @param ref       a reference to the new FsVolumeImpl instance.
    */
-  void addVolume(FsVolumeImpl newVolume) {
+  void addVolume(FsVolumeReference ref) {
     while (true) {
       final FsVolumeImpl[] curVolumes = volumes.get();
       final List<FsVolumeImpl> volumeList = Lists.newArrayList(curVolumes);
-      volumeList.add(newVolume);
+      volumeList.add((FsVolumeImpl)ref.getVolume());
       if (volumes.compareAndSet(curVolumes,
           volumeList.toArray(new FsVolumeImpl[volumeList.size()]))) {
         break;
@@ -274,12 +278,15 @@ class FsVolumeList {
         if (FsDatasetImpl.LOG.isDebugEnabled()) {
           FsDatasetImpl.LOG.debug(
               "The volume list has been changed concurrently, " +
-                  "retry to remove volume: " + newVolume);
+                  "retry to remove volume: " + ref.getVolume().getStorageID());
         }
       }
     }
-
-    FsDatasetImpl.LOG.info("Added new volume: " + newVolume.toString());
+    if (blockScanner != null) {
+      blockScanner.addVolumeScanner(ref);
+    }
+    FsDatasetImpl.LOG.info("Added new volume: " +
+        ref.getVolume().getStorageID());
   }
 
   /**
@@ -293,6 +300,9 @@ class FsVolumeList {
       if (volumeList.remove(target)) {
         if (volumes.compareAndSet(curVolumes,
             volumeList.toArray(new FsVolumeImpl[volumeList.size()]))) {
+          if (blockScanner != null) {
+            blockScanner.removeVolumeScanner(target);
+          }
           try {
             target.closeAndWait();
           } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df4edd9a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/RollingLogsImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/RollingLogsImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/RollingLogsImpl.java
deleted file mode 100644
index 121127d..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/RollingLogsImpl.java
+++ /dev/null
@@ -1,241 +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.datanode.fsdataset.impl;
-
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileNotFoundException;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.io.OutputStreamWriter;
-import java.io.PrintWriter;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.hadoop.hdfs.server.datanode.DataBlockScanner;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.RollingLogs;
-
-import com.google.common.base.Charsets;
-
-class RollingLogsImpl implements RollingLogs {
-  private static final String CURR_SUFFIX = ".curr";
-  private static final String PREV_SUFFIX = ".prev";
-
-  static boolean isFilePresent(String dir, String filePrefix) {
-    return new File(dir, filePrefix + CURR_SUFFIX).exists() ||
-           new File(dir, filePrefix + PREV_SUFFIX).exists();
-  }
-
-  private final File curr;
-  private final File prev;
-  private PrintWriter out; //require synchronized access
-
-  private final Appender appender = new Appender() {
-    @Override
-    public Appendable append(CharSequence csq) {
-      synchronized(RollingLogsImpl.this) {
-        if (out == null) {
-          throw new IllegalStateException(RollingLogsImpl.this
-              + " is not yet opened.");
-        }
-        out.print(csq);
-        out.flush();
-      }
-      return this;
-    }
-
-    @Override
-    public Appendable append(char c) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public Appendable append(CharSequence csq, int start, int end) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public void close() {
-      synchronized(RollingLogsImpl.this) {
-        if (out != null) {
-          out.close();
-          out = null;
-        }
-      }
-    }
-  };
-
-
-  private final AtomicInteger numReaders = new AtomicInteger();
-
-  RollingLogsImpl(String dir, String filePrefix) throws FileNotFoundException{
-    curr = new File(dir, filePrefix + CURR_SUFFIX);
-    prev = new File(dir, filePrefix + PREV_SUFFIX);
-    out = new PrintWriter(new OutputStreamWriter(new FileOutputStream(
-        curr, true), Charsets.UTF_8));
-  }
-
-  @Override
-  public Reader iterator(boolean skipPrevFile) throws IOException {
-    numReaders.incrementAndGet(); 
-    return new Reader(skipPrevFile);
-  }
-
-  @Override
-  public Appender appender() {
-    return appender;
-  }
-
-  @Override
-  public boolean roll() throws IOException {
-    if (numReaders.get() > 0) {
-      return false;
-    }
-    if (!prev.delete() && prev.exists()) {
-      throw new IOException("Failed to delete " + prev);
-    }
-
-    synchronized(this) {
-      appender.close();
-      final boolean renamed = curr.renameTo(prev);
-      out = new PrintWriter(new OutputStreamWriter(new FileOutputStream(
-          curr, true), Charsets.UTF_8));
-      if (!renamed) {
-        throw new IOException("Failed to rename " + curr + " to " + prev);
-      }
-    }
-    return true;
-  }
-
-  @Override
-  public String toString() {
-    return curr.toString();
-  }
-  
-  /**
-   * This is used to read the lines in order.
-   * If the data is not read completely (i.e, untill hasNext() returns
-   * false), it needs to be explicitly 
-   */
-  private class Reader implements RollingLogs.LineIterator {
-    private File file;
-    private File lastReadFile;
-    private BufferedReader reader;
-    private String line;
-    private boolean closed = false;
-    
-    private Reader(boolean skipPrevFile) throws IOException {
-      reader = null;
-      file = skipPrevFile? curr : prev;
-      readNext();        
-    }
-
-    @Override
-    public boolean isPrevious() {
-      return file == prev;
-    }
-
-    @Override
-    public boolean isLastReadFromPrevious() {
-      return lastReadFile == prev;
-    }
-
-    private boolean openFile() throws IOException {
-
-      for(int i=0; i<2; i++) {
-        if (reader != null || i > 0) {
-          // move to next file
-          file = isPrevious()? curr : null;
-        }
-        if (file == null) {
-          return false;
-        }
-        if (file.exists()) {
-          break;
-        }
-      }
-      
-      if (reader != null ) {
-        reader.close();
-        reader = null;
-      }
-      
-      reader = new BufferedReader(new InputStreamReader(new FileInputStream(
-          file), Charsets.UTF_8));
-      return true;
-    }
-    
-    // read next line if possible.
-    private void readNext() throws IOException {
-      line = null;
-      try {
-        if (reader != null && (line = reader.readLine()) != null) {
-          return;
-        }
-        // move to the next file.
-        if (openFile()) {
-          readNext();
-        }
-      } finally {
-        if (!hasNext()) {
-          close();
-        }
-      }
-    }
-    
-    @Override
-    public boolean hasNext() {
-      return line != null;
-    }
-
-    @Override
-    public String next() {
-      String curLine = line;
-      try {
-        lastReadFile = file;
-        readNext();
-      } catch (IOException e) {
-        DataBlockScanner.LOG.warn("Failed to read next line.", e);
-      }
-      return curLine;
-    }
-
-    @Override
-    public void remove() {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public void close() throws IOException {
-      if (!closed) {
-        try {
-          if (reader != null) {
-            reader.close();
-          }
-        } finally {
-          file = null;
-          reader = null;
-          closed = true;
-          final int n = numReaders.decrementAndGet();
-          assert(n >= 0);
-        }
-      }
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df4edd9a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index 4d60792..c24f7be 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -1005,6 +1005,26 @@
 </property>
 
 <property>
+  <name>dfs.datanode.scan.period.hours</name>
+  <value>0</value>
+  <description>
+        If this is 0 or negative, the DataNode's block scanner will be
+        disabled.  If this is positive, the DataNode will not scan any
+        individual block more than once in the specified scan period.
+  </description>
+</property>
+
+<property>
+  <name>dfs.block.scanner.volume.bytes.per.second</name>
+  <value>1048576</value>
+  <description>
+        If this is 0, the DataNode's block scanner will be disabled.  If this
+        is positive, this is the number of bytes per second that the DataNode's
+        block scanner will try to scan from each volume.
+  </description>
+</property>
+
+<property>
   <name>dfs.datanode.readahead.bytes</name>
   <value>4193404</value>
   <description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df4edd9a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
index 15f5f2e..0eef46f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
@@ -1637,4 +1637,20 @@ public class DFSTestUtil {
       }
     }, 100, waitTime);
   }
+
+ /**
+   * Change the length of a block at datanode dnIndex
+   */
+  public static boolean changeReplicaLength(MiniDFSCluster cluster,
+      ExtendedBlock blk, int dnIndex, int lenDelta) throws IOException {
+    File blockFile = cluster.getBlockFile(dnIndex, blk);
+    if (blockFile != null && blockFile.exists()) {
+      RandomAccessFile raFile = new RandomAccessFile(blockFile, "rw");
+      raFile.setLength(raFile.length()+lenDelta);
+      raFile.close();
+      return true;
+    }
+    LOG.info("failed to change length of block " + blk);
+    return false;
+  }
 }


[45/50] [abbrv] hadoop git commit: HADOOP-11499. Check of executorThreadsStarted in ValueQueue#submitRefillTask() evades lock acquisition. Contributed by Ted Yu

Posted by zh...@apache.org.
HADOOP-11499. Check of executorThreadsStarted in ValueQueue#submitRefillTask() evades lock acquisition. Contributed by Ted Yu


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

Branch: refs/heads/HDFS-EC
Commit: 7ab153e4a54757f825f3b37168a42d237d0497c0
Parents: 4216800
Author: Jason Lowe <jl...@apache.org>
Authored: Mon Jan 26 16:56:14 2015 +0000
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:31 2015 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt         |  3 +++
 .../org/apache/hadoop/crypto/key/kms/ValueQueue.java    | 12 +++++++-----
 2 files changed, 10 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ab153e4/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 662f580..598f750 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -764,6 +764,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11482. Use correct UGI when KMSClientProvider is called by a proxy
     user. Contributed by Arun Suresh.
 
+    HADOOP-11499. Check of executorThreadsStarted in
+    ValueQueue#submitRefillTask() evades lock acquisition (Ted Yu via jlowe)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ab153e4/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/ValueQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/ValueQueue.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/ValueQueue.java
index 8e67ecc..32451d8 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/ValueQueue.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/ValueQueue.java
@@ -308,11 +308,13 @@ public class ValueQueue <E> {
       final Queue<E> keyQueue) throws InterruptedException {
     if (!executorThreadsStarted) {
       synchronized (this) {
-        // To ensure all requests are first queued, make coreThreads =
-        // maxThreads
-        // and pre-start all the Core Threads.
-        executor.prestartAllCoreThreads();
-        executorThreadsStarted = true;
+        if (!executorThreadsStarted) {
+          // To ensure all requests are first queued, make coreThreads =
+          // maxThreads
+          // and pre-start all the Core Threads.
+          executor.prestartAllCoreThreads();
+          executorThreadsStarted = true;
+        }
       }
     }
     // The submit/execute method of the ThreadPoolExecutor is bypassed and


[24/50] [abbrv] hadoop git commit: MAPREDUCE-3283. mapred classpath CLI does not display the complete classpath. Contributed by Varun Saxena.

Posted by zh...@apache.org.
MAPREDUCE-3283. mapred classpath CLI does not display the complete classpath. Contributed by Varun Saxena.


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

Branch: refs/heads/HDFS-EC
Commit: c1ad0a881b9188db71da5d9d0b7735da9cbcd04e
Parents: a8c59ba
Author: cnauroth <cn...@apache.org>
Authored: Wed Jan 21 13:50:39 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:27 2015 -0800

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt    |  3 +++
 hadoop-mapreduce-project/bin/mapred     |  4 +---
 hadoop-mapreduce-project/bin/mapred.cmd | 10 +++++++++-
 3 files changed, 13 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c1ad0a88/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index a7379ec..489369d 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -307,6 +307,9 @@ Release 2.7.0 - UNRELEASED
     MAPREDUCE-6206. TestAggregatedTransferRate fails on non-US systems (Jens
     Rabe via jlowe)
 
+    MAPREDUCE-3283. mapred classpath CLI does not display the complete classpath
+    (Varun Saxena via cnauroth)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c1ad0a88/hadoop-mapreduce-project/bin/mapred
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/bin/mapred b/hadoop-mapreduce-project/bin/mapred
index 066c438..6d0c781 100755
--- a/hadoop-mapreduce-project/bin/mapred
+++ b/hadoop-mapreduce-project/bin/mapred
@@ -78,9 +78,7 @@ case ${COMMAND} in
     HADOOP_OPTS="${HADOOP_OPTS} ${HADOOP_CLIENT_OPTS}"
   ;;
   classpath)
-    hadoop_finalize
-    echo "${CLASSPATH}"
-    exit 0
+    hadoop_do_classpath_subcommand "$@" 
   ;;
   distcp)
     CLASS=org.apache.hadoop.tools.DistCp

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c1ad0a88/hadoop-mapreduce-project/bin/mapred.cmd
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/bin/mapred.cmd b/hadoop-mapreduce-project/bin/mapred.cmd
index bb59c03..4085599 100644
--- a/hadoop-mapreduce-project/bin/mapred.cmd
+++ b/hadoop-mapreduce-project/bin/mapred.cmd
@@ -95,6 +95,14 @@ if "%1" == "--loglevel" (
   @rem add modules to CLASSPATH
   set CLASSPATH=%CLASSPATH%;%HADOOP_MAPRED_HOME%\modules\*
 
+  if %mapred-command% == classpath (
+    if not defined mapred-command-arguments (
+      @rem No need to bother starting up a JVM for this simple case.
+      @echo %CLASSPATH%
+      exit /b
+    )
+  )
+
   call :%mapred-command% %mapred-command-arguments%
   set java_arguments=%JAVA_HEAP_MAX% %HADOOP_OPTS% -classpath %CLASSPATH% %CLASS% %mapred-command-arguments%
   call %JAVA% %java_arguments%
@@ -103,7 +111,7 @@ goto :eof
 
 
 :classpath
-  @echo %CLASSPATH%
+  set CLASS=org.apache.hadoop.util.Classpath
   goto :eof
 
 :job


[36/50] [abbrv] hadoop git commit: YARN-3082. Non thread safe access to systemCredentials in NodeHeartbeatResponse processing. Contributed by Anubhav Dhoot.

Posted by zh...@apache.org.
YARN-3082. Non thread safe access to systemCredentials in NodeHeartbeatResponse processing. Contributed by Anubhav Dhoot.


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

Branch: refs/heads/HDFS-EC
Commit: a06d2d65d38ae464d02809b1c6718d20b960ac8b
Parents: 8a96c6d
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Fri Jan 23 16:04:18 2015 +0900
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:29 2015 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +
 .../impl/pb/NodeHeartbeatResponsePBImpl.java    |  3 +-
 .../nodemanager/TestNodeStatusUpdater.java      | 63 ++++++++++++++++++++
 3 files changed, 68 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a06d2d65/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index bdc31db..7f0628d 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -400,6 +400,9 @@ Release 2.7.0 - UNRELEASED
     YARN-3078. LogCLIHelpers lacks of a blank space before string 'does not exist'.
     (Sam Liu via ozawa)
 
+    YARN-3082. Non thread safe access to systemCredentials in NodeHeartbeatResponse
+    processing. (Anubhav Dhoot via ozawa)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a06d2d65/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.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/NodeHeartbeatResponsePBImpl.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/NodeHeartbeatResponsePBImpl.java
index 1e91514..630a5bf 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/NodeHeartbeatResponsePBImpl.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/NodeHeartbeatResponsePBImpl.java
@@ -104,7 +104,8 @@ public class NodeHeartbeatResponsePBImpl extends
     for (Map.Entry<ApplicationId, ByteBuffer> entry : systemCredentials.entrySet()) {
       builder.addSystemCredentialsForApps(SystemCredentialsForAppsProto.newBuilder()
         .setAppId(convertToProtoFormat(entry.getKey()))
-        .setCredentialsForApp(ProtoUtils.convertToProtoFormat(entry.getValue())));
+        .setCredentialsForApp(ProtoUtils.convertToProtoFormat(
+            entry.getValue().duplicate())));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a06d2d65/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 46d7b10..71a420e 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
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.nodemanager;
 
+import static org.apache.hadoop.yarn.server.utils.YarnServerBuilderUtils.newNodeHeartbeatResponse;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -37,7 +38,10 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -74,12 +78,14 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeHeartbeatResponseProto;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.server.api.ResourceTracker;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.NodeHeartbeatResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.records.MasterKey;
 import org.apache.hadoop.yarn.server.api.records.NodeAction;
 import org.apache.hadoop.yarn.server.api.records.NodeStatus;
@@ -1463,6 +1469,63 @@ public class TestNodeStatusUpdater {
     nm.stop();
   }
 
+  @Test
+  public void testConcurrentAccessToSystemCredentials(){
+    final Map<ApplicationId, ByteBuffer> testCredentials = new HashMap<>();
+    ByteBuffer byteBuffer = ByteBuffer.wrap(new byte[300]);
+    ApplicationId applicationId = ApplicationId.newInstance(123456, 120);
+    testCredentials.put(applicationId, byteBuffer);
+
+    final List<Throwable> exceptions = Collections.synchronizedList(new
+        ArrayList<Throwable>());
+
+    final int NUM_THREADS = 10;
+    final CountDownLatch allDone = new CountDownLatch(NUM_THREADS);
+    final ExecutorService threadPool = Executors.newFixedThreadPool(
+        NUM_THREADS);
+
+    final AtomicBoolean stop = new AtomicBoolean(false);
+
+    try {
+      for (int i = 0; i < NUM_THREADS; i++) {
+        threadPool.submit(new Runnable() {
+          @Override
+          public void run() {
+            try {
+              for (int i = 0; i < 100 && !stop.get(); i++) {
+                NodeHeartbeatResponse nodeHeartBeatResponse =
+                    newNodeHeartbeatResponse(0, NodeAction.NORMAL,
+                        null, null, null, null, 0);
+                nodeHeartBeatResponse.setSystemCredentialsForApps(
+                    testCredentials);
+                NodeHeartbeatResponseProto proto =
+                    ((NodeHeartbeatResponsePBImpl)nodeHeartBeatResponse)
+                        .getProto();
+                Assert.assertNotNull(proto);
+              }
+            } catch (Throwable t) {
+              exceptions.add(t);
+              stop.set(true);
+            } finally {
+              allDone.countDown();
+            }
+          }
+        });
+      }
+
+      int testTimeout = 2;
+      Assert.assertTrue("Timeout waiting for more than " + testTimeout + " " +
+              "seconds",
+          allDone.await(testTimeout, TimeUnit.SECONDS));
+    } catch (InterruptedException ie) {
+      exceptions.add(ie);
+    } finally {
+      threadPool.shutdownNow();
+    }
+    Assert.assertTrue("Test failed with exception(s)" + exceptions,
+        exceptions.isEmpty());
+  }
+
   // Add new containers info into NM context each time node heart beats.
   private class MyNMContext extends NMContext {
 


[12/50] [abbrv] hadoop git commit: HADOOP-11484: move CHANGES.txt entry to 3.0

Posted by zh...@apache.org.
HADOOP-11484: move CHANGES.txt entry to 3.0


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

Branch: refs/heads/HDFS-EC
Commit: dc9978a70457f6ed9d94ee68b40c87bf0282cd73
Parents: a865d7c
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Wed Jan 21 11:32:31 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:26 2015 -0800

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/dc9978a7/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 3bd67fe..2ab8ea3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -284,6 +284,9 @@ Trunk (Unreleased)
     HDFS-7643. Test case to ensure lazy persist files cannot be truncated.
     (Yi Liu via Arpit Agarwal)
 
+    HADOOP-11484. hadoop-mapreduce-client-nativetask fails to build on ARM
+    AARCH64 due to x86 asm statements (Edward Nevill via Colin P. McCabe)
+
 Release 2.7.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -753,9 +756,6 @@ Release 2.7.0 - UNRELEASED
     HDFS-7610. Fix removal of dynamically added DN volumes (Lei (Eddy) Xu via
     Colin P. McCabe)
 
-    HADOOP-11484. hadoop-mapreduce-client-nativetask fails to build on ARM
-    AARCH64 due to x86 asm statements (Edward Nevill via Colin P. McCabe)
-
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES


[33/50] [abbrv] hadoop git commit: HDFS-7667. Various typos and improvements to HDFS Federation doc (Charles Lamb via aw)

Posted by zh...@apache.org.
HDFS-7667. Various typos and improvements to HDFS Federation doc  (Charles Lamb via aw)


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

Branch: refs/heads/HDFS-EC
Commit: d56e750e4f413018c1b80736fc4cbfa32f933c0d
Parents: 2e02d86
Author: Allen Wittenauer <aw...@apache.org>
Authored: Fri Jan 23 13:37:46 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:29 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../hadoop-hdfs/src/site/apt/Federation.apt.vm  | 207 +++++++++----------
 2 files changed, 105 insertions(+), 105 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d56e750e/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 9176ec7..c9bee1a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -290,6 +290,9 @@ Trunk (Unreleased)
     HADOOP-11484. hadoop-mapreduce-client-nativetask fails to build on ARM
     AARCH64 due to x86 asm statements (Edward Nevill via Colin P. McCabe)
 
+    HDFS-7667. Various typos and improvements to HDFS Federation doc
+    (Charles Lamb via aw)
+
 Release 2.7.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d56e750e/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/Federation.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/Federation.apt.vm b/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/Federation.apt.vm
index 29278b7..17aaf3c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/Federation.apt.vm
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/Federation.apt.vm
@@ -32,16 +32,16 @@ HDFS Federation
 
   * <<Namespace>>
 
-    * Consists of directories, files and blocks
+    * Consists of directories, files and blocks.
 
     * It supports all the namespace related file system operations such as
       create, delete, modify and list files and directories.
 
-  * <<Block Storage Service>> has two parts
+  * <<Block Storage Service>>, which has two parts:
 
-    * Block Management (which is done in Namenode)
+    * Block Management (performed in the Namenode)
 
-      * Provides datanode cluster membership by handling registrations, and
+      * Provides Datanode cluster membership by handling registrations, and
         periodic heart beats.
 
       * Processes block reports and maintains location of blocks.
@@ -49,29 +49,29 @@ HDFS Federation
       * Supports block related operations such as create, delete, modify and
         get block location.
 
-      * Manages replica placement and replication of a block for under
-        replicated blocks and deletes blocks that are over replicated.
+      * Manages replica placement, block replication for under
+        replicated blocks, and deletes blocks that are over replicated.
 
-    * Storage - is provided by datanodes by storing blocks on the local file
-      system and allows read/write access.
+    * Storage - is provided by Datanodes by storing blocks on the local file
+      system and allowing read/write access.
 
   The prior HDFS architecture allows only a single namespace for the
-  entire cluster. A single Namenode manages this namespace. HDFS
-  Federation addresses limitation of the prior architecture by adding
-  support multiple Namenodes/namespaces to HDFS file system.
+  entire cluster. In that configuration, a single Namenode manages the
+  namespace. HDFS Federation addresses this limitation by adding
+  support for multiple Namenodes/namespaces to HDFS.
 
 * {Multiple Namenodes/Namespaces}
 
   In order to scale the name service horizontally, federation uses multiple
-  independent Namenodes/namespaces. The Namenodes are federated, that is, the
+  independent Namenodes/namespaces. The Namenodes are federated; the
   Namenodes are independent and do not require coordination with each other.
-  The datanodes are used as common storage for blocks by all the Namenodes.
-  Each datanode registers with all the Namenodes in the cluster. Datanodes
-  send periodic heartbeats and block reports and handles commands from the
-  Namenodes.
+  The Datanodes are used as common storage for blocks by all the Namenodes.
+  Each Datanode registers with all the Namenodes in the cluster. Datanodes
+  send periodic heartbeats and block reports. They also handle
+  commands from the Namenodes.
 
-  Users may use {{{./ViewFs.html}ViewFs}} to create personalized namespace views,
-  where ViewFs is analogous to client side mount tables in some Unix/Linux systems.
+  Users may use {{{./ViewFs.html}ViewFs}} to create personalized namespace views.
+  ViewFs is analogous to client side mount tables in some Unix/Linux systems.
 
 [./images/federation.gif] HDFS Federation Architecture
 
@@ -79,66 +79,67 @@ HDFS Federation
   <<Block Pool>>
 
   A Block Pool is a set of blocks that belong to a single namespace.
-  Datanodes store blocks for all the block pools in the cluster.
-  It is managed independently of other block pools. This allows a namespace
-  to generate Block IDs for new blocks without the need for coordination
-  with the other namespaces. The failure of a Namenode does not prevent
-  the datanode from serving other Namenodes in the cluster.
+  Datanodes store blocks for all the block pools in the cluster.  Each
+  Block Pool is managed independently. This allows a namespace to
+  generate Block IDs for new blocks without the need for coordination
+  with the other namespaces. A Namenode failure does not prevent the
+  Datanode from serving other Namenodes in the cluster.
 
   A Namespace and its block pool together are called Namespace Volume.
   It is a self-contained unit of management. When a Namenode/namespace
-  is deleted, the corresponding block pool at the datanodes is deleted.
+  is deleted, the corresponding block pool at the Datanodes is deleted.
   Each namespace volume is upgraded as a unit, during cluster upgrade.
 
   <<ClusterID>>
 
-  A new identifier <<ClusterID>> is added to identify all the nodes in
-  the cluster.  When a Namenode is formatted, this identifier is provided
-  or auto generated. This ID should be used for formatting the other
-  Namenodes into the cluster.
+  A <<ClusterID>> identifier is used to identify all the nodes in the
+  cluster.  When a Namenode is formatted, this identifier is either
+  provided or auto generated. This ID should be used for formatting
+  the other Namenodes into the cluster.
 
 ** Key Benefits
 
-  * Namespace Scalability - HDFS cluster storage scales horizontally but
-    the namespace does not. Large deployments or deployments using lot
-    of small files benefit from scaling the namespace by adding more
-    Namenodes to the cluster
+  * Namespace Scalability - Federation adds namespace horizontal
+    scaling. Large deployments or deployments using lot of small files
+    benefit from namespace scaling by allowing more Namenodes to be
+    added to the cluster.
 
-  * Performance - File system operation throughput is limited by a single
-    Namenode in the prior architecture. Adding more Namenodes to the cluster
-    scales the file system read/write operations throughput.
+  * Performance - File system throughput is not limited by a single
+    Namenode. Adding more Namenodes to the cluster scales the file
+    system read/write throughput.
 
-  * Isolation - A single Namenode offers no isolation in multi user
-    environment. An experimental application can overload the Namenode
-    and slow down production critical applications. With multiple Namenodes,
-    different categories of applications and users can be isolated to
-    different namespaces.
+  * Isolation - A single Namenode offers no isolation in a multi user
+    environment. For example, an experimental application can overload
+    the Namenode and slow down production critical applications. By using
+    multiple Namenodes, different categories of applications and users
+    can be isolated to different namespaces.
 
 * {Federation Configuration}
 
-  Federation configuration is <<backward compatible>> and allows existing
-  single Namenode configuration to work without any change. The new
-  configuration is designed such that all the nodes in the cluster have
-  same configuration without the need for deploying different configuration
-  based on the type of the node in the cluster.
+  Federation configuration is <<backward compatible>> and allows
+  existing single Namenode configurations to work without any
+  change. The new configuration is designed such that all the nodes in
+  the cluster have the same configuration without the need for
+  deploying different configurations based on the type of the node in
+  the cluster.
 
-  A new abstraction called <<<NameServiceID>>> is added with
-  federation. The Namenode and its corresponding secondary/backup/checkpointer
-  nodes belong to this. To support single configuration file, the Namenode and
-  secondary/backup/checkpointer configuration parameters are suffixed with
-  <<<NameServiceID>>> and are added to the same configuration file.
+  Federation adds a new <<<NameServiceID>>> abstraction. A Namenode
+  and its corresponding secondary/backup/checkpointer nodes all belong
+  to a NameServiceId. In order to support a single configuration file,
+  the Namenode and secondary/backup/checkpointer configuration
+  parameters are suffixed with the <<<NameServiceID>>>.
 
 
 ** Configuration:
 
-  <<Step 1>>: Add the following parameters to your configuration:
-  <<<dfs.nameservices>>>: Configure with list of comma separated
-  NameServiceIDs. This will be used by Datanodes to determine all the
+  <<Step 1>>: Add the <<<dfs.nameservices>>> parameter to your
+  configuration and configure it with a list of comma separated
+  NameServiceIDs. This will be used by the Datanodes to determine the
   Namenodes in the cluster.
 
   <<Step 2>>: For each Namenode and Secondary Namenode/BackupNode/Checkpointer
-  add the following configuration suffixed with the corresponding
-  <<<NameServiceID>>> into the common configuration file.
+  add the following configuration parameters suffixed with the corresponding
+  <<<NameServiceID>>> into the common configuration file:
 
 *---------------------+--------------------------------------------+
 || Daemon             || Configuration Parameter                   |
@@ -160,7 +161,7 @@ HDFS Federation
 |                     | <<<dfs.secondary.namenode.keytab.file>>>   |
 *---------------------+--------------------------------------------+
 
-  Here is an example configuration with two namenodes:
+  Here is an example configuration with two Namenodes:
 
 ----
 <configuration>
@@ -199,16 +200,16 @@ HDFS Federation
 
 ** Formatting Namenodes
 
-  <<Step 1>>: Format a namenode using the following command:
+  <<Step 1>>: Format a Namenode using the following command:
 
 ----
 [hdfs]$ $HADOOP_PREFIX/bin/hdfs namenode -format [-clusterId <cluster_id>]
 ----
-  Choose a unique cluster_id, which will not conflict other clusters in
-  your environment. If it is not provided, then a unique ClusterID is
+  Choose a unique cluster_id which will not conflict other clusters in
+  your environment. If a cluster_id is not provided, then a unique one is
   auto generated.
 
-  <<Step 2>>: Format additional namenode using the following command:
+  <<Step 2>>: Format additional Namenodes using the following command:
 
 ----
 [hdfs]$ $HADOOP_PREFIX/bin/hdfs namenode -format -clusterId <cluster_id>
@@ -219,40 +220,38 @@ HDFS Federation
 
 ** Upgrading from an older release and configuring federation
 
-  Older releases supported a single Namenode.
-  Upgrade the cluster to newer release to enable federation
+  Older releases only support a single Namenode.
+  Upgrade the cluster to newer release in order to enable federation
   During upgrade you can provide a ClusterID as follows:
 
 ----
-[hdfs]$ $HADOOP_PREFIX/bin/hdfs start namenode --config $HADOOP_CONF_DIR  -upgrade -clusterId <cluster_ID>
+[hdfs]$ $HADOOP_PREFIX/bin/hdfs --daemon start namenode -upgrade -clusterId <cluster_ID>
 ----
-  If ClusterID is not provided, it is auto generated.
+  If cluster_id is not provided, it is auto generated.
 
 ** Adding a new Namenode to an existing HDFS cluster
 
-  Follow the following steps:
+  Perform the following steps:
 
-  * Add configuration parameter <<<dfs.nameservices>>> to the configuration.
+  * Add <<<dfs.nameservices>>> to the configuration.
 
-  * Update the configuration with NameServiceID suffix. Configuration
-    key names have changed post release 0.20. You must use new configuration
-    parameter names, for federation.
+  * Update the configuration with the NameServiceID suffix. Configuration
+    key names changed post release 0.20. You must use the new configuration
+    parameter names in order to use federation.
 
-  * Add new Namenode related config to the configuration files.
+  * Add the new Namenode related config to the configuration file.
 
   * Propagate the configuration file to the all the nodes in the cluster.
 
-  * Start the new Namenode, Secondary/Backup.
+  * Start the new Namenode and Secondary/Backup.
 
-  * Refresh the datanodes to pickup the newly added Namenode by running
-    the following command:
+  * Refresh the Datanodes to pickup the newly added Namenode by running
+    the following command against all the Datanodes in the cluster:
 
 ----
-[hdfs]$ $HADOOP_PREFIX/bin/hdfs dfadmin -refreshNameNode <datanode_host_name>:<datanode_rpc_port>
+[hdfs]$ $HADOOP_PREFIX/bin/hdfs dfsadmin -refreshNameNode <datanode_host_name>:<datanode_rpc_port>
 ----
 
-  * The above command must be run against all the datanodes in the cluster.
-
 * {Managing the cluster}
 
 **  Starting and stopping cluster
@@ -270,28 +269,28 @@ HDFS Federation
 ----
 
   These commands can be run from any node where the HDFS configuration is
-  available.  The command uses configuration to determine the Namenodes
-  in the cluster and starts the Namenode process on those nodes. The
-  datanodes are started on nodes specified in the <<<slaves>>> file. The
-  script can be used as reference for building your own scripts for
-  starting and stopping the cluster.
+  available.  The command uses the configuration to determine the Namenodes
+  in the cluster and then starts the Namenode process on those nodes. The
+  Datanodes are started on the nodes specified in the <<<slaves>>> file. The
+  script can be used as a reference for building your own scripts to
+  start and stop the cluster.
 
 **  Balancer
 
-  Balancer has been changed to work with multiple Namenodes in the cluster to
-  balance the cluster. Balancer can be run using the command:
+  The Balancer has been changed to work with multiple
+  Namenodes. The Balancer can be run using the command:
 
 ----
 [hdfs]$ $HADOOP_PREFIX/bin/hdfs --daemon start balancer [-policy <policy>]
 ----
 
-  Policy could be:
+  The policy parameter can be any of the following:
 
   * <<<datanode>>> - this is the <default> policy. This balances the storage at
-    the datanode level. This is similar to balancing policy from prior releases.
+    the Datanode level. This is similar to balancing policy from prior releases.
 
-  * <<<blockpool>>> - this balances the storage at the block pool level.
-    Balancing at block pool level balances storage at the datanode level also.
+  * <<<blockpool>>> - this balances the storage at the block pool
+    level which also balances at the Datanode level.
 
   Note that Balancer only balances the data and does not balance the namespace.
   For the complete command usage, see {{{../hadoop-common/CommandsManual.html#balancer}balancer}}.
@@ -299,44 +298,42 @@ HDFS Federation
 ** Decommissioning
 
   Decommissioning is similar to prior releases. The nodes that need to be
-  decomissioned are added to the exclude file at all the Namenode. Each
+  decomissioned are added to the exclude file at all of the Namenodes. Each
   Namenode decommissions its Block Pool. When all the Namenodes finish
-  decommissioning a datanode, the datanode is considered to be decommissioned.
+  decommissioning a Datanode, the Datanode is considered decommissioned.
 
-  <<Step 1>>: To distributed an exclude file to all the Namenodes, use the
+  <<Step 1>>: To distribute an exclude file to all the Namenodes, use the
   following command:
 
 ----
-[hdfs]$ $HADOOP_PREFIX/sbin/distributed-exclude.sh <exclude_file>
+[hdfs]$ $HADOOP_PREFIX/sbin/distribute-exclude.sh <exclude_file>
 ----
 
-  <<Step 2>>: Refresh all the Namenodes to pick up the new exclude file.
+  <<Step 2>>: Refresh all the Namenodes to pick up the new exclude file:
 
 ----
 [hdfs]$ $HADOOP_PREFIX/sbin/refresh-namenodes.sh
 ----
 
-  The above command uses HDFS configuration to determine the Namenodes
-  configured in the cluster and refreshes all the Namenodes to pick up
+  The above command uses HDFS configuration to determine the
+  configured Namenodes in the cluster and refreshes them to pick up
   the new exclude file.
 
 ** Cluster Web Console
 
-  Similar to Namenode status web page, a Cluster Web Console is added in
-  federation to monitor the federated cluster at
+  Similar to the Namenode status web page, when using federation a
+  Cluster Web Console is available to monitor the federated cluster at
   <<<http://<any_nn_host:port>/dfsclusterhealth.jsp>>>.
   Any Namenode in the cluster can be used to access this web page.
 
-  The web page provides the following information:
+  The Cluster Web Console provides the following information:
 
-  * Cluster summary that shows number of files, number of blocks and
-    total configured storage capacity, available and used storage information
+  * A cluster summary that shows the number of files, number of blocks,
+    total configured storage capacity, and the available and used storage
     for the entire cluster.
 
-  * Provides list of Namenodes and summary that includes number of files,
-    blocks, missing blocks, number of live and dead data nodes for each
-    Namenode. It also provides a link to conveniently access Namenode web UI.
-
-  * It also provides decommissioning status of datanodes.
-
+  * A list of Namenodes and a summary that includes the number of files,
+    blocks, missing blocks, and live and dead data nodes for each
+    Namenode. It also provides a link to access each Namenode's web UI.
 
+  * The decommissioning status of Datanodes.


[16/50] [abbrv] hadoop git commit: MAPREDUCE-5785. Derive heap size or mapreduce.*.memory.mb automatically. (Gera Shegalov and Karthik Kambatla via gera)

Posted by zh...@apache.org.
MAPREDUCE-5785. Derive heap size or mapreduce.*.memory.mb automatically. (Gera Shegalov and Karthik Kambatla via gera)


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

Branch: refs/heads/HDFS-EC
Commit: a691658a86562415b9836c46198ff33d125c68bf
Parents: 4ee89ce
Author: Gera Shegalov <ge...@apache.org>
Authored: Wed Jan 21 18:41:43 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:27 2015 -0800

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt            |   2 +
 .../apache/hadoop/mapred/MapReduceChildJVM.java |  32 +----
 .../v2/app/job/impl/TaskAttemptImpl.java        |  15 +--
 .../v2/app/job/impl/TestMapReduceChildJVM.java  |  93 +++++++++++--
 .../java/org/apache/hadoop/mapred/JobConf.java  | 129 ++++++++++++++++++-
 .../java/org/apache/hadoop/mapred/Task.java     |   6 +-
 .../apache/hadoop/mapreduce/MRJobConfig.java    |   5 +
 .../src/main/resources/mapred-default.xml       |  38 ++++--
 8 files changed, 252 insertions(+), 68 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a691658a/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 489369d..b28fc65 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -3,6 +3,8 @@ Hadoop MapReduce Change Log
 Trunk (Unreleased)
 
   INCOMPATIBLE CHANGES
+    MAPREDUCE-5785. Derive heap size or mapreduce.*.memory.mb automatically.
+    (Gera Shegalov and Karthik Kambatla via gera)
 
   NEW FEATURES
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a691658a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/MapReduceChildJVM.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/MapReduceChildJVM.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/MapReduceChildJVM.java
index 817b3a5..936dc5a 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/MapReduceChildJVM.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/MapReduceChildJVM.java
@@ -27,6 +27,7 @@ import java.util.Vector;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.TaskLog.LogName;
 import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.mapreduce.TypeConverter;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
@@ -99,36 +100,7 @@ public class MapReduceChildJVM {
   }
 
   private static String getChildJavaOpts(JobConf jobConf, boolean isMapTask) {
-    String userClasspath = "";
-    String adminClasspath = "";
-    if (isMapTask) {
-      userClasspath = 
-          jobConf.get(
-              JobConf.MAPRED_MAP_TASK_JAVA_OPTS, 
-              jobConf.get(
-                  JobConf.MAPRED_TASK_JAVA_OPTS, 
-                  JobConf.DEFAULT_MAPRED_TASK_JAVA_OPTS)
-          );
-      adminClasspath = 
-          jobConf.get(
-              MRJobConfig.MAPRED_MAP_ADMIN_JAVA_OPTS,
-              MRJobConfig.DEFAULT_MAPRED_ADMIN_JAVA_OPTS);
-    } else {
-      userClasspath =
-          jobConf.get(
-              JobConf.MAPRED_REDUCE_TASK_JAVA_OPTS, 
-              jobConf.get(
-                  JobConf.MAPRED_TASK_JAVA_OPTS,
-                  JobConf.DEFAULT_MAPRED_TASK_JAVA_OPTS)
-              );
-      adminClasspath =
-          jobConf.get(
-              MRJobConfig.MAPRED_REDUCE_ADMIN_JAVA_OPTS,
-              MRJobConfig.DEFAULT_MAPRED_ADMIN_JAVA_OPTS);
-    }
-    
-    // Add admin classpath first so it can be overridden by user.
-    return adminClasspath + " " + userClasspath;
+    return jobConf.getTaskJavaOpts(isMapTask ? TaskType.MAP : TaskType.REDUCE);
   }
 
   public static List<String> getVMCommand(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a691658a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
index dfc6a3f..f4b434b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
@@ -563,19 +563,8 @@ public abstract class TaskAttemptImpl implements
     stateMachine = stateMachineFactory.make(this);
   }
 
-  private int getMemoryRequired(Configuration conf, TaskType taskType) {
-    int memory = 1024;
-    if (taskType == TaskType.MAP)  {
-      memory =
-          conf.getInt(MRJobConfig.MAP_MEMORY_MB,
-              MRJobConfig.DEFAULT_MAP_MEMORY_MB);
-    } else if (taskType == TaskType.REDUCE) {
-      memory =
-          conf.getInt(MRJobConfig.REDUCE_MEMORY_MB,
-              MRJobConfig.DEFAULT_REDUCE_MEMORY_MB);
-    }
-    
-    return memory;
+  private int getMemoryRequired(JobConf conf, TaskType taskType) {
+    return conf.getMemoryRequired(TypeConverter.fromYarn(taskType));
   }
 
   private int getCpuRequired(Configuration conf, TaskType taskType) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a691658a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestMapReduceChildJVM.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestMapReduceChildJVM.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestMapReduceChildJVM.java
index b1e9cf0..57573cc 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestMapReduceChildJVM.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestMapReduceChildJVM.java
@@ -18,8 +18,10 @@
 
 package org.apache.hadoop.mapreduce.v2.app.job.impl;
 
+import java.util.ArrayList;
 import java.util.Map;
 
+import org.apache.hadoop.mapreduce.TaskType;
 import org.junit.Assert;
 
 import org.apache.commons.logging.Log;
@@ -56,8 +58,8 @@ public class TestMapReduceChildJVM {
     Assert.assertEquals(
       "[" + MRApps.crossPlatformify("JAVA_HOME") + "/bin/java" +
       " -Djava.net.preferIPv4Stack=true" +
-      " -Dhadoop.metrics.log.level=WARN" +
-      "  -Xmx200m -Djava.io.tmpdir=" + MRApps.crossPlatformify("PWD") + "/tmp" +
+      " -Dhadoop.metrics.log.level=WARN " +
+      "  -Xmx820m -Djava.io.tmpdir=" + MRApps.crossPlatformify("PWD") + "/tmp" +
       " -Dlog4j.configuration=container-log4j.properties" +
       " -Dyarn.app.container.log.dir=<LOG_DIR>" +
       " -Dyarn.app.container.log.filesize=0" +
@@ -67,7 +69,7 @@ public class TestMapReduceChildJVM {
       " attempt_0_0000_m_000000_0" +
       " 0" +
       " 1><LOG_DIR>/stdout" +
-      " 2><LOG_DIR>/stderr ]", app.myCommandLine);
+      " 2><LOG_DIR>/stderr ]", app.launchCmdList.get(0));
     
     Assert.assertTrue("HADOOP_ROOT_LOGGER not set for job",
       app.cmdEnvironment.containsKey("HADOOP_ROOT_LOGGER"));
@@ -119,8 +121,8 @@ public class TestMapReduceChildJVM {
     Assert.assertEquals(
         "[" + MRApps.crossPlatformify("JAVA_HOME") + "/bin/java" +
             " -Djava.net.preferIPv4Stack=true" +
-            " -Dhadoop.metrics.log.level=WARN" +
-            "  -Xmx200m -Djava.io.tmpdir=" + MRApps.crossPlatformify("PWD") + "/tmp" +
+            " -Dhadoop.metrics.log.level=WARN " +
+            "  -Xmx820m -Djava.io.tmpdir=" + MRApps.crossPlatformify("PWD") + "/tmp" +
             " -Dlog4j.configuration=container-log4j.properties" +
             " -Dyarn.app.container.log.dir=<LOG_DIR>" +
             " -Dyarn.app.container.log.filesize=0" +
@@ -134,7 +136,7 @@ public class TestMapReduceChildJVM {
             " attempt_0_0000_r_000000_0" +
             " 0" +
             " 1><LOG_DIR>/stdout" +
-            " 2><LOG_DIR>/stderr ]", app.myCommandLine);
+            " 2><LOG_DIR>/stderr ]", app.launchCmdList.get(0));
 
     Assert.assertTrue("HADOOP_ROOT_LOGGER not set for job",
         app.cmdEnvironment.containsKey("HADOOP_ROOT_LOGGER"));
@@ -161,8 +163,8 @@ public class TestMapReduceChildJVM {
     Assert.assertEquals(
       "[" + MRApps.crossPlatformify("JAVA_HOME") + "/bin/java" +
       " -Djava.net.preferIPv4Stack=true" +
-      " -Dhadoop.metrics.log.level=WARN" +
-      "  -Xmx200m -Djava.io.tmpdir=" + MRApps.crossPlatformify("PWD") + "/tmp" +
+      " -Dhadoop.metrics.log.level=WARN " +
+      "  -Xmx820m -Djava.io.tmpdir=" + MRApps.crossPlatformify("PWD") + "/tmp" +
       " -Dlog4j.configuration=" + testLogPropertieFile +
       " -Dyarn.app.container.log.dir=<LOG_DIR>" +
       " -Dyarn.app.container.log.filesize=0" +
@@ -172,12 +174,81 @@ public class TestMapReduceChildJVM {
       " attempt_0_0000_m_000000_0" +
       " 0" +
       " 1><LOG_DIR>/stdout" +
-      " 2><LOG_DIR>/stderr ]", app.myCommandLine);
+      " 2><LOG_DIR>/stderr ]", app.launchCmdList.get(0));
+  }
+
+  @Test
+  public void testAutoHeapSizes() throws Exception {
+    // Don't specify heap size or memory-mb
+    testAutoHeapSize(-1, -1, null);
+
+    // Don't specify heap size
+    testAutoHeapSize(512, 768, null);
+    testAutoHeapSize(100, 768, null);
+    testAutoHeapSize(512, 100, null);
+    // Specify heap size
+    testAutoHeapSize(512, 768, "-Xmx100m");
+    testAutoHeapSize(512, 768, "-Xmx500m");
+
+    // Specify heap size but not the memory
+    testAutoHeapSize(-1, -1, "-Xmx100m");
+    testAutoHeapSize(-1, -1, "-Xmx500m");
+  }
+
+  private void testAutoHeapSize(int mapMb, int redMb, String xmxArg)
+      throws Exception {
+    JobConf conf = new JobConf();
+    float heapRatio = conf.getFloat(MRJobConfig.HEAP_MEMORY_MB_RATIO,
+        MRJobConfig.DEFAULT_HEAP_MEMORY_MB_RATIO);
+
+    // Verify map and reduce java opts are not set by default
+    Assert.assertNull("Default map java opts!",
+        conf.get(MRJobConfig.MAP_JAVA_OPTS));
+    Assert.assertNull("Default reduce java opts!",
+        conf.get(MRJobConfig.REDUCE_JAVA_OPTS));
+    // Set the memory-mbs and java-opts
+    if (mapMb > 0) {
+      conf.setInt(MRJobConfig.MAP_MEMORY_MB, mapMb);
+    } else {
+      mapMb = conf.getMemoryRequired(TaskType.MAP);
+    }
+
+    if (redMb > 0) {
+      conf.setInt(MRJobConfig.REDUCE_MEMORY_MB, redMb);
+    } else {
+      redMb = conf.getMemoryRequired(TaskType.REDUCE);
+    }
+    if (xmxArg != null) {
+      conf.set(MRJobConfig.MAP_JAVA_OPTS, xmxArg);
+      conf.set(MRJobConfig.REDUCE_JAVA_OPTS, xmxArg);
+    }
+
+    // Submit job to let unspecified fields be picked up
+    MyMRApp app = new MyMRApp(1, 1, true, this.getClass().getName(), true);
+    Job job = app.submit(conf);
+    app.waitForState(job, JobState.SUCCEEDED);
+    app.verifyCompleted();
+
+    // Go through the tasks and verify the values are as expected
+    for (String cmd : app.launchCmdList) {
+      final boolean isMap = cmd.contains("_m_");
+      int heapMb;
+      if (xmxArg == null) {
+        heapMb = (int)(Math.ceil((isMap ? mapMb : redMb) * heapRatio));
+      } else {
+        final String javaOpts = conf.get(isMap
+            ? MRJobConfig.MAP_JAVA_OPTS
+            : MRJobConfig.REDUCE_JAVA_OPTS);
+        heapMb = JobConf.parseMaximumHeapSizeMB(javaOpts);
+      }
+      Assert.assertEquals("Incorrect heapsize in the command opts",
+          heapMb, JobConf.parseMaximumHeapSizeMB(cmd));
+    }
   }
 
   private static final class MyMRApp extends MRApp {
 
-    private String myCommandLine;
+    private ArrayList<String> launchCmdList = new ArrayList<>();
     private Map<String, String> cmdEnvironment;
 
     public MyMRApp(int maps, int reduces, boolean autoComplete,
@@ -196,7 +267,7 @@ public class TestMapReduceChildJVM {
                 launchEvent.getContainerLaunchContext();
             String cmdString = launchContext.getCommands().toString();
             LOG.info("launchContext " + cmdString);
-            myCommandLine = cmdString;
+            launchCmdList.add(cmdString);
             cmdEnvironment = launchContext.getEnvironment();
           }
           super.handle(event);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a691658a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java
index 03f1160..98a643f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java
@@ -20,8 +20,10 @@ package org.apache.hadoop.mapred;
 
 
 import java.io.IOException;
+import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -44,6 +46,7 @@ import org.apache.hadoop.mapred.lib.KeyFieldBasedComparator;
 import org.apache.hadoop.mapred.lib.KeyFieldBasedPartitioner;
 import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.mapreduce.filecache.DistributedCache;
 import org.apache.hadoop.mapreduce.util.ConfigUtil;
 import org.apache.hadoop.security.Credentials;
@@ -114,6 +117,8 @@ import org.apache.log4j.Level;
 public class JobConf extends Configuration {
 
   private static final Log LOG = LogFactory.getLog(JobConf.class);
+  private static final Pattern JAVA_OPTS_XMX_PATTERN =
+          Pattern.compile(".*(?:^|\\s)-Xmx(\\d+)([gGmMkK]?)(?:$|\\s).*");
 
   static{
     ConfigUtil.loadResources();
@@ -247,9 +252,9 @@ public class JobConf extends Configuration {
    */
   public static final String MAPRED_REDUCE_TASK_JAVA_OPTS = 
     JobContext.REDUCE_JAVA_OPTS;
-  
-  public static final String DEFAULT_MAPRED_TASK_JAVA_OPTS = "-Xmx200m";
-  
+
+  public static final String DEFAULT_MAPRED_TASK_JAVA_OPTS = "";
+
   /**
    * @deprecated
    * Configuration key to set the maximum virtual memory available to the child
@@ -2022,7 +2027,123 @@ public class JobConf extends Configuration {
       LOG.warn(JobConf.deprecatedString(JobConf.MAPRED_REDUCE_TASK_ULIMIT));
     }
   }
-  
+
+  private String getConfiguredTaskJavaOpts(TaskType taskType) {
+    String userClasspath = "";
+    String adminClasspath = "";
+    if (taskType == TaskType.MAP) {
+      userClasspath = get(MAPRED_MAP_TASK_JAVA_OPTS,
+          get(MAPRED_TASK_JAVA_OPTS, DEFAULT_MAPRED_TASK_JAVA_OPTS));
+      adminClasspath = get(MRJobConfig.MAPRED_MAP_ADMIN_JAVA_OPTS,
+          MRJobConfig.DEFAULT_MAPRED_ADMIN_JAVA_OPTS);
+    } else {
+      userClasspath = get(MAPRED_REDUCE_TASK_JAVA_OPTS,
+          get(MAPRED_TASK_JAVA_OPTS, DEFAULT_MAPRED_TASK_JAVA_OPTS));
+      adminClasspath = get(MRJobConfig.MAPRED_REDUCE_ADMIN_JAVA_OPTS,
+          MRJobConfig.DEFAULT_MAPRED_ADMIN_JAVA_OPTS);
+    }
+
+    return adminClasspath + " " + userClasspath;
+  }
+
+  @Private
+  public String getTaskJavaOpts(TaskType taskType) {
+    String javaOpts = getConfiguredTaskJavaOpts(taskType);
+
+    if (!javaOpts.contains("-Xmx")) {
+      float heapRatio = getFloat(MRJobConfig.HEAP_MEMORY_MB_RATIO,
+          MRJobConfig.DEFAULT_HEAP_MEMORY_MB_RATIO);
+
+      if (heapRatio > 1.0f || heapRatio < 0) {
+        LOG.warn("Invalid value for " + MRJobConfig.HEAP_MEMORY_MB_RATIO
+            + ", using the default.");
+        heapRatio = MRJobConfig.DEFAULT_HEAP_MEMORY_MB_RATIO;
+      }
+
+      int taskContainerMb = getMemoryRequired(taskType);
+      int taskHeapSize = (int)Math.ceil(taskContainerMb * heapRatio);
+
+      String xmxArg = String.format("-Xmx%dm", taskHeapSize);
+      LOG.info("Task java-opts do not specify heap size. Setting task attempt" +
+          " jvm max heap size to " + xmxArg);
+
+      javaOpts += " " + xmxArg;
+    }
+
+    return javaOpts;
+  }
+
+  /**
+   * Parse the Maximum heap size from the java opts as specified by the -Xmx option
+   * Format: -Xmx<size>[g|G|m|M|k|K]
+   * @param javaOpts String to parse to read maximum heap size
+   * @return Maximum heap size in MB or -1 if not specified
+   */
+  @Private
+  @VisibleForTesting
+  public static int parseMaximumHeapSizeMB(String javaOpts) {
+    // Find the last matching -Xmx following word boundaries
+    Matcher m = JAVA_OPTS_XMX_PATTERN.matcher(javaOpts);
+    if (m.matches()) {
+      int size = Integer.parseInt(m.group(1));
+      if (size <= 0) {
+        return -1;
+      }
+      if (m.group(2).isEmpty()) {
+        // -Xmx specified in bytes
+        return size / (1024 * 1024);
+      }
+      char unit = m.group(2).charAt(0);
+      switch (unit) {
+        case 'g':
+        case 'G':
+          // -Xmx specified in GB
+          return size * 1024;
+        case 'm':
+        case 'M':
+          // -Xmx specified in MB
+          return size;
+        case 'k':
+        case 'K':
+          // -Xmx specified in KB
+          return size / 1024;
+      }
+    }
+    // -Xmx not specified
+    return -1;
+  }
+
+  private int getMemoryRequiredHelper(
+      String configName, int defaultValue, int heapSize, float heapRatio) {
+    int memory = getInt(configName, -1);
+    if (memory <= 0) {
+      if (heapSize > 0) {
+        memory = (int) Math.ceil(heapSize / heapRatio);
+        LOG.info("Figured value for " + configName + " from javaOpts");
+      } else {
+        memory = defaultValue;
+      }
+    }
+
+    return memory;
+  }
+
+  @Private
+  public int getMemoryRequired(TaskType taskType) {
+    int memory = 1024;
+    int heapSize = parseMaximumHeapSizeMB(getConfiguredTaskJavaOpts(taskType));
+    float heapRatio = getFloat(MRJobConfig.HEAP_MEMORY_MB_RATIO,
+        MRJobConfig.DEFAULT_HEAP_MEMORY_MB_RATIO);
+    if (taskType == TaskType.MAP) {
+      return getMemoryRequiredHelper(MRJobConfig.MAP_MEMORY_MB,
+          MRJobConfig.DEFAULT_MAP_MEMORY_MB, heapSize, heapRatio);
+    } else if (taskType == TaskType.REDUCE) {
+      return getMemoryRequiredHelper(MRJobConfig.REDUCE_MEMORY_MB,
+          MRJobConfig.DEFAULT_REDUCE_MEMORY_MB, heapSize, heapRatio);
+    } else {
+      return memory;
+    }
+  }
 
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a691658a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Task.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Task.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Task.java
index 3a4c513..5274438 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Task.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Task.java
@@ -34,6 +34,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
 
 import javax.crypto.SecretKey;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -624,8 +625,9 @@ abstract public class Task implements Writable, Configurable {
      * Using AtomicBoolean since we need an atomic read & reset method. 
      */  
     private AtomicBoolean progressFlag = new AtomicBoolean(false);
-    
-    TaskReporter(Progress taskProgress,
+
+    @VisibleForTesting
+    public TaskReporter(Progress taskProgress,
                  TaskUmbilicalProtocol umbilical) {
       this.umbilical = umbilical;
       this.taskProgress = taskProgress;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a691658a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
index 915353b..44f57f4 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
@@ -810,6 +810,11 @@ public interface MRJobConfig {
   public static final String TASK_PREEMPTION =
       "mapreduce.job.preemption";
 
+  public static final String HEAP_MEMORY_MB_RATIO =
+      "mapreduce.job.heap.memory-mb.ratio";
+
+  public static final float DEFAULT_HEAP_MEMORY_MB_RATIO = 0.8f;
+
   public static final String MR_ENCRYPTED_INTERMEDIATE_DATA =
       "mapreduce.job.encrypted-intermediate-data";
   public static final boolean DEFAULT_MR_ENCRYPTED_INTERMEDIATE_DATA = false;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a691658a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
index 30e291b..57a17a8 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
@@ -208,9 +208,11 @@
 
 <property>
   <name>mapreduce.map.memory.mb</name>
-  <value>1024</value>
+  <value>-1</value>
   <description>The amount of memory to request from the scheduler for each
-  map task.
+    map task. If this is not specified or is non-positive, it is inferred from
+    mapreduce.map.java.opts and mapreduce.job.heap.memory-mb.ratio.
+    If java-opts are also not specified, we set it to 1024.
   </description>
 </property>
 
@@ -224,9 +226,11 @@
 
 <property>
   <name>mapreduce.reduce.memory.mb</name>
-  <value>1024</value>
+  <value>-1</value>
   <description>The amount of memory to request from the scheduler for each
-  reduce task.
+    reduce task. If this is not specified or is non-positive, it is inferred
+    from mapreduce.reduce.java.opts and mapreduce.job.heap.memory-mb.ratio.
+    If java-opts are also not specified, we set it to 1024.
   </description>
 </property>
 
@@ -240,7 +244,7 @@
 
 <property>
   <name>mapred.child.java.opts</name>
-  <value>-Xmx200m</value>
+  <value></value>
   <description>Java opts for the task processes.
   The following symbol, if present, will be interpolated: @taskid@ is replaced 
   by current TaskID. Any other occurrences of '@' will go unchanged.
@@ -251,7 +255,10 @@
   Usage of -Djava.library.path can cause programs to no longer function if
   hadoop native libraries are used. These values should instead be set as part 
   of LD_LIBRARY_PATH in the map / reduce JVM env using the mapreduce.map.env and 
-  mapreduce.reduce.env config settings. 
+  mapreduce.reduce.env config settings.
+
+  If -Xmx is not set, it is inferred from mapreduce.{map|reduce}.memory.mb and
+  mapreduce.job.heap.memory-mb.ratio.
   </description>
 </property>
 
@@ -260,7 +267,9 @@
   <name>mapreduce.map.java.opts</name>
   <value></value>
   <description>Java opts only for the child processes that are maps. If set,
-  this will be used instead of mapred.child.java.opts.
+  this will be used instead of mapred.child.java.opts. If -Xmx is not set,
+  it is inferred from mapreduce.map.memory.mb and
+  mapreduce.job.heap.memory-mb.ratio.
   </description>
 </property>
 -->
@@ -270,7 +279,9 @@
   <name>mapreduce.reduce.java.opts</name>
   <value></value>
   <description>Java opts only for the child processes that are reduces. If set,
-  this will be used instead of mapred.child.java.opts.
+  this will be used instead of mapred.child.java.opts. If -Xmx is not set,
+  it is inferred from mapreduce.reduce.memory.mb and
+  mapreduce.job.heap.memory-mb.ratio.
   </description>
 </property>
 -->
@@ -1567,4 +1578,15 @@
     - HTTPS_ONLY : Service is provided only on https
   </description>
 </property>
+
+<property>
+  <name>mapreduce.job.heap.memory-mb.ratio</name>
+  <value>0.8</value>
+  <description>The ratio of heap-size to container-size. If no -Xmx is
+    specified, it is calculated as
+    (mapreduce.{map|reduce}.memory.mb * mapreduce.heap.memory-mb.ratio).
+    If -Xmx is specified but not mapreduce.{map|reduce}.memory.mb, it is
+    calculated as (heapSize / mapreduce.heap.memory-mb.ratio).
+  </description>
+</property>
 </configuration>


[29/50] [abbrv] hadoop git commit: HADOOP-11008. Remove duplicated description about proxy-user in site documents (Masatake Iwasaki via aw)

Posted by zh...@apache.org.
HADOOP-11008. Remove duplicated description about proxy-user in site documents (Masatake Iwasaki via aw)


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

Branch: refs/heads/HDFS-EC
Commit: c13d501f51bc26368ef04631c2aadc2365a05c26
Parents: abb0115
Author: Allen Wittenauer <aw...@apache.org>
Authored: Thu Jan 22 14:30:21 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:28 2015 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 +
 .../src/site/apt/SecureMode.apt.vm              | 53 +-------------
 .../src/site/apt/Superusers.apt.vm              | 74 ++++++++++++++++----
 hadoop-project/src/site/site.xml                |  2 +-
 4 files changed, 64 insertions(+), 68 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c13d501f/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 aaa7041..47eaf7b 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -747,6 +747,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11500. InputStream is left unclosed in ApplicationClassLoader.
     (Ted Yu via ozawa)
 
+    HADOOP-11008. Remove duplicated description about proxy-user in site 
+    documents (Masatake Iwasaki via aw)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c13d501f/hadoop-common-project/hadoop-common/src/site/apt/SecureMode.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/apt/SecureMode.apt.vm b/hadoop-common-project/hadoop-common/src/site/apt/SecureMode.apt.vm
index 0a11bef..0235219 100644
--- a/hadoop-common-project/hadoop-common/src/site/apt/SecureMode.apt.vm
+++ b/hadoop-common-project/hadoop-common/src/site/apt/SecureMode.apt.vm
@@ -202,58 +202,7 @@ KVNO Timestamp         Principal
 
   Some products such as Apache Oozie which access the services of Hadoop
   on behalf of end users need to be able to impersonate end users.
-  You can configure proxy user using properties
-  <<<hadoop.proxyuser.${superuser}.hosts>>> along with either or both of 
-  <<<hadoop.proxyuser.${superuser}.groups>>>
-  and <<<hadoop.proxyuser.${superuser}.users>>>.
-
-  For example, by specifying as below in core-site.xml,
-  user named <<<oozie>>> accessing from any host
-  can impersonate any user belonging to any group.
-
-----
-  <property>
-    <name>hadoop.proxyuser.oozie.hosts</name>
-    <value>*</value>
-  </property>
-  <property>
-    <name>hadoop.proxyuser.oozie.groups</name>
-    <value>*</value>
-  </property>
-----
-
-  User named <<<oozie>>> accessing from any host
-  can impersonate user1 and user2 by specifying as below in core-site.xml.
-
-----
-  <property>
-    <name>hadoop.proxyuser.oozie.hosts</name>
-    <value>*</value>
-  </property>
-  <property>
-    <name>hadoop.proxyuser.oozie.users</name>
-    <value>user1,user2</value>
-  </property>
-----
-
-  The <<<hadoop.proxyuser.${superuser}.hosts>>> accepts list of ip addresses,
-  ip address ranges in CIDR format and/or host names.
-  
-  For example, by specifying as below in core-site.xml,
-  user named <<<oozie>>> accessing from hosts in the range 
-  10.222.0.0-15 and 10.113.221.221
-  can impersonate any user belonging to any group.
-  
-----
-  <property>
-    <name>hadoop.proxyuser.oozie.hosts</name>
-    <value>10.222.0.0/16,10.113.221.221</value>
-  </property>
-  <property>
-    <name>hadoop.proxyuser.oozie.groups</name>
-    <value>*</value>
-  </property>
-----
+  See {{{./Superusers.html}the doc of proxy user}} for details.
 
 ** Secure DataNode
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c13d501f/hadoop-common-project/hadoop-common/src/site/apt/Superusers.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/apt/Superusers.apt.vm b/hadoop-common-project/hadoop-common/src/site/apt/Superusers.apt.vm
index f940884..78ed9a4 100644
--- a/hadoop-common-project/hadoop-common/src/site/apt/Superusers.apt.vm
+++ b/hadoop-common-project/hadoop-common/src/site/apt/Superusers.apt.vm
@@ -11,19 +11,19 @@
 ~~ limitations under the License. See accompanying LICENSE file.
 
   ---
-  Superusers Acting On Behalf Of Other Users
+  Proxy user - Superusers Acting On Behalf Of Other Users
   ---
   ---
   ${maven.build.timestamp}
 
-Superusers Acting On Behalf Of Other Users
+Proxy user - Superusers Acting On Behalf Of Other Users
 
 %{toc|section=1|fromDepth=0}
 
 * Introduction
 
    This document describes how a superuser can submit jobs or access hdfs
-   on behalf of another user in a secured way.
+   on behalf of another user.
 
 * Use Case
 
@@ -38,9 +38,12 @@ Superusers Acting On Behalf Of Other Users
    on a connection authenticated with super's kerberos credentials. In
    other words super is impersonating the user joe.
 
+   Some products such as Apache Oozie need this.
+
+
 * Code example
 
-   In this example super's kerberos credentials are used for login and a
+   In this example super's credentials are used for login and a
    proxy user ugi object is created for joe. The operations are performed
    within the doAs method of this proxy user ugi object.
 
@@ -63,21 +66,26 @@ Superusers Acting On Behalf Of Other Users
 
 * Configurations
 
-   The superuser must be configured on namenode and jobtracker to be
-   allowed to impersonate another user. Following configurations are
-   required.
+   You can configure proxy user using properties
+   <<<hadoop.proxyuser.${superuser}.hosts>>> along with either or both of 
+   <<<hadoop.proxyuser.${superuser}.groups>>>
+   and <<<hadoop.proxyuser.${superuser}.users>>>.
+
+   By specifying as below in core-site.xml,
+   the superuser named <<<super>>> can connect
+   only from <<<host1>>> and <<<host2>>>
+   to impersonate a user belonging to <<<group1>>> and <<<group2>>>.
 
 ----
    <property>
-     <name>hadoop.proxyuser.super.groups</name>
-     <value>group1,group2</value>
-     <description>Allow the superuser super to impersonate any members of the group group1 and group2</description>
-   </property>
-   <property>
      <name>hadoop.proxyuser.super.hosts</name>
      <value>host1,host2</value>
-     <description>The superuser can connect only from host1 and host2 to impersonate a user</description>
    </property>
+   <property>
+     <name>hadoop.proxyuser.super.groups</name>
+     <value>group1,group2</value>
+   </property>
+
 ----
 
    If these configurations are not present, impersonation will not be
@@ -85,11 +93,47 @@ Superusers Acting On Behalf Of Other Users
 
    If more lax security is preferred, the wildcard value * may be used to
    allow impersonation from any host or of any user.
+   For example, by specifying as below in core-site.xml,
+   user named <<<oozie>>> accessing from any host
+   can impersonate any user belonging to any group.
+
+----
+  <property>
+    <name>hadoop.proxyuser.oozie.hosts</name>
+    <value>*</value>
+  </property>
+  <property>
+    <name>hadoop.proxyuser.oozie.groups</name>
+    <value>*</value>
+  </property>
+----
+
+   The <<<hadoop.proxyuser.${superuser}.hosts>>> accepts list of ip addresses,
+   ip address ranges in CIDR format and/or host names.
+   For example, by specifying as below,
+   user named <<<super>>> accessing from hosts in the range 
+   <<<10.222.0.0-15>>> and <<<10.113.221.221>>> can impersonate
+   <<<user1>>> and <<<user2>>>.
+      
+----
+   <property>
+     <name>hadoop.proxyuser.super.hosts</name>
+     <value>10.222.0.0/16,10.113.221.221</value>
+   </property>
+   <property>
+     <name>hadoop.proxyuser.super.users</name>
+     <value>user1,user2</value>
+   </property>
+----
+
 
 * Caveats
 
-   The superuser must have kerberos credentials to be able to impersonate
-   another user. It cannot use delegation tokens for this feature. It
+   If the cluster is running in {{{./SecureMode.html}Secure Mode}},
+   the superuser must have kerberos credentials to be able to impersonate
+   another user.
+
+   It cannot use delegation tokens for this feature. It
    would be wrong if superuser adds its own delegation token to the proxy
    user ugi, as it will allow the proxy user to connect to the service
    with the privileges of the superuser.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c13d501f/hadoop-project/src/site/site.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/src/site/site.xml b/hadoop-project/src/site/site.xml
index 637f7eb..6fa6648 100644
--- a/hadoop-project/src/site/site.xml
+++ b/hadoop-project/src/site/site.xml
@@ -60,7 +60,7 @@
     <menu name="Common" inherit="top">
       <item name="CLI Mini Cluster" href="hadoop-project-dist/hadoop-common/CLIMiniCluster.html"/>
       <item name="Native Libraries" href="hadoop-project-dist/hadoop-common/NativeLibraries.html"/>
-      <item name="Superusers" href="hadoop-project-dist/hadoop-common/Superusers.html"/>
+      <item name="Proxy User" href="hadoop-project-dist/hadoop-common/Superusers.html"/>
       <item name="Secure Mode" href="hadoop-project-dist/hadoop-common/SecureMode.html"/>
       <item name="Service Level Authorization" href="hadoop-project-dist/hadoop-common/ServiceLevelAuth.html"/>
       <item name="HTTP Authentication" href="hadoop-project-dist/hadoop-common/HttpAuthentication.html"/>


[31/50] [abbrv] hadoop git commit: HDFS-3519. Checkpoint upload may interfere with a concurrent saveNamespace. Contributed by Ming Ma.

Posted by zh...@apache.org.
HDFS-3519. Checkpoint upload may interfere with a concurrent saveNamespace. 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/d3b92a12
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d3b92a12
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d3b92a12

Branch: refs/heads/HDFS-EC
Commit: d3b92a1230dab9009ee71ca4aa1a796be8927fb8
Parents: 8262acf
Author: cnauroth <cn...@apache.org>
Authored: Thu Jan 22 16:26:21 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:28 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../hadoop/hdfs/server/namenode/FSImage.java    | 56 ++++++++++++++++----
 .../hdfs/server/namenode/ImageServlet.java      | 18 +++----
 .../namenode/ha/TestStandbyCheckpoints.java     |  2 +-
 4 files changed, 58 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3b92a12/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 7c5c639..74eb160 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -765,6 +765,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7575. Upgrade should generate a unique storage ID for each
     volume. (Arpit Agarwal)
 
+    HDFS-3519. Checkpoint upload may interfere with a concurrent saveNamespace.
+    (Ming Ma via cnauroth)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3b92a12/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
index 8ac6926..3b5d2c3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
@@ -29,9 +29,11 @@ import java.util.Collection;
 import java.util.Collections;
 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.Set;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -97,6 +99,15 @@ public class FSImage implements Closeable {
 
   protected NNStorageRetentionManager archivalManager;
 
+  /* Used to make sure there are no concurrent checkpoints for a given txid
+   * The checkpoint here could be one of the following operations.
+   * a. checkpoint when NN is in standby.
+   * b. admin saveNameSpace operation.
+   * c. download checkpoint file from any remote checkpointer.
+  */
+  private final Set<Long> currentlyCheckpointing =
+      Collections.<Long>synchronizedSet(new HashSet<Long>());
+
   /**
    * Construct an FSImage
    * @param conf Configuration
@@ -1058,18 +1069,26 @@ public class FSImage implements Closeable {
       editLog.endCurrentLogSegment(true);
     }
     long imageTxId = getLastAppliedOrWrittenTxId();
+    if (!addToCheckpointing(imageTxId)) {
+      throw new IOException(
+          "FS image is being downloaded from another NN at txid " + imageTxId);
+    }
     try {
-      saveFSImageInAllDirs(source, nnf, imageTxId, canceler);
-      storage.writeAll();
-    } finally {
-      if (editLogWasOpen) {
-        editLog.startLogSegmentAndWriteHeaderTxn(imageTxId + 1);
-        // Take this opportunity to note the current transaction.
-        // Even if the namespace save was cancelled, this marker
-        // is only used to determine what transaction ID is required
-        // for startup. So, it doesn't hurt to update it unnecessarily.
-        storage.writeTransactionIdFileToStorage(imageTxId + 1);
+      try {
+        saveFSImageInAllDirs(source, nnf, imageTxId, canceler);
+        storage.writeAll();
+      } finally {
+        if (editLogWasOpen) {
+          editLog.startLogSegmentAndWriteHeaderTxn(imageTxId + 1);
+          // Take this opportunity to note the current transaction.
+          // Even if the namespace save was cancelled, this marker
+          // is only used to determine what transaction ID is required
+          // for startup. So, it doesn't hurt to update it unnecessarily.
+          storage.writeTransactionIdFileToStorage(imageTxId + 1);
+        }
       }
+    } finally {
+      removeFromCheckpointing(imageTxId);
     }
   }
 
@@ -1078,7 +1097,22 @@ public class FSImage implements Closeable {
    */
   protected synchronized void saveFSImageInAllDirs(FSNamesystem source, long txid)
       throws IOException {
-    saveFSImageInAllDirs(source, NameNodeFile.IMAGE, txid, null);
+    if (!addToCheckpointing(txid)) {
+      throw new IOException(("FS image is being downloaded from another NN"));
+    }
+    try {
+      saveFSImageInAllDirs(source, NameNodeFile.IMAGE, txid, null);
+    } finally {
+      removeFromCheckpointing(txid);
+    }
+  }
+
+  public boolean addToCheckpointing(long txid) {
+    return currentlyCheckpointing.add(txid);
+  }
+
+  public void removeFromCheckpointing(long txid) {
+    currentlyCheckpointing.remove(txid);
   }
 
   private synchronized void saveFSImageInAllDirs(FSNamesystem source,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3b92a12/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageServlet.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageServlet.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageServlet.java
index d10aacc..702c8f1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageServlet.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageServlet.java
@@ -81,9 +81,6 @@ public class ImageServlet extends HttpServlet {
   private static final String LATEST_FSIMAGE_VALUE = "latest";
   private static final String IMAGE_FILE_TYPE = "imageFile";
 
-  private static final Set<Long> currentlyDownloadingCheckpoints =
-    Collections.synchronizedSet(new HashSet<Long>());
-  
   @Override
   public void doGet(final HttpServletRequest request,
       final HttpServletResponse response) throws ServletException, IOException {
@@ -467,17 +464,20 @@ public class ImageServlet extends HttpServlet {
 
               final NameNodeFile nnf = parsedParams.getNameNodeFile();
 
-              if (!currentlyDownloadingCheckpoints.add(txid)) {
+              if (!nnImage.addToCheckpointing(txid)) {
                 response.sendError(HttpServletResponse.SC_CONFLICT,
-                    "Another checkpointer is already in the process of uploading a"
-                        + " checkpoint made at transaction ID " + txid);
+                    "Either current namenode is checkpointing or another"
+                        + " checkpointer is already in the process of "
+                        + "uploading a checkpoint made at transaction ID "
+                        + txid);
                 return null;
               }
               try {
                 if (nnImage.getStorage().findImageFile(nnf, txid) != null) {
                   response.sendError(HttpServletResponse.SC_CONFLICT,
-                      "Another checkpointer already uploaded an checkpoint "
-                          + "for txid " + txid);
+                      "Either current namenode has checkpointed or "
+                          + "another checkpointer already uploaded an "
+                          + "checkpoint for txid " + txid);
                   return null;
                 }
 
@@ -502,7 +502,7 @@ public class ImageServlet extends HttpServlet {
                   stream.close();
                 }
               } finally {
-                currentlyDownloadingCheckpoints.remove(txid);
+                nnImage.removeFromCheckpointing(txid);
               }
               return null;
             }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3b92a12/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java
index 1d75c30..33af0e2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java
@@ -466,7 +466,7 @@ public class TestStandbyCheckpoints {
         throws IOException {
       CompressionOutputStream ret = super.createOutputStream(out);
       CompressionOutputStream spy = Mockito.spy(ret);
-      Mockito.doAnswer(new GenericTestUtils.SleepAnswer(2))
+      Mockito.doAnswer(new GenericTestUtils.SleepAnswer(5))
         .when(spy).write(Mockito.<byte[]>any(), Mockito.anyInt(), Mockito.anyInt());
       return spy;
     }


[28/50] [abbrv] hadoop git commit: HDFS-7575. Upgrade should generate a unique storage ID for each volume. (Contributed by Arpit Agarwal)

Posted by zh...@apache.org.
HDFS-7575. Upgrade should generate a unique storage ID for each volume. (Contributed by Arpit Agarwal)


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

Branch: refs/heads/HDFS-EC
Commit: abb011549865e845059e9b388377fba178b3b391
Parents: 3f90fca
Author: Arpit Agarwal <ar...@apache.org>
Authored: Thu Jan 22 14:08:20 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:28 2015 -0800

----------------------------------------------------------------------
 .../hdfs/server/datanode/DataStorage.java       |  35 +++--
 .../hdfs/server/protocol/DatanodeStorage.java   |  19 ++-
 .../hadoop/hdfs/TestDFSUpgradeFromImage.java    |  19 ++-
 .../hadoop/hdfs/TestDatanodeLayoutUpgrade.java  |   2 +-
 ...estDatanodeStartupFixesLegacyStorageIDs.java | 139 +++++++++++++++++++
 .../apache/hadoop/hdfs/UpgradeUtilities.java    |   2 +-
 .../server/datanode/SimulatedFSDataset.java     |   2 +-
 .../fsdataset/impl/TestFsDatasetImpl.java       |   2 +-
 .../testUpgradeFrom22FixesStorageIDs.tgz        | Bin 0 -> 3260 bytes
 .../testUpgradeFrom22FixesStorageIDs.txt        |  25 ++++
 .../testUpgradeFrom22via26FixesStorageIDs.tgz   | Bin 0 -> 3635 bytes
 .../testUpgradeFrom22via26FixesStorageIDs.txt   |  25 ++++
 .../testUpgradeFrom26PreservesStorageIDs.tgz    | Bin 0 -> 3852 bytes
 .../testUpgradeFrom26PreservesStorageIDs.txt    |  25 ++++
 14 files changed, 274 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/abb01154/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 3ea8ce3..0602dfb 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
@@ -37,6 +37,7 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
@@ -142,11 +143,20 @@ public class DataStorage extends Storage {
     this.datanodeUuid = newDatanodeUuid;
   }
 
-  /** Create an ID for this storage. */
-  public synchronized void createStorageID(StorageDirectory sd) {
-    if (sd.getStorageUuid() == null) {
+  /** Create an ID for this storage.
+   * @return true if a new storage ID was generated.
+   * */
+  public synchronized boolean createStorageID(
+      StorageDirectory sd, boolean regenerateStorageIds) {
+    final String oldStorageID = sd.getStorageUuid();
+    if (oldStorageID == null || regenerateStorageIds) {
       sd.setStorageUuid(DatanodeStorage.generateUuid());
+      LOG.info("Generated new storageID " + sd.getStorageUuid() +
+          " for directory " + sd.getRoot() +
+          (oldStorageID == null ? "" : (" to replace " + oldStorageID)));
+      return true;
     }
+    return false;
   }
 
   /**
@@ -677,20 +687,25 @@ public class DataStorage extends Storage {
           + sd.getRoot().getCanonicalPath() + ": namenode clusterID = "
           + nsInfo.getClusterID() + "; datanode clusterID = " + getClusterID());
     }
-    
-    // After addition of the federation feature, ctime check is only 
-    // meaningful at BlockPoolSliceStorage level. 
 
-    // regular start up. 
+    // 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 == HdfsConstants.DATANODE_LAYOUT_VERSION) {
-      createStorageID(sd);
+      createStorageID(sd, !haveValidStorageId);
       return; // regular startup
     }
-    
+
     // do upgrade
     if (this.layoutVersion > HdfsConstants.DATANODE_LAYOUT_VERSION) {
       doUpgrade(datanode, sd, nsInfo);  // upgrade
-      createStorageID(sd);
+      createStorageID(sd, !haveValidStorageId);
       return;
     }
     

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abb01154/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorage.java
index 4fe07b9..4d224d5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorage.java
@@ -47,6 +47,7 @@ public class DatanodeStorage {
   private final String storageID;
   private final State state;
   private final StorageType storageType;
+  private static final String STORAGE_ID_PREFIX = "DS-";
 
   /**
    * Create a storage with {@link State#NORMAL} and {@link StorageType#DEFAULT}.
@@ -80,7 +81,23 @@ public class DatanodeStorage {
    * @return unique storage ID
    */
   public static String generateUuid() {
-    return "DS-" + UUID.randomUUID();
+    return STORAGE_ID_PREFIX + UUID.randomUUID();
+  }
+
+  /**
+   * Verify that a given string is a storage ID in the "DS-..uuid.." format.
+   */
+  public static boolean isValidStorageId(final String storageID) {
+    try {
+      // Attempt to parse the UUID.
+      if (storageID != null && storageID.indexOf(STORAGE_ID_PREFIX) == 0) {
+        UUID.fromString(storageID.substring(STORAGE_ID_PREFIX.length()));
+        return true;
+      }
+    } catch (IllegalArgumentException iae) {
+    }
+
+    return false;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abb01154/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
index 88ad0cc..ad907f6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
@@ -24,7 +24,6 @@ import static org.junit.Assert.fail;
 
 import java.io.BufferedReader;
 import java.io.File;
-import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.FileReader;
 import java.io.IOException;
@@ -91,6 +90,10 @@ public class TestDFSUpgradeFromImage {
     }
   }
   
+  public interface ClusterVerifier {
+    public void verifyClusterPostUpgrade(final MiniDFSCluster cluster) throws IOException;
+  }
+
   final LinkedList<ReferenceFileInfo> refList = new LinkedList<ReferenceFileInfo>();
   Iterator<ReferenceFileInfo> refIter;
   
@@ -119,7 +122,7 @@ public class TestDFSUpgradeFromImage {
       if (line.length() <= 0 || line.startsWith("#")) {
         continue;
       }
-      String[] arr = line.split("\\s+\t\\s+");
+      String[] arr = line.split("\\s+");
       if (arr.length < 1) {
         continue;
       }
@@ -288,7 +291,7 @@ public class TestDFSUpgradeFromImage {
   public void testUpgradeFromRel22Image() throws IOException {
     unpackStorage(HADOOP22_IMAGE, HADOOP_DFS_DIR_TXT);
     upgradeAndVerify(new MiniDFSCluster.Builder(upgradeConf).
-        numDataNodes(4));
+        numDataNodes(4), null);
   }
   
   /**
@@ -316,7 +319,7 @@ public class TestDFSUpgradeFromImage {
     // Upgrade should now fail
     try {
       upgradeAndVerify(new MiniDFSCluster.Builder(upgradeConf).
-          numDataNodes(4));
+          numDataNodes(4), null);
       fail("Upgrade did not fail with bad MD5");
     } catch (IOException ioe) {
       String msg = StringUtils.stringifyException(ioe);
@@ -573,7 +576,7 @@ public class TestDFSUpgradeFromImage {
     } while (dirList.hasMore());
   }
   
-  void upgradeAndVerify(MiniDFSCluster.Builder bld)
+  void upgradeAndVerify(MiniDFSCluster.Builder bld, ClusterVerifier verifier)
       throws IOException {
     MiniDFSCluster cluster = null;
     try {
@@ -592,6 +595,10 @@ public class TestDFSUpgradeFromImage {
       }
       recoverAllLeases(dfsClient, new Path("/"));
       verifyFileSystem(dfs);
+
+      if (verifier != null) {
+        verifier.verifyClusterPostUpgrade(cluster);
+      }
     } finally {
       if (cluster != null) { cluster.shutdown(); }
     } 
@@ -611,6 +618,6 @@ public class TestDFSUpgradeFromImage {
         "data1");
     upgradeAndVerify(new MiniDFSCluster.Builder(conf).
           numDataNodes(1).enableManagedDfsDirsRedundancy(false).
-          manageDataDfsDirs(false));
+          manageDataDfsDirs(false), null);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abb01154/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeLayoutUpgrade.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeLayoutUpgrade.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeLayoutUpgrade.java
index 0966301..343320c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeLayoutUpgrade.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeLayoutUpgrade.java
@@ -43,6 +43,6 @@ public class TestDatanodeLayoutUpgrade {
         System.getProperty("test.build.data") + File.separator +
             "dfs" + File.separator + "name");
     upgrade.upgradeAndVerify(new MiniDFSCluster.Builder(conf).numDataNodes(1)
-    .manageDataDfsDirs(false).manageNameDfsDirs(false));
+    .manageDataDfsDirs(false).manageNameDfsDirs(false), null);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abb01154/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeStartupFixesLegacyStorageIDs.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeStartupFixesLegacyStorageIDs.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeStartupFixesLegacyStorageIDs.java
new file mode 100644
index 0000000..e262abc
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeStartupFixesLegacyStorageIDs.java
@@ -0,0 +1,139 @@
+/**
+ * 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;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
+import org.apache.hadoop.hdfs.server.protocol.StorageReport;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.TestDFSUpgradeFromImage.ClusterVerifier;
+
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+
+/**
+ * The test verifies that legacy storage IDs in older DataNode
+ * images are replaced with UUID-based storage IDs. The startup may
+ * or may not involve a Datanode Layout upgrade. Each test case uses
+ * the following resource files.
+ *
+ *    1. testCaseName.tgz - NN and DN directories corresponding
+ *                          to a specific layout version.
+ *    2. testCaseName.txt - Text file listing the checksum of each file
+ *                          in the cluster and overall checksum. See
+ *                          TestUpgradeFromImage for the file format.
+ *
+ * If any test case is renamed then the corresponding resource files must
+ * also be renamed.
+ */
+public class TestDatanodeStartupFixesLegacyStorageIDs {
+
+  /**
+   * Perform a upgrade using the test image corresponding to
+   * testCaseName.
+   *
+   * @param testCaseName
+   * @param expectedStorageId if null, then the upgrade generates a new
+   *                          unique storage ID.
+   * @throws IOException
+   */
+  private static void runLayoutUpgradeTest(final String testCaseName,
+                                           final String expectedStorageId)
+      throws IOException {
+    TestDFSUpgradeFromImage upgrade = new TestDFSUpgradeFromImage();
+    upgrade.unpackStorage(testCaseName + ".tgz", testCaseName + ".txt");
+    Configuration conf = new Configuration(TestDFSUpgradeFromImage.upgradeConf);
+    initStorageDirs(conf, testCaseName);
+    upgradeAndVerify(upgrade, conf, new ClusterVerifier() {
+      @Override
+      public void verifyClusterPostUpgrade(MiniDFSCluster cluster) throws IOException {
+        // Verify that a GUID-based storage ID was generated.
+        final String bpid = cluster.getNamesystem().getBlockPoolId();
+        StorageReport[] reports =
+            cluster.getDataNodes().get(0).getFSDataset().getStorageReports(bpid);
+        assertThat(reports.length, is(1));
+        final String storageID = reports[0].getStorage().getStorageID();
+        assertTrue(DatanodeStorage.isValidStorageId(storageID));
+
+        if (expectedStorageId != null) {
+          assertThat(storageID, is(expectedStorageId));
+        }
+      }
+    });
+  }
+
+  private static void initStorageDirs(final Configuration conf,
+                                      final String testName) {
+    conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY,
+             System.getProperty("test.build.data") + File.separator +
+                 testName + File.separator + "dfs" + File.separator + "data");
+    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
+             System.getProperty("test.build.data") + File.separator +
+                 testName + File.separator + "dfs" + File.separator + "name");
+
+  }
+
+  private static void upgradeAndVerify(final TestDFSUpgradeFromImage upgrade,
+                                       final Configuration conf,
+                                       final ClusterVerifier verifier)
+      throws IOException{
+    upgrade.upgradeAndVerify(new MiniDFSCluster.Builder(conf)
+                                 .numDataNodes(1)
+                                 .manageDataDfsDirs(false)
+                                 .manageNameDfsDirs(false), verifier);
+  }
+
+  /**
+   * Upgrade from 2.2 (no storage IDs per volume) correctly generates
+   * GUID-based storage IDs. Test case for HDFS-7575.
+   */
+  @Test (timeout=300000)
+  public void testUpgradeFrom22FixesStorageIDs() throws IOException {
+    runLayoutUpgradeTest(GenericTestUtils.getMethodName(), null);
+  }
+
+  /**
+   * Startup from a 2.6-layout that has legacy storage IDs correctly
+   * generates new storage IDs.
+   * Test case for HDFS-7575.
+   */
+  @Test (timeout=300000)
+  public void testUpgradeFrom22via26FixesStorageIDs() throws IOException {
+    runLayoutUpgradeTest(GenericTestUtils.getMethodName(), null);
+  }
+
+  /**
+   * Startup from a 2.6-layout that already has unique storage IDs does
+   * not regenerate the storage IDs.
+   * Test case for HDFS-7575.
+   */
+  @Test (timeout=300000)
+  public void testUpgradeFrom26PreservesStorageIDs() throws IOException {
+    // StorageId present in the image testUpgradeFrom26PreservesStorageId.tgz
+    runLayoutUpgradeTest(GenericTestUtils.getMethodName(),
+                         "DS-a0e39cfa-930f-4abd-813c-e22b59223774");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abb01154/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 bbaf3ed..dac26a0 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
@@ -476,7 +476,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);
+      storage.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/abb01154/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 0610b94..6ff4603 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
@@ -510,7 +510,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
   public SimulatedFSDataset(DataStorage storage, Configuration conf) {
     if (storage != null) {
       for (int i = 0; i < storage.getNumStorageDirs(); ++i) {
-        storage.createStorageID(storage.getStorageDir(i));
+        storage.createStorageID(storage.getStorageDir(i), false);
       }
       this.datanodeUuid = storage.getDatanodeUuid();
     } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abb01154/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 f3d15de..8f87f57 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
@@ -83,7 +83,7 @@ public class TestFsDatasetImpl {
 
   private static Storage.StorageDirectory createStorageDirectory(File root) {
     Storage.StorageDirectory sd = new Storage.StorageDirectory(root);
-    dsForStorageUuid.createStorageID(sd);
+    dsForStorageUuid.createStorageID(sd, false);
     return sd;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abb01154/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom22FixesStorageIDs.tgz
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom22FixesStorageIDs.tgz b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom22FixesStorageIDs.tgz
new file mode 100644
index 0000000..30b0324
Binary files /dev/null and b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom22FixesStorageIDs.tgz differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abb01154/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom22FixesStorageIDs.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom22FixesStorageIDs.txt b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom22FixesStorageIDs.txt
new file mode 100644
index 0000000..6b0e1ea
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom22FixesStorageIDs.txt
@@ -0,0 +1,25 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+# Similar to hadoop-dfs-dir.txt, except this is used for a datanode layout
+# upgrade test.
+# Uncomment the following line to produce checksum info for a new DFS image.
+#printChecksums
+
+/f01    4021661486
+/f02    4021661486
+/f03    4021661486
+/f04    4021661486
+overallCRC	3193029345

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abb01154/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom22via26FixesStorageIDs.tgz
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom22via26FixesStorageIDs.tgz b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom22via26FixesStorageIDs.tgz
new file mode 100644
index 0000000..74c1649
Binary files /dev/null and b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom22via26FixesStorageIDs.tgz differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abb01154/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom22via26FixesStorageIDs.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom22via26FixesStorageIDs.txt b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom22via26FixesStorageIDs.txt
new file mode 100644
index 0000000..6b0e1ea
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom22via26FixesStorageIDs.txt
@@ -0,0 +1,25 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+# Similar to hadoop-dfs-dir.txt, except this is used for a datanode layout
+# upgrade test.
+# Uncomment the following line to produce checksum info for a new DFS image.
+#printChecksums
+
+/f01    4021661486
+/f02    4021661486
+/f03    4021661486
+/f04    4021661486
+overallCRC	3193029345

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abb01154/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom26PreservesStorageIDs.tgz
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom26PreservesStorageIDs.tgz b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom26PreservesStorageIDs.tgz
new file mode 100644
index 0000000..69fbaf6
Binary files /dev/null and b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom26PreservesStorageIDs.tgz differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abb01154/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom26PreservesStorageIDs.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom26PreservesStorageIDs.txt b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom26PreservesStorageIDs.txt
new file mode 100644
index 0000000..6b0e1ea
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom26PreservesStorageIDs.txt
@@ -0,0 +1,25 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+# Similar to hadoop-dfs-dir.txt, except this is used for a datanode layout
+# upgrade test.
+# Uncomment the following line to produce checksum info for a new DFS image.
+#printChecksums
+
+/f01    4021661486
+/f02    4021661486
+/f03    4021661486
+/f04    4021661486
+overallCRC	3193029345


[22/50] [abbrv] hadoop git commit: HDFS-7430. Refactor the BlockScanner to use O(1) memory and use multiple threads (cmccabe)

Posted by zh...@apache.org.
HDFS-7430. Refactor the BlockScanner to use O(1) memory and use multiple threads (cmccabe)


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

Branch: refs/heads/HDFS-EC
Commit: df4edd9aea0dc9b4dff82347b2776f7069018243
Parents: a691658
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Wed Dec 17 11:27:48 2014 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:27 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   2 +
 .../hdfs/server/datanode/BPOfferService.java    |   3 -
 .../hdfs/server/datanode/BPServiceActor.java    |   6 -
 .../server/datanode/BlockPoolSliceScanner.java  | 872 -------------------
 .../hdfs/server/datanode/BlockReceiver.java     |   8 -
 .../hdfs/server/datanode/BlockScanner.java      | 308 +++++++
 .../hdfs/server/datanode/BlockSender.java       |   3 -
 .../hdfs/server/datanode/DataBlockScanner.java  | 339 -------
 .../hadoop/hdfs/server/datanode/DataNode.java   |  73 +-
 .../hdfs/server/datanode/VolumeScanner.java     | 652 ++++++++++++++
 .../server/datanode/fsdataset/FsDatasetSpi.java |  32 +-
 .../server/datanode/fsdataset/FsVolumeSpi.java  | 110 +++
 .../server/datanode/fsdataset/RollingLogs.java  |  73 --
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |  44 +-
 .../datanode/fsdataset/impl/FsVolumeImpl.java   | 347 ++++++++
 .../datanode/fsdataset/impl/FsVolumeList.java   |  24 +-
 .../fsdataset/impl/RollingLogsImpl.java         | 241 -----
 .../src/main/resources/hdfs-default.xml         |  20 +
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |  16 +
 .../hadoop/hdfs/TestDatanodeBlockScanner.java   | 551 ------------
 .../org/apache/hadoop/hdfs/TestReplication.java |   3 +-
 .../TestOverReplicatedBlocks.java               |  13 +-
 .../server/datanode/BlockReportTestBase.java    |   7 +-
 .../hdfs/server/datanode/DataNodeTestUtils.java |  24 -
 .../server/datanode/SimulatedFSDataset.java     |  22 +-
 .../hdfs/server/datanode/TestBlockScanner.java  | 680 +++++++++++++++
 .../server/datanode/TestDirectoryScanner.java   |  16 +
 .../TestMultipleNNDataBlockScanner.java         | 245 ------
 .../extdataset/ExternalDatasetImpl.java         |   7 -
 .../extdataset/ExternalRollingLogs.java         |  92 --
 .../datanode/extdataset/ExternalVolumeImpl.java |  17 +
 .../extdataset/TestExternalDataset.java         |   9 -
 .../fsdataset/impl/FsVolumeListTest.java        |  17 +-
 .../fsdataset/impl/TestFsDatasetImpl.java       |  30 +-
 .../impl/TestInterDatanodeProtocol.java         |   4 +-
 .../namenode/snapshot/SnapshotTestHelper.java   |   4 +-
 37 files changed, 2288 insertions(+), 2629 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/df4edd9a/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 25ad33b..866b765 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -140,6 +140,9 @@ Trunk (Unreleased)
     class and constructor to public; and fix FsDatasetSpi to use generic type
     instead of FsVolumeImpl.  (David Powell and Joe Pallas via szetszwo)
 
+    HDFS-7430. Rewrite the BlockScanner to use O(1) memory and use multiple
+    threads (cmccabe)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df4edd9a/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 fb958f1..60581b8 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
@@ -441,6 +441,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final int     DFS_DATANODE_MAX_RECEIVER_THREADS_DEFAULT = 4096;
   public static final String  DFS_DATANODE_SCAN_PERIOD_HOURS_KEY = "dfs.datanode.scan.period.hours";
   public static final int     DFS_DATANODE_SCAN_PERIOD_HOURS_DEFAULT = 0;
+  public static final String  DFS_BLOCK_SCANNER_VOLUME_BYTES_PER_SECOND = "dfs.block.scanner.volume.bytes.per.second";
+  public static final long    DFS_BLOCK_SCANNER_VOLUME_BYTES_PER_SECOND_DEFAULT = 1048576L;
   public static final String  DFS_DATANODE_TRANSFERTO_ALLOWED_KEY = "dfs.datanode.transferTo.allowed";
   public static final boolean DFS_DATANODE_TRANSFERTO_ALLOWED_DEFAULT = true;
   public static final String  DFS_HEARTBEAT_INTERVAL_KEY = "dfs.heartbeat.interval";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df4edd9a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
index 4a54bed..dfeacde 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
@@ -656,9 +656,6 @@ class BPOfferService {
       //
       Block toDelete[] = bcmd.getBlocks();
       try {
-        if (dn.blockScanner != null) {
-          dn.blockScanner.deleteBlocks(bcmd.getBlockPoolId(), toDelete);
-        }
         // using global fsdataset
         dn.getFSDataset().invalidate(bcmd.getBlockPoolId(), toDelete);
       } catch(IOException e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df4edd9a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
index e6409ab..e396727 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
@@ -736,12 +736,6 @@ class BPServiceActor implements Runnable {
         DatanodeCommand cmd = cacheReport();
         processCommand(new DatanodeCommand[]{ cmd });
 
-        // Now safe to start scanning the block pool.
-        // If it has already been started, this is a no-op.
-        if (dn.blockScanner != null) {
-          dn.blockScanner.addBlockPool(bpos.getBlockPoolId());
-        }
-
         //
         // There is no work to do;  sleep until hearbeat timer elapses, 
         // or work arrives, and then iterate again.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df4edd9a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java
deleted file mode 100644
index f36fea1..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java
+++ /dev/null
@@ -1,872 +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.datanode;
-
-import java.io.DataOutputStream;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.text.DateFormat;
-import java.text.SimpleDateFormat;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.SortedSet;
-import java.util.TreeSet;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.server.common.GenerationStamp;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.RollingLogs;
-import org.apache.hadoop.hdfs.util.DataTransferThrottler;
-import org.apache.hadoop.util.GSet;
-import org.apache.hadoop.util.LightWeightGSet;
-import org.apache.hadoop.util.LightWeightGSet.LinkedElement;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.util.Time;
-
-import com.google.common.annotations.VisibleForTesting;
-
-/**
- * Scans the block files under a block pool and verifies that the
- * files are not corrupt.
- * This keeps track of blocks and their last verification times.
- * Currently it does not modify the metadata for block.
- */
-
-class BlockPoolSliceScanner {
-  
-  public static final Log LOG = LogFactory.getLog(BlockPoolSliceScanner.class);
-  
-  private static final String DATA_FORMAT = "yyyy-MM-dd HH:mm:ss,SSS";
-
-  private static final int MAX_SCAN_RATE = 8 * 1024 * 1024; // 8MB per sec
-  private static final int MIN_SCAN_RATE = 1 * 1024 * 1024; // 1MB per sec
-  private static final long DEFAULT_SCAN_PERIOD_HOURS = 21*24L; // three weeks
-
-  private static final String VERIFICATION_PREFIX = "dncp_block_verification.log";
-
-  private final String blockPoolId;
-  private final long scanPeriod;
-  private final AtomicLong lastScanTime = new AtomicLong();
-
-  private final DataNode datanode;
-  private final FsDatasetSpi<? extends FsVolumeSpi> dataset;
-  
-  private final SortedSet<BlockScanInfo> blockInfoSet
-      = new TreeSet<BlockScanInfo>(BlockScanInfo.LAST_SCAN_TIME_COMPARATOR);
-
-  private final SortedSet<BlockScanInfo> newBlockInfoSet =
-      new TreeSet<BlockScanInfo>(BlockScanInfo.LAST_SCAN_TIME_COMPARATOR);
-
-  private final GSet<Block, BlockScanInfo> blockMap
-      = new LightWeightGSet<Block, BlockScanInfo>(
-          LightWeightGSet.computeCapacity(0.5, "BlockMap"));
-  
-  // processedBlocks keeps track of which blocks are scanned
-  // since the last run.
-  private volatile HashMap<Long, Integer> processedBlocks;
-  
-  private long totalScans = 0;
-  private long totalScanErrors = 0;
-  private long totalTransientErrors = 0;
-  private final AtomicInteger totalBlocksScannedInLastRun = new AtomicInteger(); // Used for test only
-  
-  private long currentPeriodStart = Time.monotonicNow();
-  private long bytesLeft = 0; // Bytes to scan in this period
-  private long totalBytesToScan = 0;
-  private boolean isNewPeriod = true;
-  private int lastScanTimeDifference = 5*60*1000;
-  
-  private final LogFileHandler verificationLog;
-  
-  private final DataTransferThrottler throttler = new DataTransferThrottler(
-       200, MAX_SCAN_RATE);
-  
-  private static enum ScanType {
-    IMMEDIATE_SCAN,  
-    VERIFICATION_SCAN,     // scanned as part of periodic verfication
-    NONE,
-  }
-
-  // Extend Block because in the DN process there's a 1-to-1 correspondence of
-  // BlockScanInfo to Block instances, so by extending rather than containing
-  // Block, we can save a bit of Object overhead (about 24 bytes per block
-  // replica.)
-  static class BlockScanInfo extends Block
-      implements LightWeightGSet.LinkedElement {
-
-    /** Compare the info by the last scan time. */
-    static final Comparator<BlockScanInfo> LAST_SCAN_TIME_COMPARATOR
-        = new Comparator<BlockPoolSliceScanner.BlockScanInfo>() {
-
-      @Override
-      public int compare(BlockScanInfo left, BlockScanInfo right) {
-        final ScanType leftNextScanType = left.nextScanType;
-        final ScanType rightNextScanType = right.nextScanType;
-        final long l = left.lastScanTime;
-        final long r = right.lastScanTime;
-        // Compare by nextScanType if they are same then compare by 
-        // lastScanTimes
-        // compare blocks itself if scantimes are same to avoid.
-        // because TreeMap uses comparator if available to check existence of
-        // the object. 
-        int compareByNextScanType = leftNextScanType.compareTo(rightNextScanType);
-        return compareByNextScanType < 0? -1: compareByNextScanType > 0? 1:  l < r? -1: l > r? 1: left.compareTo(right); 
-      }
-    };
-
-    long lastScanTime = 0;
-    ScanType lastScanType = ScanType.NONE; 
-    boolean lastScanOk = true;
-    private LinkedElement next;
-    ScanType nextScanType = ScanType.VERIFICATION_SCAN;
-    
-    BlockScanInfo(Block block) {
-      super(block);
-    }
-    
-    @Override
-    public int hashCode() {
-      return super.hashCode();
-    }
-    
-    @Override
-    public boolean equals(Object that) {
-      if (this == that) {
-        return true;
-      }
-      return super.equals(that);
-    }
-    
-    long getLastScanTime() {
-      return (lastScanType == ScanType.NONE) ? 0 : lastScanTime;
-    }
-
-    @Override
-    public void setNext(LinkedElement next) {
-      this.next = next;
-    }
-
-    @Override
-    public LinkedElement getNext() {
-      return next;
-    }
-  }
-  
-  BlockPoolSliceScanner(String bpid, DataNode datanode,
-      FsDatasetSpi<? extends FsVolumeSpi> dataset, Configuration conf) {
-    this.datanode = datanode;
-    this.dataset = dataset;
-    this.blockPoolId  = bpid;
-    
-    long hours = conf.getInt(DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, 
-                             DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_DEFAULT);
-    if (hours <= 0) {
-      hours = DEFAULT_SCAN_PERIOD_HOURS;
-    }
-    this.scanPeriod = hours * 3600 * 1000;
-    LOG.info("Periodic Block Verification Scanner initialized with interval "
-        + hours + " hours for block pool " + bpid);
-
-    // get the list of blocks and arrange them in random order
-    List<FinalizedReplica> arr = dataset.getFinalizedBlocksOnPersistentStorage(blockPoolId);
-    Collections.shuffle(arr);
-    
-    long scanTime = -1;
-    for (Block block : arr) {
-      BlockScanInfo info = new BlockScanInfo( block );
-      info.lastScanTime = scanTime--; 
-      //still keep 'info.lastScanType' to NONE.
-      addBlockInfo(info, false);
-    }
-
-    RollingLogs rollingLogs = null;
-    try {
-       rollingLogs = dataset.createRollingLogs(blockPoolId, VERIFICATION_PREFIX);
-    } catch (IOException e) {
-      LOG.warn("Could not open verfication log. " +
-               "Verification times are not stored.");
-    }
-    verificationLog = rollingLogs == null? null: new LogFileHandler(rollingLogs);
-  }
-  
-  String getBlockPoolId() {
-    return blockPoolId;
-  }
-  
-  private void updateBytesToScan(long len, long lastScanTime) {
-    // len could be negative when a block is deleted.
-    totalBytesToScan += len;
-    if ( lastScanTime < currentPeriodStart ) {
-      bytesLeft += len;
-    }
-    // Should we change throttler bandwidth every time bytesLeft changes?
-    // not really required.
-  }
-
-  /**
-   * Add the BlockScanInfo to sorted set of blockScanInfo
-   * @param info BlockScanInfo to be added
-   * @param isNewBlock true if the block is the new Block, false if
-   *          BlockScanInfo is being updated with new scanTime
-   */
-  private synchronized void addBlockInfo(BlockScanInfo info,
-      boolean isNewBlock) {
-    boolean added = false;
-    if (isNewBlock) {
-      // check whether the block already present
-      boolean exists = blockInfoSet.contains(info);
-      added = !exists && newBlockInfoSet.add(info);
-    } else {
-      added = blockInfoSet.add(info);
-    }
-    blockMap.put(info);
-    
-    if (added) {
-      updateBytesToScan(info.getNumBytes(), info.lastScanTime);
-    }
-  }
-
-  private synchronized void delBlockInfo(BlockScanInfo info) {
-    boolean exists = blockInfoSet.remove(info);
-    if (!exists){
-      exists = newBlockInfoSet.remove(info);
-    }
-    blockMap.remove(info);
-
-    if (exists) {
-      updateBytesToScan(-info.getNumBytes(), info.lastScanTime);
-    }
-  }
-
-  /** Update blockMap by the given LogEntry */
-  private synchronized void updateBlockInfo(LogEntry e) {
-    BlockScanInfo info = blockMap.get(new Block(e.blockId, 0, e.genStamp));
-    
-    if (info != null && e.verificationTime > 0 && 
-        info.lastScanTime < e.verificationTime) {
-      delBlockInfo(info);
-      if (info.nextScanType != ScanType.IMMEDIATE_SCAN) {
-        info.lastScanTime = e.verificationTime;
-      }
-      info.lastScanType = ScanType.VERIFICATION_SCAN;
-      addBlockInfo(info, false);
-    }
-  }
-
-  private synchronized long getNewBlockScanTime() {
-    /* If there are a lot of blocks, this returns a random time with in 
-     * the scan period. Otherwise something sooner.
-     */
-    long period = Math.min(scanPeriod, 
-                           Math.max(blockMap.size(),1) * 600 * 1000L);
-    int periodInt = Math.abs((int)period);
-    return Time.monotonicNow() - scanPeriod +
-        DFSUtil.getRandom().nextInt(periodInt);
-  }
-
-  /** Adds block to list of blocks 
-   * @param scanNow - true if we want to make that particular block a high 
-   * priority one to scan immediately
-   **/
-  synchronized void addBlock(ExtendedBlock block, boolean scanNow) {
-    BlockScanInfo info = blockMap.get(block.getLocalBlock());
-    long lastScanTime = 0;
-    if (info != null) {
-      lastScanTime = info.lastScanTime;
-    }
-    // If the particular block is scanned in last 5 minutes, the  no need to 
-    // verify that block again
-    if (scanNow && Time.monotonicNow() - lastScanTime < 
-        lastScanTimeDifference) {
-      return;
-    }
-    
-    if ( info != null ) {
-      LOG.warn("Adding an already existing block " + block);
-      delBlockInfo(info);
-    }
-    
-    info = new BlockScanInfo(block.getLocalBlock());    
-    info.lastScanTime = getNewBlockScanTime();
-    if (scanNow) {
-      // Create a new BlockScanInfo object and set the lastScanTime to 0
-      // which will make it the high priority block
-      LOG.info("Adding block for immediate verification " + block);
-      info.nextScanType = ScanType.IMMEDIATE_SCAN;
-    }
-    
-    addBlockInfo(info, true);
-    adjustThrottler();
-  }
-  
-  /** Deletes the block from internal structures */
-  synchronized void deleteBlock(Block block) {
-    BlockScanInfo info = blockMap.get(block);
-    if (info != null) {
-      delBlockInfo(info);
-    }
-  }
-
-  @VisibleForTesting
-  long getTotalScans() {
-    return totalScans;
-  }
-
-  /** @return the last scan time for the block pool. */
-  long getLastScanTime() {
-    return lastScanTime.get();
-  }
-
-  /** @return the last scan time the given block. */
-  synchronized long getLastScanTime(Block block) {
-    BlockScanInfo info = blockMap.get(block);
-    return info == null? 0: info.lastScanTime;
-  }
-
-  /** Deletes blocks from internal structures */
-  void deleteBlocks(Block[] blocks) {
-    for ( Block b : blocks ) {
-      deleteBlock(b);
-    }
-  }
-  
-  private synchronized void updateScanStatus(BlockScanInfo info,
-                                             ScanType type,
-                                             boolean scanOk) {
-    delBlockInfo(info);
-
-    long now = Time.monotonicNow();
-    info.lastScanType = type;
-    info.lastScanTime = now;
-    info.lastScanOk = scanOk;
-    info.nextScanType = ScanType.VERIFICATION_SCAN;
-    addBlockInfo(info, false);
-        
-    // Don't update meta data if the verification failed.
-    if (!scanOk) {
-      return;
-    }
-    
-    if (verificationLog != null) {
-      verificationLog.append(now, info.getGenerationStamp(),
-          info.getBlockId());
-    }
-  }
-  
-  private void handleScanFailure(ExtendedBlock block) {
-    LOG.info("Reporting bad " + block);
-    try {
-      datanode.reportBadBlocks(block);
-    } catch (IOException ie) {
-      // it is bad, but not bad enough to shutdown the scanner
-      LOG.warn("Cannot report bad " + block.getBlockId());
-    }
-  }
-  
-  @VisibleForTesting
-  synchronized void setLastScanTimeDifference(int lastScanTimeDifference) {
-    this.lastScanTimeDifference = lastScanTimeDifference;
-  }
-  
-  static private class LogEntry {
-
-    long blockId = -1;
-    long verificationTime = -1;
-    long genStamp = GenerationStamp.GRANDFATHER_GENERATION_STAMP;
-    
-    /**
-     * The format consists of single line with multiple entries. each 
-     * entry is in the form : name="value".
-     * This simple text and easily extendable and easily parseable with a
-     * regex.
-     */
-    private static final Pattern entryPattern =
-      Pattern.compile("\\G\\s*([^=\\p{Space}]+)=\"(.*?)\"\\s*");
-    
-    static String toString(long verificationTime, long genStamp, long blockId,
-        DateFormat dateFormat) {
-      return "\ndate=\"" + dateFormat.format(new Date(verificationTime))
-          + "\"\t time=\"" + verificationTime
-          + "\"\t genstamp=\"" + genStamp
-          + "\"\t id=\"" + blockId + "\"";
-    }
-
-    static LogEntry parseEntry(String line) {
-      LogEntry entry = new LogEntry();
-      
-      Matcher matcher = entryPattern.matcher(line);
-      while (matcher.find()) {
-        String name = matcher.group(1);
-        String value = matcher.group(2);
-        
-        try {
-          if (name.equals("id")) {
-            entry.blockId = Long.parseLong(value);
-          } else if (name.equals("time")) {
-            entry.verificationTime = Long.parseLong(value);
-          } else if (name.equals("genstamp")) {
-            entry.genStamp = Long.parseLong(value);
-          }
-        } catch(NumberFormatException nfe) {
-          LOG.warn("Cannot parse line: " + line, nfe);
-          return null;
-        }
-      }
-      
-      return entry;
-    }
-  }
-  
-  private synchronized void adjustThrottler() {
-    long timeLeft = Math.max(1L,
-        currentPeriodStart + scanPeriod - Time.monotonicNow());
-    long bw = Math.max((bytesLeft * 1000) / timeLeft, MIN_SCAN_RATE);
-    throttler.setBandwidth(Math.min(bw, MAX_SCAN_RATE));
-  }
-  
-  @VisibleForTesting
-  void verifyBlock(ExtendedBlock block) {
-    BlockSender blockSender = null;
-
-    /* In case of failure, attempt to read second time to reduce
-     * transient errors. How do we flush block data from kernel 
-     * buffers before the second read? 
-     */
-    for (int i=0; i<2; i++) {
-      boolean second = (i > 0);
-      
-      try {
-        adjustThrottler();
-        
-        blockSender = new BlockSender(block, 0, -1, false, true, true, 
-            datanode, null, CachingStrategy.newDropBehind());
-
-        DataOutputStream out = 
-                new DataOutputStream(new IOUtils.NullOutputStream());
-        
-        blockSender.sendBlock(out, null, throttler);
-
-        LOG.info((second ? "Second " : "") +
-                 "Verification succeeded for " + block);
-        
-        if ( second ) {
-          totalTransientErrors++;
-        }
-        
-        updateScanStatus((BlockScanInfo)block.getLocalBlock(),
-            ScanType.VERIFICATION_SCAN, true);
-
-        return;
-      } catch (IOException e) {
-        updateScanStatus((BlockScanInfo)block.getLocalBlock(),
-            ScanType.VERIFICATION_SCAN, false);
-
-        // If the block does not exists anymore, then its not an error
-        if (!dataset.contains(block)) {
-          LOG.info(block + " is no longer in the dataset");
-          deleteBlock(block.getLocalBlock());
-          return;
-        }
-
-        // If the block exists, the exception may due to a race with write:
-        // The BlockSender got an old block path in rbw. BlockReceiver removed
-        // the rbw block from rbw to finalized but BlockSender tried to open the
-        // file before BlockReceiver updated the VolumeMap. The state of the
-        // block can be changed again now, so ignore this error here. If there
-        // is a block really deleted by mistake, DirectoryScan should catch it.
-        if (e instanceof FileNotFoundException ) {
-          LOG.info("Verification failed for " + block +
-              " - may be due to race with write");
-          deleteBlock(block.getLocalBlock());
-          return;
-        }
-
-        LOG.warn((second ? "Second " : "First ") + "Verification failed for "
-            + block, e);
-        
-        if (second) {
-          totalScanErrors++;
-          datanode.getMetrics().incrBlockVerificationFailures();
-          handleScanFailure(block);
-          return;
-        } 
-      } finally {
-        IOUtils.closeStream(blockSender);
-        datanode.getMetrics().incrBlocksVerified();
-        totalScans++;
-      }
-    }
-  }
-  
-  private synchronized long getEarliestScanTime() {
-    if (!blockInfoSet.isEmpty()) {
-      return blockInfoSet.first().lastScanTime;
-    }
-    return Long.MAX_VALUE; 
-  }
-  
-  private synchronized boolean isFirstBlockProcessed() {
-    if (!blockInfoSet.isEmpty()) {
-      if (blockInfoSet.first().nextScanType == ScanType.IMMEDIATE_SCAN) {
-        return false;
-      }
-      long blockId = blockInfoSet.first().getBlockId();
-      if ((processedBlocks.get(blockId) != null)
-          && (processedBlocks.get(blockId) == 1)) {
-        return true;
-      }
-    }
-    return false;
-  }
-  
-  // Picks one block and verifies it
-  private void verifyFirstBlock() {
-    BlockScanInfo block = null;
-    synchronized (this) {
-      if (!blockInfoSet.isEmpty()) {
-        block = blockInfoSet.first();
-      }
-    }
-    if ( block != null ) {
-      verifyBlock(new ExtendedBlock(blockPoolId, block));
-      processedBlocks.put(block.getBlockId(), 1);
-    }
-  }
-  
-  // Used for tests only
-  int getBlocksScannedInLastRun() {
-    return totalBlocksScannedInLastRun.get();
-  }
-
-  /**
-   * Reads the current and previous log files (if any) and marks the blocks
-   * processed if they were processed within last scan period. Copies the log
-   * records of recently scanned blocks from previous to current file. 
-   * Returns false if the process was interrupted because the thread is marked 
-   * to exit.
-   */
-  private boolean assignInitialVerificationTimes() {
-    //First updates the last verification times from the log file.
-    if (verificationLog != null) {
-      long now = Time.monotonicNow();
-      RollingLogs.LineIterator logIterator = null;
-      try {
-        logIterator = verificationLog.logs.iterator(false);
-        // update verification times from the verificationLog.
-        while (logIterator.hasNext()) {
-          if (!datanode.shouldRun
-              || datanode.blockScanner.blockScannerThread.isInterrupted()) {
-            return false;
-          }
-          LogEntry entry = LogEntry.parseEntry(logIterator.next());
-          if (entry != null) {
-            updateBlockInfo(entry);
-            if (now - entry.verificationTime < scanPeriod) {
-              BlockScanInfo info = blockMap.get(new Block(entry.blockId, 0,
-                  entry.genStamp));
-              if (info != null) {
-                if (processedBlocks.get(entry.blockId) == null) {
-                  if (isNewPeriod) {
-                    updateBytesLeft(-info.getNumBytes());
-                  }
-                  processedBlocks.put(entry.blockId, 1);
-                }
-                if (logIterator.isLastReadFromPrevious()) {
-                  // write the log entry to current file
-                  // so that the entry is preserved for later runs.
-                  verificationLog.append(entry.verificationTime, entry.genStamp,
-                      entry.blockId);
-                }
-              }
-            }
-          }
-        }
-      } catch (IOException e) {
-        LOG.warn("Failed to read previous verification times.", e);
-      } finally {
-        IOUtils.closeStream(logIterator);
-      }
-      isNewPeriod = false;
-    }
-    
-    
-    /* Before this loop, entries in blockInfoSet that are not
-     * updated above have lastScanTime of <= 0 . Loop until first entry has
-     * lastModificationTime > 0.
-     */    
-    synchronized (this) {
-      final int numBlocks = Math.max(blockMap.size(), 1);
-      // Initially spread the block reads over half of scan period
-      // so that we don't keep scanning the blocks too quickly when restarted.
-      long verifyInterval = Math.min(scanPeriod/(2L * numBlocks), 10*60*1000L);
-      long lastScanTime = Time.monotonicNow() - scanPeriod;
-
-      if (!blockInfoSet.isEmpty()) {
-        BlockScanInfo info;
-        while ((info =  blockInfoSet.first()).lastScanTime < 0) {
-          delBlockInfo(info);        
-          info.lastScanTime = lastScanTime;
-          lastScanTime += verifyInterval;
-          addBlockInfo(info, false);
-        }
-      }
-    }
-    
-    return true;
-  }
-  
-  private synchronized void updateBytesLeft(long len) {
-    bytesLeft += len;
-  }
-  
-  private synchronized void startNewPeriod() {
-    LOG.info("Starting a new period : work left in prev period : "
-        + String.format("%.2f%%", totalBytesToScan == 0 ? 0
-            : (bytesLeft * 100.0) / totalBytesToScan));
-
-    // reset the byte counts :
-    bytesLeft = totalBytesToScan;
-    currentPeriodStart = Time.monotonicNow();
-    isNewPeriod = true;
-  }
-  
-  private synchronized boolean workRemainingInCurrentPeriod() {
-    if (bytesLeft <= 0 && Time.monotonicNow() < currentPeriodStart + scanPeriod) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Skipping scan since bytesLeft=" + bytesLeft + ", Start=" +
-                  currentPeriodStart + ", period=" + scanPeriod + ", now=" +
-                  Time.monotonicNow() + " " + blockPoolId);
-      }
-      return false;
-    } else {
-      return true;
-    }
-  }
-
-  void scanBlockPoolSlice() {
-    if (!workRemainingInCurrentPeriod()) {
-      return;
-    }
-
-    // Create a new processedBlocks structure
-    processedBlocks = new HashMap<Long, Integer>();
-    if (!assignInitialVerificationTimes()) {
-      return;
-    }
-    // Start scanning
-    try {
-      scan();
-    } finally {
-      totalBlocksScannedInLastRun.set(processedBlocks.size());
-      lastScanTime.set(Time.monotonicNow());
-    }
-  }
-
-  /**
-   * Shuts down this BlockPoolSliceScanner and releases any internal resources.
-   */
-  void shutdown() {
-    if (verificationLog != null) {
-      verificationLog.close();
-    }
-  }
-  
-  private void scan() {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Starting to scan blockpool: " + blockPoolId);
-    }
-    try {
-      adjustThrottler();
-        
-      while (datanode.shouldRun
-          && !datanode.blockScanner.blockScannerThread.isInterrupted()
-          && datanode.isBPServiceAlive(blockPoolId)) {
-        long now = Time.monotonicNow();
-        synchronized (this) {
-          if ( now >= (currentPeriodStart + scanPeriod)) {
-            startNewPeriod();
-          }
-        }
-        if (((now - getEarliestScanTime()) >= scanPeriod)
-            || ((!blockInfoSet.isEmpty()) && !(this.isFirstBlockProcessed()))) {
-          verifyFirstBlock();
-        } else {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("All remaining blocks were processed recently, "
-                + "so this run is complete");
-          }
-          break;
-        }
-      }
-    } catch (RuntimeException e) {
-      LOG.warn("RuntimeException during BlockPoolScanner.scan()", e);
-      throw e;
-    } finally {
-      rollVerificationLogs();
-      rollNewBlocksInfo();
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Done scanning block pool: " + blockPoolId);
-      }
-    }
-  }
-
-  // add new blocks to scan in next iteration
-  private synchronized void rollNewBlocksInfo() {
-    for (BlockScanInfo newBlock : newBlockInfoSet) {
-      blockInfoSet.add(newBlock);
-    }
-    newBlockInfoSet.clear();
-  }
-
-  private synchronized void rollVerificationLogs() {
-    if (verificationLog != null) {
-      try {
-        verificationLog.logs.roll();
-      } catch (IOException ex) {
-        LOG.warn("Received exception: ", ex);
-        verificationLog.close();
-      }
-    }
-  }
-
-  
-  synchronized void printBlockReport(StringBuilder buffer, 
-                                     boolean summaryOnly) {
-    long oneHour = 3600*1000;
-    long oneDay = 24*oneHour;
-    long oneWeek = 7*oneDay;
-    long fourWeeks = 4*oneWeek;
-    
-    int inOneHour = 0;
-    int inOneDay = 0;
-    int inOneWeek = 0;
-    int inFourWeeks = 0;
-    int inScanPeriod = 0;
-    int neverScanned = 0;
-    
-    DateFormat dateFormat = new SimpleDateFormat(DATA_FORMAT);
-    
-    int total = blockInfoSet.size();
-    
-    long now = Time.monotonicNow();
-    
-    Date date = new Date();
-    
-    for(Iterator<BlockScanInfo> it = blockInfoSet.iterator(); it.hasNext();) {
-      BlockScanInfo info = it.next();
-      
-      long scanTime = info.getLastScanTime();
-      long diff = now - scanTime;
-      
-      if (diff <= oneHour) inOneHour++;
-      if (diff <= oneDay) inOneDay++;
-      if (diff <= oneWeek) inOneWeek++;
-      if (diff <= fourWeeks) inFourWeeks++;
-      if (diff <= scanPeriod) inScanPeriod++;      
-      if (scanTime <= 0) neverScanned++;
-      
-      if (!summaryOnly) {
-        date.setTime(scanTime);
-        String scanType = 
-          (info.lastScanType == ScanType.VERIFICATION_SCAN) ? "local" : "none"; 
-        buffer.append(String.format("%-26s : status : %-6s type : %-6s" +
-                                    " scan time : " +
-                                    "%-15d %s%n", info,
-                                    (info.lastScanOk ? "ok" : "failed"),
-                                    scanType, scanTime,
-                                    (scanTime <= 0) ? "not yet verified" : 
-                                      dateFormat.format(date)));
-      }
-    }
-    
-    double pctPeriodLeft = (scanPeriod + currentPeriodStart - now)
-                           *100.0/scanPeriod;
-    double pctProgress = (totalBytesToScan == 0) ? 100 :
-                         (totalBytesToScan-bytesLeft)*100.0/totalBytesToScan;
-                         
-    buffer.append(String.format("%nTotal Blocks                 : %6d" +
-                                "%nVerified in last hour        : %6d" +
-                                "%nVerified in last day         : %6d" +
-                                "%nVerified in last week        : %6d" +
-                                "%nVerified in last four weeks  : %6d" +
-                                "%nVerified in SCAN_PERIOD      : %6d" +
-                                "%nNot yet verified             : %6d" +
-                                "%nVerified since restart       : %6d" +
-                                "%nScans since restart          : %6d" +
-                                "%nScan errors since restart    : %6d" +
-                                "%nTransient scan errors        : %6d" +
-                                "%nCurrent scan rate limit KBps : %6d" +
-                                "%nProgress this period         : %6.0f%%" +
-                                "%nTime left in cur period      : %6.2f%%" +
-                                "%n", 
-                                total, inOneHour, inOneDay, inOneWeek,
-                                inFourWeeks, inScanPeriod, neverScanned,
-                                totalScans, totalScans, 
-                                totalScanErrors, totalTransientErrors, 
-                                Math.round(throttler.getBandwidth()/1024.0),
-                                pctProgress, pctPeriodLeft));
-  }
-  
-  /**
-   * This class takes care of log file used to store the last verification
-   * times of the blocks.
-   */
-  private static class LogFileHandler {
-    private final DateFormat dateFormat = new SimpleDateFormat(DATA_FORMAT);
-
-    private final RollingLogs logs;
-
-    private LogFileHandler(RollingLogs logs)  {
-      this.logs = logs;
-    }
-
-    void append(long verificationTime, long genStamp, long blockId) {
-      final String m = LogEntry.toString(verificationTime, genStamp, blockId,
-          dateFormat);
-      try {
-        logs.appender().append(m);
-      } catch (IOException e) {
-        LOG.warn("Failed to append to " + logs + ", m=" + m, e);
-      }
-    }
-
-    void close() {
-      try {
-        logs.appender().close();
-      } catch (IOException e) {
-        LOG.warn("Failed to close the appender of " + logs, e);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df4edd9a/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 df8dd5c..12041a6 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
@@ -193,20 +193,12 @@ class BlockReceiver implements Closeable {
           break;
         case PIPELINE_SETUP_APPEND:
           replicaHandler = datanode.data.append(block, newGs, minBytesRcvd);
-          if (datanode.blockScanner != null) { // remove from block scanner
-            datanode.blockScanner.deleteBlock(block.getBlockPoolId(),
-                block.getLocalBlock());
-          }
           block.setGenerationStamp(newGs);
           datanode.notifyNamenodeReceivingBlock(
               block, replicaHandler.getReplica().getStorageUuid());
           break;
         case PIPELINE_SETUP_APPEND_RECOVERY:
           replicaHandler = datanode.data.recoverAppend(block, newGs, minBytesRcvd);
-          if (datanode.blockScanner != null) { // remove from block scanner
-            datanode.blockScanner.deleteBlock(block.getBlockPoolId(),
-                block.getLocalBlock());
-          }
           block.setGenerationStamp(newGs);
           datanode.notifyNamenodeReceivingBlock(
               block, replicaHandler.getReplica().getStorageUuid());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df4edd9a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockScanner.java
new file mode 100644
index 0000000..7429fff
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockScanner.java
@@ -0,0 +1,308 @@
+/**
+ * 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.datanode;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SCANNER_VOLUME_BYTES_PER_SECOND;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SCANNER_VOLUME_BYTES_PER_SECOND_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_DEFAULT;
+
+import java.io.IOException;
+import java.util.Map.Entry;
+import java.util.TreeMap;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdfs.server.datanode.VolumeScanner.ScanResultHandler;
+import com.google.common.base.Preconditions;
+import com.google.common.util.concurrent.Uninterruptibles;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.io.IOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+@InterfaceAudience.Private
+public class BlockScanner {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockScanner.class);
+
+  /**
+   * The DataNode that this scanner is associated with.
+   */
+  private final DataNode datanode;
+
+  /**
+   * Maps Storage IDs to VolumeScanner objects.
+   */
+  private final TreeMap<String, VolumeScanner> scanners =
+      new TreeMap<String, VolumeScanner>();
+
+  /**
+   * The scanner configuration.
+   */
+  private final Conf conf;
+
+  /**
+   * The cached scanner configuration.
+   */
+  static class Conf {
+    // These are a few internal configuration keys used for unit tests.
+    // They can't be set unless the static boolean allowUnitTestSettings has
+    // been set to true.
+
+    @VisibleForTesting
+    static final String INTERNAL_DFS_DATANODE_SCAN_PERIOD_MS =
+        "internal.dfs.datanode.scan.period.ms.key";
+
+    @VisibleForTesting
+    static final String INTERNAL_VOLUME_SCANNER_SCAN_RESULT_HANDLER =
+        "internal.volume.scanner.scan.result.handler";
+
+    @VisibleForTesting
+    static final String INTERNAL_DFS_BLOCK_SCANNER_MAX_STALENESS_MS =
+        "internal.dfs.block.scanner.max_staleness.ms";
+
+    @VisibleForTesting
+    static final long INTERNAL_DFS_BLOCK_SCANNER_MAX_STALENESS_MS_DEFAULT =
+        TimeUnit.MILLISECONDS.convert(15, TimeUnit.MINUTES);
+
+    @VisibleForTesting
+    static final String INTERNAL_DFS_BLOCK_SCANNER_CURSOR_SAVE_INTERVAL_MS =
+        "dfs.block.scanner.cursor.save.interval.ms";
+
+    @VisibleForTesting
+    static final long
+        INTERNAL_DFS_BLOCK_SCANNER_CURSOR_SAVE_INTERVAL_MS_DEFAULT =
+            TimeUnit.MILLISECONDS.convert(10, TimeUnit.MINUTES);
+
+    static boolean allowUnitTestSettings = false;
+    final long targetBytesPerSec;
+    final long maxStalenessMs;
+    final long scanPeriodMs;
+    final long cursorSaveMs;
+    final Class<? extends ScanResultHandler> resultHandler;
+
+    private static long getUnitTestLong(Configuration conf, String key,
+                                        long defVal) {
+      if (allowUnitTestSettings) {
+        return conf.getLong(key, defVal);
+      } else {
+        return defVal;
+      }
+    }
+
+    @SuppressWarnings("unchecked")
+    Conf(Configuration conf) {
+      this.targetBytesPerSec = Math.max(0L, conf.getLong(
+          DFS_BLOCK_SCANNER_VOLUME_BYTES_PER_SECOND,
+          DFS_BLOCK_SCANNER_VOLUME_BYTES_PER_SECOND_DEFAULT));
+      this.maxStalenessMs = Math.max(0L, getUnitTestLong(conf,
+          INTERNAL_DFS_BLOCK_SCANNER_MAX_STALENESS_MS,
+          INTERNAL_DFS_BLOCK_SCANNER_MAX_STALENESS_MS_DEFAULT));
+      this.scanPeriodMs = Math.max(0L,
+          getUnitTestLong(conf, INTERNAL_DFS_DATANODE_SCAN_PERIOD_MS,
+              TimeUnit.MILLISECONDS.convert(conf.getLong(
+                  DFS_DATANODE_SCAN_PERIOD_HOURS_KEY,
+                  DFS_DATANODE_SCAN_PERIOD_HOURS_DEFAULT), TimeUnit.HOURS)));
+      this.cursorSaveMs = Math.max(0L, getUnitTestLong(conf,
+          INTERNAL_DFS_BLOCK_SCANNER_CURSOR_SAVE_INTERVAL_MS,
+          INTERNAL_DFS_BLOCK_SCANNER_CURSOR_SAVE_INTERVAL_MS_DEFAULT));
+      if (allowUnitTestSettings) {
+        this.resultHandler = (Class<? extends ScanResultHandler>)
+            conf.getClass(INTERNAL_VOLUME_SCANNER_SCAN_RESULT_HANDLER,
+                          ScanResultHandler.class);
+      } else {
+        this.resultHandler = ScanResultHandler.class;
+      }
+    }
+  }
+
+  public BlockScanner(DataNode datanode, Configuration conf) {
+    this.datanode = datanode;
+    this.conf = new Conf(conf);
+    if (isEnabled()) {
+      LOG.info("Initialized block scanner with targetBytesPerSec {}",
+          this.conf.targetBytesPerSec);
+    } else {
+      LOG.info("Disabled block scanner.");
+    }
+  }
+
+  /**
+   * Returns true if the block scanner is enabled.<p/>
+   *
+   * If the block scanner is disabled, no volume scanners will be created, and
+   * no threads will start.
+   */
+  public boolean isEnabled() {
+    return (conf.scanPeriodMs) > 0 && (conf.targetBytesPerSec > 0);
+  }
+
+ /**
+  * Set up a scanner for the given block pool and volume.
+  *
+  * @param ref              A reference to the volume.
+  */
+  public synchronized void addVolumeScanner(FsVolumeReference ref) {
+    boolean success = false;
+    try {
+      FsVolumeSpi volume = ref.getVolume();
+      if (!isEnabled()) {
+        LOG.debug("Not adding volume scanner for {}, because the block " +
+            "scanner is disabled.", volume.getBasePath());
+        return;
+      }
+      VolumeScanner scanner = scanners.get(volume.getStorageID());
+      if (scanner != null) {
+        LOG.error("Already have a scanner for volume {}.",
+            volume.getBasePath());
+        return;
+      }
+      LOG.debug("Adding scanner for volume {} (StorageID {})",
+          volume.getBasePath(), volume.getStorageID());
+      scanner = new VolumeScanner(conf, datanode, ref);
+      scanner.start();
+      scanners.put(volume.getStorageID(), scanner);
+      success = true;
+    } finally {
+      if (!success) {
+        // If we didn't create a new VolumeScanner object, we don't
+        // need this reference to the volume.
+        IOUtils.cleanup(null, ref);
+      }
+    }
+  }
+
+  /**
+   * Stops and removes a volume scanner.<p/>
+   *
+   * This function will block until the volume scanner has stopped.
+   *
+   * @param volume           The volume to remove.
+   */
+  public synchronized void removeVolumeScanner(FsVolumeSpi volume) {
+    if (!isEnabled()) {
+      LOG.debug("Not removing volume scanner for {}, because the block " +
+          "scanner is disabled.", volume.getStorageID());
+      return;
+    }
+    VolumeScanner scanner = scanners.get(volume.getStorageID());
+    if (scanner == null) {
+      LOG.warn("No scanner found to remove for volumeId {}",
+          volume.getStorageID());
+      return;
+    }
+    LOG.info("Removing scanner for volume {} (StorageID {})",
+        volume.getBasePath(), volume.getStorageID());
+    scanner.shutdown();
+    scanners.remove(volume.getStorageID());
+    Uninterruptibles.joinUninterruptibly(scanner, 5, TimeUnit.MINUTES);
+  }
+
+  /**
+   * Stops and removes all volume scanners.<p/>
+   *
+   * This function will block until all the volume scanners have stopped.
+   */
+  public synchronized void removeAllVolumeScanners() {
+    for (Entry<String, VolumeScanner> entry : scanners.entrySet()) {
+      entry.getValue().shutdown();
+    }
+    for (Entry<String, VolumeScanner> entry : scanners.entrySet()) {
+      Uninterruptibles.joinUninterruptibly(entry.getValue(),
+          5, TimeUnit.MINUTES);
+    }
+    scanners.clear();
+  }
+
+  /**
+   * Enable scanning a given block pool id.
+   *
+   * @param bpid        The block pool id to enable scanning for.
+   */
+  synchronized void enableBlockPoolId(String bpid) {
+    Preconditions.checkNotNull(bpid);
+    for (VolumeScanner scanner : scanners.values()) {
+      scanner.enableBlockPoolId(bpid);
+    }
+  }
+
+  /**
+   * Disable scanning a given block pool id.
+   *
+   * @param bpid        The block pool id to disable scanning for.
+   */
+  synchronized void disableBlockPoolId(String bpid) {
+    Preconditions.checkNotNull(bpid);
+    for (VolumeScanner scanner : scanners.values()) {
+      scanner.disableBlockPoolId(bpid);
+    }
+  }
+
+  @VisibleForTesting
+  synchronized VolumeScanner.Statistics getVolumeStats(String volumeId) {
+    VolumeScanner scanner = scanners.get(volumeId);
+    if (scanner == null) {
+      return null;
+    }
+    return scanner.getStatistics();
+  }
+
+  synchronized void printStats(StringBuilder p) {
+    // print out all bpids that we're scanning ?
+    for (Entry<String, VolumeScanner> entry : scanners.entrySet()) {
+      entry.getValue().printStats(p);
+    }
+  }
+
+  @InterfaceAudience.Private
+  public static class Servlet extends HttpServlet {
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public void doGet(HttpServletRequest request,
+                      HttpServletResponse response) throws IOException {
+      response.setContentType("text/plain");
+
+      DataNode datanode = (DataNode)
+          getServletContext().getAttribute("datanode");
+      BlockScanner blockScanner = datanode.getBlockScanner();
+
+      StringBuilder buffer = new StringBuilder(8 * 1024);
+      if (!blockScanner.isEnabled()) {
+        LOG.warn("Periodic block scanner is not running");
+        buffer.append("Periodic block scanner is not running. " +
+            "Please check the datanode log if this is unexpected.");
+      } else {
+        buffer.append("Block Scanner Statistics\n\n");
+        blockScanner.printStats(buffer);
+      }
+      String resp = buffer.toString();
+      LOG.trace("Returned Servlet info {}", resp);
+      response.getWriter().write(resp);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df4edd9a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
index 2d312d7..182b366 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
@@ -600,9 +600,6 @@ class BlockSender implements java.io.Closeable {
         String ioem = e.getMessage();
         if (!ioem.startsWith("Broken pipe") && !ioem.startsWith("Connection reset")) {
           LOG.error("BlockSender.sendChunks() exception: ", e);
-          //Something might be wrong with the block. Make this block the high 
-          //priority block for verification.
-          datanode.blockScanner.addBlock(block, true);
         }
       }
       throw ioeToSocketException(e);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df4edd9a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataBlockScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataBlockScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataBlockScanner.java
deleted file mode 100644
index 450c2b1..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataBlockScanner.java
+++ /dev/null
@@ -1,339 +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.datanode;
-
-import java.io.IOException;
-import java.util.TreeMap;
-
-import javax.servlet.http.HttpServlet;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-
-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.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
-
-import com.google.common.annotations.VisibleForTesting;
-
-/**
- * DataBlockScanner manages block scanning for all the block pools. For each
- * block pool a {@link BlockPoolSliceScanner} is created which runs in a separate
- * thread to scan the blocks for that block pool. When a {@link BPOfferService}
- * becomes alive or dies, blockPoolScannerMap in this class is updated.
- */
-@InterfaceAudience.Private
-public class DataBlockScanner implements Runnable {
-  public static final Log LOG = LogFactory.getLog(DataBlockScanner.class);
-  private final DataNode datanode;
-  private final FsDatasetSpi<? extends FsVolumeSpi> dataset;
-  private final Configuration conf;
-  
-  static final int SLEEP_PERIOD_MS = 5 * 1000;
-
-  /**
-   * Map to find the BlockPoolScanner for a given block pool id. This is updated
-   * when a BPOfferService becomes alive or dies.
-   */
-  private final TreeMap<String, BlockPoolSliceScanner> blockPoolScannerMap = 
-    new TreeMap<String, BlockPoolSliceScanner>();
-  Thread blockScannerThread = null;
-  
-  DataBlockScanner(DataNode datanode,
-      FsDatasetSpi<? extends FsVolumeSpi> dataset,
-      Configuration conf) {
-    this.datanode = datanode;
-    this.dataset = dataset;
-    this.conf = conf;
-  }
-  
-  @Override
-  public void run() {
-    String currentBpId = "";
-    boolean firstRun = true;
-    while (datanode.shouldRun && !Thread.interrupted()) {
-      //Sleep everytime except in the first iteration.
-      if (!firstRun) {
-        try {
-          Thread.sleep(SLEEP_PERIOD_MS);
-        } catch (InterruptedException ex) {
-          // Interrupt itself again to set the interrupt status
-          blockScannerThread.interrupt();
-          continue;
-        }
-      } else {
-        firstRun = false;
-      }
-      
-      BlockPoolSliceScanner bpScanner = getNextBPScanner(currentBpId);
-      if (bpScanner == null) {
-        // Possible if thread is interrupted
-        continue;
-      }
-      currentBpId = bpScanner.getBlockPoolId();
-      // If BPOfferService for this pool is not alive, don't process it
-      if (!datanode.isBPServiceAlive(currentBpId)) {
-        LOG.warn("Block Pool " + currentBpId + " is not alive");
-        // Remove in case BP service died abruptly without proper shutdown
-        removeBlockPool(currentBpId);
-        continue;
-      }
-      bpScanner.scanBlockPoolSlice();
-    }
-
-    // Call shutdown for each allocated BlockPoolSliceScanner.
-    for (BlockPoolSliceScanner bpss: blockPoolScannerMap.values()) {
-      bpss.shutdown();
-    }
-  }
-
-  // Wait for at least one block pool to be up
-  private void waitForInit() {
-    while ((getBlockPoolSetSize() < datanode.getAllBpOs().length)
-        || (getBlockPoolSetSize() < 1)) {
-      try {
-        Thread.sleep(SLEEP_PERIOD_MS);
-      } catch (InterruptedException e) {
-        blockScannerThread.interrupt();
-        return;
-      }
-    }
-  }
-  
-  /**
-   * Find next block pool id to scan. There should be only one current
-   * verification log file. Find which block pool contains the current
-   * verification log file and that is used as the starting block pool id. If no
-   * current files are found start with first block-pool in the blockPoolSet.
-   * However, if more than one current files are found, the one with latest 
-   * modification time is used to find the next block pool id.
-   */
-  private BlockPoolSliceScanner getNextBPScanner(String currentBpId) {
-    
-    String nextBpId = null;
-    while (datanode.shouldRun && !blockScannerThread.isInterrupted()) {
-      waitForInit();
-      synchronized (this) {
-        if (getBlockPoolSetSize() > 0) {          
-          // Find nextBpId by the minimum of the last scan time
-          long lastScanTime = 0;
-          for (String bpid : blockPoolScannerMap.keySet()) {
-            final long t = getBPScanner(bpid).getLastScanTime();
-            if (t != 0L) {
-              if (bpid == null || t < lastScanTime) {
-                lastScanTime =  t;
-                nextBpId = bpid;
-              }
-            }
-          }
-          
-          // nextBpId can still be null if no current log is found,
-          // find nextBpId sequentially.
-          if (nextBpId == null) {
-            nextBpId = blockPoolScannerMap.higherKey(currentBpId);
-            if (nextBpId == null) {
-              nextBpId = blockPoolScannerMap.firstKey();
-            }
-          }
-          if (nextBpId != null) {
-            return getBPScanner(nextBpId);
-          }
-        }
-      }
-      LOG.warn("No block pool is up, going to wait");
-      try {
-        Thread.sleep(5000);
-      } catch (InterruptedException ex) {
-        LOG.warn("Received exception: " + ex);
-        blockScannerThread.interrupt();
-        return null;
-      }
-    }
-    return null;
-  }
-
-  private synchronized int getBlockPoolSetSize() {
-    return blockPoolScannerMap.size();
-  }
-  
-  @VisibleForTesting
-  synchronized BlockPoolSliceScanner getBPScanner(String bpid) {
-    return blockPoolScannerMap.get(bpid);
-  }
-  
-  private synchronized String[] getBpIdList() {
-    return blockPoolScannerMap.keySet().toArray(
-        new String[blockPoolScannerMap.keySet().size()]);
-  }
-  
-  public void addBlock(ExtendedBlock block, boolean scanNow) {
-    BlockPoolSliceScanner bpScanner = getBPScanner(block.getBlockPoolId());
-    if (bpScanner != null) {
-      bpScanner.addBlock(block, scanNow);
-    } else {
-      LOG.warn("No block pool scanner found for block pool id: "
-          + block.getBlockPoolId());
-    }
-  }
-  
-  boolean isInitialized(String bpid) {
-    return getBPScanner(bpid) != null;
-  }
-
-  public synchronized void printBlockReport(StringBuilder buffer,
-      boolean summary) {
-    String[] bpIdList = getBpIdList();
-    if (bpIdList == null || bpIdList.length == 0) {
-      buffer.append("Periodic block scanner is not yet initialized. "
-          + "Please check back again after some time.");
-      return;
-    }
-    for (String bpid : bpIdList) {
-      BlockPoolSliceScanner bpScanner = getBPScanner(bpid);
-      buffer.append("\n\nBlock report for block pool: "+bpid + "\n");
-      bpScanner.printBlockReport(buffer, summary);
-      buffer.append("\n");
-    }
-  }
-  
-  public void deleteBlock(String poolId, Block toDelete) {
-    BlockPoolSliceScanner bpScanner = getBPScanner(poolId);
-    if (bpScanner != null) {
-      bpScanner.deleteBlock(toDelete);
-    } else {
-      LOG.warn("No block pool scanner found for block pool id: "
-          + poolId);
-    }
-  }
-
-  public void deleteBlocks(String poolId, Block[] toDelete) {
-    BlockPoolSliceScanner bpScanner = getBPScanner(poolId);
-    if (bpScanner != null) {
-      bpScanner.deleteBlocks(toDelete);
-    } else {
-      LOG.warn("No block pool scanner found for block pool id: "
-          + poolId);
-    }
-  }
-  
-  public void shutdown() {
-    synchronized (this) {
-      if (blockScannerThread != null) {
-        blockScannerThread.interrupt();
-      }
-    }
-
-    // We cannot join within the synchronized block, because it would create a
-    // deadlock situation.  blockScannerThread calls other synchronized methods.
-    if (blockScannerThread != null) {
-      try {
-        blockScannerThread.join();
-      } catch (InterruptedException e) {
-        // shutting down anyway
-      }
-    }
-  }
-
-  public synchronized void addBlockPool(String blockPoolId) {
-    if (blockPoolScannerMap.get(blockPoolId) != null) {
-      return;
-    }
-    BlockPoolSliceScanner bpScanner = new BlockPoolSliceScanner(blockPoolId,
-        datanode, dataset, conf);
-    blockPoolScannerMap.put(blockPoolId, bpScanner);
-    LOG.info("Added bpid=" + blockPoolId + " to blockPoolScannerMap, new size="
-        + blockPoolScannerMap.size());
-  }
-  
-  public synchronized void removeBlockPool(String blockPoolId) {
-    BlockPoolSliceScanner bpss = blockPoolScannerMap.remove(blockPoolId);
-    if (bpss != null) {
-      bpss.shutdown();
-    }
-    LOG.info("Removed bpid="+blockPoolId+" from blockPoolScannerMap");
-  }
-  
-  @VisibleForTesting
-  long getBlocksScannedInLastRun(String bpid) throws IOException {
-    BlockPoolSliceScanner bpScanner = getBPScanner(bpid);
-    if (bpScanner == null) {
-      throw new IOException("Block Pool: "+bpid+" is not running");
-    } else {
-      return bpScanner.getBlocksScannedInLastRun();
-    }
-  }
-
-  @VisibleForTesting
-  long getTotalScans(String bpid) throws IOException {
-    BlockPoolSliceScanner bpScanner = getBPScanner(bpid);
-    if (bpScanner == null) {
-      throw new IOException("Block Pool: "+bpid+" is not running");
-    } else {
-      return bpScanner.getTotalScans();
-    }
-  }
-
-  @VisibleForTesting
-  public void setLastScanTimeDifference(ExtendedBlock block, int lastScanTimeDifference) {
-    BlockPoolSliceScanner bpScanner = getBPScanner(block.getBlockPoolId());
-    if (bpScanner != null) {
-      bpScanner.setLastScanTimeDifference(lastScanTimeDifference);
-    } else {
-      LOG.warn("No block pool scanner found for block pool id: "
-          + block.getBlockPoolId());
-    }
-  }
-  
-  public void start() {
-    blockScannerThread = new Thread(this);
-    blockScannerThread.setDaemon(true);
-    blockScannerThread.start();
-  }
-  
-  @InterfaceAudience.Private
-  public static class Servlet extends HttpServlet {
-    private static final long serialVersionUID = 1L;
-
-    @Override
-    public void doGet(HttpServletRequest request, 
-                      HttpServletResponse response) throws IOException {
-      response.setContentType("text/plain");
-      
-      DataNode datanode = (DataNode) getServletContext().getAttribute("datanode");
-      DataBlockScanner blockScanner = datanode.blockScanner;
-      
-      boolean summary = (request.getParameter("listblocks") == null);
-      
-      StringBuilder buffer = new StringBuilder(8*1024);
-      if (blockScanner == null) {
-        LOG.warn("Periodic block scanner is not running");
-        buffer.append("Periodic block scanner is not running. " +
-                      "Please check the datanode log if this is unexpected.");
-      } else {
-        blockScanner.printBlockReport(buffer, summary);
-      }
-      response.getWriter().write(buffer.toString()); // extra copy!
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df4edd9a/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 12df9d6..c77bc3d 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
@@ -316,7 +316,7 @@ public class DataNode extends ReconfigurableBase
   BlockPoolTokenSecretManager blockPoolTokenSecretManager;
   private boolean hasAnyBlockPoolRegistered = false;
   
-  volatile DataBlockScanner blockScanner = null;
+  private final BlockScanner blockScanner;
   private DirectoryScanner directoryScanner = null;
   
   /** Activated plug-ins. */
@@ -365,6 +365,7 @@ public class DataNode extends ReconfigurableBase
     this.usersWithLocalPathAccess = null;
     this.connectToDnViaHostname = false;
     this.getHdfsBlockLocationsEnabled = false;
+    this.blockScanner = new BlockScanner(this, conf);
   }
 
   /**
@@ -375,6 +376,7 @@ public class DataNode extends ReconfigurableBase
            final List<StorageLocation> dataDirs,
            final SecureResources resources) throws IOException {
     super(conf);
+    this.blockScanner = new BlockScanner(this, conf);
     this.lastDiskErrorCheck = 0;
     this.maxNumberOfBlocksToLog = conf.getLong(DFS_MAX_NUM_BLOCKS_TO_LOG_KEY,
         DFS_MAX_NUM_BLOCKS_TO_LOG_DEFAULT);
@@ -671,7 +673,8 @@ public class DataNode extends ReconfigurableBase
     this.infoServer.setAttribute("datanode", this);
     this.infoServer.setAttribute(JspHelper.CURRENT_CONF, conf);
     this.infoServer.addServlet(null, "/blockScannerReport",
-                               DataBlockScanner.Servlet.class);
+                               BlockScanner.Servlet.class);
+
     this.infoServer.start();
     InetSocketAddress jettyAddr = infoServer.getConnectorAddress(0);
 
@@ -772,56 +775,12 @@ public class DataNode extends ReconfigurableBase
     // Not a superuser.
     throw new AccessControlException();
   }
-  
-/**
- * Initialize the datanode's periodic scanners:
- *     {@link DataBlockScanner}
- *     {@link DirectoryScanner}
- * They report results on a per-blockpool basis but do their scanning 
- * on a per-Volume basis to minimize competition for disk iops.
- * 
- * @param conf - Configuration has the run intervals and other 
- *               parameters for these periodic scanners
- */
-  private void initPeriodicScanners(Configuration conf) {
-    initDataBlockScanner(conf);
-    initDirectoryScanner(conf);
-  }
-  
+
   private void shutdownPeriodicScanners() {
     shutdownDirectoryScanner();
-    shutdownDataBlockScanner();
-  }
-  
-  /**
-   * See {@link DataBlockScanner}
-   */
-  private synchronized void initDataBlockScanner(Configuration conf) {
-    if (blockScanner != null) {
-      return;
-    }
-    String reason = null;
-    assert data != null;
-    if (conf.getInt(DFS_DATANODE_SCAN_PERIOD_HOURS_KEY,
-                    DFS_DATANODE_SCAN_PERIOD_HOURS_DEFAULT) < 0) {
-      reason = "verification is turned off by configuration";
-    } else if ("SimulatedFSDataset".equals(data.getClass().getSimpleName())) {
-      reason = "verifcation is not supported by SimulatedFSDataset";
-    } 
-    if (reason == null) {
-      blockScanner = new DataBlockScanner(this, data, conf);
-      blockScanner.start();
-    } else {
-      LOG.info("Periodic Block Verification scan disabled because " + reason);
-    }
+    blockScanner.removeAllVolumeScanners();
   }
-  
-  private void shutdownDataBlockScanner() {
-    if (blockScanner != null) {
-      blockScanner.shutdown();
-    }
-  }
-  
+
   /**
    * See {@link DirectoryScanner}
    */
@@ -1250,9 +1209,8 @@ public class DataNode extends ReconfigurableBase
       // registering anywhere. If that's the case, we wouldn't have
       // a block pool id
       String bpId = bpos.getBlockPoolId();
-      if (blockScanner != null) {
-        blockScanner.removeBlockPool(bpId);
-      }
+
+      blockScanner.disableBlockPoolId(bpId);
 
       if (data != null) {
         data.shutdownBlockPool(bpId);
@@ -1296,9 +1254,9 @@ public class DataNode extends ReconfigurableBase
     // failures.
     checkDiskError();
 
-    initPeriodicScanners(conf);
-    
+    initDirectoryScanner(conf);
     data.addBlockPool(nsInfo.getBlockPoolID(), conf);
+    blockScanner.enableBlockPoolId(bpos.getBlockPoolId());
   }
 
   BPOfferService[] getAllBpOs() {
@@ -2168,10 +2126,6 @@ public class DataNode extends ReconfigurableBase
       LOG.warn("Cannot find BPOfferService for reporting block received for bpid="
           + block.getBlockPoolId());
     }
-    FsVolumeSpi volume = getFSDataset().getVolume(block);
-    if (blockScanner != null && !volume.isTransientStorage()) {
-      blockScanner.addBlock(block, false);
-    }
   }
 
   /** Start a single datanode daemon and wait for it to finish.
@@ -2445,8 +2399,9 @@ public class DataNode extends ReconfigurableBase
     return data;
   }
 
+  @VisibleForTesting
   /** @return the block scanner. */
-  public DataBlockScanner getBlockScanner() {
+  public BlockScanner getBlockScanner() {
     return blockScanner;
   }
 


[19/50] [abbrv] hadoop git commit: HDFS-7430. Refactor the BlockScanner to use O(1) memory and use multiple threads (cmccabe)

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/df4edd9a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java
index cffb930..4c703ba 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java
@@ -48,7 +48,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.datanode.BlockPoolSliceStorage;
-import org.apache.hadoop.hdfs.server.datanode.DataBlockScanner;
+import org.apache.hadoop.hdfs.server.datanode.BlockScanner;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DirectoryScanner;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
@@ -89,7 +89,7 @@ public class SnapshotTestHelper {
     GenericTestUtils.disableLog(LogFactory.getLog(DirectoryScanner.class));
     GenericTestUtils.disableLog(LogFactory.getLog(MetricsSystemImpl.class));
     
-    GenericTestUtils.disableLog(DataBlockScanner.LOG);
+    GenericTestUtils.disableLog(BlockScanner.LOG);
     GenericTestUtils.disableLog(HttpServer2.LOG);
     GenericTestUtils.disableLog(DataNode.LOG);
     GenericTestUtils.disableLog(BlockPoolSliceStorage.LOG);


[40/50] [abbrv] hadoop git commit: HADOOP-11419 improve hadoop-maven-plugins. (Hervé Boutemy via stevel)

Posted by zh...@apache.org.
HADOOP-11419 improve hadoop-maven-plugins.  (Hervé Boutemy via stevel)


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

Branch: refs/heads/HDFS-EC
Commit: 5adb125218dd687c2cc632cefe00a8d746963f35
Parents: 38cbafd
Author: Steve Loughran <st...@apache.org>
Authored: Sun Jan 25 16:12:58 2015 +0000
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:30 2015 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt  |  2 ++
 hadoop-maven-plugins/pom.xml                     | 19 +++++--------------
 .../hadoop/maven/plugin/protoc/ProtocMojo.java   |  7 ++-----
 .../hadoop/maven/plugin/util/FileSetUtils.java   |  2 +-
 .../plugin/versioninfo/VersionInfoMojo.java      |  3 +--
 5 files changed, 11 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5adb1252/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 c0617e8..8618e38 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -497,6 +497,8 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11489 Dropping dependency on io.netty from hadoop-nfs' pom.xml
     (Ted Yu via ozawa)
 
+    HADOOP-11419 Improve hadoop-maven-plugins. (Herve Boutemy via stevel)
+
   OPTIMIZATIONS
 
     HADOOP-11323. WritableComparator#compare keeps reference to byte array.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5adb1252/hadoop-maven-plugins/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-maven-plugins/pom.xml b/hadoop-maven-plugins/pom.xml
index 4beaf7c..b48b9ac 100644
--- a/hadoop-maven-plugins/pom.xml
+++ b/hadoop-maven-plugins/pom.xml
@@ -28,6 +28,7 @@
   <name>Apache Hadoop Maven Plugins</name>
   <properties>
     <maven.dependency.version>3.0</maven.dependency.version>
+    <maven.plugin-tools.version>3.4</maven.plugin-tools.version>
   </properties>
   <dependencies>
     <dependency>
@@ -43,30 +44,20 @@
     <dependency>
       <groupId>org.apache.maven.plugin-tools</groupId>
       <artifactId>maven-plugin-annotations</artifactId>
-      <version>${maven.dependency.version}</version>
+      <version>${maven.plugin-tools.version}</version>
       <scope>provided</scope>
     </dependency>
-    <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <scope>test</scope>
-    </dependency>
   </dependencies>
   <build>
     <plugins>
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-plugin-plugin</artifactId>
-        <version>${maven.dependency.version}</version>
-        <configuration>
-          <skipErrorNoDescriptorsFound>true</skipErrorNoDescriptorsFound>
-        </configuration>
+        <version>${maven.plugin-tools.version}</version>
         <executions>
           <execution>
-            <id>mojo-descriptor</id>
-            <goals>
-              <goal>descriptor</goal>
-            </goals>
+            <id>default-descriptor</id>
+            <phase>process-classes</phase>
           </execution>
         </executions>
       </plugin>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5adb1252/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/protoc/ProtocMojo.java
----------------------------------------------------------------------
diff --git a/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/protoc/ProtocMojo.java b/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/protoc/ProtocMojo.java
index 86ba7bf..465b713 100644
--- a/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/protoc/ProtocMojo.java
+++ b/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/protoc/ProtocMojo.java
@@ -31,7 +31,7 @@ import java.util.List;
 @Mojo(name="protoc", defaultPhase = LifecyclePhase.GENERATE_SOURCES)
 public class ProtocMojo extends AbstractMojo {
 
-  @Parameter(defaultValue="${project}")
+  @Parameter(defaultValue="${project}", readonly=true)
   private MavenProject project;
 
   @Parameter
@@ -43,7 +43,7 @@ public class ProtocMojo extends AbstractMojo {
   @Parameter(required=true)
   private FileSet source;
 
-  @Parameter
+  @Parameter(defaultValue="protoc")
   private String protocCommand;
 
   @Parameter(required=true)
@@ -51,9 +51,6 @@ public class ProtocMojo extends AbstractMojo {
 
   public void execute() throws MojoExecutionException {
     try {
-      if (protocCommand == null || protocCommand.trim().isEmpty()) {
-        protocCommand = "protoc";
-      }
       List<String> command = new ArrayList<String>();
       command.add(protocCommand);
       command.add("--version");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5adb1252/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/util/FileSetUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/util/FileSetUtils.java b/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/util/FileSetUtils.java
index 8bd66cc..2f2d2fb 100644
--- a/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/util/FileSetUtils.java
+++ b/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/util/FileSetUtils.java
@@ -33,7 +33,7 @@ public class FileSetUtils {
    * @param list List of all elements
    * @return String containing every element, comma-separated
    */
-  private static String getCommaSeparatedList(List list) {
+  private static String getCommaSeparatedList(List<String> list) {
     StringBuilder buffer = new StringBuilder();
     String separator = "";
     for (Object e : list) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5adb1252/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/versioninfo/VersionInfoMojo.java
----------------------------------------------------------------------
diff --git a/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/versioninfo/VersionInfoMojo.java b/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/versioninfo/VersionInfoMojo.java
index 563bd59..f342463 100644
--- a/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/versioninfo/VersionInfoMojo.java
+++ b/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/versioninfo/VersionInfoMojo.java
@@ -18,7 +18,6 @@ import org.apache.hadoop.maven.plugin.util.FileSetUtils;
 import org.apache.maven.model.FileSet;
 import org.apache.maven.plugin.AbstractMojo;
 import org.apache.maven.plugin.MojoExecutionException;
-import org.apache.maven.plugins.annotations.LifecyclePhase;
 import org.apache.maven.plugins.annotations.Mojo;
 import org.apache.maven.plugins.annotations.Parameter;
 import org.apache.maven.project.MavenProject;
@@ -47,7 +46,7 @@ import java.util.TimeZone;
 @Mojo(name="version-info")
 public class VersionInfoMojo extends AbstractMojo {
 
-  @Parameter(defaultValue="${project}")
+  @Parameter(defaultValue="${project}", readonly=true)
   private MavenProject project;
 
   @Parameter(required=true)


[26/50] [abbrv] hadoop git commit: HADOOP-11500. InputStream is left unclosed in ApplicationClassLoader. Contributed by Ted Yu.

Posted by zh...@apache.org.
HADOOP-11500. InputStream is left unclosed in ApplicationClassLoader. Contributed by Ted Yu.


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

Branch: refs/heads/HDFS-EC
Commit: 3f90fca19117ed9d59391f23a30140bd4ff09f98
Parents: ea85ae4
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Fri Jan 23 00:50:19 2015 +0900
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:28 2015 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt                | 3 +++
 .../java/org/apache/hadoop/util/ApplicationClassLoader.java    | 6 ++----
 2 files changed, 5 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3f90fca1/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 eb9015c..aaa7041 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -744,6 +744,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11209. Configuration#updatingResource/finalParameters are not
     thread-safe. (Varun Saxena via ozawa)
 
+    HADOOP-11500. InputStream is left unclosed in ApplicationClassLoader.
+    (Ted Yu via ozawa)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3f90fca1/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ApplicationClassLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ApplicationClassLoader.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ApplicationClassLoader.java
index 9f16b61..6d37c28 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ApplicationClassLoader.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ApplicationClassLoader.java
@@ -67,10 +67,8 @@ public class ApplicationClassLoader extends URLClassLoader {
   };
 
   static {
-    InputStream is = null;
-    try {
-      is = ApplicationClassLoader.class.getClassLoader().
-          getResourceAsStream(PROPERTIES_FILE);
+    try (InputStream is = ApplicationClassLoader.class.getClassLoader()
+        .getResourceAsStream(PROPERTIES_FILE);) {
       if (is == null) {
         throw new ExceptionInInitializerError("properties file " +
             PROPERTIES_FILE + " is not found");


[30/50] [abbrv] hadoop git commit: HDFS-7575. Fix CHANGES.txt

Posted by zh...@apache.org.
HDFS-7575. Fix CHANGES.txt


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

Branch: refs/heads/HDFS-EC
Commit: 8262acf91ee7e34cd7235e7ac942ba470df06acc
Parents: c13d501
Author: Arpit Agarwal <ar...@apache.org>
Authored: Thu Jan 22 14:33:06 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:28 2015 -0800

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8262acf9/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 866b765..7c5c639 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -762,6 +762,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7548. Corrupt block reporting delayed until datablock scanner thread
     detects it (Rushabh Shah via kihwal)
 
+    HDFS-7575. Upgrade should generate a unique storage ID for each
+    volume. (Arpit Agarwal)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES


[25/50] [abbrv] hadoop git commit: HADOOP-11493. Fix some typos in kms-acls.xml description. (Contributed by Charles Lamb)

Posted by zh...@apache.org.
HADOOP-11493. Fix some typos in kms-acls.xml description. (Contributed by Charles Lamb)


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

Branch: refs/heads/HDFS-EC
Commit: c265a0e3f687ae0cda01577f91c9401575545eac
Parents: d3b92a1
Author: Akira Ajisaka <aa...@apache.org>
Authored: Fri Jan 23 11:48:19 2015 +0900
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:28 2015 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt                | 3 +++
 hadoop-common-project/hadoop-kms/src/main/conf/kms-acls.xml    | 2 +-
 hadoop-common-project/hadoop-kms/src/site/apt/index.apt.vm     | 6 +++---
 .../hadoop-kms/src/test/resources/mini-kms-acls-default.xml    | 4 ++--
 4 files changed, 9 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c265a0e3/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 47eaf7b..bab2220 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -750,6 +750,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11008. Remove duplicated description about proxy-user in site 
     documents (Masatake Iwasaki via aw)
 
+    HADOOP-11493. Fix some typos in kms-acls.xml description.
+    (Charles Lamb via aajisaka)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c265a0e3/hadoop-common-project/hadoop-kms/src/main/conf/kms-acls.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/main/conf/kms-acls.xml b/hadoop-common-project/hadoop-kms/src/main/conf/kms-acls.xml
index 1d5b649..cba69f4 100644
--- a/hadoop-common-project/hadoop-kms/src/main/conf/kms-acls.xml
+++ b/hadoop-common-project/hadoop-kms/src/main/conf/kms-acls.xml
@@ -41,7 +41,7 @@
     <value>*</value>
     <description>
       ACL for rollover-key operations.
-      If the user does is not in the GET ACL, the key material is not returned
+      If the user is not in the GET ACL, the key material is not returned
       as part of the response.
     </description>
   </property>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c265a0e3/hadoop-common-project/hadoop-kms/src/site/apt/index.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/site/apt/index.apt.vm b/hadoop-common-project/hadoop-kms/src/site/apt/index.apt.vm
index 80d9a48..a2dcce3 100644
--- a/hadoop-common-project/hadoop-kms/src/site/apt/index.apt.vm
+++ b/hadoop-common-project/hadoop-kms/src/site/apt/index.apt.vm
@@ -299,7 +299,7 @@ $ keytool -genkey -alias tomcat -keyalg RSA
     <value>*</value>
     <description>
       ACL for create-key operations.
-      If the user does is not in the GET ACL, the key material is not returned
+      If the user is not in the GET ACL, the key material is not returned
       as part of the response.
     </description>
   </property>
@@ -309,7 +309,7 @@ $ keytool -genkey -alias tomcat -keyalg RSA
     <value>hdfs,foo</value>
     <description>
       Blacklist for create-key operations.
-      If the user does is in the Blacklist, the key material is not returned
+      If the user is in the Blacklist, the key material is not returned
       as part of the response.
     </description>
   </property>
@@ -335,7 +335,7 @@ $ keytool -genkey -alias tomcat -keyalg RSA
     <value>*</value>
     <description>
       ACL for rollover-key operations.
-      If the user does is not in the GET ACL, the key material is not returned
+      If the user is not in the GET ACL, the key material is not returned
       as part of the response.
     </description>
   </property>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c265a0e3/hadoop-common-project/hadoop-kms/src/test/resources/mini-kms-acls-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/test/resources/mini-kms-acls-default.xml b/hadoop-common-project/hadoop-kms/src/test/resources/mini-kms-acls-default.xml
index 24a46b8..6ac4155 100644
--- a/hadoop-common-project/hadoop-kms/src/test/resources/mini-kms-acls-default.xml
+++ b/hadoop-common-project/hadoop-kms/src/test/resources/mini-kms-acls-default.xml
@@ -23,7 +23,7 @@
     <value>*</value>
     <description>
       ACL for create-key operations.
-      If the user does is not in the GET ACL, the key material is not returned
+      If the user is not in the GET ACL, the key material is not returned
       as part of the response.
     </description>
   </property>
@@ -41,7 +41,7 @@
     <value>*</value>
     <description>
       ACL for rollover-key operations.
-      If the user does is not in the GET ACL, the key material is not returned
+      If the user is not in the GET ACL, the key material is not returned
       as part of the response.
     </description>
   </property>


[09/50] [abbrv] hadoop git commit: HDFS-7548. Corrupt block reporting delayed until datablock scanner thread detects it. Contributed by Rushabh Shah.

Posted by zh...@apache.org.
HDFS-7548. Corrupt block reporting delayed until datablock scanner thread detects it. Contributed by Rushabh Shah.


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

Branch: refs/heads/HDFS-EC
Commit: a8c59ba0cef5b904b6f499c8e073203abb91d2b4
Parents: ce16949
Author: Kihwal Lee <ki...@apache.org>
Authored: Wed Jan 21 14:41:31 2015 -0600
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:26 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +
 .../server/datanode/BlockPoolSliceScanner.java  | 49 ++++++++++++++--
 .../hdfs/server/datanode/BlockSender.java       |  3 +
 .../hdfs/server/datanode/DataBlockScanner.java  | 15 ++++-
 .../hadoop/hdfs/server/datanode/DataNode.java   |  2 +-
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |  4 +-
 .../hadoop/hdfs/TestDatanodeBlockScanner.java   | 60 +++++++++++++++++++-
 7 files changed, 125 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8c59ba0/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 0a301f8..25ad33b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -756,6 +756,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7610. Fix removal of dynamically added DN volumes (Lei (Eddy) Xu via
     Colin P. McCabe)
 
+    HDFS-7548. Corrupt block reporting delayed until datablock scanner thread
+    detects it (Rushabh Shah via kihwal)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8c59ba0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java
index 61f1e7e..f36fea1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java
@@ -105,6 +105,7 @@ class BlockPoolSliceScanner {
   private long bytesLeft = 0; // Bytes to scan in this period
   private long totalBytesToScan = 0;
   private boolean isNewPeriod = true;
+  private int lastScanTimeDifference = 5*60*1000;
   
   private final LogFileHandler verificationLog;
   
@@ -112,6 +113,7 @@ class BlockPoolSliceScanner {
        200, MAX_SCAN_RATE);
   
   private static enum ScanType {
+    IMMEDIATE_SCAN,  
     VERIFICATION_SCAN,     // scanned as part of periodic verfication
     NONE,
   }
@@ -129,12 +131,17 @@ class BlockPoolSliceScanner {
 
       @Override
       public int compare(BlockScanInfo left, BlockScanInfo right) {
+        final ScanType leftNextScanType = left.nextScanType;
+        final ScanType rightNextScanType = right.nextScanType;
         final long l = left.lastScanTime;
         final long r = right.lastScanTime;
+        // Compare by nextScanType if they are same then compare by 
+        // lastScanTimes
         // compare blocks itself if scantimes are same to avoid.
         // because TreeMap uses comparator if available to check existence of
         // the object. 
-        return l < r? -1: l > r? 1: left.compareTo(right); 
+        int compareByNextScanType = leftNextScanType.compareTo(rightNextScanType);
+        return compareByNextScanType < 0? -1: compareByNextScanType > 0? 1:  l < r? -1: l > r? 1: left.compareTo(right); 
       }
     };
 
@@ -142,6 +149,7 @@ class BlockPoolSliceScanner {
     ScanType lastScanType = ScanType.NONE; 
     boolean lastScanOk = true;
     private LinkedElement next;
+    ScanType nextScanType = ScanType.VERIFICATION_SCAN;
     
     BlockScanInfo(Block block) {
       super(block);
@@ -265,10 +273,12 @@ class BlockPoolSliceScanner {
   private synchronized void updateBlockInfo(LogEntry e) {
     BlockScanInfo info = blockMap.get(new Block(e.blockId, 0, e.genStamp));
     
-    if(info != null && e.verificationTime > 0 && 
+    if (info != null && e.verificationTime > 0 && 
         info.lastScanTime < e.verificationTime) {
       delBlockInfo(info);
-      info.lastScanTime = e.verificationTime;
+      if (info.nextScanType != ScanType.IMMEDIATE_SCAN) {
+        info.lastScanTime = e.verificationTime;
+      }
       info.lastScanType = ScanType.VERIFICATION_SCAN;
       addBlockInfo(info, false);
     }
@@ -285,9 +295,23 @@ class BlockPoolSliceScanner {
         DFSUtil.getRandom().nextInt(periodInt);
   }
 
-  /** Adds block to list of blocks */
-  synchronized void addBlock(ExtendedBlock block) {
+  /** Adds block to list of blocks 
+   * @param scanNow - true if we want to make that particular block a high 
+   * priority one to scan immediately
+   **/
+  synchronized void addBlock(ExtendedBlock block, boolean scanNow) {
     BlockScanInfo info = blockMap.get(block.getLocalBlock());
+    long lastScanTime = 0;
+    if (info != null) {
+      lastScanTime = info.lastScanTime;
+    }
+    // If the particular block is scanned in last 5 minutes, the  no need to 
+    // verify that block again
+    if (scanNow && Time.monotonicNow() - lastScanTime < 
+        lastScanTimeDifference) {
+      return;
+    }
+    
     if ( info != null ) {
       LOG.warn("Adding an already existing block " + block);
       delBlockInfo(info);
@@ -295,6 +319,12 @@ class BlockPoolSliceScanner {
     
     info = new BlockScanInfo(block.getLocalBlock());    
     info.lastScanTime = getNewBlockScanTime();
+    if (scanNow) {
+      // Create a new BlockScanInfo object and set the lastScanTime to 0
+      // which will make it the high priority block
+      LOG.info("Adding block for immediate verification " + block);
+      info.nextScanType = ScanType.IMMEDIATE_SCAN;
+    }
     
     addBlockInfo(info, true);
     adjustThrottler();
@@ -340,6 +370,7 @@ class BlockPoolSliceScanner {
     info.lastScanType = type;
     info.lastScanTime = now;
     info.lastScanOk = scanOk;
+    info.nextScanType = ScanType.VERIFICATION_SCAN;
     addBlockInfo(info, false);
         
     // Don't update meta data if the verification failed.
@@ -363,6 +394,11 @@ class BlockPoolSliceScanner {
     }
   }
   
+  @VisibleForTesting
+  synchronized void setLastScanTimeDifference(int lastScanTimeDifference) {
+    this.lastScanTimeDifference = lastScanTimeDifference;
+  }
+  
   static private class LogEntry {
 
     long blockId = -1;
@@ -502,6 +538,9 @@ class BlockPoolSliceScanner {
   
   private synchronized boolean isFirstBlockProcessed() {
     if (!blockInfoSet.isEmpty()) {
+      if (blockInfoSet.first().nextScanType == ScanType.IMMEDIATE_SCAN) {
+        return false;
+      }
       long blockId = blockInfoSet.first().getBlockId();
       if ((processedBlocks.get(blockId) != null)
           && (processedBlocks.get(blockId) == 1)) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8c59ba0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
index 182b366..2d312d7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
@@ -600,6 +600,9 @@ class BlockSender implements java.io.Closeable {
         String ioem = e.getMessage();
         if (!ioem.startsWith("Broken pipe") && !ioem.startsWith("Connection reset")) {
           LOG.error("BlockSender.sendChunks() exception: ", e);
+          //Something might be wrong with the block. Make this block the high 
+          //priority block for verification.
+          datanode.blockScanner.addBlock(block, true);
         }
       }
       throw ioeToSocketException(e);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8c59ba0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataBlockScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataBlockScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataBlockScanner.java
index bee3625..450c2b1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataBlockScanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataBlockScanner.java
@@ -186,10 +186,10 @@ public class DataBlockScanner implements Runnable {
         new String[blockPoolScannerMap.keySet().size()]);
   }
   
-  public void addBlock(ExtendedBlock block) {
+  public void addBlock(ExtendedBlock block, boolean scanNow) {
     BlockPoolSliceScanner bpScanner = getBPScanner(block.getBlockPoolId());
     if (bpScanner != null) {
-      bpScanner.addBlock(block);
+      bpScanner.addBlock(block, scanNow);
     } else {
       LOG.warn("No block pool scanner found for block pool id: "
           + block.getBlockPoolId());
@@ -293,6 +293,17 @@ public class DataBlockScanner implements Runnable {
     }
   }
 
+  @VisibleForTesting
+  public void setLastScanTimeDifference(ExtendedBlock block, int lastScanTimeDifference) {
+    BlockPoolSliceScanner bpScanner = getBPScanner(block.getBlockPoolId());
+    if (bpScanner != null) {
+      bpScanner.setLastScanTimeDifference(lastScanTimeDifference);
+    } else {
+      LOG.warn("No block pool scanner found for block pool id: "
+          + block.getBlockPoolId());
+    }
+  }
+  
   public void start() {
     blockScannerThread = new Thread(this);
     blockScannerThread.setDaemon(true);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8c59ba0/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 84528e7..12df9d6 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
@@ -2170,7 +2170,7 @@ public class DataNode extends ReconfigurableBase
     }
     FsVolumeSpi volume = getFSDataset().getVolume(block);
     if (blockScanner != null && !volume.isTransientStorage()) {
-      blockScanner.addBlock(block);
+      blockScanner.addBlock(block, false);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8c59ba0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
index d8cc287..f990faf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
@@ -771,7 +771,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     }
 
     // Replace the old block if any to reschedule the scanning.
-    datanode.getBlockScanner().addBlock(block);
+    datanode.getBlockScanner().addBlock(block, false);
     return replicaInfo;
   }
 
@@ -2035,7 +2035,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
         final DataBlockScanner blockScanner = datanode.getBlockScanner();
         if (!vol.isTransientStorage()) {
           if (blockScanner != null) {
-            blockScanner.addBlock(new ExtendedBlock(bpid, diskBlockInfo));
+            blockScanner.addBlock(new ExtendedBlock(bpid, diskBlockInfo), false);
           }
         } else {
           ramDiskReplicaTracker.addReplica(bpid, blockId, (FsVolumeImpl) vol);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8c59ba0/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeBlockScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeBlockScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeBlockScanner.java
index bf0182b..9e78c10 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeBlockScanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeBlockScanner.java
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.hdfs;
 
-import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -28,7 +27,10 @@ import java.io.IOException;
 import java.io.RandomAccessFile;
 import java.net.InetSocketAddress;
 import java.net.URL;
+import java.util.Collections;
+import java.util.HashSet;
 import java.util.Random;
+import java.util.Set;
 import java.util.concurrent.TimeoutException;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
@@ -42,6 +44,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.server.datanode.DataBlockScanner;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
@@ -490,4 +493,59 @@ public class TestDatanodeBlockScanner {
       cluster.shutdown();
     }
   }
+  
+/**
+ * This test verifies whether block is added to the first location of 
+ * BlockPoolSliceScanner#blockInfoSet
+ */
+  @Test
+  public void testAddBlockInfoToFirstLocation() throws Exception {
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(new Configuration())
+        .numDataNodes(1).build();
+    FileSystem fs = null;
+    try {
+      fs = cluster.getFileSystem();
+      DataNode dataNode = cluster.getDataNodes().get(0);
+      // Creating a bunch of blocks
+      for (int i = 1; i < 10; i++) {
+        Path fileName = new Path("/test" + i);
+        DFSTestUtil.createFile(fs, fileName, 1024, (short) 1, 1000L);
+      } 
+      // Get block of the first file created (file1)
+      ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, new Path("/test1"));
+      dataNode.getBlockScanner().setLastScanTimeDifference(block, 0);
+      // Let it sleep for more than 5 seconds so that BlockPoolSliceScanner can
+      // scan the first set of blocks
+      Thread.sleep(10000);
+      Long scanTime1Fortest1Block = DataNodeTestUtils.getLatestScanTime(
+          dataNode, block);
+      // Create another set of blocks
+      for (int i = 10; i < 20; i++) {
+        Path fileName = new Path("/test" + i);
+        DFSTestUtil.createFile(fs, fileName, 1024, (short) 1, 1000L);
+      }
+      dataNode.getBlockScanner().addBlock(block, true);
+      // Sleep so that BlockPoolSliceScanner can scan the second set of blocks
+      // and one block which we scheduled to rescan
+      Thread.sleep(10000);
+      // Get the lastScanTime of all of the second set of blocks
+      Set<Long> lastScanTimeSet = new HashSet<Long>();
+      for (int i = 10; i < 20; i++) {
+        long lastScanTime = DataNodeTestUtils.getLatestScanTime(dataNode,
+            DFSTestUtil.getFirstBlock(fs, new Path("/test" + i)));
+        lastScanTimeSet.add(lastScanTime);
+      }
+      Long scanTime2Fortest1Block = DataNodeTestUtils.getLatestScanTime(
+          dataNode, DFSTestUtil.getFirstBlock(fs, new Path("/test1")));
+      Long minimumLastScanTime = Collections.min(lastScanTimeSet);
+      assertTrue("The second scanTime for test1 block should be greater than "
+         + "first scanTime", scanTime2Fortest1Block > scanTime1Fortest1Block);
+      assertTrue("The second scanTime for test1 block should be less than or"
+         + " equal to minimum of the lastScanTime of second set of blocks",
+          scanTime2Fortest1Block <= minimumLastScanTime);
+    } finally {
+      IOUtils.closeStream(fs);
+      cluster.shutdown();
+    }
+  }
 }


[48/50] [abbrv] hadoop git commit: HADOOP-11450. Cleanup DistCpV1 not to use deprecated methods and fix javadocs. Contributed by Varun Saxena.

Posted by zh...@apache.org.
HADOOP-11450. Cleanup DistCpV1 not to use deprecated methods and fix javadocs. Contributed by Varun Saxena.


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

Branch: refs/heads/HDFS-EC
Commit: b2de93dd330ba4a2e3a087bdafd840067f4cc43a
Parents: f343538
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Mon Jan 26 12:58:38 2015 +0900
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:31 2015 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |   3 +
 .../java/org/apache/hadoop/tools/DistCpV1.java  | 152 ++++++++-----------
 2 files changed, 68 insertions(+), 87 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2de93dd/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 8618e38..662f580 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -499,6 +499,9 @@ Release 2.7.0 - UNRELEASED
 
     HADOOP-11419 Improve hadoop-maven-plugins. (Herve Boutemy via stevel)
 
+    HADOOP-11450. Cleanup DistCpV1 not to use deprecated methods and fix
+    javadocs. (Varun Saxena via ozawa)
+
   OPTIMIZATIONS
 
     HADOOP-11323. WritableComparator#compare keeps reference to byte array.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2de93dd/hadoop-tools/hadoop-extras/src/main/java/org/apache/hadoop/tools/DistCpV1.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-extras/src/main/java/org/apache/hadoop/tools/DistCpV1.java b/hadoop-tools/hadoop-extras/src/main/java/org/apache/hadoop/tools/DistCpV1.java
index c44b67b..f46c421 100644
--- a/hadoop-tools/hadoop-extras/src/main/java/org/apache/hadoop/tools/DistCpV1.java
+++ b/hadoop-tools/hadoop-extras/src/main/java/org/apache/hadoop/tools/DistCpV1.java
@@ -51,9 +51,11 @@ import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.SequenceFile.Reader;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.SequenceFile.Writer;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.mapred.FileOutputFormat;
 import org.apache.hadoop.mapred.FileSplit;
@@ -73,6 +75,7 @@ import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
+import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
 
 /**
  * A Map-reduce program to recursively copy directories between
@@ -283,9 +286,8 @@ public class DistCpV1 implements Tool {
       long last = 0L;
       long acc = 0L;
       long cbrem = srcst.getLen();
-      SequenceFile.Reader sl = null;
-      try {
-        sl = new SequenceFile.Reader(fs, src, job);
+      try (SequenceFile.Reader sl =
+          new SequenceFile.Reader(job, Reader.file(src))) {
         for (; sl.next(key, value); last = sl.getPosition()) {
           // if adding this split would put this split past the target size,
           // cut the last split and put this next file in the next split.
@@ -299,9 +301,6 @@ public class DistCpV1 implements Tool {
           acc += key.get();
         }
       }
-      finally {
-        checkAndClose(sl);
-      }
       if (cbrem != 0) {
         splits.add(new FileSplit(src, pos, cbrem, (String[])null));
       }
@@ -438,32 +437,28 @@ public class DistCpV1 implements Tool {
      */
     private long doCopyFile(FileStatus srcstat, Path tmpfile, Path absdst,
                             Reporter reporter) throws IOException {
-      FSDataInputStream in = null;
-      FSDataOutputStream out = null;
       long bytesCopied = 0L;
-      try {
-        Path srcPath = srcstat.getPath();
-        // open src file
-        in = srcPath.getFileSystem(job).open(srcPath);
+      Path srcPath = srcstat.getPath();
+      // open src file
+      try (FSDataInputStream in = srcPath.getFileSystem(job).open(srcPath)) {
         reporter.incrCounter(Counter.BYTESEXPECTED, srcstat.getLen());
         // open tmp file
-        out = create(tmpfile, reporter, srcstat);
-        LOG.info("Copying file " + srcPath + " of size " +
-                 srcstat.getLen() + " bytes...");
+        try (FSDataOutputStream out = create(tmpfile, reporter, srcstat)) {
+          LOG.info("Copying file " + srcPath + " of size " +
+                   srcstat.getLen() + " bytes...");
         
-        // copy file
-        for(int bytesRead; (bytesRead = in.read(buffer)) >= 0; ) {
-          out.write(buffer, 0, bytesRead);
-          bytesCopied += bytesRead;
-          reporter.setStatus(
-              String.format("%.2f ", bytesCopied*100.0/srcstat.getLen())
-              + absdst + " [ " +
-              StringUtils.humanReadableInt(bytesCopied) + " / " +
-              StringUtils.humanReadableInt(srcstat.getLen()) + " ]");
+          // copy file
+          for(int bytesRead; (bytesRead = in.read(buffer)) >= 0; ) {
+            out.write(buffer, 0, bytesRead);
+            bytesCopied += bytesRead;
+            reporter.setStatus(
+                String.format("%.2f ", bytesCopied*100.0/srcstat.getLen())
+                + absdst + " [ " +
+                TraditionalBinaryPrefix.long2String(bytesCopied, "", 1) + " / "
+                + TraditionalBinaryPrefix.long2String(srcstat.getLen(), "", 1)
+                + " ]");
+          }
         }
-      } finally {
-        checkAndClose(in);
-        checkAndClose(out);
       }
       return bytesCopied;
     }
@@ -471,7 +466,8 @@ public class DistCpV1 implements Tool {
     /**
      * Copy a file to a destination.
      * @param srcstat src path and metadata
-     * @param dstpath dst path
+     * @param relativedst relative dst path
+     * @param outc Log of skipped files
      * @param reporter
      * @throws IOException if copy fails(even if the validation of copy fails)
      */
@@ -570,7 +566,8 @@ public class DistCpV1 implements Tool {
     }
 
     static String bytesString(long b) {
-      return b + " bytes (" + StringUtils.humanReadableInt(b) + ")";
+      return b + " bytes (" +
+          TraditionalBinaryPrefix.long2String(b, "", 1) + ")";
     }
 
     /**
@@ -762,6 +759,7 @@ public class DistCpV1 implements Tool {
 
   /**
    * Driver to copy srcPath to destPath depending on required protocol.
+   * @param conf configuration
    * @param args arguments
    */
   static void copy(final Configuration conf, final Arguments args
@@ -838,10 +836,8 @@ public class DistCpV1 implements Tool {
 
     FileSystem dstfs = destPath.getFileSystem(conf);
     Path dstdirlist = new Path(jobconf.get(DST_DIR_LIST_LABEL));
-    SequenceFile.Reader in = null;
-    try {
-      in = new SequenceFile.Reader(dstdirlist.getFileSystem(jobconf),
-          dstdirlist, jobconf);
+    try (SequenceFile.Reader in =
+        new SequenceFile.Reader(jobconf, Reader.file(dstdirlist))) {
       Text dsttext = new Text();
       FilePair pair = new FilePair(); 
       for(; in.next(dsttext, pair); ) {
@@ -849,8 +845,6 @@ public class DistCpV1 implements Tool {
         updateDestStatus(pair.input, dstfs.getFileStatus(absdst),
             preseved, dstfs);
       }
-    } finally {
-      checkAndClose(in);
     }
   }
 
@@ -876,6 +870,8 @@ public class DistCpV1 implements Tool {
      * @param preservedAttributes Preserved attributes 
      * @param filelimit File limit
      * @param sizelimit Size limit
+     * @param mapredSslConf ssl configuration
+     * @param dryrun
      */
     Arguments(List<Path> srcs, Path basedir, Path dst, Path log,
         EnumSet<Options> flags, String preservedAttributes,
@@ -1266,15 +1262,18 @@ public class DistCpV1 implements Tool {
     long fileCount = 0L, dirCount = 0L, byteCount = 0L, cbsyncs = 0L,
          skipFileCount = 0L, skipByteCount = 0L;
     try (
-        SequenceFile.Writer src_writer = SequenceFile.createWriter(jobfs,
-            jobConf, srcfilelist, LongWritable.class, FilePair.class,
-            SequenceFile.CompressionType.NONE);
-        SequenceFile.Writer dst_writer = SequenceFile.createWriter(jobfs,
-            jobConf, dstfilelist, Text.class, Text.class,
-            SequenceFile.CompressionType.NONE);
-        SequenceFile.Writer dir_writer = SequenceFile.createWriter(jobfs,
-            jobConf, dstdirlist, Text.class, FilePair.class,
-            SequenceFile.CompressionType.NONE)
+        SequenceFile.Writer src_writer = SequenceFile.createWriter(jobConf,
+            Writer.file(srcfilelist), Writer.keyClass(LongWritable.class),
+            Writer.valueClass(FilePair.class), Writer.compression(
+            SequenceFile.CompressionType.NONE));
+        SequenceFile.Writer dst_writer = SequenceFile.createWriter(jobConf,
+            Writer.file(dstfilelist), Writer.keyClass(Text.class),
+            Writer.valueClass(Text.class), Writer.compression(
+            SequenceFile.CompressionType.NONE));
+        SequenceFile.Writer dir_writer = SequenceFile.createWriter(jobConf,
+            Writer.file(dstdirlist), Writer.keyClass(Text.class),
+            Writer.valueClass(FilePair.class), Writer.compression(
+            SequenceFile.CompressionType.NONE));
     ) {
       // handle the case where the destination directory doesn't exist
       // and we've only a single src directory OR we're updating/overwriting
@@ -1286,7 +1285,8 @@ public class DistCpV1 implements Tool {
       HashSet<Path> parentDirsToCopy = new HashSet<Path>();
       if (args.basedir != null) {
         FileSystem basefs = args.basedir.getFileSystem(conf);
-        basedir = args.basedir.makeQualified(basefs);
+        basedir = args.basedir.makeQualified(
+            basefs.getUri(), basefs.getWorkingDirectory());
         if (!basefs.isDirectory(basedir)) {
           throw new IOException("Basedir " + basedir + " is not a directory.");
         }
@@ -1307,7 +1307,8 @@ public class DistCpV1 implements Tool {
 
         if (basedir != null) {
           root = basedir;
-          Path parent = src.getParent().makeQualified(srcfs);
+          Path parent = src.getParent().makeQualified(
+              srcfs.getUri(), srcfs.getWorkingDirectory());
           while (parent != null && !parent.equals(basedir)) {
             if (!parentDirsToCopy.contains(parent)){
               parentDirsToCopy.add(parent);
@@ -1427,11 +1428,12 @@ public class DistCpV1 implements Tool {
     }
     LOG.info("sourcePathsCount(files+directories)=" + srcCount);
     LOG.info("filesToCopyCount=" + fileCount);
-    LOG.info("bytesToCopyCount=" + StringUtils.humanReadableInt(byteCount));
+    LOG.info("bytesToCopyCount=" +
+             TraditionalBinaryPrefix.long2String(byteCount, "", 1));
     if (update) {
       LOG.info("filesToSkipCopyCount=" + skipFileCount);
       LOG.info("bytesToSkipCopyCount=" +
-               StringUtils.humanReadableInt(skipByteCount));
+               TraditionalBinaryPrefix.long2String(skipByteCount, "", 1));
     }
     if (args.dryrun) {
       return false;
@@ -1475,7 +1477,8 @@ public class DistCpV1 implements Tool {
 
     LOG.info("sourcePathsCount=" + srcCount);
     LOG.info("filesToCopyCount=" + fileCount);
-    LOG.info("bytesToCopyCount=" + StringUtils.humanReadableInt(byteCount));
+    LOG.info("bytesToCopyCount=" +
+             TraditionalBinaryPrefix.long2String(byteCount, "", 1));
     jobConf.setInt(SRC_COUNT_LABEL, srcCount);
     jobConf.setLong(TOTAL_SIZE_LABEL, byteCount);
     
@@ -1559,10 +1562,10 @@ public class DistCpV1 implements Tool {
 
     //write dst lsr results
     final Path dstlsr = new Path(jobdir, "_distcp_dst_lsr");
-    final SequenceFile.Writer writer = SequenceFile.createWriter(jobfs, jobconf,
-        dstlsr, Text.class, NullWritable.class,
-        SequenceFile.CompressionType.NONE);
-    try {
+    try (final SequenceFile.Writer writer = SequenceFile.createWriter(jobconf,
+        Writer.file(dstlsr), Writer.keyClass(Text.class),
+        Writer.valueClass(NullWritable.class), Writer.compression(
+        SequenceFile.CompressionType.NONE))) {
       //do lsr to get all file statuses in dstroot
       final Stack<FileStatus> lsrstack = new Stack<FileStatus>();
       for(lsrstack.push(dstroot); !lsrstack.isEmpty(); ) {
@@ -1575,8 +1578,6 @@ public class DistCpV1 implements Tool {
           }
         }
       }
-    } finally {
-      checkAndClose(writer);
     }
 
     //sort lsr results
@@ -1586,13 +1587,11 @@ public class DistCpV1 implements Tool {
     sorter.sort(dstlsr, sortedlsr);
 
     //compare lsr list and dst list  
-    SequenceFile.Reader lsrin = null;
-    SequenceFile.Reader dstin = null;
     long deletedPathsCount = 0;
-    try {
-      lsrin = new SequenceFile.Reader(jobfs, sortedlsr, jobconf);
-      dstin = new SequenceFile.Reader(jobfs, dstsorted, jobconf);
-
+    try (SequenceFile.Reader lsrin =
+             new SequenceFile.Reader(jobconf, Reader.file(sortedlsr));
+         SequenceFile.Reader  dstin =
+             new SequenceFile.Reader(jobconf, Reader.file(dstsorted))) {
       //compare sorted lsr list and sorted dst list
       final Text lsrpath = new Text();
       final Text dstpath = new Text();
@@ -1623,9 +1622,6 @@ public class DistCpV1 implements Tool {
           }
         }
       }
-    } finally {
-      checkAndClose(lsrin);
-      checkAndClose(dstin);
     }
     return deletedPathsCount;
   }
@@ -1644,13 +1640,11 @@ public class DistCpV1 implements Tool {
   /** Check whether the file list have duplication. */
   static private void checkDuplication(FileSystem fs, Path file, Path sorted,
     Configuration conf) throws IOException {
-    SequenceFile.Reader in = null;
-    try {
-      SequenceFile.Sorter sorter = new SequenceFile.Sorter(fs,
-        new Text.Comparator(), Text.class, Text.class, conf);
-      sorter.sort(file, sorted);
-      in = new SequenceFile.Reader(fs, sorted, conf);
-
+    SequenceFile.Sorter sorter = new SequenceFile.Sorter(fs,
+      new Text.Comparator(), Text.class, Text.class, conf);
+    sorter.sort(file, sorted);
+    try (SequenceFile.Reader in =
+         new SequenceFile.Reader(conf, Reader.file(sorted))) {
       Text prevdst = null, curdst = new Text();
       Text prevsrc = null, cursrc = new Text(); 
       for(; in.next(curdst, cursrc); ) {
@@ -1665,24 +1659,8 @@ public class DistCpV1 implements Tool {
         cursrc = new Text();
       }
     }
-    finally {
-      checkAndClose(in);
-    }
   } 
 
-  static boolean checkAndClose(java.io.Closeable io) {
-    if (io != null) {
-      try {
-        io.close();
-      }
-      catch(IOException ioe) {
-        LOG.warn(StringUtils.stringifyException(ioe));
-        return false;
-      }
-    }
-    return true;
-  }
-
   /** An exception class for duplicated source files. */
   public static class DuplicationException extends IOException {
     private static final long serialVersionUID = 1L;


[23/50] [abbrv] hadoop git commit: YARN-3078. LogCLIHelpers lacks of a blank space before string 'does not exist'. Contributed by Sam Liu.

Posted by zh...@apache.org.
YARN-3078. LogCLIHelpers lacks of a blank space before string 'does not exist'. Contributed by Sam Liu.


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

Branch: refs/heads/HDFS-EC
Commit: 73309fbc8d8addb61893025677893da8b81f0e87
Parents: df4edd9
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Thu Jan 22 12:25:05 2015 +0900
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:27 2015 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                  | 3 +++
 .../org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java     | 4 ++--
 2 files changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/73309fbc/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index cab71a3..bdc31db 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -397,6 +397,9 @@ Release 2.7.0 - UNRELEASED
     YARN-2731. Fixed RegisterApplicationMasterResponsePBImpl to properly invoke 
     maybeInitBuilder. (Carlo Curino via wangda)
 
+    YARN-3078. LogCLIHelpers lacks of a blank space before string 'does not exist'.
+    (Sam Liu via ozawa)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/73309fbc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java
index 1546ece..df9bd32 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java
@@ -212,11 +212,11 @@ public class LogCLIHelpers implements Configurable {
   }
 
   private static void logDirNotExist(String remoteAppLogDir) {
-    System.out.println(remoteAppLogDir + "does not exist.");
+    System.out.println(remoteAppLogDir + " does not exist.");
     System.out.println("Log aggregation has not completed or is not enabled.");
   }
 
   private static void emptyLogDir(String remoteAppLogDir) {
-    System.out.println(remoteAppLogDir + "does not have any log files.");
+    System.out.println(remoteAppLogDir + " does not have any log files.");
   }
 }


[08/50] [abbrv] hadoop git commit: HDFS-7496: add to CHANGES.txt

Posted by zh...@apache.org.
HDFS-7496: add to CHANGES.txt


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

Branch: refs/heads/HDFS-EC
Commit: 8c130aecc562c9334581f13f06264e83ae352d1f
Parents: 9014305
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Tue Jan 20 19:32:19 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:25 2015 -0800

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c130aec/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 964e278..b032a48 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -744,6 +744,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7641. Update archival storage user doc for list/set/get block storage
     policies. (yliu)
 
+    HDFS-7496. Fix FsVolume removal race conditions on the DataNode by
+    reference-counting the volume instances (lei via cmccabe)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES


[02/50] [abbrv] hadoop git commit: HDFS-7610. Add CHANGES.txt

Posted by zh...@apache.org.
HDFS-7610. Add CHANGES.txt


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

Branch: refs/heads/HDFS-EC
Commit: 499592392d82b9d9bfb03d9050da6ed19061c26d
Parents: ef4453d
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Tue Jan 20 20:14:11 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:25 2015 -0800

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/49959239/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 b032a48..2c7fbc7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -747,6 +747,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7496. Fix FsVolume removal race conditions on the DataNode by
     reference-counting the volume instances (lei via cmccabe)
 
+    HDFS-7610. Fix removal of dynamically added DN volumes (Lei (Eddy) Xu via
+    Colin P. McCabe)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES


[20/50] [abbrv] hadoop git commit: HDFS-7430. Refactor the BlockScanner to use O(1) memory and use multiple threads (cmccabe)

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/df4edd9a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeBlockScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeBlockScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeBlockScanner.java
deleted file mode 100644
index 9e78c10..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeBlockScanner.java
+++ /dev/null
@@ -1,551 +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;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.RandomAccessFile;
-import java.net.InetSocketAddress;
-import java.net.URL;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Random;
-import java.util.Set;
-import java.util.concurrent.TimeoutException;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-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.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
-import org.apache.hadoop.hdfs.server.datanode.DataBlockScanner;
-import org.apache.hadoop.hdfs.server.datanode.DataNode;
-import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
-import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.util.Time;
-import org.apache.log4j.Level;
-import org.junit.Test;
-
-/**
- * This test verifies that block verification occurs on the datanode
- */
-public class TestDatanodeBlockScanner {
-  
-  private static final Log LOG = 
-                 LogFactory.getLog(TestDatanodeBlockScanner.class);
-  
-  private static final long TIMEOUT = 20000; // 20 sec.
-  
-  private static final Pattern pattern =
-             Pattern.compile(".*?(blk_[-]*\\d+).*?scan time\\s*:\\s*(\\d+)");
-  
-  private static final Pattern pattern_blockVerify =
-             Pattern.compile(".*?(SCAN_PERIOD)\\s*:\\s*(\\d+.*?)");
-  
-  static {
-    ((Log4JLogger)FSNamesystem.auditLog).getLogger().setLevel(Level.WARN);
-  }
-  /**
-   * This connects to datanode and fetches block verification data.
-   * It repeats this until the given block has a verification time > newTime.
-   * @param newTime - validation timestamps before newTime are "old", the
-   *            result of previous validations.  This method waits until a "new"
-   *            validation timestamp is obtained.  If no validator runs soon
-   *            enough, the method will time out.
-   * @return - the new validation timestamp
-   * @throws IOException
-   * @throws TimeoutException
-   */
-  private static long waitForVerification(int infoPort, FileSystem fs, 
-                          Path file, int blocksValidated, 
-                          long newTime, long timeout) 
-  throws IOException, TimeoutException {
-    URL url = new URL("http://localhost:" + infoPort +
-                      "/blockScannerReport?listblocks");
-    long lastWarnTime = Time.monotonicNow();
-    if (newTime <= 0) newTime = 1L;
-    long verificationTime = 0;
-    
-    String block = DFSTestUtil.getFirstBlock(fs, file).getBlockName();
-    long failtime = (timeout <= 0) ? Long.MAX_VALUE 
-        : Time.monotonicNow() + timeout;
-    while (verificationTime < newTime) {
-      if (failtime < Time.monotonicNow()) {
-        throw new TimeoutException("failed to achieve block verification after "
-            + timeout + " msec.  Current verification timestamp = "
-            + verificationTime + ", requested verification time > " 
-            + newTime);
-      }
-      String response = DFSTestUtil.urlGet(url);
-      if(blocksValidated >= 0) {
-        for(Matcher matcher = pattern_blockVerify.matcher(response); matcher.find();) {
-          if (block.equals(matcher.group(1))) {
-            assertEquals(1, blocksValidated);
-            break;
-          }
-        }
-      }
-      for(Matcher matcher = pattern.matcher(response); matcher.find();) {
-        if (block.equals(matcher.group(1))) {
-          verificationTime = Long.parseLong(matcher.group(2));
-          break;
-        }
-      }
-      
-      if (verificationTime < newTime) {
-        long now = Time.monotonicNow();
-        if ((now - lastWarnTime) >= 5*1000) {
-          LOG.info("Waiting for verification of " + block);
-          lastWarnTime = now; 
-        }
-        try {
-          Thread.sleep(500);
-        } catch (InterruptedException ignored) {}
-      }
-    }
-    
-    return verificationTime;
-  }
-
-  @Test
-  public void testDatanodeBlockScanner() throws IOException, TimeoutException {
-    long startTime = Time.monotonicNow();
-    
-    Configuration conf = new HdfsConfiguration();
-    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
-    cluster.waitActive();
-    
-    FileSystem fs = cluster.getFileSystem();
-    Path file1 = new Path("/tmp/testBlockVerification/file1");
-    Path file2 = new Path("/tmp/testBlockVerification/file2");
-    
-    /*
-     * Write the first file and restart the cluster.
-     */
-    DFSTestUtil.createFile(fs, file1, 10, (short)1, 0);
-    cluster.shutdown();
-
-    cluster = new MiniDFSCluster.Builder(conf)
-                                .numDataNodes(1)
-                                .format(false).build();
-    cluster.waitActive();
-    
-    DFSClient dfsClient =  new DFSClient(new InetSocketAddress("localhost", 
-                                         cluster.getNameNodePort()), conf);
-    fs = cluster.getFileSystem();
-    DatanodeInfo dn = dfsClient.datanodeReport(DatanodeReportType.LIVE)[0];
-    
-    /*
-     * The cluster restarted. The block should be verified by now.
-     */
-    assertTrue(waitForVerification(dn.getInfoPort(), fs, file1, 1, startTime,
-        TIMEOUT) >= startTime);
-    
-    /*
-     * Create a new file and read the block. The block should be marked 
-     * verified since the client reads the block and verifies checksum. 
-     */
-    DFSTestUtil.createFile(fs, file2, 10, (short)1, 0);
-    IOUtils.copyBytes(fs.open(file2), new IOUtils.NullOutputStream(), 
-                      conf, true); 
-    assertTrue(waitForVerification(dn.getInfoPort(), fs, file2, 2, startTime,
-        TIMEOUT) >= startTime);
-    
-    cluster.shutdown();
-  }
-
-  @Test
-  public void testBlockCorruptionPolicy() throws Exception {
-    Configuration conf = new HdfsConfiguration();
-    conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 1000L);
-    Random random = new Random();
-    FileSystem fs = null;
-    int rand = random.nextInt(3);
-
-    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
-    cluster.waitActive();
-    fs = cluster.getFileSystem();
-    Path file1 = new Path("/tmp/testBlockVerification/file1");
-    DFSTestUtil.createFile(fs, file1, 1024, (short)3, 0);
-    ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, file1);
-    
-    DFSTestUtil.waitReplication(fs, file1, (short)3);
-    assertFalse(DFSTestUtil.allBlockReplicasCorrupt(cluster, file1, 0));
-
-    // Corrupt random replica of block 
-    assertTrue(cluster.corruptReplica(rand, block));
-
-    // Restart the datanode hoping the corrupt block to be reported
-    cluster.restartDataNode(rand);
-
-    // We have 2 good replicas and block is not corrupt
-    DFSTestUtil.waitReplication(fs, file1, (short)2);
-    assertFalse(DFSTestUtil.allBlockReplicasCorrupt(cluster, file1, 0));
-  
-    // Corrupt all replicas. Now, block should be marked as corrupt
-    // and we should get all the replicas 
-    assertTrue(cluster.corruptReplica(0, block));
-    assertTrue(cluster.corruptReplica(1, block));
-    assertTrue(cluster.corruptReplica(2, block));
-
-    // Trigger each of the DNs to scan this block immediately.
-    // The block pool scanner doesn't run frequently enough on its own
-    // to notice these, and due to HDFS-1371, the client won't report
-    // bad blocks to the NN when all replicas are bad.
-    for (DataNode dn : cluster.getDataNodes()) {
-      DataNodeTestUtils.runBlockScannerForBlock(dn, block);
-    }
-
-    // We now have the blocks to be marked as corrupt and we get back all
-    // its replicas
-    DFSTestUtil.waitReplication(fs, file1, (short)3);
-    assertTrue(DFSTestUtil.allBlockReplicasCorrupt(cluster, file1, 0));
-    cluster.shutdown();
-  }
-  
-  /**
-   * testBlockCorruptionRecoveryPolicy.
-   * This tests recovery of corrupt replicas, first for one corrupt replica
-   * then for two. The test invokes blockCorruptionRecoveryPolicy which
-   * 1. Creates a block with desired number of replicas
-   * 2. Corrupts the desired number of replicas and restarts the datanodes
-   *    containing the corrupt replica. Additionaly we also read the block
-   *    in case restarting does not report corrupt replicas.
-   *    Restarting or reading from the datanode would trigger reportBadBlocks 
-   *    to namenode.
-   *    NameNode adds it to corruptReplicasMap and neededReplication
-   * 3. Test waits until all corrupt replicas are reported, meanwhile
-   *    Re-replciation brings the block back to healthy state
-   * 4. Test again waits until the block is reported with expected number
-   *    of good replicas.
-   */
-  @Test
-  public void testBlockCorruptionRecoveryPolicy1() throws Exception {
-    // Test recovery of 1 corrupt replica
-    LOG.info("Testing corrupt replica recovery for one corrupt replica");
-    blockCorruptionRecoveryPolicy(4, (short)3, 1);
-  }
-
-  @Test
-  public void testBlockCorruptionRecoveryPolicy2() throws Exception {
-    // Test recovery of 2 corrupt replicas
-    LOG.info("Testing corrupt replica recovery for two corrupt replicas");
-    blockCorruptionRecoveryPolicy(5, (short)3, 2);
-  }
-  
-  private void blockCorruptionRecoveryPolicy(int numDataNodes, 
-                                             short numReplicas,
-                                             int numCorruptReplicas) 
-                                             throws Exception {
-    Configuration conf = new HdfsConfiguration();
-    conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 30L);
-    conf.setLong(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 3);
-    conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 3L);
-    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, false);
-    conf.setLong(DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY, 5L);
-
-    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDataNodes).build();
-    cluster.waitActive();
-    FileSystem fs = cluster.getFileSystem();
-    Path file1 = new Path("/tmp/testBlockCorruptRecovery/file");
-    DFSTestUtil.createFile(fs, file1, 1024, numReplicas, 0);
-    ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, file1);
-    final int ITERATIONS = 10;
-
-    // Wait until block is replicated to numReplicas
-    DFSTestUtil.waitReplication(fs, file1, numReplicas);
-
-    for (int k = 0; ; k++) {
-      // Corrupt numCorruptReplicas replicas of block 
-      int[] corruptReplicasDNIDs = new int[numCorruptReplicas];
-      for (int i=0, j=0; (j != numCorruptReplicas) && (i < numDataNodes); i++) {
-        if (cluster.corruptReplica(i, block)) {
-          corruptReplicasDNIDs[j++] = i;
-          LOG.info("successfully corrupted block " + block + " on node " 
-                   + i + " " + cluster.getDataNodes().get(i).getDisplayName());
-        }
-      }
-      
-      // Restart the datanodes containing corrupt replicas 
-      // so they would be reported to namenode and re-replicated
-      // They MUST be restarted in reverse order from highest to lowest index,
-      // because the act of restarting them removes them from the ArrayList
-      // and causes the indexes of all nodes above them in the list to change.
-      for (int i = numCorruptReplicas - 1; i >= 0 ; i--) {
-        LOG.info("restarting node with corrupt replica: position " 
-            + i + " node " + corruptReplicasDNIDs[i] + " " 
-            + cluster.getDataNodes().get(corruptReplicasDNIDs[i]).getDisplayName());
-        cluster.restartDataNode(corruptReplicasDNIDs[i]);
-      }
-
-      // Loop until all corrupt replicas are reported
-      try {
-        DFSTestUtil.waitCorruptReplicas(fs, cluster.getNamesystem(), file1, 
-                                        block, numCorruptReplicas);
-      } catch(TimeoutException e) {
-        if (k > ITERATIONS) {
-          throw e;
-        }
-        LOG.info("Timed out waiting for corrupt replicas, trying again, iteration " + k);
-        continue;
-      }
-      break;
-    }
-    
-    // Loop until the block recovers after replication
-    DFSTestUtil.waitReplication(fs, file1, numReplicas);
-    assertFalse(DFSTestUtil.allBlockReplicasCorrupt(cluster, file1, 0));
-
-    // Make sure the corrupt replica is invalidated and removed from
-    // corruptReplicasMap
-    DFSTestUtil.waitCorruptReplicas(fs, cluster.getNamesystem(), file1, 
-        block, 0);
-    cluster.shutdown();
-  }
-  
-  /** Test if NameNode handles truncated blocks in block report */
-  @Test
-  public void testTruncatedBlockReport() throws Exception {
-    final Configuration conf = new HdfsConfiguration();
-    final short REPLICATION_FACTOR = (short)2;
-    final Path fileName = new Path("/file1");
-
-    conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 3L);
-    conf.setLong(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 3);
-    conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 3L);
-    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, false);
-
-    long startTime = Time.monotonicNow();
-    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
-                                               .numDataNodes(REPLICATION_FACTOR)
-                                               .build();
-    cluster.waitActive();
-    
-    ExtendedBlock block;
-    try {
-      FileSystem fs = cluster.getFileSystem();
-      DFSTestUtil.createFile(fs, fileName, 1, REPLICATION_FACTOR, 0);
-      DFSTestUtil.waitReplication(fs, fileName, REPLICATION_FACTOR);
-      block = DFSTestUtil.getFirstBlock(fs, fileName);
-    } finally {
-      cluster.shutdown();
-    }
-
-    // Restart cluster and confirm block is verified on datanode 0,
-    // then truncate it on datanode 0.
-    cluster = new MiniDFSCluster.Builder(conf)
-                                .numDataNodes(REPLICATION_FACTOR)
-                                .format(false)
-                                .build();
-    cluster.waitActive();
-    try {
-      FileSystem fs = cluster.getFileSystem();
-      int infoPort = cluster.getDataNodes().get(0).getInfoPort();
-      assertTrue(waitForVerification(infoPort, fs, fileName, 1, startTime, TIMEOUT) >= startTime);
-      
-      // Truncate replica of block
-      if (!changeReplicaLength(cluster, block, 0, -1)) {
-        throw new IOException(
-            "failed to find or change length of replica on node 0 "
-            + cluster.getDataNodes().get(0).getDisplayName());
-      }      
-    } finally {
-      cluster.shutdown();
-    }
-
-    // Restart the cluster, add a node, and check that the truncated block is 
-    // handled correctly
-    cluster = new MiniDFSCluster.Builder(conf)
-                                .numDataNodes(REPLICATION_FACTOR)
-                                .format(false)
-                                .build();
-    cluster.startDataNodes(conf, 1, true, null, null);
-    cluster.waitActive();  // now we have 3 datanodes
-
-    // Assure the cluster has left safe mode.
-    cluster.waitClusterUp();
-    assertFalse("failed to leave safe mode", 
-        cluster.getNameNode().isInSafeMode());
-
-    try {
-      // wait for truncated block be detected by block scanner,
-      // and the block to be replicated
-      DFSTestUtil.waitReplication(
-          cluster.getFileSystem(), fileName, REPLICATION_FACTOR);
-      
-      // Make sure that truncated block will be deleted
-      waitForBlockDeleted(cluster, block, 0, TIMEOUT);
-    } finally {
-      cluster.shutdown();
-    }
-  }
-  
-  /**
-   * Change the length of a block at datanode dnIndex
-   */
-  static boolean changeReplicaLength(MiniDFSCluster cluster, ExtendedBlock blk,
-      int dnIndex, int lenDelta) throws IOException {
-    File blockFile = cluster.getBlockFile(dnIndex, blk);
-    if (blockFile != null && blockFile.exists()) {
-      RandomAccessFile raFile = new RandomAccessFile(blockFile, "rw");
-      raFile.setLength(raFile.length()+lenDelta);
-      raFile.close();
-      return true;
-    }
-    LOG.info("failed to change length of block " + blk);
-    return false;
-  }
-  
-  private static void waitForBlockDeleted(MiniDFSCluster cluster,
-      ExtendedBlock blk, int dnIndex, long timeout) throws TimeoutException,
-      InterruptedException {
-    File blockFile = cluster.getBlockFile(dnIndex, blk);
-    long failtime = Time.monotonicNow()
-                    + ((timeout > 0) ? timeout : Long.MAX_VALUE);
-    while (blockFile != null && blockFile.exists()) {
-      if (failtime < Time.monotonicNow()) {
-        throw new TimeoutException("waited too long for blocks to be deleted: "
-            + blockFile.getPath() + (blockFile.exists() ? " still exists; " : " is absent; "));
-      }
-      Thread.sleep(100);
-      blockFile = cluster.getBlockFile(dnIndex, blk);
-    }
-  }
-  
-  private static final String BASE_PATH = (new File("/data/current/finalized"))
-      .getAbsolutePath();
-  
-  @Test
-  public void testReplicaInfoParsing() throws Exception {
-    testReplicaInfoParsingSingle(BASE_PATH);
-    testReplicaInfoParsingSingle(BASE_PATH + "/subdir1");
-    testReplicaInfoParsingSingle(BASE_PATH + "/subdir1/subdir2/subdir3");
-  }
-  
-  private static void testReplicaInfoParsingSingle(String subDirPath) {
-    File testFile = new File(subDirPath);
-    assertEquals(BASE_PATH, ReplicaInfo.parseBaseDir(testFile).baseDirPath);
-  }
-
-  @Test
-  public void testDuplicateScans() throws Exception {
-    long startTime = Time.monotonicNow();
-    MiniDFSCluster cluster = new MiniDFSCluster.Builder(new Configuration())
-        .numDataNodes(1).build();
-    FileSystem fs = null;
-    try {
-      fs = cluster.getFileSystem();
-      DataNode dataNode = cluster.getDataNodes().get(0);
-      int infoPort = dataNode.getInfoPort();
-      long scanTimeBefore = 0, scanTimeAfter = 0;
-      for (int i = 1; i < 10; i++) {
-        Path fileName = new Path("/test" + i);
-        DFSTestUtil.createFile(fs, fileName, 1024, (short) 1, 1000L);
-        waitForVerification(infoPort, fs, fileName, i, startTime, TIMEOUT);
-        if (i > 1) {
-          scanTimeAfter = DataNodeTestUtils.getLatestScanTime(dataNode,
-              DFSTestUtil.getFirstBlock(fs, new Path("/test" + (i - 1))));
-          assertFalse("scan time shoud not be 0", scanTimeAfter == 0);
-          assertEquals("There should not be duplicate scan", scanTimeBefore,
-              scanTimeAfter);
-        }
-
-        scanTimeBefore = DataNodeTestUtils.getLatestScanTime(dataNode,
-            DFSTestUtil.getFirstBlock(fs, new Path("/test" + i)));
-      }
-      cluster.restartDataNode(0);
-      Thread.sleep(10000);
-      dataNode = cluster.getDataNodes().get(0);
-      scanTimeAfter = DataNodeTestUtils.getLatestScanTime(dataNode,
-          DFSTestUtil.getFirstBlock(fs, new Path("/test" + (9))));
-      assertEquals("There should not be duplicate scan", scanTimeBefore,
-          scanTimeAfter);
-    } finally {
-      IOUtils.closeStream(fs);
-      cluster.shutdown();
-    }
-  }
-  
-/**
- * This test verifies whether block is added to the first location of 
- * BlockPoolSliceScanner#blockInfoSet
- */
-  @Test
-  public void testAddBlockInfoToFirstLocation() throws Exception {
-    MiniDFSCluster cluster = new MiniDFSCluster.Builder(new Configuration())
-        .numDataNodes(1).build();
-    FileSystem fs = null;
-    try {
-      fs = cluster.getFileSystem();
-      DataNode dataNode = cluster.getDataNodes().get(0);
-      // Creating a bunch of blocks
-      for (int i = 1; i < 10; i++) {
-        Path fileName = new Path("/test" + i);
-        DFSTestUtil.createFile(fs, fileName, 1024, (short) 1, 1000L);
-      } 
-      // Get block of the first file created (file1)
-      ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, new Path("/test1"));
-      dataNode.getBlockScanner().setLastScanTimeDifference(block, 0);
-      // Let it sleep for more than 5 seconds so that BlockPoolSliceScanner can
-      // scan the first set of blocks
-      Thread.sleep(10000);
-      Long scanTime1Fortest1Block = DataNodeTestUtils.getLatestScanTime(
-          dataNode, block);
-      // Create another set of blocks
-      for (int i = 10; i < 20; i++) {
-        Path fileName = new Path("/test" + i);
-        DFSTestUtil.createFile(fs, fileName, 1024, (short) 1, 1000L);
-      }
-      dataNode.getBlockScanner().addBlock(block, true);
-      // Sleep so that BlockPoolSliceScanner can scan the second set of blocks
-      // and one block which we scheduled to rescan
-      Thread.sleep(10000);
-      // Get the lastScanTime of all of the second set of blocks
-      Set<Long> lastScanTimeSet = new HashSet<Long>();
-      for (int i = 10; i < 20; i++) {
-        long lastScanTime = DataNodeTestUtils.getLatestScanTime(dataNode,
-            DFSTestUtil.getFirstBlock(fs, new Path("/test" + i)));
-        lastScanTimeSet.add(lastScanTime);
-      }
-      Long scanTime2Fortest1Block = DataNodeTestUtils.getLatestScanTime(
-          dataNode, DFSTestUtil.getFirstBlock(fs, new Path("/test1")));
-      Long minimumLastScanTime = Collections.min(lastScanTimeSet);
-      assertTrue("The second scanTime for test1 block should be greater than "
-         + "first scanTime", scanTime2Fortest1Block > scanTime1Fortest1Block);
-      assertTrue("The second scanTime for test1 block should be less than or"
-         + " equal to minimum of the lastScanTime of second set of blocks",
-          scanTime2Fortest1Block <= minimumLastScanTime);
-    } finally {
-      IOUtils.closeStream(fs);
-      cluster.shutdown();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df4edd9a/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 b88b5c2..d116f82 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
@@ -444,8 +444,7 @@ public class TestReplication {
 
     // Change the length of a replica
     for (int i=0; i<cluster.getDataNodes().size(); i++) {
-      if (TestDatanodeBlockScanner.changeReplicaLength(cluster, block, i,
-          lenDelta)) {
+      if (DFSTestUtil.changeReplicaLength(cluster, block, i, lenDelta)) {
         break;
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df4edd9a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java
index f8f476d..2942d0f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java
@@ -34,7 +34,6 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
-import org.apache.hadoop.hdfs.TestDatanodeBlockScanner;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -54,6 +53,7 @@ public class TestOverReplicatedBlocks {
   @Test
   public void testProcesOverReplicateBlock() throws Exception {
     Configuration conf = new HdfsConfiguration();
+    conf.setLong(DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, 100L);
     conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 1000L);
     conf.set(
         DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY,
@@ -71,13 +71,14 @@ public class TestOverReplicatedBlocks {
       assertTrue(cluster.corruptReplica(0, block));
       DataNodeProperties dnProps = cluster.stopDataNode(0);
       // remove block scanner log to trigger block scanning
-      File scanLog = new File(MiniDFSCluster.getFinalizedDir(
+      File scanCursor = new File(new File(MiniDFSCluster.getFinalizedDir(
           cluster.getInstanceStorageDir(0, 0),
-          cluster.getNamesystem().getBlockPoolId()).getParent().toString()
-          + "/../dncp_block_verification.log.prev");
+          cluster.getNamesystem().getBlockPoolId()).getParent()).getParent(),
+          "scanner.cursor");
       //wait for one minute for deletion to succeed;
-      for(int i=0; !scanLog.delete(); i++) {
-        assertTrue("Could not delete log file in one minute", i < 60);
+      for(int i = 0; !scanCursor.delete(); i++) {
+        assertTrue("Could not delete " + scanCursor.getAbsolutePath() +
+            " in one minute", i < 60);
         try {
           Thread.sleep(1000);
         } catch (InterruptedException ignored) {}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df4edd9a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/BlockReportTestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/BlockReportTestBase.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/BlockReportTestBase.java
index e9557da..68c66a3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/BlockReportTestBase.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/BlockReportTestBase.java
@@ -30,6 +30,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Random;
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
 import org.apache.commons.logging.Log;
@@ -82,8 +83,8 @@ public abstract class BlockReportTestBase {
 
   private static short REPL_FACTOR = 1;
   private static final int RAND_LIMIT = 2000;
-  private static final long DN_RESCAN_INTERVAL = 5000;
-  private static final long DN_RESCAN_EXTRA_WAIT = 2 * DN_RESCAN_INTERVAL;
+  private static final long DN_RESCAN_INTERVAL = 1;
+  private static final long DN_RESCAN_EXTRA_WAIT = 3 * DN_RESCAN_INTERVAL;
   private static final int DN_N0 = 0;
   private static final int FILE_START = 0;
 
@@ -294,7 +295,7 @@ public abstract class BlockReportTestBase {
       }
     }
 
-    waitTil(DN_RESCAN_EXTRA_WAIT);
+    waitTil(TimeUnit.SECONDS.toMillis(DN_RESCAN_EXTRA_WAIT));
 
     // all blocks belong to the same file, hence same BP
     String poolId = cluster.getNamesystem().getBlockPoolId();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df4edd9a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java
index f50afd4..fd51e52 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java
@@ -113,30 +113,6 @@ public class DataNodeTestUtils {
     return DataNode.createInterDataNodeProtocolProxy(datanodeid, conf,
         dn.getDnConf().socketTimeout, dn.getDnConf().connectToDnViaHostname);
   }
-  
-  public static void runBlockScannerForBlock(DataNode dn, ExtendedBlock b) {
-    BlockPoolSliceScanner bpScanner = getBlockPoolScanner(dn, b);
-    bpScanner.verifyBlock(new ExtendedBlock(b.getBlockPoolId(),
-        new BlockPoolSliceScanner.BlockScanInfo(b.getLocalBlock())));
-  }
-
-  private static BlockPoolSliceScanner getBlockPoolScanner(DataNode dn,
-      ExtendedBlock b) {
-    DataBlockScanner scanner = dn.getBlockScanner();
-    BlockPoolSliceScanner bpScanner = scanner.getBPScanner(b.getBlockPoolId());
-    return bpScanner;
-  }
-
-  public static long getLatestScanTime(DataNode dn, ExtendedBlock b) {
-    BlockPoolSliceScanner scanner = getBlockPoolScanner(dn, b);
-    return scanner.getLastScanTime(b.getLocalBlock());
-  }
-
-  public static void shutdownBlockScanner(DataNode dn) {
-    if (dn.blockScanner != null) {
-      dn.blockScanner.shutdown();
-    }
-  }
 
   /**
    * This method is used for testing. 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df4edd9a/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 16b6350..0610b94 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
@@ -50,7 +50,6 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaInputStreams;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.RollingLogs;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl;
 import org.apache.hadoop.hdfs.server.datanode.metrics.FSDatasetMBean;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
@@ -484,6 +483,22 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
     @Override
     public void releaseReservedSpace(long bytesToRelease) {
     }
+
+    @Override
+    public BlockIterator newBlockIterator(String bpid, String name) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public BlockIterator loadBlockIterator(String bpid, String name)
+        throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public FsDatasetSpi getDataset() {
+      throw new UnsupportedOperationException();
+    }
   }
 
   private final Map<String, Map<Block, BInfo>> blockMap
@@ -1238,11 +1253,6 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
   }
 
   @Override
-  public RollingLogs createRollingLogs(String bpid, String prefix) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
   public FsVolumeSpi getVolume(ExtendedBlock b) {
     return volume;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df4edd9a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockScanner.java
new file mode 100644
index 0000000..7eaa2bf
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockScanner.java
@@ -0,0 +1,680 @@
+/**
+ * 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.datanode;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SCANNER_VOLUME_BYTES_PER_SECOND;
+import static org.apache.hadoop.hdfs.server.datanode.BlockScanner.Conf.INTERNAL_DFS_DATANODE_SCAN_PERIOD_MS;
+import static org.apache.hadoop.hdfs.server.datanode.BlockScanner.Conf.INTERNAL_VOLUME_SCANNER_SCAN_RESULT_HANDLER;
+import static org.apache.hadoop.hdfs.server.datanode.BlockScanner.Conf.INTERNAL_DFS_BLOCK_SCANNER_CURSOR_SAVE_INTERVAL_MS;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Semaphore;
+
+import com.google.common.base.Supplier;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
+import org.apache.hadoop.hdfs.server.datanode.VolumeScanner.ScanResultHandler;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi.BlockIterator;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl;
+import org.apache.hadoop.hdfs.server.datanode.VolumeScanner.Statistics;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.Time;
+import org.apache.log4j.Level;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TestBlockScanner {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(TestBlockScanner.class);
+
+  @Before
+  public void before() {
+    BlockScanner.Conf.allowUnitTestSettings = true;
+    GenericTestUtils.setLogLevel(BlockScanner.LOG, Level.ALL);
+    GenericTestUtils.setLogLevel(VolumeScanner.LOG, Level.ALL);
+    GenericTestUtils.setLogLevel(FsVolumeImpl.LOG, Level.ALL);
+  }
+
+  private static void disableBlockScanner(Configuration conf) {
+    conf.setLong(DFS_BLOCK_SCANNER_VOLUME_BYTES_PER_SECOND, 0L);
+  }
+
+  private static class TestContext implements Closeable {
+    final int numNameServices;
+    final MiniDFSCluster cluster;
+    final DistributedFileSystem[] dfs;
+    final String[] bpids;
+    final DataNode datanode;
+    final BlockScanner blockScanner;
+    final FsDatasetSpi<? extends FsVolumeSpi> data;
+    final List<? extends FsVolumeSpi> volumes;
+
+    TestContext(Configuration conf, int numNameServices) throws Exception {
+      this.numNameServices = numNameServices;
+      MiniDFSCluster.Builder bld = new MiniDFSCluster.Builder(conf).
+          numDataNodes(1).
+          storagesPerDatanode(1);
+      if (numNameServices > 1) {
+        bld.nnTopology(MiniDFSNNTopology.
+              simpleFederatedTopology(numNameServices));
+      }
+      cluster = bld.build();
+      cluster.waitActive();
+      dfs = new DistributedFileSystem[numNameServices];
+      for (int i = 0; i < numNameServices; i++) {
+        dfs[i] = cluster.getFileSystem(i);
+      }
+      bpids = new String[numNameServices];
+      for (int i = 0; i < numNameServices; i++) {
+        bpids[i] = cluster.getNamesystem(i).getBlockPoolId();
+      }
+      datanode = cluster.getDataNodes().get(0);
+      blockScanner = datanode.getBlockScanner();
+      for (int i = 0; i < numNameServices; i++) {
+        dfs[i].mkdirs(new Path("/test"));
+      }
+      data = datanode.getFSDataset();
+      volumes = data.getVolumes();
+    }
+
+    @Override
+    public void close() throws IOException {
+      if (cluster != null) {
+        for (int i = 0; i < numNameServices; i++) {
+          dfs[i].delete(new Path("/test"), true);
+        }
+        cluster.shutdown();
+      }
+    }
+
+    public void createFiles(int nsIdx, int numFiles, int length)
+          throws Exception {
+      for (int blockIdx = 0; blockIdx < numFiles; blockIdx++) {
+        DFSTestUtil.createFile(dfs[nsIdx], getPath(blockIdx), length,
+            (short)1, 123L);
+      }
+    }
+
+    public Path getPath(int fileIdx) {
+      return new Path("/test/" + fileIdx);
+    }
+
+    public ExtendedBlock getFileBlock(int nsIdx, int fileIdx)
+          throws Exception {
+      return DFSTestUtil.getFirstBlock(dfs[nsIdx], getPath(fileIdx));
+    }
+  }
+
+  /**
+   * Test iterating through a bunch of blocks in a volume using a volume
+   * iterator.<p/>
+   *
+   * We will rewind the iterator when about halfway through the blocks.
+   *
+   * @param numFiles        The number of files to create.
+   * @param maxStaleness    The maximum staleness to allow with the iterator.
+   * @throws Exception
+   */
+  private void testVolumeIteratorImpl(int numFiles,
+              long maxStaleness) throws Exception {
+    Configuration conf = new Configuration();
+    disableBlockScanner(conf);
+    TestContext ctx = new TestContext(conf, 1);
+    ctx.createFiles(0, numFiles, 1);
+    assertEquals(1, ctx.volumes.size());
+    FsVolumeSpi volume = ctx.volumes.get(0);
+    ExtendedBlock savedBlock = null, loadedBlock = null;
+    boolean testedRewind = false, testedSave = false, testedLoad = false;
+    int blocksProcessed = 0, savedBlocksProcessed = 0;
+    try {
+      BPOfferService bpos[] = ctx.datanode.getAllBpOs();
+      assertEquals(1, bpos.length);
+      BlockIterator iter = volume.newBlockIterator(ctx.bpids[0], "test");
+      assertEquals(ctx.bpids[0], iter.getBlockPoolId());
+      iter.setMaxStalenessMs(maxStaleness);
+      while (true) {
+        HashSet<ExtendedBlock> blocks = new HashSet<ExtendedBlock>();
+        for (int blockIdx = 0; blockIdx < numFiles; blockIdx++) {
+          blocks.add(ctx.getFileBlock(0, blockIdx));
+        }
+        while (true) {
+          ExtendedBlock block = iter.nextBlock();
+          if (block == null) {
+            break;
+          }
+          blocksProcessed++;
+          LOG.info("BlockIterator for {} found block {}, blocksProcessed = {}",
+              volume, block, blocksProcessed);
+          if (testedSave && (savedBlock == null)) {
+            savedBlock = block;
+          }
+          if (testedLoad && (loadedBlock == null)) {
+            loadedBlock = block;
+            // The block that we get back right after loading the iterator
+            // should be the same block we got back right after saving
+            // the iterator.
+            assertEquals(savedBlock, loadedBlock);
+          }
+          boolean blockRemoved = blocks.remove(block);
+          assertTrue("Found unknown block " + block, blockRemoved);
+          if (blocksProcessed > (numFiles / 3)) {
+            if (!testedSave) {
+              LOG.info("Processed {} blocks out of {}.  Saving iterator.",
+                  blocksProcessed, numFiles);
+              iter.save();
+              testedSave = true;
+              savedBlocksProcessed = blocksProcessed;
+            }
+          }
+          if (blocksProcessed > (numFiles / 2)) {
+            if (!testedRewind) {
+              LOG.info("Processed {} blocks out of {}.  Rewinding iterator.",
+                  blocksProcessed, numFiles);
+              iter.rewind();
+              break;
+            }
+          }
+          if (blocksProcessed > ((2 * numFiles) / 3)) {
+            if (!testedLoad) {
+              LOG.info("Processed {} blocks out of {}.  Loading iterator.",
+                  blocksProcessed, numFiles);
+              iter = volume.loadBlockIterator(ctx.bpids[0], "test");
+              iter.setMaxStalenessMs(maxStaleness);
+              break;
+            }
+          }
+        }
+        if (!testedRewind) {
+          testedRewind = true;
+          blocksProcessed = 0;
+          LOG.info("Starting again at the beginning...");
+          continue;
+        }
+        if (!testedLoad) {
+          testedLoad = true;
+          blocksProcessed = savedBlocksProcessed;
+          LOG.info("Starting again at the load point...");
+          continue;
+        }
+        assertEquals(numFiles, blocksProcessed);
+        break;
+      }
+    } finally {
+      ctx.close();
+    }
+  }
+
+  @Test(timeout=60000)
+  public void testVolumeIteratorWithoutCaching() throws Exception {
+    testVolumeIteratorImpl(5, 0);
+  }
+
+  @Test(timeout=60000)
+  public void testVolumeIteratorWithCaching() throws Exception {
+    testVolumeIteratorImpl(600, 100);
+  }
+
+  @Test(timeout=60000)
+  public void testDisableVolumeScanner() throws Exception {
+    Configuration conf = new Configuration();
+    disableBlockScanner(conf);
+    TestContext ctx = new TestContext(conf, 1);
+    try {
+      Assert.assertFalse(ctx.datanode.getBlockScanner().isEnabled());
+    } finally {
+      ctx.close();
+    }
+  }
+
+  public static class TestScanResultHandler extends ScanResultHandler {
+    static class Info {
+      boolean shouldRun = false;
+      final Set<ExtendedBlock> badBlocks = new HashSet<ExtendedBlock>();
+      final Set<ExtendedBlock> goodBlocks = new HashSet<ExtendedBlock>();
+      long blocksScanned = 0;
+      Semaphore sem = null;
+    }
+
+    private VolumeScanner scanner;
+
+    final static ConcurrentHashMap<String, Info> infos =
+        new ConcurrentHashMap<String, Info>();
+
+    static Info getInfo(FsVolumeSpi volume) {
+      Info newInfo = new Info();
+      Info prevInfo = infos.
+          putIfAbsent(volume.getStorageID(), newInfo);
+      return prevInfo == null ? newInfo : prevInfo;
+    }
+
+    @Override
+    public void setup(VolumeScanner scanner) {
+      this.scanner = scanner;
+      Info info = getInfo(scanner.volume);
+      LOG.info("about to start scanning.");
+      synchronized (info) {
+        while (!info.shouldRun) {
+          try {
+            info.wait();
+          } catch (InterruptedException e) {
+          }
+        }
+      }
+      LOG.info("starting scanning.");
+    }
+
+    @Override
+    public void handle(ExtendedBlock block, IOException e) {
+      LOG.info("handling block {} (exception {})", block, e);
+      Info info = getInfo(scanner.volume);
+      Semaphore sem;
+      synchronized (info) {
+        sem = info.sem;
+      }
+      if (sem != null) {
+        try {
+          sem.acquire();
+        } catch (InterruptedException ie) {
+          throw new RuntimeException("interrupted");
+        }
+      }
+      synchronized (info) {
+        if (!info.shouldRun) {
+          throw new RuntimeException("stopping volumescanner thread.");
+        }
+        if (e == null) {
+          info.goodBlocks.add(block);
+        } else {
+          info.badBlocks.add(block);
+        }
+        info.blocksScanned++;
+      }
+    }
+  }
+
+  private void testScanAllBlocksImpl(final boolean rescan) throws Exception {
+    Configuration conf = new Configuration();
+    conf.setLong(DFS_BLOCK_SCANNER_VOLUME_BYTES_PER_SECOND, 1048576L);
+    if (rescan) {
+      conf.setLong(INTERNAL_DFS_DATANODE_SCAN_PERIOD_MS, 100L);
+    } else {
+      conf.setLong(DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, 100L);
+    }
+    conf.set(INTERNAL_VOLUME_SCANNER_SCAN_RESULT_HANDLER,
+        TestScanResultHandler.class.getName());
+    final TestContext ctx = new TestContext(conf, 1);
+    final int NUM_EXPECTED_BLOCKS = 10;
+    ctx.createFiles(0, NUM_EXPECTED_BLOCKS, 1);
+    final Set<ExtendedBlock> expectedBlocks = new HashSet<ExtendedBlock>();
+    for (int i = 0; i < NUM_EXPECTED_BLOCKS; i++) {
+      expectedBlocks.add(ctx.getFileBlock(0, i));
+    }
+    TestScanResultHandler.Info info =
+        TestScanResultHandler.getInfo(ctx.volumes.get(0));
+    synchronized (info) {
+      info.shouldRun = true;
+      info.notify();
+    }
+    GenericTestUtils.waitFor(new Supplier<Boolean>(){
+      @Override
+      public Boolean get() {
+        TestScanResultHandler.Info info =
+            TestScanResultHandler.getInfo(ctx.volumes.get(0));
+        int numFoundBlocks = 0;
+        StringBuilder foundBlocksBld = new StringBuilder();
+        String prefix = "";
+        synchronized (info) {
+          for (ExtendedBlock block : info.goodBlocks) {
+            assertTrue(expectedBlocks.contains(block));
+            numFoundBlocks++;
+            foundBlocksBld.append(prefix).append(block);
+            prefix = ", ";
+          }
+          LOG.info("numFoundBlocks = {}.  blocksScanned = {}. Found blocks {}",
+              numFoundBlocks, info.blocksScanned, foundBlocksBld.toString());
+          if (rescan) {
+            return (numFoundBlocks == NUM_EXPECTED_BLOCKS) &&
+                     (info.blocksScanned >= 2 * NUM_EXPECTED_BLOCKS);
+          } else {
+            return numFoundBlocks == NUM_EXPECTED_BLOCKS;
+          }
+        }
+      }
+    }, 10, 60000);
+    if (!rescan) {
+      synchronized (info) {
+        assertEquals(NUM_EXPECTED_BLOCKS, info.blocksScanned);
+      }
+      Statistics stats = ctx.blockScanner.getVolumeStats(
+          ctx.volumes.get(0).getStorageID());
+      assertEquals(5 * NUM_EXPECTED_BLOCKS, stats.bytesScannedInPastHour);
+      assertEquals(NUM_EXPECTED_BLOCKS, stats.blocksScannedSinceRestart);
+      assertEquals(NUM_EXPECTED_BLOCKS, stats.blocksScannedInCurrentPeriod);
+      assertEquals(0, stats.scanErrorsSinceRestart);
+      assertEquals(1, stats.scansSinceRestart);
+    }
+    ctx.close();
+  }
+
+  /**
+   * Test scanning all blocks.  Set the scan period high enough that
+   * we shouldn't rescan any block during this test.
+   */
+  @Test(timeout=60000)
+  public void testScanAllBlocksNoRescan() throws Exception {
+    testScanAllBlocksImpl(false);
+  }
+
+  /**
+   * Test scanning all blocks.  Set the scan period high enough that
+   * we should rescan all blocks at least twice during this test.
+   */
+  @Test(timeout=60000)
+  public void testScanAllBlocksWithRescan() throws Exception {
+    testScanAllBlocksImpl(true);
+  }
+
+  /**
+   * Test that we don't scan too many blocks per second.
+   */
+  @Test(timeout=120000)
+  public void testScanRateLimit() throws Exception {
+    Configuration conf = new Configuration();
+    // Limit scan bytes per second dramatically
+    conf.setLong(DFS_BLOCK_SCANNER_VOLUME_BYTES_PER_SECOND, 4096L);
+    // Scan continuously
+    conf.setLong(INTERNAL_DFS_DATANODE_SCAN_PERIOD_MS, 1L);
+    conf.set(INTERNAL_VOLUME_SCANNER_SCAN_RESULT_HANDLER,
+        TestScanResultHandler.class.getName());
+    final TestContext ctx = new TestContext(conf, 1);
+    final int NUM_EXPECTED_BLOCKS = 5;
+    ctx.createFiles(0, NUM_EXPECTED_BLOCKS, 4096);
+    final TestScanResultHandler.Info info =
+        TestScanResultHandler.getInfo(ctx.volumes.get(0));
+    long startMs = Time.monotonicNow();
+    synchronized (info) {
+      info.shouldRun = true;
+      info.notify();
+    }
+    Thread.sleep(5000);
+    synchronized (info) {
+      long endMs = Time.monotonicNow();
+      // Should scan no more than one block a second.
+      long maxBlocksScanned = ((endMs + 999 - startMs) / 1000);
+      assertTrue(info.blocksScanned < maxBlocksScanned);
+    }
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        synchronized (info) {
+          return info.blocksScanned > 0;
+        }
+      }
+    }, 1, 30000);
+    ctx.close();
+  }
+
+  @Test(timeout=120000)
+  public void testCorruptBlockHandling() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setLong(DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, 100L);
+    conf.set(INTERNAL_VOLUME_SCANNER_SCAN_RESULT_HANDLER,
+        TestScanResultHandler.class.getName());
+    final TestContext ctx = new TestContext(conf, 1);
+    final int NUM_EXPECTED_BLOCKS = 5;
+    final int CORRUPT_INDEX = 3;
+    ctx.createFiles(0, NUM_EXPECTED_BLOCKS, 4);
+    ExtendedBlock badBlock = ctx.getFileBlock(0, CORRUPT_INDEX);
+    ctx.cluster.corruptBlockOnDataNodes(badBlock);
+    final TestScanResultHandler.Info info =
+        TestScanResultHandler.getInfo(ctx.volumes.get(0));
+    synchronized (info) {
+      info.shouldRun = true;
+      info.notify();
+    }
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        synchronized (info) {
+          return info.blocksScanned == NUM_EXPECTED_BLOCKS;
+        }
+      }
+    }, 3, 30000);
+    synchronized (info) {
+      assertTrue(info.badBlocks.contains(badBlock));
+      for (int i = 0; i < NUM_EXPECTED_BLOCKS; i++) {
+        if (i != CORRUPT_INDEX) {
+          ExtendedBlock block = ctx.getFileBlock(0, i);
+          assertTrue(info.goodBlocks.contains(block));
+        }
+      }
+    }
+    ctx.close();
+  }
+
+  /**
+   * Test that we save the scan cursor when shutting down the datanode, and
+   * restart scanning from there when the datanode is restarted.
+   */
+  @Test(timeout=120000)
+  public void testDatanodeCursor() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setLong(DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, 100L);
+    conf.set(INTERNAL_VOLUME_SCANNER_SCAN_RESULT_HANDLER,
+        TestScanResultHandler.class.getName());
+    conf.setLong(INTERNAL_DFS_BLOCK_SCANNER_CURSOR_SAVE_INTERVAL_MS, 0L);
+    final TestContext ctx = new TestContext(conf, 1);
+    final int NUM_EXPECTED_BLOCKS = 10;
+    ctx.createFiles(0, NUM_EXPECTED_BLOCKS, 1);
+    final TestScanResultHandler.Info info =
+        TestScanResultHandler.getInfo(ctx.volumes.get(0));
+    synchronized (info) {
+      info.sem = new Semaphore(5);
+      info.shouldRun = true;
+      info.notify();
+    }
+    // Scan the first 5 blocks
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        synchronized (info) {
+          return info.blocksScanned == 5;
+        }
+      }
+    }, 3, 30000);
+    synchronized (info) {
+      assertEquals(5, info.goodBlocks.size());
+      assertEquals(5, info.blocksScanned);
+      info.shouldRun = false;
+    }
+    ctx.datanode.shutdown();
+    String vPath = ctx.volumes.get(0).getBasePath();
+    File cursorPath = new File(new File(new File(vPath, "current"),
+          ctx.bpids[0]), "scanner.cursor");
+    assertTrue("Failed to find cursor save file in " +
+        cursorPath.getAbsolutePath(), cursorPath.exists());
+    Set<ExtendedBlock> prevGoodBlocks = new HashSet<ExtendedBlock>();
+    synchronized (info) {
+      info.sem = new Semaphore(4);
+      prevGoodBlocks.addAll(info.goodBlocks);
+      info.goodBlocks.clear();
+    }
+
+    // The block that we were scanning when we shut down the DN won't get
+    // recorded.
+    // After restarting the datanode, we should scan the next 4 blocks.
+    ctx.cluster.restartDataNode(0);
+    synchronized (info) {
+      info.shouldRun = true;
+      info.notify();
+    }
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        synchronized (info) {
+          if (info.blocksScanned != 9) {
+            LOG.info("Waiting for blocksScanned to reach 9.  It is at {}",
+                info.blocksScanned);
+          }
+          return info.blocksScanned == 9;
+        }
+      }
+    }, 3, 30000);
+    synchronized (info) {
+      assertEquals(4, info.goodBlocks.size());
+      info.goodBlocks.addAll(prevGoodBlocks);
+      assertEquals(9, info.goodBlocks.size());
+      assertEquals(9, info.blocksScanned);
+    }
+    ctx.datanode.shutdown();
+
+    // After restarting the datanode, we should not scan any more blocks.
+    // This is because we reached the end of the block pool earlier, and
+    // the scan period is much, much longer than the test time.
+    synchronized (info) {
+      info.sem = null;
+      info.shouldRun = false;
+      info.goodBlocks.clear();
+    }
+    ctx.cluster.restartDataNode(0);
+    synchronized (info) {
+      info.shouldRun = true;
+      info.notify();
+    }
+    Thread.sleep(3000);
+    synchronized (info) {
+      assertTrue(info.goodBlocks.isEmpty());
+    }
+    ctx.close();
+  }
+
+  @Test(timeout=120000)
+  public void testMultipleBlockPoolScanning() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setLong(DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, 100L);
+    conf.set(INTERNAL_VOLUME_SCANNER_SCAN_RESULT_HANDLER,
+        TestScanResultHandler.class.getName());
+    final TestContext ctx = new TestContext(conf, 3);
+
+    // We scan 5 bytes per file (1 byte in file, 4 bytes of checksum)
+    final int BYTES_SCANNED_PER_FILE = 5;
+    final int NUM_FILES[] = new int[] { 1, 5, 10 };
+    int TOTAL_FILES = 0;
+    for (int i = 0; i < NUM_FILES.length; i++) {
+      TOTAL_FILES += NUM_FILES[i];
+    }
+    ctx.createFiles(0, NUM_FILES[0], 1);
+    ctx.createFiles(0, NUM_FILES[1], 1);
+    ctx.createFiles(0, NUM_FILES[2], 1);
+
+    // start scanning
+    final TestScanResultHandler.Info info =
+        TestScanResultHandler.getInfo(ctx.volumes.get(0));
+    synchronized (info) {
+      info.shouldRun = true;
+      info.notify();
+    }
+
+    // Wait for all the block pools to be scanned.
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        synchronized (info) {
+          Statistics stats = ctx.blockScanner.getVolumeStats(
+              ctx.volumes.get(0).getStorageID());
+          if (stats.scansSinceRestart < 3) {
+            LOG.info("Waiting for scansSinceRestart to reach 3 (it is {})",
+                stats.scansSinceRestart);
+            return false;
+          }
+          if (!stats.eof) {
+            LOG.info("Waiting for eof.");
+            return false;
+          }
+          return true;
+        }
+      }
+    }, 3, 30000);
+
+    Statistics stats = ctx.blockScanner.getVolumeStats(
+        ctx.volumes.get(0).getStorageID());
+    assertEquals(TOTAL_FILES, stats.blocksScannedSinceRestart);
+    assertEquals(BYTES_SCANNED_PER_FILE * TOTAL_FILES,
+        stats.bytesScannedInPastHour);
+    ctx.close();
+  }
+
+  @Test(timeout=120000)
+  public void testNextSorted() throws Exception {
+    List<String> arr = new LinkedList<String>();
+    arr.add("1");
+    arr.add("3");
+    arr.add("5");
+    arr.add("7");
+    Assert.assertEquals("3", FsVolumeImpl.nextSorted(arr, "2"));
+    Assert.assertEquals("3", FsVolumeImpl.nextSorted(arr, "1"));
+    Assert.assertEquals("1", FsVolumeImpl.nextSorted(arr, ""));
+    Assert.assertEquals("1", FsVolumeImpl.nextSorted(arr, null));
+    Assert.assertEquals(null, FsVolumeImpl.nextSorted(arr, "9"));
+  }
+
+  @Test(timeout=120000)
+  public void testCalculateNeededBytesPerSec() throws Exception {
+    // If we didn't check anything the last hour, we should scan now.
+    Assert.assertTrue(
+        VolumeScanner.calculateShouldScan(100, 0));
+
+    // If, on average, we checked 101 bytes/s checked during the last hour,
+    // stop checking now.
+    Assert.assertFalse(
+        VolumeScanner.calculateShouldScan(100, 101 * 3600));
+
+    // Target is 1 byte / s, but we didn't scan anything in the last minute.
+    // Should scan now.
+    Assert.assertTrue(
+        VolumeScanner.calculateShouldScan(1, 3540));
+
+    // Target is 1000000 byte / s, but we didn't scan anything in the last
+    // minute.  Should scan now.
+    Assert.assertTrue(
+        VolumeScanner.calculateShouldScan(100000L, 354000000L));
+
+    Assert.assertFalse(
+        VolumeScanner.calculateShouldScan(100000L, 365000000L));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df4edd9a/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 1b8f243..82a1684 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
@@ -589,6 +589,22 @@ public class TestDirectoryScanner {
     @Override
     public void releaseReservedSpace(long bytesToRelease) {
     }
+
+    @Override
+    public BlockIterator newBlockIterator(String bpid, String name) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public BlockIterator loadBlockIterator(String bpid, String name)
+          throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public FsDatasetSpi getDataset() {
+      throw new UnsupportedOperationException();
+    }
   }
 
   private final static TestFsVolumeSpi TEST_VOLUME = new TestFsVolumeSpi();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df4edd9a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestMultipleNNDataBlockScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestMultipleNNDataBlockScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestMultipleNNDataBlockScanner.java
deleted file mode 100644
index 55b1739..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestMultipleNNDataBlockScanner.java
+++ /dev/null
@@ -1,245 +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.datanode;
-
-import java.io.IOException;
-
-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.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DFSTestUtil;
-import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.MiniDFSNNTopology;
-import static org.apache.hadoop.hdfs.server.datanode.DataBlockScanner.SLEEP_PERIOD_MS;
-import org.apache.log4j.Level;
-import org.junit.Assert;
-import org.junit.Test;
-import static org.junit.Assert.fail;
-
-
-public class TestMultipleNNDataBlockScanner {
-  private static final Log LOG = 
-    LogFactory.getLog(TestMultipleNNDataBlockScanner.class);
-  Configuration conf;
-  MiniDFSCluster cluster = null;
-  final String[] bpids = new String[3];
-  final FileSystem[] fs = new FileSystem[3];
-  
-  public void setUp() throws IOException {
-    conf = new HdfsConfiguration();
-    conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 100);
-    conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, 100);
-    cluster = new MiniDFSCluster.Builder(conf)
-        .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(3))
-        .build();
-    for (int i = 0; i < 3; i++) {
-      cluster.waitActive(i);
-    }
-    for (int i = 0; i < 3; i++) {
-      bpids[i] = cluster.getNamesystem(i).getBlockPoolId();
-    }
-    for (int i = 0; i < 3; i++) {
-      fs[i] = cluster.getFileSystem(i);
-    }
-    // Create 2 files on each namenode with 10 blocks each
-    for (int i = 0; i < 3; i++) {
-      DFSTestUtil.createFile(fs[i], new Path("file1"), 1000, (short) 1, 0);
-      DFSTestUtil.createFile(fs[i], new Path("file2"), 1000, (short) 1, 1);
-    }
-  }
-  
-  @Test(timeout=120000)
-  public void testDataBlockScanner() throws IOException, InterruptedException {
-    setUp();
-    try {
-      DataNode dn = cluster.getDataNodes().get(0);
-      for (int i = 0; i < 3; i++) {
-        long blocksScanned = 0;
-        while (blocksScanned != 20) {
-          blocksScanned = dn.blockScanner.getBlocksScannedInLastRun(bpids[i]);
-          LOG.info("Waiting for all blocks to be scanned for bpid=" + bpids[i]
-              + "; Scanned so far=" + blocksScanned);
-          Thread.sleep(5000);
-        }
-      }
-
-      StringBuilder buffer = new StringBuilder();
-      dn.blockScanner.printBlockReport(buffer, false);
-      LOG.info("Block Report\n" + buffer.toString());
-    } finally {
-      cluster.shutdown();
-    }
-  }
-  
-  @Test(timeout=120000)
-  public void testBlockScannerAfterRefresh() throws IOException,
-      InterruptedException {
-    setUp();
-    try {
-      Configuration dnConf = cluster.getDataNodes().get(0).getConf();
-      Configuration conf = new HdfsConfiguration(dnConf);
-      StringBuilder namenodesBuilder = new StringBuilder();
-
-      String bpidToShutdown = cluster.getNamesystem(2).getBlockPoolId();
-      for (int i = 0; i < 2; i++) {
-        String nsId = DFSUtil.getNamenodeNameServiceId(cluster
-            .getConfiguration(i));
-        namenodesBuilder.append(nsId);
-        namenodesBuilder.append(",");
-      }
-
-      conf.set(DFSConfigKeys.DFS_NAMESERVICES, namenodesBuilder
-          .toString());
-      DataNode dn = cluster.getDataNodes().get(0);
-      dn.refreshNamenodes(conf);
-
-      try {
-        while (true) {
-          dn.blockScanner.getBlocksScannedInLastRun(bpidToShutdown);
-          Thread.sleep(1000);
-        }
-      } catch (IOException ex) {
-        // Expected
-        LOG.info(ex.getMessage());
-      }
-
-      namenodesBuilder.append(DFSUtil.getNamenodeNameServiceId(cluster
-          .getConfiguration(2)));
-      conf.set(DFSConfigKeys.DFS_NAMESERVICES, namenodesBuilder
-          .toString());
-      dn.refreshNamenodes(conf);
-
-      for (int i = 0; i < 3; i++) {
-        long blocksScanned = 0;
-        while (blocksScanned != 20) {
-          blocksScanned = dn.blockScanner.getBlocksScannedInLastRun(bpids[i]);
-          LOG.info("Waiting for all blocks to be scanned for bpid=" + bpids[i]
-              + "; Scanned so far=" + blocksScanned);
-          Thread.sleep(5000);
-        }
-      }
-    } finally {
-      cluster.shutdown();
-    }
-  }
-  
-  @Test(timeout=120000)
-  public void testBlockScannerAfterRestart() throws IOException,
-      InterruptedException {
-    setUp();
-    try {
-      cluster.restartDataNode(0);
-      cluster.waitActive();
-      DataNode dn = cluster.getDataNodes().get(0);
-      for (int i = 0; i < 3; i++) {
-        while (!dn.blockScanner.isInitialized(bpids[i])) {
-          Thread.sleep(1000);
-        }
-        long blocksScanned = 0;
-        while (blocksScanned != 20) {
-          if (dn.blockScanner != null) {
-            blocksScanned = dn.blockScanner.getBlocksScannedInLastRun(bpids[i]);
-            LOG.info("Waiting for all blocks to be scanned for bpid="
-                + bpids[i] + "; Scanned so far=" + blocksScanned);
-          }
-          Thread.sleep(5000);
-        }
-      }
-    } finally {
-      cluster.shutdown();
-    }
-  }
-  
-  @Test(timeout=120000)
-  public void test2NNBlockRescanInterval() throws IOException {
-    ((Log4JLogger)BlockPoolSliceScanner.LOG).getLogger().setLevel(Level.ALL);
-    Configuration conf = new HdfsConfiguration();
-    cluster = new MiniDFSCluster.Builder(conf)
-        .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(3))
-        .build();
-
-    try {
-      FileSystem fs = cluster.getFileSystem(1);
-      Path file2 = new Path("/test/testBlockScanInterval");
-      DFSTestUtil.createFile(fs, file2, 30, (short) 1, 0);
-
-      fs = cluster.getFileSystem(0);
-      Path file1 = new Path("/test/testBlockScanInterval");
-      DFSTestUtil.createFile(fs, file1, 30, (short) 1, 0);
-      for (int i = 0; i < 8; i++) {
-        LOG.info("Verifying that the blockscanner scans exactly once");
-        waitAndScanBlocks(1, 1);
-      }
-    } finally {
-      cluster.shutdown();
-    }
-  }
-
-  /**
-   * HDFS-3828: DN rescans blocks too frequently
-   * 
-   * @throws Exception
-   */
-  @Test(timeout=120000)
-  public void testBlockRescanInterval() throws IOException {
-    ((Log4JLogger)BlockPoolSliceScanner.LOG).getLogger().setLevel(Level.ALL);
-    Configuration conf = new HdfsConfiguration();
-    cluster = new MiniDFSCluster.Builder(conf).build();
-
-    try {
-      FileSystem fs = cluster.getFileSystem();
-      Path file1 = new Path("/test/testBlockScanInterval");
-      DFSTestUtil.createFile(fs, file1, 30, (short) 1, 0);
-      for (int i = 0; i < 4; i++) {
-        LOG.info("Verifying that the blockscanner scans exactly once");
-        waitAndScanBlocks(1, 1);
-      }
-    } finally {
-      cluster.shutdown();
-    }
-  }
-
-  void waitAndScanBlocks(long scansLastRun, long scansTotal)
-      throws IOException {
-    // DataBlockScanner will run for every 5 seconds so we are checking for
-    // every 5 seconds
-    int n = 5;
-    String bpid = cluster.getNamesystem(0).getBlockPoolId();
-    DataNode dn = cluster.getDataNodes().get(0);
-    long blocksScanned, total;
-    do {
-      try {
-        Thread.sleep(SLEEP_PERIOD_MS);
-      } catch (InterruptedException e) {
-        fail("Interrupted: " + e);
-      }
-      blocksScanned = dn.blockScanner.getBlocksScannedInLastRun(bpid);
-      total = dn.blockScanner.getTotalScans(bpid);
-      LOG.info("bpid = " + bpid + " blocksScanned = " + blocksScanned + " total=" + total);
-    } while (n-- > 0 && (blocksScanned != scansLastRun || scansTotal != total));
-    Assert.assertEquals(scansTotal, total);
-    Assert.assertEquals(scansLastRun, blocksScanned);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df4edd9a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
index f256ee6..8fd51d2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
@@ -35,7 +35,6 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaInputStreams;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.RollingLogs;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
@@ -51,12 +50,6 @@ public class ExternalDatasetImpl implements FsDatasetSpi<ExternalVolumeImpl> {
       StorageType.DEFAULT);
 
   @Override
-  public RollingLogs createRollingLogs(String bpid, String prefix)
-      throws IOException {
-    return new ExternalRollingLogs();
-  }
-
-  @Override
   public List<ExternalVolumeImpl> getVolumes() {
     return null;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df4edd9a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalRollingLogs.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalRollingLogs.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalRollingLogs.java
deleted file mode 100644
index c9fb7c8..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalRollingLogs.java
+++ /dev/null
@@ -1,92 +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.datanode.extdataset;
-
-import java.io.IOException;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.RollingLogs;
-
-public class ExternalRollingLogs implements RollingLogs {
-
-  private class ExternalLineIterator implements LineIterator {
-    @Override
-    public boolean isPrevious() {
-      return false;
-    }
-
-    @Override
-    public boolean isLastReadFromPrevious() {
-      return false;
-    }
-
-    @Override
-    public boolean hasNext() {
-      return false;
-    }
-
-    @Override
-    public String next() {
-      return null;
-    }
-
-    @Override
-    public void remove() {
-    }
-
-    @Override
-    public void close() throws IOException {
-    }
-  }
-
-  private class ExternalAppender implements Appender {
-    @Override
-    public Appendable append(CharSequence cs) throws IOException {
-      return null;
-    }
-
-    @Override
-    public Appendable append(CharSequence cs, int i, int i1)
-	throws IOException {
-      return null;
-    }
-
-    @Override
-    public Appendable append(char c) throws IOException {
-      return null;
-    }
-
-    @Override
-    public void close() throws IOException {
-    }
-  }
-
-  @Override
-  public LineIterator iterator(boolean skipPrevious) throws IOException {
-    return new ExternalLineIterator();
-  }
-
-  @Override
-  public Appender appender() {
-    return new ExternalAppender();
-  }
-
-  @Override
-  public boolean roll() throws IOException {
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df4edd9a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalVolumeImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalVolumeImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalVolumeImpl.java
index 857e946..0ea33bb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalVolumeImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalVolumeImpl.java
@@ -23,6 +23,7 @@ import java.io.IOException;
 import java.nio.channels.ClosedChannelException;
 
 import org.apache.hadoop.hdfs.StorageType;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 
@@ -79,4 +80,20 @@ public class ExternalVolumeImpl implements FsVolumeSpi {
   @Override
   public void releaseReservedSpace(long bytesToRelease) {
   }
+
+  @Override
+  public BlockIterator newBlockIterator(String bpid, String name) {
+    return null;
+  }
+
+  @Override
+  public BlockIterator loadBlockIterator(String bpid, String name)
+      throws IOException {
+    return null;
+  }
+
+  @Override
+  public FsDatasetSpi getDataset() {
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df4edd9a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/TestExternalDataset.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/TestExternalDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/TestExternalDataset.java
index 791bb76..82a6951 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/TestExternalDataset.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/TestExternalDataset.java
@@ -22,7 +22,6 @@ import org.apache.hadoop.hdfs.server.datanode.Replica;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipelineInterface;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.RollingLogs;
 import org.junit.Test;
 
 /**
@@ -80,14 +79,6 @@ public class TestExternalDataset {
   }
 
   /**
-   * Tests instantiating a RollingLogs subclass.
-   */
-  @Test
-  public void testInstantiateRollingLogs() throws Throwable {
-    RollingLogs inst = new ExternalRollingLogs();
-  }
-
-  /**
    * Tests instantiating an FsVolumeSpi subclass.
    */
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df4edd9a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeListTest.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeListTest.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeListTest.java
index f92d949..691d390 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeListTest.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeListTest.java
@@ -19,7 +19,9 @@ package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystemTestHelper;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.StorageType;
+import org.apache.hadoop.hdfs.server.datanode.BlockScanner;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.RoundRobinVolumeChoosingPolicy;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.VolumeChoosingPolicy;
@@ -41,16 +43,21 @@ public class FsVolumeListTest {
       new RoundRobinVolumeChoosingPolicy<>();
   private FsDatasetImpl dataset = null;
   private String baseDir;
+  private BlockScanner blockScanner;
 
   @Before
   public void setUp() {
     dataset = mock(FsDatasetImpl.class);
     baseDir = new FileSystemTestHelper().getTestRootDir();
+    Configuration blockScannerConf = new Configuration();
+    blockScannerConf.setInt(DFSConfigKeys.
+        DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, -1);
+    blockScanner = new BlockScanner(null, blockScannerConf);
   }
 
   @Test
   public void testGetNextVolumeWithClosedVolume() throws IOException {
-    FsVolumeList volumeList = new FsVolumeList(0, blockChooser);
+    FsVolumeList volumeList = new FsVolumeList(0, blockScanner, blockChooser);
     List<FsVolumeImpl> volumes = new ArrayList<>();
     for (int i = 0; i < 3; i++) {
       File curDir = new File(baseDir, "nextvolume-" + i);
@@ -59,7 +66,7 @@ public class FsVolumeListTest {
           conf, StorageType.DEFAULT);
       volume.setCapacityForTesting(1024 * 1024 * 1024);
       volumes.add(volume);
-      volumeList.addVolume(volume);
+      volumeList.addVolume(volume.obtainReference());
     }
 
     // Close the second volume.
@@ -75,7 +82,7 @@ public class FsVolumeListTest {
 
   @Test
   public void testCheckDirsWithClosedVolume() throws IOException {
-    FsVolumeList volumeList = new FsVolumeList(0, blockChooser);
+    FsVolumeList volumeList = new FsVolumeList(0, blockScanner, blockChooser);
     List<FsVolumeImpl> volumes = new ArrayList<>();
     for (int i = 0; i < 3; i++) {
       File curDir = new File(baseDir, "volume-" + i);
@@ -83,7 +90,7 @@ public class FsVolumeListTest {
       FsVolumeImpl volume = new FsVolumeImpl(dataset, "storage-id", curDir,
           conf, StorageType.DEFAULT);
       volumes.add(volume);
-      volumeList.addVolume(volume);
+      volumeList.addVolume(volume.obtainReference());
     }
 
     // Close the 2nd volume.
@@ -91,4 +98,4 @@ public class FsVolumeListTest {
     // checkDirs() should ignore the 2nd volume since it is closed.
     volumeList.checkDirs();
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df4edd9a/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 ca936b3..f3d15de 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
@@ -22,17 +22,17 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.StorageType;
-import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
+import org.apache.hadoop.hdfs.server.datanode.BlockScanner;
 import org.apache.hadoop.hdfs.server.datanode.DNConf;
-import org.apache.hadoop.hdfs.server.datanode.DataBlockScanner;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataStorage;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaHandler;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.RoundRobinVolumeChoosingPolicy;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -51,19 +51,17 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY;
 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.Matchers.any;
 import static org.mockito.Matchers.anyListOf;
-import static org.mockito.Matchers.anyString;
 import static org.mockito.Matchers.eq;
 import static org.mockito.Mockito.doAnswer;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.never;
-import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
@@ -81,7 +79,6 @@ public class TestFsDatasetImpl {
   private Configuration conf;
   private DataNode datanode;
   private DataStorage storage;
-  private DataBlockScanner scanner;
   private FsDatasetImpl dataset;
 
   private static Storage.StorageDirectory createStorageDirectory(File root) {
@@ -112,13 +109,14 @@ public class TestFsDatasetImpl {
   public void setUp() throws IOException {
     datanode = mock(DataNode.class);
     storage = mock(DataStorage.class);
-    scanner = mock(DataBlockScanner.class);
     this.conf = new Configuration();
+    this.conf.setLong(DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, 0);
     final DNConf dnConf = new DNConf(conf);
 
     when(datanode.getConf()).thenReturn(conf);
     when(datanode.getDnConf()).thenReturn(dnConf);
-    when(datanode.getBlockScanner()).thenReturn(scanner);
+    final BlockScanner disabledBlockScanner = new BlockScanner(datanode, conf);
+    when(datanode.getBlockScanner()).thenReturn(disabledBlockScanner);
 
     createStorageDirs(storage, conf, NUM_INIT_VOLUMES);
     dataset = new FsDatasetImpl(datanode, storage, conf);
@@ -208,10 +206,6 @@ public class TestFsDatasetImpl {
     assertEquals("The replica infos on this volume has been removed from the "
                  + "volumeMap.", NUM_BLOCKS / NUM_INIT_VOLUMES,
                  totalNumReplicas);
-
-    // Verify that every BlockPool deletes the removed blocks from the volume.
-    verify(scanner, times(BLOCK_POOL_IDS.length))
-        .deleteBlocks(anyString(), any(Block[].class));
   }
 
   @Test(timeout = 5000)
@@ -245,7 +239,9 @@ public class TestFsDatasetImpl {
   public void testChangeVolumeWithRunningCheckDirs() throws IOException {
     RoundRobinVolumeChoosingPolicy<FsVolumeImpl> blockChooser =
         new RoundRobinVolumeChoosingPolicy<>();
-    final FsVolumeList volumeList = new FsVolumeList(0, blockChooser);
+    final BlockScanner blockScanner = new BlockScanner(datanode, conf);
+    final FsVolumeList volumeList =
+        new FsVolumeList(0, blockScanner, blockChooser);
     final List<FsVolumeImpl> oldVolumes = new ArrayList<>();
 
     // Initialize FsVolumeList with 5 mock volumes.
@@ -254,19 +250,23 @@ public class TestFsDatasetImpl {
       FsVolumeImpl volume = mock(FsVolumeImpl.class);
       oldVolumes.add(volume);
       when(volume.getBasePath()).thenReturn("data" + i);
-      volumeList.addVolume(volume);
+      FsVolumeReference ref = mock(FsVolumeReference.class);
+      when(ref.getVolume()).thenReturn(volume);
+      volumeList.addVolume(ref);
     }
 
     // When call checkDirs() on the 2nd volume, anther "thread" removes the 5th
     // volume and add another volume. It does not affect checkDirs() running.
     final FsVolumeImpl newVolume = mock(FsVolumeImpl.class);
+    final FsVolumeReference newRef = mock(FsVolumeReference.class);
+    when(newRef.getVolume()).thenReturn(newVolume);
     FsVolumeImpl blockedVolume = volumeList.getVolumes().get(1);
     doAnswer(new Answer() {
       @Override
       public Object answer(InvocationOnMock invocationOnMock)
           throws Throwable {
         volumeList.removeVolume(new File("data4"));
-        volumeList.addVolume(newVolume);
+        volumeList.addVolume(newRef);
         return null;
       }
     }).when(blockedVolume).checkDirs();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df4edd9a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestInterDatanodeProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestInterDatanodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestInterDatanodeProtocol.java
index 3609684..6cc3d7e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestInterDatanodeProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestInterDatanodeProtocol.java
@@ -184,8 +184,8 @@ public class TestInterDatanodeProtocol {
       InterDatanodeProtocol idp = DataNodeTestUtils.createInterDatanodeProtocolProxy(
           datanode, datanodeinfo[0], conf, useDnHostname);
       
-      //stop block scanner, so we could compare lastScanTime
-      DataNodeTestUtils.shutdownBlockScanner(datanode);
+      // Stop the block scanners.
+      datanode.getBlockScanner().removeAllVolumeScanners();
 
       //verify BlockMetaDataInfo
       ExtendedBlock b = locatedblock.getBlock();


[07/50] [abbrv] hadoop git commit: HDFS-7641. Update archival storage user doc for list/set/get block storage policies. (yliu)

Posted by zh...@apache.org.
HDFS-7641. Update archival storage user doc for list/set/get block storage policies. (yliu)


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

Branch: refs/heads/HDFS-EC
Commit: a6d1f21e2a540856d29e59d1fe4dd824343dc19a
Parents: 3111cfb
Author: yliu <yl...@apache.org>
Authored: Wed Jan 21 02:06:18 2015 +0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:25 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +++
 .../src/site/apt/ArchivalStorage.apt.vm         | 22 ++++++++++----------
 2 files changed, 14 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a6d1f21e/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 932fee8..964e278 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -741,6 +741,9 @@ Release 2.7.0 - UNRELEASED
 
     HDFS-7637. Fix the check condition for reserved path. (Yi Liu via jing9)
 
+    HDFS-7641. Update archival storage user doc for list/set/get block storage
+    policies. (yliu)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a6d1f21e/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/ArchivalStorage.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/ArchivalStorage.apt.vm b/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/ArchivalStorage.apt.vm
index 69674c7..5336ea3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/ArchivalStorage.apt.vm
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/ArchivalStorage.apt.vm
@@ -189,7 +189,7 @@ hdfs mover [-p <files/dirs> | -f <local file name>]
   * Command:
 
 +------------------------------------------+
-hdfs storagepolicies
+hdfs storagepolicies -listPolicies
 +------------------------------------------+
 
   * Arguments: none.
@@ -201,16 +201,16 @@ hdfs storagepolicies
   * Command:
 
 +------------------------------------------+
-hdfs dfsadmin -setStoragePolicy <path> <policyName>
+hdfs storagepolicies -setStoragePolicy -path <path> -policy <policy>
 +------------------------------------------+
 
   * Arguments:
 
-*----------------------+-----------------------------------------------------+
-| <<<\<path\>>>>       | The path referring to either a directory or a file. |
-*----------------------+-----------------------------------------------------+
-| <<<\<policyName\>>>> | The name of the storage policy.                     |
-*----------------------+-----------------------------------------------------+
+*--------------------------+-----------------------------------------------------+
+| <<<-path \<path\>>>>     | The path referring to either a directory or a file. |
+*--------------------------+-----------------------------------------------------+
+| <<<-policy \<policy\>>>> | The name of the storage policy.                     |
+*--------------------------+-----------------------------------------------------+
 
   []
 
@@ -221,13 +221,13 @@ hdfs dfsadmin -setStoragePolicy <path> <policyName>
   * Command:
 
 +------------------------------------------+
-hdfs dfsadmin -getStoragePolicy <path>
+hdfs storagepolicies -getStoragePolicy -path <path>
 +------------------------------------------+
 
   * Arguments:
 
-*----------------------+-----------------------------------------------------+
-| <<<\<path\>>>>       | The path referring to either a directory or a file. |
-*----------------------+-----------------------------------------------------+
+*----------------------------+-----------------------------------------------------+
+| <<<-path \<path\>>>>       | The path referring to either a directory or a file. |
+*----------------------------+-----------------------------------------------------+
 
   []


[06/50] [abbrv] hadoop git commit: HDFS-7623. Add htrace configuration properties to core-default.xml and update user doc about how to enable htrace. (yliu)

Posted by zh...@apache.org.
HDFS-7623. Add htrace configuration properties to core-default.xml and update user doc about how to enable htrace. (yliu)


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

Branch: refs/heads/HDFS-EC
Commit: 3111cfb18fe2e25c4dd96620f16ed53b322dd635
Parents: 7ba6504
Author: yliu <yl...@apache.org>
Authored: Wed Jan 21 00:59:47 2015 +0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:25 2015 -0800

----------------------------------------------------------------------
 .../src/main/resources/core-default.xml         | 24 +++++++++++++++++++
 .../hadoop-common/src/site/apt/Tracing.apt.vm   | 25 +++++++++++++++-----
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +++
 3 files changed, 46 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3111cfb1/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 e7a382d..bebc263 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
@@ -1653,4 +1653,28 @@ for ldap providers in the same way as above does.
   </description>
 </property>
 
+<property>
+  <name>hadoop.htrace.sampler</name>
+  <value>NeverSampler</value>
+  <description>
+    Configure the samplers for HTrace, the value can be NeverSampler,
+    AlwaysSampler or ProbabilitySampler. NeverSampler: HTrace is OFF 
+    for all spans; AlwaysSampler: HTrace is ON for all spans;
+    ProbabilitySampler: HTrace is ON for some percentage% of top-level 
+    spans.
+  </description>
+</property>
+
+<property>
+  <name>hadoop.htrace.spanreceiver.classes</name>
+  <value></value>
+  <description>
+    A comma separated list of the fully-qualified class name of classes 
+    implementing SpanReceiver. The tracing system works by collecting 
+    information in structs called 'Spans'. It is up to you to choose 
+    how you want to receive this information by implementing the 
+    SpanReceiver interface.
+  </description>
+</property>
+
 </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3111cfb1/hadoop-common-project/hadoop-common/src/site/apt/Tracing.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/apt/Tracing.apt.vm b/hadoop-common-project/hadoop-common/src/site/apt/Tracing.apt.vm
index 9eda220..f04da33 100644
--- a/hadoop-common-project/hadoop-common/src/site/apt/Tracing.apt.vm
+++ b/hadoop-common-project/hadoop-common/src/site/apt/Tracing.apt.vm
@@ -16,19 +16,32 @@
   ---
   ${maven.build.timestamp}
 
-Enabling Dapper-like Tracing in HDFS
+Enabling Dapper-like Tracing in Hadoop
 
 %{toc|section=1|fromDepth=0}
 
-* {Dapper-like Tracing in HDFS}
+* {Dapper-like Tracing in Hadoop}
 
 ** HTrace
 
   {{{https://issues.apache.org/jira/browse/HDFS-5274}HDFS-5274}}
   added support for tracing requests through HDFS,
-  using the open source tracing library, {{{https://github.com/cloudera/htrace}HTrace}}.
+  using the open source tracing library, {{{https://git-wip-us.apache.org/repos/asf/incubator-htrace.git}Apache HTrace}}.
   Setting up tracing is quite simple, however it requires some very minor changes to your client code.
 
+** Samplers
+  Configure the samplers in <<<core-site.xml>>> property: <<<hadoop.htrace.sampler>>>.
+  The value can be NeverSampler, AlwaysSampler or ProbabilitySampler. NeverSampler: HTrace is OFF 
+  for all spans; AlwaysSampler: HTrace is ON for all spans; ProbabilitySampler: HTrace is ON for 
+  some percentage% of top-level spans.
+
++----
+  <property>
+    <name>hadoop.htrace.sampler</name>
+    <value>NeverSampler</value>
+  </property>
++----
+
 ** SpanReceivers
 
   The tracing system works by collecting information in structs called 'Spans'.
@@ -42,7 +55,7 @@ public void receiveSpan(Span span);
   Configure what SpanReceivers you'd like to use
   by putting a comma separated list of the fully-qualified class name of
   classes implementing SpanReceiver
-  in <<<hdfs-site.xml>>> property: <<<hadoop.htrace.spanreceiver.classes>>>.
+  in <<<core-site.xml>>> property: <<<hadoop.htrace.spanreceiver.classes>>>.
 
 +----
   <property>
@@ -83,11 +96,11 @@ public void receiveSpan(Span span);
   $ git clone https://github.com/cloudera/htrace
   $ cd htrace/htrace-zipkin
   $ mvn compile assembly:single
-  $ cp target/htrace-zipkin-*-jar-with-dependencies.jar $HADOOP_HOME/share/hadoop/hdfs/lib/
+  $ cp target/htrace-zipkin-*-jar-with-dependencies.jar $HADOOP_HOME/share/hadoop/common/lib/
 +----
 
   The sample configuration for <<<ZipkinSpanReceiver>>> is shown below.
-  By adding these to <<<hdfs-site.xml>>> of NameNode and DataNodes,
+  By adding these to <<<core-site.xml>>> of NameNode and DataNodes,
   <<<ZipkinSpanReceiver>>> is initialized on the startup.
   You also need this configuration on the client node in addition to the servers.
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3111cfb1/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 49fce24..932fee8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -536,6 +536,9 @@ Release 2.7.0 - UNRELEASED
 
     HDFS-7640. Print NFS Client in the NFS log. (Brandon Li via wheat9)
 
+    HDFS-7623. Add htrace configuration properties to core-default.xml and
+    update user doc about how to enable htrace. (yliu)
+
   OPTIMIZATIONS
 
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.


[27/50] [abbrv] hadoop git commit: HADOOP-11209. Configuration#updatingResource/finalParameters are not thread-safe. Contributed by Varun Saxena.

Posted by zh...@apache.org.
HADOOP-11209. Configuration#updatingResource/finalParameters are not thread-safe. Contributed by Varun Saxena.


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

Branch: refs/heads/HDFS-EC
Commit: ea85ae4f80e0e6781bbe483fd5babb0669ea141c
Parents: 73309fb
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Thu Jan 22 14:15:59 2015 +0900
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:28 2015 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 ++
 .../org/apache/hadoop/conf/Configuration.java   | 46 +++++++++++++-------
 .../apache/hadoop/conf/TestConfiguration.java   | 46 ++++++++++++++++++++
 3 files changed, 80 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea85ae4f/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 abe699a..eb9015c 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -741,6 +741,9 @@ Release 2.7.0 - UNRELEASED
     architecture because it is slower there (Suman Somasundar via Colin P.
     McCabe)
 
+    HADOOP-11209. Configuration#updatingResource/finalParameters are not
+    thread-safe. (Varun Saxena via ozawa)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea85ae4f/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
index afcea44..54ee46d 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
@@ -52,6 +52,7 @@ import java.util.Properties;
 import java.util.Set;
 import java.util.StringTokenizer;
 import java.util.WeakHashMap;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
@@ -228,7 +229,8 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
   /**
    * List of configuration parameters marked <b>final</b>. 
    */
-  private Set<String> finalParameters = new HashSet<String>();
+  private Set<String> finalParameters = Collections.newSetFromMap(
+      new ConcurrentHashMap<String, Boolean>());
   
   private boolean loadDefaults = true;
   
@@ -258,7 +260,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
    * Stores the mapping of key to the resource which modifies or loads 
    * the key most recently
    */
-  private HashMap<String, String[]> updatingResource;
+  private Map<String, String[]> updatingResource;
  
   /**
    * Class to keep the information about the keys which replace the deprecated
@@ -685,7 +687,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
    */
   public Configuration(boolean loadDefaults) {
     this.loadDefaults = loadDefaults;
-    updatingResource = new HashMap<String, String[]>();
+    updatingResource = new ConcurrentHashMap<String, String[]>();
     synchronized(Configuration.class) {
       REGISTRY.put(this, null);
     }
@@ -708,8 +710,11 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
        this.overlay = (Properties)other.overlay.clone();
      }
 
-     this.updatingResource = new HashMap<String, String[]>(other.updatingResource);
-     this.finalParameters = new HashSet<String>(other.finalParameters);
+     this.updatingResource = new ConcurrentHashMap<String, String[]>(
+         other.updatingResource);
+     this.finalParameters = Collections.newSetFromMap(
+         new ConcurrentHashMap<String, Boolean>());
+     this.finalParameters.addAll(other.finalParameters);
    }
    
     synchronized(Configuration.class) {
@@ -2314,20 +2319,27 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
    * @return final parameter set.
    */
   public Set<String> getFinalParameters() {
-    return new HashSet<String>(finalParameters);
+    Set<String> setFinalParams = Collections.newSetFromMap(
+        new ConcurrentHashMap<String, Boolean>());
+    setFinalParams.addAll(finalParameters);
+    return setFinalParams;
   }
 
   protected synchronized Properties getProps() {
     if (properties == null) {
       properties = new Properties();
-      HashMap<String, String[]> backup = 
-        new HashMap<String, String[]>(updatingResource);
+      Map<String, String[]> backup =
+          new ConcurrentHashMap<String, String[]>(updatingResource);
       loadResources(properties, resources, quietmode);
-      if (overlay!= null) {
+
+      if (overlay != null) {
         properties.putAll(overlay);
         for (Map.Entry<Object,Object> item: overlay.entrySet()) {
           String key = (String)item.getKey();
-          updatingResource.put(key, backup.get(key));
+          String[] source = backup.get(key);
+          if(source != null) {
+            updatingResource.put(key, source);
+          }
         }
       }
     }
@@ -2576,16 +2588,18 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     if (value != null || allowNullValueProperties) {
       if (!finalParameters.contains(attr)) {
         if (value==null && allowNullValueProperties) {
-	  value = DEFAULT_STRING_CHECK;
-	}
+          value = DEFAULT_STRING_CHECK;
+        }
         properties.setProperty(attr, value);
-        updatingResource.put(attr, source);
+        if(source != null) {
+          updatingResource.put(attr, source);
+        }
       } else if (!value.equals(properties.getProperty(attr))) {
         LOG.warn(name+":an attempt to override final parameter: "+attr
             +";  Ignoring.");
       }
     }
-    if (finalParameter) {
+    if (finalParameter && attr != null) {
       finalParameters.add(attr);
     }
   }
@@ -2788,7 +2802,9 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
       String value = org.apache.hadoop.io.Text.readString(in);
       set(key, value); 
       String sources[] = WritableUtils.readCompressedStringArray(in);
-      updatingResource.put(key, sources);
+      if(sources != null) {
+        updatingResource.put(key, sources);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea85ae4f/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
index d21500f..55bcdc6 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
@@ -1308,6 +1308,52 @@ public class TestConfiguration extends TestCase {
     assertTrue("my.var is not final", finalParameters.contains("my.var"));
   }
 
+  /**
+   * A test to check whether this thread goes into infinite loop because of
+   * destruction of data structure by resize of Map. This problem was reported
+   * by SPARK-2546.
+   * @throws Exception
+   */
+  public void testConcurrentAccesses() throws Exception {
+    out = new BufferedWriter(new FileWriter(CONFIG));
+    startConfig();
+    declareProperty("some.config", "xyz", "xyz", false);
+    endConfig();
+    Path fileResource = new Path(CONFIG);
+    Configuration conf = new Configuration();
+    conf.addResource(fileResource);
+
+    class ConfigModifyThread extends Thread {
+      final private Configuration config;
+      final private String prefix;
+
+      public ConfigModifyThread(Configuration conf, String prefix) {
+        config = conf;
+        this.prefix = prefix;
+      }
+
+      @Override
+      public void run() {
+        for (int i = 0; i < 100000; i++) {
+          config.set("some.config.value-" + prefix + i, "value");
+        }
+      }
+    }
+
+    ArrayList<ConfigModifyThread> threads = new ArrayList<>();
+    for (int i = 0; i < 100; i++) {
+      threads.add(new ConfigModifyThread(conf, String.valueOf(i)));
+    }
+    for (Thread t: threads) {
+      t.start();
+    }
+    for (Thread t: threads) {
+      t.join();
+    }
+    // If this test completes without going into infinite loop,
+    // it's expected behaviour.
+  }
+
   public static void main(String[] argv) throws Exception {
     junit.textui.TestRunner.main(new String[]{
       TestConfiguration.class.getName()


[34/50] [abbrv] hadoop git commit: HADOOP-11507 Hadoop RPC Authentication problem with different user locale. (Talat UYARER via stevel)

Posted by zh...@apache.org.
HADOOP-11507 Hadoop RPC Authentication problem with different user locale. (Talat UYARER via stevel)


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

Branch: refs/heads/HDFS-EC
Commit: 5177c148d22a08f6cd9015ab6f8d7050ccbbade2
Parents: 7d62e90
Author: Steve Loughran <st...@apache.org>
Authored: Fri Jan 23 19:36:37 2015 +0000
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:29 2015 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt                   | 3 +++
 .../java/org/apache/hadoop/security/SaslPropertiesResolver.java   | 3 ++-
 2 files changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5177c148/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 bab2220..6bedd4d 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -753,6 +753,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11493. Fix some typos in kms-acls.xml description.
     (Charles Lamb via aajisaka)
 
+    HADOOP-11507 Hadoop RPC Authentication problem with different user locale.
+    (Talat UYARER via stevel)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5177c148/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslPropertiesResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslPropertiesResolver.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslPropertiesResolver.java
index c4fc965..0b49cfb 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslPropertiesResolver.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslPropertiesResolver.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.security;
 
 import java.net.InetAddress;
+import java.util.Locale;
 import java.util.Map;
 import java.util.TreeMap;
 
@@ -65,7 +66,7 @@ public class SaslPropertiesResolver implements Configurable{
         CommonConfigurationKeysPublic.HADOOP_RPC_PROTECTION,
         QualityOfProtection.AUTHENTICATION.toString());
     for (int i=0; i < qop.length; i++) {
-      qop[i] = QualityOfProtection.valueOf(qop[i].toUpperCase()).getSaslQop();
+      qop[i] = QualityOfProtection.valueOf(qop[i].toUpperCase(Locale.ENGLISH)).getSaslQop();
     }
     properties.put(Sasl.QOP, StringUtils.join(",", qop));
     properties.put(Sasl.SERVER_AUTH, "true");


[13/50] [abbrv] hadoop git commit: HADOOP-11327. BloomFilter#not() omits the last bit, resulting in an incorrect filter. Contributed by Eric Payne

Posted by zh...@apache.org.
HADOOP-11327. BloomFilter#not() omits the last bit, resulting in an incorrect filter. Contributed by Eric Payne


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

Branch: refs/heads/HDFS-EC
Commit: cf76280c7dd12cbe1ad9d03ef57851b2ee4b30a8
Parents: bd457d3
Author: Jason Lowe <jl...@apache.org>
Authored: Wed Jan 21 19:04:29 2015 +0000
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:26 2015 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt          |  3 +++
 .../java/org/apache/hadoop/util/bloom/BloomFilter.java   |  2 +-
 .../org/apache/hadoop/util/bloom/TestBloomFilters.java   | 11 +++++++++++
 3 files changed, 15 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf76280c/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 2951002..c54800f 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -731,6 +731,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-10668. TestZKFailoverControllerStress#testExpireBackAndForth
     occasionally fails. (Ming Ma via cnauroth)
 
+    HADOOP-11327. BloomFilter#not() omits the last bit, resulting in an
+    incorrect filter (Eric Payne via jlowe)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf76280c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/bloom/BloomFilter.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/bloom/BloomFilter.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/bloom/BloomFilter.java
index e2dea6d..f8b9519 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/bloom/BloomFilter.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/bloom/BloomFilter.java
@@ -157,7 +157,7 @@ public class BloomFilter extends Filter {
 
   @Override
   public void not() {
-    bits.flip(0, vectorSize - 1);
+    bits.flip(0, vectorSize);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf76280c/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/bloom/TestBloomFilters.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/bloom/TestBloomFilters.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/bloom/TestBloomFilters.java
index 93fa6d5..6ff854d 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/bloom/TestBloomFilters.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/bloom/TestBloomFilters.java
@@ -23,6 +23,7 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
 import java.util.AbstractCollection;
+import java.util.BitSet;
 import java.util.Iterator;
 
 import org.apache.hadoop.util.bloom.BloomFilterCommonTester.BloomFilterTestStrategy;
@@ -237,4 +238,14 @@ public class TestBloomFilters {
                 BloomFilterTestStrategy.FILTER_AND_STRATEGY,
                 BloomFilterTestStrategy.FILTER_XOR_STRATEGY)).test();
   }
+
+  @Test
+  public void testNot() {
+    BloomFilter bf = new BloomFilter(8, 1, Hash.JENKINS_HASH);
+    bf.bits = BitSet.valueOf(new byte[] { (byte) 0x95 });
+    BitSet origBitSet = (BitSet) bf.bits.clone();
+    bf.not();
+    assertFalse("BloomFilter#not should have inverted all bits",
+                bf.bits.intersects(origBitSet));
+  }
 }


[11/50] [abbrv] hadoop git commit: HDFS-3443. Fix NPE when namenode transition to active during startup by adding checkNNStartup() in NameNodeRpcServer. Contributed by Vinayakumar B

Posted by zh...@apache.org.
HDFS-3443. Fix NPE when namenode transition to active during startup by adding checkNNStartup() in NameNodeRpcServer.  Contributed by Vinayakumar B


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

Branch: refs/heads/HDFS-EC
Commit: 8a690f9f41cb18dd2a3bc58c38387fa3bc9a3d65
Parents: dc9978a
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Wed Jan 21 11:32:32 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:26 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../hadoop/hdfs/server/namenode/NameNode.java   |  12 +-
 .../hdfs/server/namenode/NameNodeRpcServer.java | 187 +++++++++++++++----
 3 files changed, 165 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a690f9f/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 2ab8ea3..0a301f8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -783,6 +783,9 @@ Release 2.6.1 - UNRELEASED
     HDFS-7443. Datanode upgrade to BLOCKID_BASED_LAYOUT fails if duplicate
     block files are present in the same volume (cmccabe)
 
+    HDFS-3443. Fix NPE when namenode transition to active during startup by
+    adding checkNNStartup() in NameNodeRpcServer.  (Vinayakumar B via szetszwo)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a690f9f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
index a71d158..fea7c62 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
@@ -79,6 +79,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_DEFAULT;
@@ -265,6 +266,7 @@ public class NameNode implements NameNodeStatusMXBean {
   private final boolean haEnabled;
   private final HAContext haContext;
   protected final boolean allowStaleStandbyReads;
+  private AtomicBoolean started = new AtomicBoolean(false); 
 
   
   /** httpServer */
@@ -775,6 +777,7 @@ public class NameNode implements NameNodeStatusMXBean {
       this.stop();
       throw e;
     }
+    this.started.set(true);
   }
 
   protected HAState createHAState(StartupOption startOpt) {
@@ -1743,7 +1746,14 @@ public class NameNode implements NameNodeStatusMXBean {
   public boolean isActiveState() {
     return (state.equals(ACTIVE_STATE));
   }
-  
+
+  /**
+   * Returns whether the NameNode is completely started
+   */
+  boolean isStarted() {
+    return this.started.get();
+  }
+
   /**
    * Check that a request to change this node's HA state is valid.
    * In particular, verifies that, if auto failover is enabled, non-forced

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a690f9f/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 6ef8fd6..a3ac455 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
@@ -69,7 +69,6 @@ import org.apache.hadoop.ha.protocolPB.HAServiceProtocolServerSideTranslatorPB;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HDFSPolicyProvider;
-import org.apache.hadoop.hdfs.inotify.Event;
 import org.apache.hadoop.hdfs.inotify.EventBatch;
 import org.apache.hadoop.hdfs.inotify.EventBatchList;
 import org.apache.hadoop.hdfs.protocol.AclException;
@@ -479,12 +478,14 @@ class NameNodeRpcServer implements NamenodeProtocols {
       throw new IllegalArgumentException(
         "Unexpected not positive size: "+size);
     }
+    checkNNStartup();
     namesystem.checkSuperuserPrivilege();
     return namesystem.getBlockManager().getBlocks(datanode, size); 
   }
 
   @Override // NamenodeProtocol
   public ExportedBlockKeys getBlockKeys() throws IOException {
+    checkNNStartup();
     namesystem.checkSuperuserPrivilege();
     return namesystem.getBlockManager().getBlockKeys();
   }
@@ -493,6 +494,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   public void errorReport(NamenodeRegistration registration,
                           int errorCode, 
                           String msg) throws IOException {
+    checkNNStartup();
     namesystem.checkOperation(OperationCategory.UNCHECKED);
     namesystem.checkSuperuserPrivilege();
     verifyRequest(registration);
@@ -505,6 +507,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   @Override // NamenodeProtocol
   public NamenodeRegistration registerSubordinateNamenode(
       NamenodeRegistration registration) throws IOException {
+    checkNNStartup();
     namesystem.checkSuperuserPrivilege();
     verifyLayoutVersion(registration.getVersion());
     NamenodeRegistration myRegistration = nn.setRegistration();
@@ -514,7 +517,8 @@ class NameNodeRpcServer implements NamenodeProtocols {
 
   @Override // NamenodeProtocol
   public NamenodeCommand startCheckpoint(NamenodeRegistration registration)
-  throws IOException {
+      throws IOException {
+    checkNNStartup();
     namesystem.checkSuperuserPrivilege();
     verifyRequest(registration);
     if(!nn.isRole(NamenodeRole.NAMENODE))
@@ -537,6 +541,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   @Override // NamenodeProtocol
   public void endCheckpoint(NamenodeRegistration registration,
                             CheckpointSignature sig) throws IOException {
+    checkNNStartup();
     namesystem.checkSuperuserPrivilege();
     CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
     if (cacheEntry != null && cacheEntry.isSuccess()) {
@@ -554,18 +559,21 @@ class NameNodeRpcServer implements NamenodeProtocols {
   @Override // ClientProtocol
   public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
       throws IOException {
+    checkNNStartup();
     return namesystem.getDelegationToken(renewer);
   }
 
   @Override // ClientProtocol
   public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
       throws InvalidToken, IOException {
+    checkNNStartup();
     return namesystem.renewDelegationToken(token);
   }
 
   @Override // ClientProtocol
   public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
       throws IOException {
+    checkNNStartup();
     namesystem.cancelDelegationToken(token);
   }
   
@@ -574,6 +582,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
                                           long offset, 
                                           long length) 
       throws IOException {
+    checkNNStartup();
     metrics.incrGetBlockLocations();
     return namesystem.getBlockLocations(getClientMachine(), 
                                         src, offset, length);
@@ -581,6 +590,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   
   @Override // ClientProtocol
   public FsServerDefaults getServerDefaults() throws IOException {
+    checkNNStartup();
     return namesystem.getServerDefaults();
   }
 
@@ -590,6 +600,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
       boolean createParent, short replication, long blockSize, 
       CryptoProtocolVersion[] supportedVersions)
       throws IOException {
+    checkNNStartup();
     String clientMachine = getClientMachine();
     if (stateChangeLog.isDebugEnabled()) {
       stateChangeLog.debug("*DIR* NameNode.create: file "
@@ -624,6 +635,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   @Override // ClientProtocol
   public LastBlockWithStatus append(String src, String clientName) 
       throws IOException {
+    checkNNStartup();
     String clientMachine = getClientMachine();
     if (stateChangeLog.isDebugEnabled()) {
       stateChangeLog.debug("*DIR* NameNode.append: file "
@@ -649,36 +661,42 @@ class NameNodeRpcServer implements NamenodeProtocols {
 
   @Override // ClientProtocol
   public boolean recoverLease(String src, String clientName) throws IOException {
+    checkNNStartup();
     String clientMachine = getClientMachine();
     return namesystem.recoverLease(src, clientName, clientMachine);
   }
 
   @Override // ClientProtocol
   public boolean setReplication(String src, short replication) 
-    throws IOException {  
+    throws IOException {
+    checkNNStartup();
     return namesystem.setReplication(src, replication);
   }
 
   @Override
   public void setStoragePolicy(String src, String policyName)
       throws IOException {
+    checkNNStartup();
     namesystem.setStoragePolicy(src, policyName);
   }
 
   @Override
   public BlockStoragePolicy[] getStoragePolicies() throws IOException {
+    checkNNStartup();
     return namesystem.getStoragePolicies();
   }
 
   @Override // ClientProtocol
   public void setPermission(String src, FsPermission permissions)
       throws IOException {
+    checkNNStartup();
     namesystem.setPermission(src, permissions);
   }
 
   @Override // ClientProtocol
   public void setOwner(String src, String username, String groupname)
       throws IOException {
+    checkNNStartup();
     namesystem.setOwner(src, username, groupname);
   }
   
@@ -687,6 +705,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
       ExtendedBlock previous, DatanodeInfo[] excludedNodes, long fileId,
       String[] favoredNodes)
       throws IOException {
+    checkNNStartup();
     if (stateChangeLog.isDebugEnabled()) {
       stateChangeLog.debug("*BLOCK* NameNode.addBlock: file " + src
           + " fileId=" + fileId + " for " + clientName);
@@ -714,6 +733,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
       final DatanodeInfo[] excludes,
       final int numAdditionalNodes, final String clientName
       ) throws IOException {
+    checkNNStartup();
     if (LOG.isDebugEnabled()) {
       LOG.debug("getAdditionalDatanode: src=" + src
           + ", fileId=" + fileId
@@ -742,6 +762,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   @Override // ClientProtocol
   public void abandonBlock(ExtendedBlock b, long fileId, String src,
         String holder) throws IOException {
+    checkNNStartup();
     if(stateChangeLog.isDebugEnabled()) {
       stateChangeLog.debug("*BLOCK* NameNode.abandonBlock: "
           +b+" of file "+src);
@@ -755,6 +776,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   public boolean complete(String src, String clientName,
                           ExtendedBlock last,  long fileId)
       throws IOException {
+    checkNNStartup();
     if(stateChangeLog.isDebugEnabled()) {
       stateChangeLog.debug("*DIR* NameNode.complete: "
           + src + " fileId=" + fileId +" for " + clientName);
@@ -770,12 +792,14 @@ class NameNodeRpcServer implements NamenodeProtocols {
    */
   @Override // ClientProtocol, DatanodeProtocol
   public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
+    checkNNStartup();
     namesystem.reportBadBlocks(blocks);
   }
 
   @Override // ClientProtocol
   public LocatedBlock updateBlockForPipeline(ExtendedBlock block, String clientName)
       throws IOException {
+    checkNNStartup();
     return namesystem.updateBlockForPipeline(block, clientName);
   }
 
@@ -784,6 +808,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   public void updatePipeline(String clientName, ExtendedBlock oldBlock,
       ExtendedBlock newBlock, DatanodeID[] newNodes, String[] newStorageIDs)
       throws IOException {
+    checkNNStartup();
     CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
     if (cacheEntry != null && cacheEntry.isSuccess()) {
       return; // Return previous response
@@ -805,6 +830,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
       boolean closeFile, boolean deleteblock, DatanodeID[] newtargets,
       String[] newtargetstorages)
       throws IOException {
+    checkNNStartup();
     namesystem.commitBlockSynchronization(block, newgenerationstamp,
         newlength, closeFile, deleteblock, newtargets, newtargetstorages);
   }
@@ -812,12 +838,14 @@ class NameNodeRpcServer implements NamenodeProtocols {
   @Override // ClientProtocol
   public long getPreferredBlockSize(String filename) 
       throws IOException {
+    checkNNStartup();
     return namesystem.getPreferredBlockSize(filename);
   }
     
   @Deprecated
   @Override // ClientProtocol
   public boolean rename(String src, String dst) throws IOException {
+    checkNNStartup();
     if(stateChangeLog.isDebugEnabled()) {
       stateChangeLog.debug("*DIR* NameNode.rename: " + src + " to " + dst);
     }
@@ -845,6 +873,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   
   @Override // ClientProtocol
   public void concat(String trg, String[] src) throws IOException {
+    checkNNStartup();
     CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
     if (cacheEntry != null && cacheEntry.isSuccess()) {
       return; // Return previous response
@@ -862,6 +891,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   @Override // ClientProtocol
   public void rename2(String src, String dst, Options.Rename... options)
       throws IOException {
+    checkNNStartup();
     if(stateChangeLog.isDebugEnabled()) {
       stateChangeLog.debug("*DIR* NameNode.rename: " + src + " to " + dst);
     }
@@ -886,6 +916,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   @Override // ClientProtocol
   public boolean truncate(String src, long newLength, String clientName)
       throws IOException {
+    checkNNStartup();
     if(stateChangeLog.isDebugEnabled()) {
       stateChangeLog.debug("*DIR* NameNode.truncate: " + src + " to " +
           newLength);
@@ -901,6 +932,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
 
   @Override // ClientProtocol
   public boolean delete(String src, boolean recursive) throws IOException {
+    checkNNStartup();
     if (stateChangeLog.isDebugEnabled()) {
       stateChangeLog.debug("*DIR* Namenode.delete: src=" + src
           + ", recursive=" + recursive);
@@ -935,6 +967,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   @Override // ClientProtocol
   public boolean mkdirs(String src, FsPermission masked, boolean createParent)
       throws IOException {
+    checkNNStartup();
     if(stateChangeLog.isDebugEnabled()) {
       stateChangeLog.debug("*DIR* NameNode.mkdirs: " + src);
     }
@@ -949,12 +982,14 @@ class NameNodeRpcServer implements NamenodeProtocols {
 
   @Override // ClientProtocol
   public void renewLease(String clientName) throws IOException {
+    checkNNStartup();
     namesystem.renewLease(clientName);        
   }
 
   @Override // ClientProtocol
   public DirectoryListing getListing(String src, byte[] startAfter,
       boolean needLocation) throws IOException {
+    checkNNStartup();
     DirectoryListing files = namesystem.getListing(
         src, startAfter, needLocation);
     if (files != null) {
@@ -966,23 +1001,27 @@ class NameNodeRpcServer implements NamenodeProtocols {
 
   @Override // ClientProtocol
   public HdfsFileStatus getFileInfo(String src)  throws IOException {
+    checkNNStartup();
     metrics.incrFileInfoOps();
     return namesystem.getFileInfo(src, true);
   }
   
   @Override // ClientProtocol
   public boolean isFileClosed(String src) throws IOException{
+    checkNNStartup();
     return namesystem.isFileClosed(src);
   }
   
   @Override // ClientProtocol
-  public HdfsFileStatus getFileLinkInfo(String src) throws IOException { 
+  public HdfsFileStatus getFileLinkInfo(String src) throws IOException {
+    checkNNStartup();
     metrics.incrFileInfoOps();
     return namesystem.getFileInfo(src, false);
   }
   
   @Override // ClientProtocol
   public long[] getStats() throws IOException {
+    checkNNStartup();
     namesystem.checkOperation(OperationCategory.READ);
     return namesystem.getStats();
   }
@@ -990,6 +1029,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   @Override // ClientProtocol
   public DatanodeInfo[] getDatanodeReport(DatanodeReportType type)
   throws IOException {
+    checkNNStartup();
     DatanodeInfo results[] = namesystem.datanodeReport(type);
     return results;
   }
@@ -997,6 +1037,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   @Override // ClientProtocol
   public DatanodeStorageReport[] getDatanodeStorageReport(
       DatanodeReportType type) throws IOException {
+    checkNNStartup();
     final DatanodeStorageReport[] reports = namesystem.getDatanodeStorageReport(type);
     return reports;
   }
@@ -1004,6 +1045,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   @Override // ClientProtocol
   public boolean setSafeMode(SafeModeAction action, boolean isChecked)
       throws IOException {
+    checkNNStartup();
     OperationCategory opCategory = OperationCategory.UNCHECKED;
     if (isChecked) {
       if (action == SafeModeAction.SAFEMODE_GET) {
@@ -1018,11 +1060,13 @@ class NameNodeRpcServer implements NamenodeProtocols {
 
   @Override // ClientProtocol
   public boolean restoreFailedStorage(String arg) throws IOException { 
+    checkNNStartup();
     return namesystem.restoreFailedStorage(arg);
   }
 
   @Override // ClientProtocol
   public void saveNamespace() throws IOException {
+    checkNNStartup();
     CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
     if (cacheEntry != null && cacheEntry.isSuccess()) {
       return; // Return previous response
@@ -1038,17 +1082,20 @@ class NameNodeRpcServer implements NamenodeProtocols {
   
   @Override // ClientProtocol
   public long rollEdits() throws AccessControlException, IOException {
+    checkNNStartup();
     CheckpointSignature sig = namesystem.rollEditLog();
     return sig.getCurSegmentTxId();
   }
 
   @Override // ClientProtocol
   public void refreshNodes() throws IOException {
+    checkNNStartup();
     namesystem.refreshNodes();
   }
 
   @Override // NamenodeProtocol
   public long getTransactionID() throws IOException {
+    checkNNStartup();
     namesystem.checkOperation(OperationCategory.UNCHECKED);
     namesystem.checkSuperuserPrivilege();
     return namesystem.getFSImage().getLastAppliedOrWrittenTxId();
@@ -1056,6 +1103,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   
   @Override // NamenodeProtocol
   public long getMostRecentCheckpointTxId() throws IOException {
+    checkNNStartup();
     namesystem.checkOperation(OperationCategory.UNCHECKED);
     namesystem.checkSuperuserPrivilege();
     return namesystem.getFSImage().getMostRecentCheckpointTxId();
@@ -1063,13 +1111,15 @@ class NameNodeRpcServer implements NamenodeProtocols {
   
   @Override // NamenodeProtocol
   public CheckpointSignature rollEditLog() throws IOException {
+    checkNNStartup();
     namesystem.checkSuperuserPrivilege();
     return namesystem.rollEditLog();
   }
   
   @Override // NamenodeProtocol
   public RemoteEditLogManifest getEditLogManifest(long sinceTxId)
-  throws IOException {
+      throws IOException {
+    checkNNStartup();
     namesystem.checkOperation(OperationCategory.READ);
     namesystem.checkSuperuserPrivilege();
     return namesystem.getEditLog().getEditLogManifest(sinceTxId);
@@ -1077,11 +1127,13 @@ class NameNodeRpcServer implements NamenodeProtocols {
     
   @Override // ClientProtocol
   public void finalizeUpgrade() throws IOException {
+    checkNNStartup();
     namesystem.finalizeUpgrade();
   }
 
   @Override // ClientProtocol
   public RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action) throws IOException {
+    checkNNStartup();
     LOG.info("rollingUpgrade " + action);
     switch(action) {
     case QUERY:
@@ -1098,12 +1150,14 @@ class NameNodeRpcServer implements NamenodeProtocols {
 
   @Override // ClientProtocol
   public void metaSave(String filename) throws IOException {
+    checkNNStartup();
     namesystem.metaSave(filename);
   }
 
   @Override // ClientProtocol
   public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie)
       throws IOException {
+    checkNNStartup();
     String[] cookieTab = new String[] { cookie };
     Collection<FSNamesystem.CorruptFileBlockInfo> fbs =
       namesystem.listCorruptFileBlocks(path, cookieTab);
@@ -1124,17 +1178,20 @@ class NameNodeRpcServer implements NamenodeProtocols {
    */
   @Override // ClientProtocol
   public void setBalancerBandwidth(long bandwidth) throws IOException {
+    checkNNStartup();
     namesystem.setBalancerBandwidth(bandwidth);
   }
   
   @Override // ClientProtocol
   public ContentSummary getContentSummary(String path) throws IOException {
+    checkNNStartup();
     return namesystem.getContentSummary(path);
   }
 
   @Override // ClientProtocol
   public void setQuota(String path, long namespaceQuota, long diskspaceQuota) 
       throws IOException {
+    checkNNStartup();
     namesystem.setQuota(path, namespaceQuota, diskspaceQuota);
   }
   
@@ -1142,18 +1199,21 @@ class NameNodeRpcServer implements NamenodeProtocols {
   public void fsync(String src, long fileId, String clientName,
                     long lastBlockLength)
       throws IOException {
+    checkNNStartup();
     namesystem.fsync(src, fileId, clientName, lastBlockLength);
   }
 
   @Override // ClientProtocol
   public void setTimes(String src, long mtime, long atime) 
       throws IOException {
+    checkNNStartup();
     namesystem.setTimes(src, mtime, atime);
   }
 
   @Override // ClientProtocol
   public void createSymlink(String target, String link, FsPermission dirPerms,
       boolean createParent) throws IOException {
+    checkNNStartup();
     CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
     if (cacheEntry != null && cacheEntry.isSuccess()) {
       return; // Return previous response
@@ -1184,6 +1244,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
 
   @Override // ClientProtocol
   public String getLinkTarget(String path) throws IOException {
+    checkNNStartup();
     metrics.incrGetLinkTargetOps();
     HdfsFileStatus stat = null;
     try {
@@ -1206,6 +1267,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   @Override // DatanodeProtocol
   public DatanodeRegistration registerDatanode(DatanodeRegistration nodeReg)
       throws IOException {
+    checkNNStartup();
     verifySoftwareVersion(nodeReg);
     namesystem.registerDatanode(nodeReg);
     return nodeReg;
@@ -1216,6 +1278,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
       StorageReport[] report, long dnCacheCapacity, long dnCacheUsed,
       int xmitsInProgress, int xceiverCount,
       int failedVolumes) throws IOException {
+    checkNNStartup();
     verifyRequest(nodeReg);
     return namesystem.handleHeartbeat(nodeReg, report,
         dnCacheCapacity, dnCacheUsed, xceiverCount, xmitsInProgress,
@@ -1225,6 +1288,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   @Override // DatanodeProtocol
   public DatanodeCommand blockReport(DatanodeRegistration nodeReg,
       String poolId, StorageBlockReport[] reports) throws IOException {
+    checkNNStartup();
     verifyRequest(nodeReg);
     if(blockStateChangeLog.isDebugEnabled()) {
       blockStateChangeLog.debug("*BLOCK* NameNode.blockReport: "
@@ -1256,6 +1320,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   @Override
   public DatanodeCommand cacheReport(DatanodeRegistration nodeReg,
       String poolId, List<Long> blockIds) throws IOException {
+    checkNNStartup();
     verifyRequest(nodeReg);
     if (blockStateChangeLog.isDebugEnabled()) {
       blockStateChangeLog.debug("*BLOCK* NameNode.cacheReport: "
@@ -1268,6 +1333,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   @Override // DatanodeProtocol
   public void blockReceivedAndDeleted(DatanodeRegistration nodeReg, String poolId,
       StorageReceivedDeletedBlocks[] receivedAndDeletedBlocks) throws IOException {
+    checkNNStartup();
     verifyRequest(nodeReg);
     metrics.incrBlockReceivedAndDeletedOps();
     if(blockStateChangeLog.isDebugEnabled()) {
@@ -1283,6 +1349,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   @Override // DatanodeProtocol
   public void errorReport(DatanodeRegistration nodeReg,
                           int errorCode, String msg) throws IOException { 
+    checkNNStartup();
     String dnName = 
        (nodeReg == null) ? "Unknown DataNode" : nodeReg.toString();
 
@@ -1304,6 +1371,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
     
   @Override // DatanodeProtocol, NamenodeProtocol
   public NamespaceInfo versionRequest() throws IOException {
+    checkNNStartup();
     namesystem.checkSuperuserPrivilege();
     return namesystem.getNamespaceInfo();
   }
@@ -1328,6 +1396,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
 
   @Override // RefreshAuthorizationPolicyProtocol
   public void refreshServiceAcl() throws IOException {
+    checkNNStartup();
     if (!serviceAuthEnabled) {
       throw new AuthorizationException("Service Level Authorization not enabled!");
     }
@@ -1378,28 +1447,32 @@ class NameNodeRpcServer implements NamenodeProtocols {
   }
 
   @Override // HAServiceProtocol
-  public synchronized void monitorHealth() 
-      throws HealthCheckFailedException, AccessControlException {
+  public synchronized void monitorHealth() throws HealthCheckFailedException,
+      AccessControlException, IOException {
+    checkNNStartup();
     nn.monitorHealth();
   }
   
   @Override // HAServiceProtocol
   public synchronized void transitionToActive(StateChangeRequestInfo req) 
-      throws ServiceFailedException, AccessControlException {
+      throws ServiceFailedException, AccessControlException, IOException {
+    checkNNStartup();
     nn.checkHaStateChange(req);
     nn.transitionToActive();
   }
   
   @Override // HAServiceProtocol
   public synchronized void transitionToStandby(StateChangeRequestInfo req) 
-      throws ServiceFailedException, AccessControlException {
+      throws ServiceFailedException, AccessControlException, IOException {
+    checkNNStartup();
     nn.checkHaStateChange(req);
     nn.transitionToStandby();
   }
 
   @Override // HAServiceProtocol
   public synchronized HAServiceStatus getServiceStatus() 
-      throws AccessControlException, ServiceFailedException {
+      throws AccessControlException, ServiceFailedException, IOException {
+    checkNNStartup();
     return nn.getServiceStatus();
   }
 
@@ -1456,12 +1529,14 @@ class NameNodeRpcServer implements NamenodeProtocols {
 
   @Override
   public DataEncryptionKey getDataEncryptionKey() throws IOException {
+    checkNNStartup();
     return namesystem.getBlockManager().generateDataEncryptionKey();
   }
 
   @Override
   public String createSnapshot(String snapshotRoot, String snapshotName)
       throws IOException {
+    checkNNStartup();
     if (!checkPathLength(snapshotRoot)) {
       throw new IOException("createSnapshot: Pathname too long.  Limit "
           + MAX_PATH_LENGTH + " characters, " + MAX_PATH_DEPTH + " levels.");
@@ -1486,6 +1561,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   @Override
   public void deleteSnapshot(String snapshotRoot, String snapshotName)
       throws IOException {
+    checkNNStartup();
     metrics.incrDeleteSnapshotOps();
     CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
     if (cacheEntry != null && cacheEntry.isSuccess()) {
@@ -1503,6 +1579,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   @Override
   // Client Protocol
   public void allowSnapshot(String snapshotRoot) throws IOException {
+    checkNNStartup();
     metrics.incrAllowSnapshotOps();
     namesystem.allowSnapshot(snapshotRoot);
   }
@@ -1510,13 +1587,16 @@ class NameNodeRpcServer implements NamenodeProtocols {
   @Override
   // Client Protocol
   public void disallowSnapshot(String snapshot) throws IOException {
+    checkNNStartup();
     metrics.incrDisAllowSnapshotOps();
     namesystem.disallowSnapshot(snapshot);
   }
 
   @Override
+  // ClientProtocol
   public void renameSnapshot(String snapshotRoot, String snapshotOldName,
       String snapshotNewName) throws IOException {
+    checkNNStartup();
     if (snapshotNewName == null || snapshotNewName.isEmpty()) {
       throw new IOException("The new snapshot name is null or empty.");
     }
@@ -1538,24 +1618,27 @@ class NameNodeRpcServer implements NamenodeProtocols {
   @Override // Client Protocol
   public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
       throws IOException {
+    checkNNStartup();
     SnapshottableDirectoryStatus[] status = namesystem
         .getSnapshottableDirListing();
     metrics.incrListSnapshottableDirOps();
     return status;
   }
 
-  @Override
+  @Override // ClientProtocol
   public SnapshotDiffReport getSnapshotDiffReport(String snapshotRoot,
       String earlierSnapshotName, String laterSnapshotName) throws IOException {
+    checkNNStartup();
     SnapshotDiffReport report = namesystem.getSnapshotDiffReport(snapshotRoot,
         earlierSnapshotName, laterSnapshotName);
     metrics.incrSnapshotDiffReportOps();
     return report;
   }
 
-  @Override
+  @Override // ClientProtocol
   public long addCacheDirective(
       CacheDirectiveInfo path, EnumSet<CacheFlag> flags) throws IOException {
+    checkNNStartup();
     CacheEntryWithPayload cacheEntry = RetryCache.waitForCompletion
       (retryCache, null);
     if (cacheEntry != null && cacheEntry.isSuccess()) {
@@ -1573,9 +1656,10 @@ class NameNodeRpcServer implements NamenodeProtocols {
     return ret;
   }
 
-  @Override
+  @Override // ClientProtocol
   public void modifyCacheDirective(
       CacheDirectiveInfo directive, EnumSet<CacheFlag> flags) throws IOException {
+    checkNNStartup();
     CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
     if (cacheEntry != null && cacheEntry.isSuccess()) {
       return;
@@ -1590,8 +1674,9 @@ class NameNodeRpcServer implements NamenodeProtocols {
     }
   }
 
-  @Override
+  @Override // ClientProtocol
   public void removeCacheDirective(long id) throws IOException {
+    checkNNStartup();
     CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
     if (cacheEntry != null && cacheEntry.isSuccess()) {
       return;
@@ -1605,17 +1690,19 @@ class NameNodeRpcServer implements NamenodeProtocols {
     }
   }
 
-  @Override
+  @Override // ClientProtocol
   public BatchedEntries<CacheDirectiveEntry> listCacheDirectives(long prevId,
       CacheDirectiveInfo filter) throws IOException {
+    checkNNStartup();
     if (filter == null) {
       filter = new CacheDirectiveInfo.Builder().build();
     }
     return namesystem.listCacheDirectives(prevId, filter);
   }
 
-  @Override
+  @Override //ClientProtocol
   public void addCachePool(CachePoolInfo info) throws IOException {
+    checkNNStartup();
     CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
     if (cacheEntry != null && cacheEntry.isSuccess()) {
       return; // Return previous response
@@ -1629,8 +1716,9 @@ class NameNodeRpcServer implements NamenodeProtocols {
     }
   }
 
-  @Override
+  @Override // ClientProtocol
   public void modifyCachePool(CachePoolInfo info) throws IOException {
+    checkNNStartup();
     CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
     if (cacheEntry != null && cacheEntry.isSuccess()) {
       return; // Return previous response
@@ -1644,8 +1732,9 @@ class NameNodeRpcServer implements NamenodeProtocols {
     }
   }
 
-  @Override
+  @Override // ClientProtocol
   public void removeCachePool(String cachePoolName) throws IOException {
+    checkNNStartup();
     CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
     if (cacheEntry != null && cacheEntry.isSuccess()) {
       return;
@@ -1659,47 +1748,55 @@ class NameNodeRpcServer implements NamenodeProtocols {
     }
   }
 
-  @Override
+  @Override // ClientProtocol
   public BatchedEntries<CachePoolEntry> listCachePools(String prevKey)
       throws IOException {
+    checkNNStartup();
     return namesystem.listCachePools(prevKey != null ? prevKey : "");
   }
 
-  @Override
+  @Override // ClientProtocol
   public void modifyAclEntries(String src, List<AclEntry> aclSpec)
       throws IOException {
+    checkNNStartup();
     namesystem.modifyAclEntries(src, aclSpec);
   }
 
-  @Override
+  @Override // ClienProtocol
   public void removeAclEntries(String src, List<AclEntry> aclSpec)
       throws IOException {
+    checkNNStartup();
     namesystem.removeAclEntries(src, aclSpec);
   }
 
-  @Override
+  @Override // ClientProtocol
   public void removeDefaultAcl(String src) throws IOException {
+    checkNNStartup();
     namesystem.removeDefaultAcl(src);
   }
 
-  @Override
+  @Override // ClientProtocol
   public void removeAcl(String src) throws IOException {
+    checkNNStartup();
     namesystem.removeAcl(src);
   }
 
-  @Override
+  @Override // ClientProtocol
   public void setAcl(String src, List<AclEntry> aclSpec) throws IOException {
+    checkNNStartup();
     namesystem.setAcl(src, aclSpec);
   }
 
-  @Override
+  @Override // ClientProtocol
   public AclStatus getAclStatus(String src) throws IOException {
+    checkNNStartup();
     return namesystem.getAclStatus(src);
   }
   
-  @Override
+  @Override // ClientProtocol
   public void createEncryptionZone(String src, String keyName)
     throws IOException {
+    checkNNStartup();
     final CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
     if (cacheEntry != null && cacheEntry.isSuccess()) {
       return;
@@ -1713,21 +1810,24 @@ class NameNodeRpcServer implements NamenodeProtocols {
     }
   }
 
-  @Override
+  @Override // ClientProtocol
   public EncryptionZone getEZForPath(String src)
     throws IOException {
+    checkNNStartup();
     return namesystem.getEZForPath(src);
   }
 
-  @Override
+  @Override // ClientProtocol
   public BatchedEntries<EncryptionZone> listEncryptionZones(
       long prevId) throws IOException {
+    checkNNStartup();
     return namesystem.listEncryptionZones(prevId);
   }
 
-  @Override
+  @Override // ClientProtocol
   public void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag)
       throws IOException {
+    checkNNStartup();
     CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
     if (cacheEntry != null && cacheEntry.isSuccess()) {
       return; // Return previous response
@@ -1741,19 +1841,22 @@ class NameNodeRpcServer implements NamenodeProtocols {
     }
   }
   
-  @Override
+  @Override // ClientProtocol
   public List<XAttr> getXAttrs(String src, List<XAttr> xAttrs) 
       throws IOException {
+    checkNNStartup();
     return namesystem.getXAttrs(src, xAttrs);
   }
 
-  @Override
+  @Override // ClientProtocol
   public List<XAttr> listXAttrs(String src) throws IOException {
+    checkNNStartup();
     return namesystem.listXAttrs(src);
   }
   
-  @Override
+  @Override // ClientProtocol
   public void removeXAttr(String src, XAttr xAttr) throws IOException {
+    checkNNStartup();
     CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
     if (cacheEntry != null && cacheEntry.isSuccess()) {
       return; // Return previous response
@@ -1767,13 +1870,21 @@ class NameNodeRpcServer implements NamenodeProtocols {
     }
   }
 
-  @Override
+  private void checkNNStartup() throws IOException {
+    if (!this.nn.isStarted()) {
+      throw new IOException(this.nn.getRole() + " still not started");
+    }
+  }
+
+  @Override // ClientProtocol
   public void checkAccess(String path, FsAction mode) throws IOException {
+    checkNNStartup();
     namesystem.checkAccess(path, mode);
   }
 
   @Override // ClientProtocol
   public long getCurrentEditLogTxid() throws IOException {
+    checkNNStartup();
     namesystem.checkOperation(OperationCategory.READ); // only active
     namesystem.checkSuperuserPrivilege();
     // if it's not yet open for write, we may be in the process of transitioning
@@ -1802,6 +1913,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
 
   @Override // ClientProtocol
   public EventBatchList getEditsFromTxid(long txid) throws IOException {
+    checkNNStartup();
     namesystem.checkOperation(OperationCategory.READ); // only active
     namesystem.checkSuperuserPrivilege();
     int maxEventsPerRPC = nn.conf.getInt(
@@ -1885,20 +1997,23 @@ class NameNodeRpcServer implements NamenodeProtocols {
     return new EventBatchList(batches, firstSeenTxid, maxSeenTxid, syncTxid);
   }
 
-  @Override
+  @Override // TraceAdminProtocol
   public SpanReceiverInfo[] listSpanReceivers() throws IOException {
+    checkNNStartup();
     namesystem.checkSuperuserPrivilege();
     return nn.spanReceiverHost.listSpanReceivers();
   }
 
-  @Override
+  @Override // TraceAdminProtocol
   public long addSpanReceiver(SpanReceiverInfo info) throws IOException {
+    checkNNStartup();
     namesystem.checkSuperuserPrivilege();
     return nn.spanReceiverHost.addSpanReceiver(info);
   }
 
-  @Override
+  @Override // TraceAdminProtocol
   public void removeSpanReceiver(long id) throws IOException {
+    checkNNStartup();
     namesystem.checkSuperuserPrivilege();
     nn.spanReceiverHost.removeSpanReceiver(id);
   }


[38/50] [abbrv] hadoop git commit: HDFS-3750. API docs don't include HDFS (Jolly Chen via aw)

Posted by zh...@apache.org.
HDFS-3750. API docs don't include HDFS (Jolly Chen via aw)


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

Branch: refs/heads/HDFS-EC
Commit: cc9ed5249769c91e216bef48763e084578f15d35
Parents: f3c4294
Author: Allen Wittenauer <aw...@apache.org>
Authored: Fri Jan 23 14:10:44 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:30 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 2 ++
 pom.xml                                     | 6 +++++-
 2 files changed, 7 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc9ed524/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 6849229..053b2eb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -293,6 +293,8 @@ Trunk (Unreleased)
     HDFS-7667. Various typos and improvements to HDFS Federation doc
     (Charles Lamb via aw)
 
+    HDFS-3750. API docs don't include HDFS (Jolly Chen via aw)
+
 Release 2.7.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc9ed524/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 72ca43d..e33865f 100644
--- a/pom.xml
+++ b/pom.xml
@@ -367,13 +367,17 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
               <reportOutputDirectory>${project.build.directory}/site</reportOutputDirectory>
               <destDir>hadoop-project/api</destDir>
               <!-- Non-public APIs -->
-              <excludePackageNames>org.apache.hadoop.authentication*,org.apache.hadoop.hdfs*,org.apache.hadoop.mapreduce.v2.proto,org.apache.hadoop.yarn.proto,org.apache.hadoop.yarn.server*,org.apache.hadoop.yarn.webapp*</excludePackageNames>
+              <excludePackageNames>org.apache.hadoop.authentication*,org.apache.hadoop.mapreduce.v2.proto,org.apache.hadoop.yarn.proto,org.apache.hadoop.yarn.server*,org.apache.hadoop.yarn.webapp*</excludePackageNames>
               <groups>
                 <group>
                   <title>Common</title>
                   <packages>org.apache.hadoop*</packages>
                 </group>
                 <group>
+                  <title>HDFS</title>
+                  <packages>org.apache.hadoop.hdfs*</packages>
+                </group>
+                <group>
                   <title>MapReduce</title>
                   <packages>org.apache.hadoop.mapred*</packages>
                 </group>


[49/50] [abbrv] hadoop git commit: YARN-3088. LinuxContainerExecutor.deleteAsUser can throw NPE if native executor returns an error. Contributed by Eric Payne

Posted by zh...@apache.org.
YARN-3088. LinuxContainerExecutor.deleteAsUser can throw NPE if native executor returns an error. Contributed by Eric Payne


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

Branch: refs/heads/HDFS-EC
Commit: b327379b2aa5da9a48ff5258df1760d6523f04d7
Parents: 21f5c51
Author: Jason Lowe <jl...@apache.org>
Authored: Mon Jan 26 15:40:21 2015 +0000
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:31 2015 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../nodemanager/LinuxContainerExecutor.java     |  9 +++-
 .../TestLinuxContainerExecutorWithMocks.java    | 49 +++++++++++++++++++-
 3 files changed, 58 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b327379b/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 0808678..872f16e 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -409,6 +409,9 @@ Release 2.7.0 - UNRELEASED
     YARN-3082. Non thread safe access to systemCredentials in NodeHeartbeatResponse
     processing. (Anubhav Dhoot via ozawa)
 
+    YARN-3088. LinuxContainerExecutor.deleteAsUser can throw NPE if native
+    executor returns an error (Eric Payne via jlowe)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b327379b/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 4606f0c..d6e6894 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
@@ -392,18 +392,23 @@ public class LinuxContainerExecutor extends ContainerExecutor {
     verifyUsernamePattern(user);
     String runAsUser = getRunAsUser(user);
 
+    String dirString = dir == null ? "" : dir.toUri().getPath();
+
     List<String> command = new ArrayList<String>(
         Arrays.asList(containerExecutorExe,
                     runAsUser,
                     user,
                     Integer.toString(Commands.DELETE_AS_USER.getValue()),
-                    dir == null ? "" : dir.toUri().getPath()));
+                    dirString));
+    List<String> pathsToDelete = new ArrayList<String>();
     if (baseDirs == null || baseDirs.length == 0) {
       LOG.info("Deleting absolute path : " + dir);
+      pathsToDelete.add(dirString);
     } else {
       for (Path baseDir : baseDirs) {
         Path del = dir == null ? baseDir : new Path(baseDir, dir);
         LOG.info("Deleting path : " + del);
+        pathsToDelete.add(del.toString());
         command.add(baseDir.toUri().getPath());
       }
     }
@@ -419,7 +424,7 @@ public class LinuxContainerExecutor extends ContainerExecutor {
       }
     } catch (IOException e) {
       int exitCode = shExec.getExitCode();
-      LOG.error("DeleteAsUser for " + dir.toUri().getPath()
+      LOG.error("DeleteAsUser for " + StringUtils.join(" ", pathsToDelete)
           + " returned with exit code: " + exitCode, e);
       LOG.error("Output from LinuxContainerExecutor's deleteAsUser follows:");
       logOutput(shExec.getOutput());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b327379b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestLinuxContainerExecutorWithMocks.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/TestLinuxContainerExecutorWithMocks.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestLinuxContainerExecutorWithMocks.java
index d54367a..98ab8e0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestLinuxContainerExecutorWithMocks.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestLinuxContainerExecutorWithMocks.java
@@ -319,10 +319,57 @@ public class TestLinuxContainerExecutorWithMocks {
     String cmd = String.valueOf(
         LinuxContainerExecutor.Commands.DELETE_AS_USER.getValue());
     Path dir = new Path("/tmp/testdir");
-    
+    Path testFile = new Path("testfile");
+    Path baseDir0 = new Path("/grid/0/BaseDir");
+    Path baseDir1 = new Path("/grid/1/BaseDir");
+
+    mockExec.deleteAsUser(appSubmitter, dir);
+    assertEquals(Arrays.asList(YarnConfiguration.DEFAULT_NM_NONSECURE_MODE_LOCAL_USER,
+        appSubmitter, cmd, "/tmp/testdir"),
+        readMockParams());
+
+    mockExec.deleteAsUser(appSubmitter, null);
+    assertEquals(Arrays.asList(YarnConfiguration.DEFAULT_NM_NONSECURE_MODE_LOCAL_USER,
+        appSubmitter, cmd, ""),
+        readMockParams());
+
+    mockExec.deleteAsUser(appSubmitter, testFile, baseDir0, baseDir1);
+    assertEquals(Arrays.asList(YarnConfiguration.DEFAULT_NM_NONSECURE_MODE_LOCAL_USER,
+        appSubmitter, cmd, testFile.toString(), baseDir0.toString(), baseDir1.toString()),
+        readMockParams());
+
+    mockExec.deleteAsUser(appSubmitter, null, baseDir0, baseDir1);
+    assertEquals(Arrays.asList(YarnConfiguration.DEFAULT_NM_NONSECURE_MODE_LOCAL_USER,
+        appSubmitter, cmd, "", baseDir0.toString(), baseDir1.toString()),
+        readMockParams());
+
+    File f = new File("./src/test/resources/mock-container-executer-with-error");
+    if (!FileUtil.canExecute(f)) {
+      FileUtil.setExecutable(f, true);
+    }
+    String executorPath = f.getAbsolutePath();
+    Configuration conf = new Configuration();
+    conf.set(YarnConfiguration.NM_LINUX_CONTAINER_EXECUTOR_PATH, executorPath);
+    mockExec.setConf(conf);
+
     mockExec.deleteAsUser(appSubmitter, dir);
     assertEquals(Arrays.asList(YarnConfiguration.DEFAULT_NM_NONSECURE_MODE_LOCAL_USER,
         appSubmitter, cmd, "/tmp/testdir"),
         readMockParams());
+
+    mockExec.deleteAsUser(appSubmitter, null);
+    assertEquals(Arrays.asList(YarnConfiguration.DEFAULT_NM_NONSECURE_MODE_LOCAL_USER,
+        appSubmitter, cmd, ""),
+        readMockParams());
+
+    mockExec.deleteAsUser(appSubmitter, testFile, baseDir0, baseDir1);
+    assertEquals(Arrays.asList(YarnConfiguration.DEFAULT_NM_NONSECURE_MODE_LOCAL_USER,
+        appSubmitter, cmd, testFile.toString(), baseDir0.toString(), baseDir1.toString()),
+        readMockParams());
+
+    mockExec.deleteAsUser(appSubmitter, null, baseDir0, baseDir1);
+    assertEquals(Arrays.asList(YarnConfiguration.DEFAULT_NM_NONSECURE_MODE_LOCAL_USER,
+        appSubmitter, cmd, "", baseDir0.toString(), baseDir1.toString()),
+        readMockParams());
   }
 }


[43/50] [abbrv] hadoop git commit: HDFS-7659. truncate should check negative value of the new length. Contributed by Yi Liu.

Posted by zh...@apache.org.
HDFS-7659. truncate should check negative value of the new length. Contributed by Yi Liu.


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

Branch: refs/heads/HDFS-EC
Commit: 27c98ccab3801456fb5fdc7f8414490ce7ada695
Parents: f8b1ce9
Author: yliu <yl...@apache.org>
Authored: Sat Jan 24 15:41:06 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:30 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                 | 3 +++
 .../src/main/java/org/apache/hadoop/hdfs/DFSClient.java     | 4 ++++
 .../apache/hadoop/hdfs/server/namenode/FSNamesystem.java    | 4 ++++
 .../hadoop/hdfs/server/namenode/TestFileTruncate.java       | 9 +++++++++
 4 files changed, 20 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/27c98cca/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 08f705a..cca755e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -143,6 +143,9 @@ Trunk (Unreleased)
     HDFS-7430. Rewrite the BlockScanner to use O(1) memory and use multiple
     threads (cmccabe)
 
+    HDFS-7659. truncate should check negative value of the new length.
+    (Yi Liu via shv)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/27c98cca/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index 1bb7f4a..21f75a5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -1984,6 +1984,10 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   public boolean truncate(String src, long newLength) throws IOException {
     checkOpen();
+    if (newLength < 0) {
+      throw new HadoopIllegalArgumentException(
+          "Cannot truncate to a negative file size: " + newLength + ".");
+    }
     TraceScope scope = getPathTraceScope("truncate", src);
     try {
       return namenode.truncate(src, newLength, clientName);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/27c98cca/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 6a8f574..fae1641 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
@@ -1911,6 +1911,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       NameNode.stateChangeLog.debug("DIR* NameSystem.truncate: src="
           + src + " newLength=" + newLength);
     }
+    if (newLength < 0) {
+      throw new HadoopIllegalArgumentException(
+          "Cannot truncate to a negative file size: " + newLength + ".");
+    }
     HdfsFileStatus stat = null;
     FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.WRITE);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/27c98cca/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 5498b12..1612a24 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
@@ -34,6 +34,7 @@ import java.io.IOException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -443,6 +444,14 @@ public class TestFileTruncate {
     } catch(IOException expected) {}
     out.close();
 
+    try {
+      fs.truncate(p, -1);
+      fail("Truncate must fail for a negative new length.");
+    } catch (HadoopIllegalArgumentException expected) {
+      GenericTestUtils.assertExceptionContains(
+          "Cannot truncate to a negative file size", expected);
+    }
+
     cluster.shutdownDataNodes();
     NameNodeAdapter.getLeaseManager(cluster.getNamesystem())
         .setLeasePeriod(LOW_SOFTLIMIT, LOW_HARDLIMIT);


[15/50] [abbrv] hadoop git commit: HADOOP-11484. hadoop-mapreduce-client-nativetask fails to build on ARM AARCH64 due to x86 asm statements (Edward Nevill via Colin P. McCabe)

Posted by zh...@apache.org.
HADOOP-11484. hadoop-mapreduce-client-nativetask fails to build on ARM AARCH64 due to x86 asm statements (Edward Nevill via Colin P. McCabe)


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

Branch: refs/heads/HDFS-EC
Commit: a865d7c4f9b3ecc6c7fb52edb50a70b5cd326f6d
Parents: cf76280
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Wed Jan 21 11:24:09 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:26 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                  | 3 +++
 .../src/main/native/src/lib/primitives.h                     | 8 ++++++++
 .../src/main/native/src/util/Checksum.cc                     | 5 +++++
 3 files changed, 16 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a865d7c4/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 1801d2f..3bd67fe 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -753,6 +753,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7610. Fix removal of dynamically added DN volumes (Lei (Eddy) Xu via
     Colin P. McCabe)
 
+    HADOOP-11484. hadoop-mapreduce-client-nativetask fails to build on ARM
+    AARCH64 due to x86 asm statements (Edward Nevill via Colin P. McCabe)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a865d7c4/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/primitives.h
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/primitives.h b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/primitives.h
index 4c0c1a7..3bf5f76 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/primitives.h
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/primitives.h
@@ -97,11 +97,18 @@ inline void simple_memcpy(void * dest, const void * src, size_t len) {
  * little-endian to big-endian or vice versa
  */
 inline uint32_t bswap(uint32_t val) {
+#ifdef __aarch64__
+  __asm__("rev %w[dst], %w[src]" : [dst]"=r"(val) : [src]"r"(val));
+#else
   __asm__("bswap %0" : "=r" (val) : "0" (val));
+#endif
   return val;
 }
 
 inline uint64_t bswap64(uint64_t val) {
+#ifdef __aarch64__
+  __asm__("rev %[dst], %[src]" : [dst]"=r"(val) : [src]"r"(val));
+#else
 #ifdef __X64
   __asm__("bswapq %0" : "=r" (val) : "0" (val));
 #else
@@ -115,6 +122,7 @@ inline uint64_t bswap64(uint64_t val) {
   return (lower << 32) + higher;
 
 #endif
+#endif
   return val;
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a865d7c4/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/util/Checksum.cc
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/util/Checksum.cc b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/util/Checksum.cc
index 191e093..be800c5 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/util/Checksum.cc
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/util/Checksum.cc
@@ -579,6 +579,11 @@ const uint32_t CRC32C_T8_7[256] = {0x00000000, 0x493C7D27, 0x9278FA4E, 0xDB44876
     0xCF56CE31, 0x14124958, 0x5D2E347F, 0xE54C35A1, 0xAC704886, 0x7734CFEF, 0x3E08B2C8, 0xC451B7CC,
     0x8D6DCAEB, 0x56294D82, 0x1F1530A5};
 
+#ifdef __aarch64__
+// Awaiting HW implementation
+#define SOFTWARE_CRC
+#endif
+
 #ifndef SOFTWARE_CRC
 #define USE_HARDWARE_CRC32C 1
 #endif


[46/50] [abbrv] hadoop git commit: YARN-3024. LocalizerRunner should give DIE action when all resources are localized. Contributed by Chengbing Liu

Posted by zh...@apache.org.
YARN-3024. LocalizerRunner should give DIE action when all resources are
localized. Contributed by Chengbing Liu


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

Branch: refs/heads/HDFS-EC
Commit: f343538850878d208864af3faea2b430d9ab0a47
Parents: 5adb125
Author: Xuan <xg...@apache.org>
Authored: Sun Jan 25 19:37:57 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:31 2015 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +
 .../localizer/ResourceLocalizationService.java  | 99 ++++++++------------
 .../TestResourceLocalizationService.java        | 71 +++++++++-----
 3 files changed, 91 insertions(+), 82 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f3435388/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 7263c6f..0808678 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -200,6 +200,9 @@ Release 2.7.0 - UNRELEASED
     YARN-2800. Remove MemoryNodeLabelsStore and add a way to enable/disable
     node labels feature. (Wangda Tan via ozawa)
 
+    YARN-3024. LocalizerRunner should give DIE action when all resources are
+    localized. (Chengbing Liu via xgong)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f3435388/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.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/localizer/ResourceLocalizationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
index 5440980..2f4fa5e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
@@ -763,7 +763,7 @@ public class ResourceLocalizationService extends CompositeService
        */
 
       if (rsrc.tryAcquire()) {
-        if (rsrc.getState().equals(ResourceState.DOWNLOADING)) {
+        if (rsrc.getState() == ResourceState.DOWNLOADING) {
           LocalResource resource = request.getResource().getRequest();
           try {
             Path publicRootPath =
@@ -895,7 +895,7 @@ public class ResourceLocalizationService extends CompositeService
          LocalizedResource nRsrc = evt.getResource();
          // Resource download should take place ONLY if resource is in
          // Downloading state
-         if (!ResourceState.DOWNLOADING.equals(nRsrc.getState())) {
+         if (nRsrc.getState() != ResourceState.DOWNLOADING) {
            i.remove();
            continue;
          }
@@ -906,7 +906,7 @@ public class ResourceLocalizationService extends CompositeService
           * 2) Resource is still in DOWNLOADING state
           */
          if (nRsrc.tryAcquire()) {
-           if (nRsrc.getState().equals(ResourceState.DOWNLOADING)) {
+           if (nRsrc.getState() == ResourceState.DOWNLOADING) {
              LocalResourceRequest nextRsrc = nRsrc.getRequest();
              LocalResource next =
                  recordFactory.newRecordInstance(LocalResource.class);
@@ -936,41 +936,9 @@ public class ResourceLocalizationService extends CompositeService
       String user = context.getUser();
       ApplicationId applicationId =
           context.getContainerId().getApplicationAttemptId().getApplicationId();
-      // The localizer has just spawned. Start giving it resources for
-      // remote-fetching.
-      if (remoteResourceStatuses.isEmpty()) {
-        LocalResource next = findNextResource();
-        if (next != null) {
-          response.setLocalizerAction(LocalizerAction.LIVE);
-          try {
-            ArrayList<ResourceLocalizationSpec> rsrcs =
-                new ArrayList<ResourceLocalizationSpec>();
-            ResourceLocalizationSpec rsrc =
-                NodeManagerBuilderUtils.newResourceLocalizationSpec(next,
-                  getPathForLocalization(next));
-            rsrcs.add(rsrc);
-            response.setResourceSpecs(rsrcs);
-          } catch (IOException e) {
-            LOG.error("local path for PRIVATE localization could not be found."
-                + "Disks might have failed.", e);
-          } catch (URISyntaxException e) {
-            // TODO fail? Already translated several times...
-          }
-        } else if (pending.isEmpty()) {
-          // TODO: Synchronization
-          response.setLocalizerAction(LocalizerAction.DIE);
-        } else {
-          response.setLocalizerAction(LocalizerAction.LIVE);
-        }
-        return response;
-      }
-      ArrayList<ResourceLocalizationSpec> rsrcs =
-          new ArrayList<ResourceLocalizationSpec>();
-       /*
-        * TODO : It doesn't support multiple downloads per ContainerLocalizer
-        * at the same time. We need to think whether we should support this.
-        */
 
+      LocalizerAction action = LocalizerAction.LIVE;
+      // Update resource statuses.
       for (LocalResourceStatus stat : remoteResourceStatuses) {
         LocalResource rsrc = stat.getResource();
         LocalResourceRequest req = null;
@@ -999,30 +967,8 @@ public class ResourceLocalizationService extends CompositeService
             // list
             assoc.getResource().unlock();
             scheduled.remove(req);
-            
-            if (pending.isEmpty()) {
-              // TODO: Synchronization
-              response.setLocalizerAction(LocalizerAction.DIE);
-              break;
-            }
-            response.setLocalizerAction(LocalizerAction.LIVE);
-            LocalResource next = findNextResource();
-            if (next != null) {
-              try {
-                ResourceLocalizationSpec resource =
-                    NodeManagerBuilderUtils.newResourceLocalizationSpec(next,
-                      getPathForLocalization(next));
-                rsrcs.add(resource);
-              } catch (IOException e) {
-                LOG.error("local path for PRIVATE localization could not be " +
-                  "found. Disks might have failed.", e);
-              } catch (URISyntaxException e) {
-                  //TODO fail? Already translated several times...
-              }
-            }
             break;
           case FETCH_PENDING:
-            response.setLocalizerAction(LocalizerAction.LIVE);
             break;
           case FETCH_FAILURE:
             final String diagnostics = stat.getException().toString();
@@ -1036,17 +982,48 @@ public class ResourceLocalizationService extends CompositeService
             // list
             assoc.getResource().unlock();
             scheduled.remove(req);
-            
             break;
           default:
             LOG.info("Unknown status: " + stat.getStatus());
-            response.setLocalizerAction(LocalizerAction.DIE);
+            action = LocalizerAction.DIE;
             getLocalResourcesTracker(req.getVisibility(), user, applicationId)
               .handle(new ResourceFailedLocalizationEvent(
                   req, stat.getException().getMessage()));
             break;
         }
       }
+      if (action == LocalizerAction.DIE) {
+        response.setLocalizerAction(action);
+        return response;
+      }
+
+      // Give the localizer resources for remote-fetching.
+      List<ResourceLocalizationSpec> rsrcs =
+          new ArrayList<ResourceLocalizationSpec>();
+
+      /*
+       * TODO : It doesn't support multiple downloads per ContainerLocalizer
+       * at the same time. We need to think whether we should support this.
+       */
+      LocalResource next = findNextResource();
+      if (next != null) {
+        try {
+          ResourceLocalizationSpec resource =
+              NodeManagerBuilderUtils.newResourceLocalizationSpec(next,
+                getPathForLocalization(next));
+          rsrcs.add(resource);
+        } catch (IOException e) {
+          LOG.error("local path for PRIVATE localization could not be " +
+            "found. Disks might have failed.", e);
+        } catch (URISyntaxException e) {
+            //TODO fail? Already translated several times...
+        }
+      } else if (pending.isEmpty()) {
+        // TODO: Synchronization
+        action = LocalizerAction.DIE;
+      }
+
+      response.setLocalizerAction(action);
       response.setResourceSpecs(rsrcs);
       return response;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f3435388/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.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/localizer/TestResourceLocalizationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java
index f968bb9..9ed18dd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java
@@ -827,10 +827,16 @@ public class TestResourceLocalizationService {
       do {
         resource2 = getPrivateMockedResource(r);
       } while (resource2 == null || resource2.equals(resource1));
+      LocalResource resource3 = null;
+      do {
+        resource3 = getPrivateMockedResource(r);
+      } while (resource3 == null || resource3.equals(resource1)
+          || resource3.equals(resource2));
       // above call to make sure we don't get identical resources.
       
       final LocalResourceRequest req1 = new LocalResourceRequest(resource1);
       final LocalResourceRequest req2 = new LocalResourceRequest(resource2);
+      final LocalResourceRequest req3 = new LocalResourceRequest(resource3);
       Map<LocalResourceVisibility, Collection<LocalResourceRequest>> rsrcs =
         new HashMap<LocalResourceVisibility, 
                     Collection<LocalResourceRequest>>();
@@ -838,6 +844,7 @@ public class TestResourceLocalizationService {
           new ArrayList<LocalResourceRequest>();
       privateResourceList.add(req1);
       privateResourceList.add(req2);
+      privateResourceList.add(req3);
       rsrcs.put(LocalResourceVisibility.PRIVATE, privateResourceList);
       spyService.handle(new ContainerLocalizationRequestEvent(c, rsrcs));
       // Sigh. Thread init of private localizer not accessible
@@ -852,30 +859,47 @@ public class TestResourceLocalizationService {
       Path localizationTokenPath = tokenPathCaptor.getValue();
 
       // heartbeat from localizer
-      LocalResourceStatus rsrcStat1 = mock(LocalResourceStatus.class);
-      LocalResourceStatus rsrcStat2 = mock(LocalResourceStatus.class);
+      LocalResourceStatus rsrc1success = mock(LocalResourceStatus.class);
+      LocalResourceStatus rsrc2pending = mock(LocalResourceStatus.class);
+      LocalResourceStatus rsrc2success = mock(LocalResourceStatus.class);
+      LocalResourceStatus rsrc3success = mock(LocalResourceStatus.class);
       LocalizerStatus stat = mock(LocalizerStatus.class);
       when(stat.getLocalizerId()).thenReturn(ctnrStr);
-      when(rsrcStat1.getResource()).thenReturn(resource1);
-      when(rsrcStat2.getResource()).thenReturn(resource2);
-      when(rsrcStat1.getLocalSize()).thenReturn(4344L);
-      when(rsrcStat2.getLocalSize()).thenReturn(2342L);
+      when(rsrc1success.getResource()).thenReturn(resource1);
+      when(rsrc2pending.getResource()).thenReturn(resource2);
+      when(rsrc2success.getResource()).thenReturn(resource2);
+      when(rsrc3success.getResource()).thenReturn(resource3);
+      when(rsrc1success.getLocalSize()).thenReturn(4344L);
+      when(rsrc2success.getLocalSize()).thenReturn(2342L);
+      when(rsrc3success.getLocalSize()).thenReturn(5345L);
       URL locPath = getPath("/cache/private/blah");
-      when(rsrcStat1.getLocalPath()).thenReturn(locPath);
-      when(rsrcStat2.getLocalPath()).thenReturn(locPath);
-      when(rsrcStat1.getStatus()).thenReturn(ResourceStatusType.FETCH_SUCCESS);
-      when(rsrcStat2.getStatus()).thenReturn(ResourceStatusType.FETCH_SUCCESS);
+      when(rsrc1success.getLocalPath()).thenReturn(locPath);
+      when(rsrc2success.getLocalPath()).thenReturn(locPath);
+      when(rsrc3success.getLocalPath()).thenReturn(locPath);
+      when(rsrc1success.getStatus()).thenReturn(ResourceStatusType.FETCH_SUCCESS);
+      when(rsrc2pending.getStatus()).thenReturn(ResourceStatusType.FETCH_PENDING);
+      when(rsrc2success.getStatus()).thenReturn(ResourceStatusType.FETCH_SUCCESS);
+      when(rsrc3success.getStatus()).thenReturn(ResourceStatusType.FETCH_SUCCESS);
+
+      // Four heartbeats with sending:
+      // 1 - empty
+      // 2 - resource1 FETCH_SUCCESS
+      // 3 - resource2 FETCH_PENDING
+      // 4 - resource2 FETCH_SUCCESS, resource3 FETCH_SUCCESS
+      List<LocalResourceStatus> rsrcs4 = new ArrayList<LocalResourceStatus>();
+      rsrcs4.add(rsrc2success);
+      rsrcs4.add(rsrc3success);
       when(stat.getResources())
         .thenReturn(Collections.<LocalResourceStatus>emptyList())
-        .thenReturn(Collections.singletonList(rsrcStat1))
-        .thenReturn(Collections.singletonList(rsrcStat2))
-        .thenReturn(Collections.<LocalResourceStatus>emptyList());
+        .thenReturn(Collections.singletonList(rsrc1success))
+        .thenReturn(Collections.singletonList(rsrc2pending))
+        .thenReturn(rsrcs4);
 
       String localPath = Path.SEPARATOR + ContainerLocalizer.USERCACHE +
           Path.SEPARATOR + "user0" + Path.SEPARATOR +
           ContainerLocalizer.FILECACHE;
-      
-      // get first resource
+
+      // First heartbeat
       LocalizerHeartbeatResponse response = spyService.heartbeat(stat);
       assertEquals(LocalizerAction.LIVE, response.getLocalizerAction());
       assertEquals(1, response.getResourceSpecs().size());
@@ -888,7 +912,7 @@ public class TestResourceLocalizationService {
       assertTrue(localizedPath.getFile().endsWith(
         localPath + Path.SEPARATOR + "10"));
 
-      // get second resource
+      // Second heartbeat
       response = spyService.heartbeat(stat);
       assertEquals(LocalizerAction.LIVE, response.getLocalizerAction());
       assertEquals(1, response.getResourceSpecs().size());
@@ -902,16 +926,21 @@ public class TestResourceLocalizationService {
       assertTrue(localizedPath.getFile().endsWith(
         localPath + Path.SEPARATOR + "0" + Path.SEPARATOR + "11"));
 
-      // empty rsrc
+      // Third heartbeat
       response = spyService.heartbeat(stat);
       assertEquals(LocalizerAction.LIVE, response.getLocalizerAction());
-      assertEquals(0, response.getResourceSpecs().size());
+      assertEquals(1, response.getResourceSpecs().size());
+      assertEquals(req3, new LocalResourceRequest(response.getResourceSpecs()
+          .get(0).getResource()));
+      localizedPath =
+          response.getResourceSpecs().get(0).getDestinationDirectory();
+      assertTrue(localizedPath.getFile().endsWith(
+          localPath + Path.SEPARATOR + "1" + Path.SEPARATOR + "12"));
 
       // get shutdown
       response = spyService.heartbeat(stat);
       assertEquals(LocalizerAction.DIE, response.getLocalizerAction());
 
-
       dispatcher.await();
       // verify container notification
       ArgumentMatcher<ContainerEvent> matchesContainerLoc =
@@ -923,8 +952,8 @@ public class TestResourceLocalizationService {
               && c.getContainerId() == evt.getContainerID();
           }
         };
-      // total 2 resource localzation calls. one for each resource.
-      verify(containerBus, times(2)).handle(argThat(matchesContainerLoc));
+      // total 3 resource localzation calls. one for each resource.
+      verify(containerBus, times(3)).handle(argThat(matchesContainerLoc));
         
       // Verify deletion of localization token.
       verify(delService).delete((String)isNull(), eq(localizationTokenPath));


[47/50] [abbrv] hadoop git commit: MAPREDUCE-6141. History server leveldb recovery store. Contributed by Jason Lowe

Posted by zh...@apache.org.
MAPREDUCE-6141. History server leveldb recovery store. Contributed by Jason Lowe


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

Branch: refs/heads/HDFS-EC
Commit: 4216800c00b2136e73102c7751b56b5d121bea20
Parents: b327379
Author: Jason Lowe <jl...@apache.org>
Authored: Mon Jan 26 16:28:55 2015 +0000
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:31 2015 -0800

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt            |   2 +
 .../mapreduce/v2/jobhistory/JHAdminConfig.java  |   7 +
 .../src/main/resources/mapred-default.xml       |   8 +
 .../hadoop-mapreduce-client-hs/pom.xml          |   4 +
 .../HistoryServerLeveldbStateStoreService.java  | 379 +++++++++++++++++++
 ...stHistoryServerLeveldbStateStoreService.java | 207 ++++++++++
 6 files changed, 607 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4216800c/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index b28fc65..35ceb2e 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -262,6 +262,8 @@ Release 2.7.0 - UNRELEASED
     cache with enabling wired encryption at the same time. 
     (Junping Du via xgong)
 
+    MAPREDUCE-6141. History server leveldb recovery store (jlowe)
+
   OPTIMIZATIONS
 
     MAPREDUCE-6169. MergeQueue should release reference to the current item 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4216800c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/JHAdminConfig.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/JHAdminConfig.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/JHAdminConfig.java
index e5a49b5..f7cba9f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/JHAdminConfig.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/JHAdminConfig.java
@@ -197,6 +197,13 @@ public class JHAdminConfig {
   public static final String MR_HS_FS_STATE_STORE_URI =
       MR_HISTORY_PREFIX + "recovery.store.fs.uri";
 
+  /**
+   * The local path where server state will be stored when
+   * HistoryServerLeveldbStateStoreService is configured as the state store
+   */
+  public static final String MR_HS_LEVELDB_STATE_STORE_PATH =
+      MR_HISTORY_PREFIX + "recovery.store.leveldb.path";
+
   /** Whether to use fixed ports with the minicluster. */
   public static final String MR_HISTORY_MINICLUSTER_FIXED_PORTS = MR_HISTORY_PREFIX
        + "minicluster.fixed.ports";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4216800c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
index 57a17a8..4535137 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
@@ -1569,6 +1569,14 @@
 </property>
 
 <property>
+  <name>mapreduce.jobhistory.recovery.store.leveldb.path</name>
+  <value>${hadoop.tmp.dir}/mapred/history/recoverystore</value>
+  <description>The URI where history server state will be stored if
+  HistoryServerLeveldbSystemStateStoreService is configured as the recovery
+  storage class.</description>
+</property>
+
+<property>
   <name>mapreduce.jobhistory.http.policy</name>
   <value>HTTP_ONLY</value>
   <description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4216800c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/pom.xml
index adeb9fa..fa8162b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/pom.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/pom.xml
@@ -63,6 +63,10 @@
       <type>test-jar</type>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.fusesource.leveldbjni</groupId>
+      <artifactId>leveldbjni-all</artifactId>
+    </dependency>
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4216800c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryServerLeveldbStateStoreService.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryServerLeveldbStateStoreService.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryServerLeveldbStateStoreService.java
new file mode 100644
index 0000000..16366b1
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryServerLeveldbStateStoreService.java
@@ -0,0 +1,379 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapreduce.v2.hs;
+
+import static org.fusesource.leveldbjni.JniDBFactory.asString;
+import static org.fusesource.leveldbjni.JniDBFactory.bytes;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.util.Map.Entry;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.mapreduce.v2.api.MRDelegationTokenIdentifier;
+import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
+import org.apache.hadoop.security.token.delegation.DelegationKey;
+import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.VersionProto;
+import org.apache.hadoop.yarn.server.records.Version;
+import org.apache.hadoop.yarn.server.records.impl.pb.VersionPBImpl;
+import org.apache.hadoop.yarn.server.utils.LeveldbIterator;
+import org.fusesource.leveldbjni.JniDBFactory;
+import org.fusesource.leveldbjni.internal.NativeDB;
+import org.iq80.leveldb.DB;
+import org.iq80.leveldb.DBException;
+import org.iq80.leveldb.Logger;
+import org.iq80.leveldb.Options;
+
+public class HistoryServerLeveldbStateStoreService extends
+    HistoryServerStateStoreService {
+
+  private static final String DB_NAME = "mr-jhs-state";
+  private static final String DB_SCHEMA_VERSION_KEY = "jhs-schema-version";
+  private static final String TOKEN_MASTER_KEY_KEY_PREFIX = "tokens/key_";
+  private static final String TOKEN_STATE_KEY_PREFIX = "tokens/token_";
+
+  private static final Version CURRENT_VERSION_INFO =
+      Version.newInstance(1, 0);
+
+  private DB db;
+
+  public static final Log LOG =
+      LogFactory.getLog(HistoryServerLeveldbStateStoreService.class);
+
+  @Override
+  protected void initStorage(Configuration conf) throws IOException {
+  }
+
+  @Override
+  protected void startStorage() throws IOException {
+    Path storeRoot = createStorageDir(getConfig());
+    Options options = new Options();
+    options.createIfMissing(false);
+    options.logger(new LeveldbLogger());
+    LOG.info("Using state database at " + storeRoot + " for recovery");
+    File dbfile = new File(storeRoot.toString());
+    try {
+      db = JniDBFactory.factory.open(dbfile, options);
+    } catch (NativeDB.DBException e) {
+      if (e.isNotFound() || e.getMessage().contains(" does not exist ")) {
+        LOG.info("Creating state database at " + dbfile);
+        options.createIfMissing(true);
+        try {
+          db = JniDBFactory.factory.open(dbfile, options);
+          // store version
+          storeVersion();
+        } catch (DBException dbErr) {
+          throw new IOException(dbErr.getMessage(), dbErr);
+        }
+      } else {
+        throw e;
+      }
+    }
+    checkVersion();
+  }
+
+  @Override
+  protected void closeStorage() throws IOException {
+    if (db != null) {
+      db.close();
+      db = null;
+    }
+  }
+
+  @Override
+  public HistoryServerState loadState() throws IOException {
+    HistoryServerState state = new HistoryServerState();
+    int numKeys = loadTokenMasterKeys(state);
+    LOG.info("Recovered " + numKeys + " token master keys");
+    int numTokens = loadTokens(state);
+    LOG.info("Recovered " + numTokens + " tokens");
+    return state;
+  }
+
+  private int loadTokenMasterKeys(HistoryServerState state)
+      throws IOException {
+    int numKeys = 0;
+    LeveldbIterator iter = null;
+    try {
+      iter = new LeveldbIterator(db);
+      iter.seek(bytes(TOKEN_MASTER_KEY_KEY_PREFIX));
+      while (iter.hasNext()) {
+        Entry<byte[],byte[]> entry = iter.next();
+        String key = asString(entry.getKey());
+        if (!key.startsWith(TOKEN_MASTER_KEY_KEY_PREFIX)) {
+          break;
+        }
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Loading master key from " + key);
+        }
+        try {
+          loadTokenMasterKey(state, entry.getValue());
+        } catch (IOException e) {
+          throw new IOException("Error loading token master key from " + key,
+              e);
+        }
+        ++numKeys;
+      }
+    } catch (DBException e) {
+      throw new IOException(e);
+    } finally {
+      if (iter != null) {
+        iter.close();
+      }
+    }
+    return numKeys;
+  }
+
+  private void loadTokenMasterKey(HistoryServerState state, byte[] data)
+      throws IOException {
+    DelegationKey key = new DelegationKey();
+    DataInputStream in =
+        new DataInputStream(new ByteArrayInputStream(data));
+    try {
+      key.readFields(in);
+    } finally {
+      IOUtils.cleanup(LOG, in);
+    }
+    state.tokenMasterKeyState.add(key);
+  }
+
+  private int loadTokens(HistoryServerState state) throws IOException {
+    int numTokens = 0;
+    LeveldbIterator iter = null;
+    try {
+      iter = new LeveldbIterator(db);
+      iter.seek(bytes(TOKEN_STATE_KEY_PREFIX));
+      while (iter.hasNext()) {
+        Entry<byte[],byte[]> entry = iter.next();
+        String key = asString(entry.getKey());
+        if (!key.startsWith(TOKEN_STATE_KEY_PREFIX)) {
+          break;
+        }
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Loading token from " + key);
+        }
+        try {
+          loadToken(state, entry.getValue());
+        } catch (IOException e) {
+          throw new IOException("Error loading token state from " + key, e);
+        }
+        ++numTokens;
+      }
+    } catch (DBException e) {
+      throw new IOException(e);
+    } finally {
+      if (iter != null) {
+        iter.close();
+      }
+    }
+    return numTokens;
+  }
+
+  private void loadToken(HistoryServerState state, byte[] data)
+      throws IOException {
+    MRDelegationTokenIdentifier tokenId = new MRDelegationTokenIdentifier();
+    long renewDate;
+    DataInputStream in = new DataInputStream(new ByteArrayInputStream(data));
+    try {
+      tokenId.readFields(in);
+      renewDate = in.readLong();
+    } finally {
+      IOUtils.cleanup(LOG, in);
+    }
+    state.tokenState.put(tokenId, renewDate);
+  }
+
+  @Override
+  public void storeToken(MRDelegationTokenIdentifier tokenId, Long renewDate)
+      throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Storing token " + tokenId.getSequenceNumber());
+    }
+
+    ByteArrayOutputStream memStream = new ByteArrayOutputStream();
+    DataOutputStream dataStream = new DataOutputStream(memStream);
+    try {
+      tokenId.write(dataStream);
+      dataStream.writeLong(renewDate);
+      dataStream.close();
+      dataStream = null;
+    } finally {
+      IOUtils.cleanup(LOG, dataStream);
+    }
+
+    String dbKey = getTokenDatabaseKey(tokenId);
+    try {
+      db.put(bytes(dbKey), memStream.toByteArray());
+    } catch (DBException e) {
+      throw new IOException(e);
+    }
+  }
+
+  @Override
+  public void updateToken(MRDelegationTokenIdentifier tokenId, Long renewDate)
+      throws IOException {
+    storeToken(tokenId, renewDate);
+  }
+
+  @Override
+  public void removeToken(MRDelegationTokenIdentifier tokenId)
+      throws IOException {
+    String dbKey = getTokenDatabaseKey(tokenId);
+    try {
+      db.delete(bytes(dbKey));
+    } catch (DBException e) {
+      throw new IOException(e);
+    }
+  }
+
+  private String getTokenDatabaseKey(MRDelegationTokenIdentifier tokenId) {
+    return TOKEN_STATE_KEY_PREFIX + tokenId.getSequenceNumber();
+  }
+
+  @Override
+  public void storeTokenMasterKey(DelegationKey masterKey)
+      throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Storing master key " + masterKey.getKeyId());
+    }
+
+    ByteArrayOutputStream memStream = new ByteArrayOutputStream();
+    DataOutputStream dataStream = new DataOutputStream(memStream);
+    try {
+      masterKey.write(dataStream);
+      dataStream.close();
+      dataStream = null;
+    } finally {
+      IOUtils.cleanup(LOG, dataStream);
+    }
+
+    String dbKey = getTokenMasterKeyDatabaseKey(masterKey);
+    try {
+      db.put(bytes(dbKey), memStream.toByteArray());
+    } catch (DBException e) {
+      throw new IOException(e);
+    }
+  }
+
+  @Override
+  public void removeTokenMasterKey(DelegationKey masterKey)
+      throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Removing master key " + masterKey.getKeyId());
+    }
+
+    String dbKey = getTokenMasterKeyDatabaseKey(masterKey);
+    try {
+      db.delete(bytes(dbKey));
+    } catch (DBException e) {
+      throw new IOException(e);
+    }
+  }
+
+  private String getTokenMasterKeyDatabaseKey(DelegationKey masterKey) {
+    return TOKEN_MASTER_KEY_KEY_PREFIX + masterKey.getKeyId();
+  }
+
+  private Path createStorageDir(Configuration conf) throws IOException {
+    String confPath = conf.get(JHAdminConfig.MR_HS_LEVELDB_STATE_STORE_PATH);
+    if (confPath == null) {
+      throw new IOException("No store location directory configured in " +
+          JHAdminConfig.MR_HS_LEVELDB_STATE_STORE_PATH);
+    }
+    Path root = new Path(confPath, DB_NAME);
+    FileSystem fs = FileSystem.getLocal(conf);
+    fs.mkdirs(root, new FsPermission((short)0700));
+    return root;
+  }
+
+  Version loadVersion() throws IOException {
+    byte[] data = db.get(bytes(DB_SCHEMA_VERSION_KEY));
+    // if version is not stored previously, treat it as 1.0.
+    if (data == null || data.length == 0) {
+      return Version.newInstance(1, 0);
+    }
+    Version version =
+        new VersionPBImpl(VersionProto.parseFrom(data));
+    return version;
+  }
+
+  private void storeVersion() throws IOException {
+    dbStoreVersion(CURRENT_VERSION_INFO);
+  }
+
+  void dbStoreVersion(Version state) throws IOException {
+    String key = DB_SCHEMA_VERSION_KEY;
+    byte[] data =
+        ((VersionPBImpl) state).getProto().toByteArray();
+    try {
+      db.put(bytes(key), data);
+    } catch (DBException e) {
+      throw new IOException(e);
+    }
+  }
+
+  Version getCurrentVersion() {
+    return CURRENT_VERSION_INFO;
+  }
+
+  /**
+   * 1) Versioning scheme: major.minor. For e.g. 1.0, 1.1, 1.2...1.25, 2.0 etc.
+   * 2) Any incompatible change of state-store is a major upgrade, and any
+   *    compatible change of state-store is a minor upgrade.
+   * 3) Within a minor upgrade, say 1.1 to 1.2:
+   *    overwrite the version info and proceed as normal.
+   * 4) Within a major upgrade, say 1.2 to 2.0:
+   *    throw exception and indicate user to use a separate upgrade tool to
+   *    upgrade state or remove incompatible old state.
+   */
+  private void checkVersion() throws IOException {
+    Version loadedVersion = loadVersion();
+    LOG.info("Loaded state version info " + loadedVersion);
+    if (loadedVersion.equals(getCurrentVersion())) {
+      return;
+    }
+    if (loadedVersion.isCompatibleTo(getCurrentVersion())) {
+      LOG.info("Storing state version info " + getCurrentVersion());
+      storeVersion();
+    } else {
+      throw new IOException(
+        "Incompatible version for state: expecting state version "
+            + getCurrentVersion() + ", but loading version " + loadedVersion);
+    }
+  }
+
+  private static class LeveldbLogger implements Logger {
+    private static final Log LOG = LogFactory.getLog(LeveldbLogger.class);
+
+    @Override
+    public void log(String message) {
+      LOG.info(message);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4216800c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestHistoryServerLeveldbStateStoreService.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestHistoryServerLeveldbStateStoreService.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestHistoryServerLeveldbStateStoreService.java
new file mode 100644
index 0000000..2af2f84
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestHistoryServerLeveldbStateStoreService.java
@@ -0,0 +1,207 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapreduce.v2.hs;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.v2.api.MRDelegationTokenIdentifier;
+import org.apache.hadoop.mapreduce.v2.hs.HistoryServerStateStoreService.HistoryServerState;
+import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
+import org.apache.hadoop.security.token.delegation.DelegationKey;
+import org.apache.hadoop.service.ServiceStateException;
+import org.apache.hadoop.yarn.server.records.Version;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestHistoryServerLeveldbStateStoreService {
+
+  private static final File testDir = new File(
+      System.getProperty("test.build.data",
+          System.getProperty("java.io.tmpdir")),
+      "TestHistoryServerLeveldbSystemStateStoreService");
+
+  private Configuration conf;
+
+  @Before
+  public void setup() {
+    FileUtil.fullyDelete(testDir);
+    testDir.mkdirs();
+    conf = new Configuration();
+    conf.setBoolean(JHAdminConfig.MR_HS_RECOVERY_ENABLE, true);
+    conf.setClass(JHAdminConfig.MR_HS_STATE_STORE,
+        HistoryServerLeveldbStateStoreService.class,
+        HistoryServerStateStoreService.class);
+    conf.set(JHAdminConfig.MR_HS_LEVELDB_STATE_STORE_PATH,
+        testDir.getAbsoluteFile().toString());
+  }
+
+  @After
+  public void cleanup() {
+    FileUtil.fullyDelete(testDir);
+  }
+
+  private HistoryServerStateStoreService createAndStartStore()
+      throws IOException {
+    HistoryServerStateStoreService store =
+        HistoryServerStateStoreServiceFactory.getStore(conf);
+    assertTrue("Factory did not create a leveldb store",
+        store instanceof HistoryServerLeveldbStateStoreService);
+    store.init(conf);
+    store.start();
+    return store;
+  }
+
+  @Test
+  public void testCheckVersion() throws IOException {
+    HistoryServerLeveldbStateStoreService store =
+        new HistoryServerLeveldbStateStoreService();
+    store.init(conf);
+    store.start();
+
+    // default version
+    Version defaultVersion = store.getCurrentVersion();
+    assertEquals(defaultVersion, store.loadVersion());
+
+    // compatible version
+    Version compatibleVersion =
+        Version.newInstance(defaultVersion.getMajorVersion(),
+          defaultVersion.getMinorVersion() + 2);
+    store.dbStoreVersion(compatibleVersion);
+    assertEquals(compatibleVersion, store.loadVersion());
+    store.close();
+    store = new HistoryServerLeveldbStateStoreService();
+    store.init(conf);
+    store.start();
+
+    // overwrite the compatible version
+    assertEquals(defaultVersion, store.loadVersion());
+
+    // incompatible version
+    Version incompatibleVersion =
+      Version.newInstance(defaultVersion.getMajorVersion() + 1,
+          defaultVersion.getMinorVersion());
+    store.dbStoreVersion(incompatibleVersion);
+    store.close();
+    store = new HistoryServerLeveldbStateStoreService();
+    try {
+      store.init(conf);
+      store.start();
+      fail("Incompatible version, should have thrown before here.");
+    } catch (ServiceStateException e) {
+      assertTrue("Exception message mismatch",
+        e.getMessage().contains("Incompatible version for state:"));
+    }
+    store.close();
+  }
+
+  @Test
+  public void testTokenStore() throws IOException {
+    HistoryServerStateStoreService store = createAndStartStore();
+
+    // verify initially the store is empty
+    HistoryServerState state = store.loadState();
+    assertTrue("token state not empty", state.tokenState.isEmpty());
+    assertTrue("key state not empty", state.tokenMasterKeyState.isEmpty());
+
+    // store a key and some tokens
+    final DelegationKey key1 = new DelegationKey(1, 2, "keyData1".getBytes());
+    final MRDelegationTokenIdentifier token1 =
+        new MRDelegationTokenIdentifier(new Text("tokenOwner1"),
+            new Text("tokenRenewer1"), new Text("tokenUser1"));
+    token1.setSequenceNumber(1);
+    final Long tokenDate1 = 1L;
+    final MRDelegationTokenIdentifier token2 =
+        new MRDelegationTokenIdentifier(new Text("tokenOwner2"),
+            new Text("tokenRenewer2"), new Text("tokenUser2"));
+    token2.setSequenceNumber(12345678);
+    final Long tokenDate2 = 87654321L;
+
+    store.storeTokenMasterKey(key1);
+    store.storeToken(token1, tokenDate1);
+    store.storeToken(token2, tokenDate2);
+    store.close();
+
+    // verify the key and tokens can be recovered
+    store = createAndStartStore();
+    state = store.loadState();
+    assertEquals("incorrect loaded token count", 2, state.tokenState.size());
+    assertTrue("missing token 1", state.tokenState.containsKey(token1));
+    assertEquals("incorrect token 1 date", tokenDate1,
+        state.tokenState.get(token1));
+    assertTrue("missing token 2", state.tokenState.containsKey(token2));
+    assertEquals("incorrect token 2 date", tokenDate2,
+        state.tokenState.get(token2));
+    assertEquals("incorrect master key count", 1,
+        state.tokenMasterKeyState.size());
+    assertTrue("missing master key 1",
+        state.tokenMasterKeyState.contains(key1));
+
+    // store some more keys and tokens, remove the previous key and one
+    // of the tokens, and renew a previous token
+    final DelegationKey key2 = new DelegationKey(3, 4, "keyData2".getBytes());
+    final DelegationKey key3 = new DelegationKey(5, 6, "keyData3".getBytes());
+    final MRDelegationTokenIdentifier token3 =
+        new MRDelegationTokenIdentifier(new Text("tokenOwner3"),
+            new Text("tokenRenewer3"), new Text("tokenUser3"));
+    token3.setSequenceNumber(12345679);
+    final Long tokenDate3 = 87654321L;
+
+    store.removeToken(token1);
+    store.storeTokenMasterKey(key2);
+    final Long newTokenDate2 = 975318642L;
+    store.updateToken(token2, newTokenDate2);
+    store.removeTokenMasterKey(key1);
+    store.storeTokenMasterKey(key3);
+    store.storeToken(token3, tokenDate3);
+    store.close();
+
+    // verify the new keys and tokens are recovered, the removed key and
+    // token are no longer present, and the renewed token has the updated
+    // expiration date
+    store = createAndStartStore();
+    state = store.loadState();
+    assertEquals("incorrect loaded token count", 2, state.tokenState.size());
+    assertFalse("token 1 not removed", state.tokenState.containsKey(token1));
+    assertTrue("missing token 2", state.tokenState.containsKey(token2));
+    assertEquals("incorrect token 2 date", newTokenDate2,
+        state.tokenState.get(token2));
+    assertTrue("missing token 3", state.tokenState.containsKey(token3));
+    assertEquals("incorrect token 3 date", tokenDate3,
+        state.tokenState.get(token3));
+    assertEquals("incorrect master key count", 2,
+        state.tokenMasterKeyState.size());
+    assertFalse("master key 1 not removed",
+        state.tokenMasterKeyState.contains(key1));
+    assertTrue("missing master key 2",
+        state.tokenMasterKeyState.contains(key2));
+    assertTrue("missing master key 3",
+        state.tokenMasterKeyState.contains(key3));
+    store.close();
+  }
+}


[44/50] [abbrv] hadoop git commit: Move truncate related messages in CHANGES.txt to branch 2 section. (shv)

Posted by zh...@apache.org.
Move truncate related messages in CHANGES.txt to branch 2 section. (shv)


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

Branch: refs/heads/HDFS-EC
Commit: e461d62cfa6111cb5d31de2b9d1a024cf6978b9f
Parents: 27c98cc
Author: Konstantin V Shvachko <sh...@apache.org>
Authored: Sat Jan 24 17:49:22 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:30 2015 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  6 ++--
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     | 34 ++++++++++----------
 2 files changed, 20 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e461d62c/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 6a34092..c0617e8 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -19,9 +19,6 @@ Trunk (Unreleased)
 
     HADOOP-11353. Add support for .hadooprc (aw)
 
-    HADOOP-11490. Expose truncate API via FileSystem and shell command.
-    (Milan Desai via shv)
-
   IMPROVEMENTS
 
     HADOOP-8017. Configure hadoop-main pom to get rid of M2E plugin execution
@@ -386,6 +383,9 @@ Release 2.7.0 - UNRELEASED
 
     HADOOP-8989. hadoop fs -find feature (Jonathan Allen via aw)
 
+    HADOOP-11490. Expose truncate API via FileSystem and shell command.
+    (Milan Desai via shv)
+
   IMPROVEMENTS
 
     HADOOP-11483. HardLink.java should use the jdk7 createLink method (aajisaka)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e461d62c/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 cca755e..21c8374 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -18,10 +18,6 @@ Trunk (Unreleased)
 
     HDFS-3125. Add JournalService to enable Journal Daemon. (suresh)
 
-    HDFS-3107. Introduce truncate. (Plamen Jeliazkov via shv)
-
-    HDFS-7056. Snapshot support for truncate. (Plamen Jeliazkov and shv)
-
   IMPROVEMENTS
 
     HDFS-4665. Move TestNetworkTopologyWithNodeGroup to common.
@@ -143,9 +139,6 @@ Trunk (Unreleased)
     HDFS-7430. Rewrite the BlockScanner to use O(1) memory and use multiple
     threads (cmccabe)
 
-    HDFS-7659. truncate should check negative value of the new length.
-    (Yi Liu via shv)
-
   OPTIMIZATIONS
 
   BUG FIXES
@@ -280,16 +273,6 @@ Trunk (Unreleased)
 
     HDFS-7581. HDFS documentation needs updating post-shell rewrite (aw)
 
-    HDFS-7606. Fix potential NPE in INodeFile.getBlocks(). (Byron Wong via shv)
-
-    HDFS-7638: Small fix and few refinements for FSN#truncate. (yliu)
-
-    HDFS-7634. Disallow truncation of Lazy persist files. (Yi Liu via
-    Arpit Agarwal)
-
-    HDFS-7643. Test case to ensure lazy persist files cannot be truncated.
-    (Yi Liu via Arpit Agarwal)
-
     HADOOP-11484. hadoop-mapreduce-client-nativetask fails to build on ARM
     AARCH64 due to x86 asm statements (Edward Nevill via Colin P. McCabe)
 
@@ -320,6 +303,10 @@ Release 2.7.0 - UNRELEASED
     
     HDFS-7449. Add metrics to NFS gateway (brandonli)
 
+    HDFS-3107. Introduce truncate. (Plamen Jeliazkov via shv)
+
+    HDFS-7056. Snapshot support for truncate. (Plamen Jeliazkov and shv)
+
   IMPROVEMENTS
 
     HDFS-7055. Add tracing to DFSInputStream (cmccabe)
@@ -785,6 +772,19 @@ Release 2.7.0 - UNRELEASED
 
     HDFS-7644. minor typo in HttpFS doc (Charles Lamb via aw)
 
+    HDFS-7606. Fix potential NPE in INodeFile.getBlocks(). (Byron Wong via shv)
+
+    HDFS-7638: Small fix and few refinements for FSN#truncate. (yliu)
+
+    HDFS-7634. Disallow truncation of Lazy persist files. (Yi Liu via
+    Arpit Agarwal)
+
+    HDFS-7643. Test case to ensure lazy persist files cannot be truncated.
+    (Yi Liu via Arpit Agarwal)
+
+    HDFS-7659. truncate should check negative value of the new length.
+    (Yi Liu via shv)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES


[37/50] [abbrv] hadoop git commit: YARN-2800. Remove MemoryNodeLabelsStore and add a way to enable/disable node labels feature. Contributed by Wangda Tan.

Posted by zh...@apache.org.
YARN-2800. Remove MemoryNodeLabelsStore and add a way to enable/disable node labels feature. Contributed by Wangda Tan.


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

Branch: refs/heads/HDFS-EC
Commit: 7d62e9070f2c240e16782e36f96f8cfda339d862
Parents: a06d2d6
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Fri Jan 23 20:37:05 2015 +0900
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:29 2015 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +
 .../hadoop/yarn/conf/YarnConfiguration.java     | 16 ++--
 .../distributedshell/TestDistributedShell.java  |  1 +
 .../nodelabels/CommonNodeLabelsManager.java     | 42 ++++++++-
 .../nodelabels/TestCommonNodeLabelsManager.java | 62 +++++++++++++-
 .../TestFileSystemNodeLabelsStore.java          |  1 +
 .../server/resourcemanager/ResourceManager.java |  6 +-
 .../nodelabels/MemoryRMNodeLabelsManager.java   | 82 ------------------
 .../yarn/server/resourcemanager/MockRM.java     |  4 +-
 .../server/resourcemanager/TestRMRestart.java   |  1 +
 .../nodelabels/NullRMNodeLabelsManager.java     | 90 ++++++++++++++++++++
 .../nodelabels/TestRMNodeLabelsManager.java     |  9 +-
 .../capacity/TestCapacityScheduler.java         |  6 +-
 .../TestCapacitySchedulerNodeLabelUpdate.java   |  4 +-
 .../capacity/TestContainerAllocation.java       |  6 +-
 .../scheduler/capacity/TestQueueParsing.java    | 12 +--
 .../resourcemanager/webapp/TestRMWebApp.java    |  6 +-
 17 files changed, 233 insertions(+), 118 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d62e907/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 7f0628d..7263c6f 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -197,6 +197,9 @@ Release 2.7.0 - UNRELEASED
 
     YARN-2984. Metrics for container's actual memory usage. (kasha)
 
+    YARN-2800. Remove MemoryNodeLabelsStore and add a way to enable/disable
+    node labels feature. (Wangda Tan via ozawa)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d62e907/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 9ac5438..2ccd894 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
@@ -1648,14 +1648,10 @@ public class YarnConfiguration extends Configuration {
   public static final String YARN_HTTP_POLICY_DEFAULT = HttpConfig.Policy.HTTP_ONLY
       .name();
   
-  public static final String NODE_LABELS_PREFIX = YARN_PREFIX + "node-labels.";
-
   /**
-   * Class for RMNodeLabelsManager Please note this value should be consistent
-   * in client nodes and RM node(s)
+   * Node-labels configurations
    */
-  public static final String RM_NODE_LABELS_MANAGER_CLASS = NODE_LABELS_PREFIX
-      + "manager-class";
+  public static final String NODE_LABELS_PREFIX = YARN_PREFIX + "node-labels.";
   
   /** URI for NodeLabelManager */
   public static final String FS_NODE_LABELS_STORE_ROOT_DIR = NODE_LABELS_PREFIX
@@ -1664,6 +1660,14 @@ public class YarnConfiguration extends Configuration {
       NODE_LABELS_PREFIX + "fs-store.retry-policy-spec";
   public static final String DEFAULT_FS_NODE_LABELS_STORE_RETRY_POLICY_SPEC =
       "2000, 500";
+  
+  /**
+   * Flag to indicate if the node labels feature enabled, by default it's
+   * disabled
+   */
+  public static final String NODE_LABELS_ENABLED = NODE_LABELS_PREFIX
+      + "enabled";
+  public static final boolean DEFAULT_NODE_LABELS_ENABLED = false;
 
   public YarnConfiguration() {
     super();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d62e907/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
index 1d3a104..46b5850 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
@@ -80,6 +80,7 @@ public class TestDistributedShell {
     conf.set("yarn.log.dir", "target");
     conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
     conf.set(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class.getName());
+    conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, true);
     
     if (yarnCluster == null) {
       yarnCluster =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d62e907/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
index aeefff1..daa1529 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
@@ -52,6 +53,7 @@ import org.apache.hadoop.yarn.nodelabels.event.RemoveClusterNodeLabels;
 import org.apache.hadoop.yarn.nodelabels.event.UpdateNodeToLabelsMappingsEvent;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableSet;
 
 public class CommonNodeLabelsManager extends AbstractService {
@@ -64,6 +66,14 @@ public class CommonNodeLabelsManager extends AbstractService {
   private static final Pattern LABEL_PATTERN = Pattern
       .compile("^[0-9a-zA-Z][0-9a-zA-Z-_]*");
   public static final int WILDCARD_PORT = 0;
+  
+  /**
+   * Error messages
+   */
+  @VisibleForTesting
+  public static final String NODE_LABELS_NOT_ENABLED_ERR =
+      "Node-label-based scheduling is disabled. Please check "
+          + YarnConfiguration.NODE_LABELS_ENABLED;
 
   /**
    * If a user doesn't specify label of a queue or node, it belongs
@@ -82,6 +92,7 @@ public class CommonNodeLabelsManager extends AbstractService {
   protected final WriteLock writeLock;
 
   protected NodeLabelsStore store;
+  private boolean nodeLabelsEnabled = false;
 
   /**
    * A <code>Host</code> can have multiple <code>Node</code>s 
@@ -193,7 +204,13 @@ public class CommonNodeLabelsManager extends AbstractService {
 
   @Override
   protected void serviceInit(Configuration conf) throws Exception {
-    initNodeLabelStore(conf);
+    // set if node labels enabled
+    nodeLabelsEnabled =
+        conf.getBoolean(YarnConfiguration.NODE_LABELS_ENABLED,
+            YarnConfiguration.DEFAULT_NODE_LABELS_ENABLED);
+    if (nodeLabelsEnabled) {
+      initNodeLabelStore(conf);
+    }
     
     labelCollections.put(NO_LABEL, new NodeLabel(NO_LABEL));
   }
@@ -250,6 +267,10 @@ public class CommonNodeLabelsManager extends AbstractService {
    */
   @SuppressWarnings("unchecked")
   public void addToCluserNodeLabels(Set<String> labels) throws IOException {
+    if (!nodeLabelsEnabled) {
+      LOG.error(NODE_LABELS_NOT_ENABLED_ERR);
+      throw new IOException(NODE_LABELS_NOT_ENABLED_ERR);
+    }
     if (null == labels || labels.isEmpty()) {
       return;
     }
@@ -304,6 +325,10 @@ public class CommonNodeLabelsManager extends AbstractService {
    */
   public void addLabelsToNode(Map<NodeId, Set<String>> addedLabelsToNode)
       throws IOException {
+    if (!nodeLabelsEnabled) {
+      LOG.error(NODE_LABELS_NOT_ENABLED_ERR);
+      throw new IOException(NODE_LABELS_NOT_ENABLED_ERR);
+    }
     addedLabelsToNode = normalizeNodeIdToLabels(addedLabelsToNode);
     checkAddLabelsToNode(addedLabelsToNode);
     internalUpdateLabelsOnNodes(addedLabelsToNode, NodeLabelUpdateOperation.ADD);
@@ -370,6 +395,11 @@ public class CommonNodeLabelsManager extends AbstractService {
    */
   public void removeFromClusterNodeLabels(Collection<String> labelsToRemove)
       throws IOException {
+    if (!nodeLabelsEnabled) {
+      LOG.error(NODE_LABELS_NOT_ENABLED_ERR);
+      throw new IOException(NODE_LABELS_NOT_ENABLED_ERR);
+    }
+    
     labelsToRemove = normalizeLabels(labelsToRemove);
     
     checkRemoveFromClusterNodeLabels(labelsToRemove);
@@ -532,6 +562,11 @@ public class CommonNodeLabelsManager extends AbstractService {
   public void
       removeLabelsFromNode(Map<NodeId, Set<String>> removeLabelsFromNode)
           throws IOException {
+    if (!nodeLabelsEnabled) {
+      LOG.error(NODE_LABELS_NOT_ENABLED_ERR);
+      throw new IOException(NODE_LABELS_NOT_ENABLED_ERR);
+    }
+    
     removeLabelsFromNode = normalizeNodeIdToLabels(removeLabelsFromNode);
     
     checkRemoveLabelsFromNode(removeLabelsFromNode);
@@ -567,6 +602,11 @@ public class CommonNodeLabelsManager extends AbstractService {
    */
   public void replaceLabelsOnNode(Map<NodeId, Set<String>> replaceLabelsToNode)
       throws IOException {
+    if (!nodeLabelsEnabled) {
+      LOG.error(NODE_LABELS_NOT_ENABLED_ERR);
+      throw new IOException(NODE_LABELS_NOT_ENABLED_ERR);
+    }
+    
     replaceLabelsToNode = normalizeNodeIdToLabels(replaceLabelsToNode);
     
     checkReplaceLabelsOnNode(replaceLabelsToNode);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d62e907/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestCommonNodeLabelsManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestCommonNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestCommonNodeLabelsManager.java
index c0b05e3..242f59c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestCommonNodeLabelsManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestCommonNodeLabelsManager.java
@@ -26,6 +26,8 @@ import java.util.Set;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -41,7 +43,9 @@ public class TestCommonNodeLabelsManager extends NodeLabelTestBase {
   @Before
   public void before() {
     mgr = new DummyCommonNodeLabelsManager();
-    mgr.init(new Configuration());
+    Configuration conf = new YarnConfiguration();
+    conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, true);
+    mgr.init(conf);
     mgr.start();
   }
 
@@ -319,4 +323,58 @@ public class TestCommonNodeLabelsManager extends NodeLabelTestBase {
         toSet("p1"), toNodeId("n1:1"), toSet("p2"), toNodeId("n1:2"),
         toSet("p1")));
   }
-}
\ No newline at end of file
+
+  private void assertNodeLabelsDisabledErrorMessage(IOException e) {
+    Assert.assertEquals(CommonNodeLabelsManager.NODE_LABELS_NOT_ENABLED_ERR,
+        e.getMessage());
+  }
+  
+  @Test(timeout = 5000)
+  public void testNodeLabelsDisabled() throws IOException {
+    DummyCommonNodeLabelsManager mgr = new DummyCommonNodeLabelsManager();
+    Configuration conf = new YarnConfiguration();
+    conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, true);
+    mgr.init(conf);
+    mgr.start();
+    
+    // add labels
+    try {
+      mgr.addToCluserNodeLabels(ImmutableSet.of("x"));
+    } catch (IOException e) {
+      assertNodeLabelsDisabledErrorMessage(e);
+    }
+    
+    // remove labels
+    try {
+      mgr.removeFromClusterNodeLabels(ImmutableSet.of("x"));
+    } catch (IOException e) {
+      assertNodeLabelsDisabledErrorMessage(e);
+    }
+    
+    // add labels to node
+    try {
+      mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("host", 0),
+          CommonNodeLabelsManager.EMPTY_STRING_SET));
+    } catch (IOException e) {
+      assertNodeLabelsDisabledErrorMessage(e);
+    }
+    
+    // remove labels from node
+    try {
+      mgr.removeLabelsFromNode(ImmutableMap.of(NodeId.newInstance("host", 0),
+          CommonNodeLabelsManager.EMPTY_STRING_SET));
+    } catch (IOException e) {
+      assertNodeLabelsDisabledErrorMessage(e);
+    }
+    
+    // replace labels on node
+    try {
+      mgr.replaceLabelsOnNode(ImmutableMap.of(NodeId.newInstance("host", 0),
+          CommonNodeLabelsManager.EMPTY_STRING_SET));
+    } catch (IOException e) {
+      assertNodeLabelsDisabledErrorMessage(e);
+    }
+    
+    mgr.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d62e907/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestFileSystemNodeLabelsStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestFileSystemNodeLabelsStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestFileSystemNodeLabelsStore.java
index 45a2d8d..55e0e17 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestFileSystemNodeLabelsStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestFileSystemNodeLabelsStore.java
@@ -63,6 +63,7 @@ public class TestFileSystemNodeLabelsStore extends NodeLabelTestBase {
   public void before() throws IOException {
     mgr = new MockNodeLabelManager();
     conf = new Configuration();
+    conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, true);
     File tempDir = File.createTempFile("nlb", ".tmp");
     tempDir.delete();
     tempDir.mkdirs();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d62e907/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
index cceee2b..4f242e93 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
@@ -59,7 +59,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMaste
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
 import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingEditPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingMonitor;
-import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.MemoryRMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.NullRMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
@@ -334,10 +333,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
   
   protected RMNodeLabelsManager createNodeLabelManager()
       throws InstantiationException, IllegalAccessException {
-    Class<? extends RMNodeLabelsManager> nlmCls =
-        conf.getClass(YarnConfiguration.RM_NODE_LABELS_MANAGER_CLASS,
-            MemoryRMNodeLabelsManager.class, RMNodeLabelsManager.class);
-    return nlmCls.newInstance();
+    return new RMNodeLabelsManager();
   }
   
   protected DelegationTokenRenewer createDelegationTokenRenewer() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d62e907/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/MemoryRMNodeLabelsManager.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/nodelabels/MemoryRMNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/MemoryRMNodeLabelsManager.java
deleted file mode 100644
index 89053ca..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/MemoryRMNodeLabelsManager.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.server.resourcemanager.nodelabels;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.nodelabels.NodeLabelsStore;
-
-public class MemoryRMNodeLabelsManager extends RMNodeLabelsManager {
-  Map<NodeId, Set<String>> lastNodeToLabels = null;
-  Collection<String> lastAddedlabels = null;
-  Collection<String> lastRemovedlabels = null;
-
-  @Override
-  public void initNodeLabelStore(Configuration conf) {
-    this.store = new NodeLabelsStore(this) {
-
-      @Override
-      public void recover() throws IOException {
-        // do nothing
-      }
-
-      @Override
-      public void removeClusterNodeLabels(Collection<String> labels)
-          throws IOException {
-        // do nothing
-      }
-
-      @Override
-      public void updateNodeToLabelsMappings(
-          Map<NodeId, Set<String>> nodeToLabels) throws IOException {
-        // do nothing
-      }
-
-      @Override
-      public void storeNewClusterNodeLabels(Set<String> label) throws IOException {
-        // do nothing
-      }
-
-      @Override
-      public void close() throws IOException {
-        // do nothing
-      }
-    };
-  }
-
-  @Override
-  protected void initDispatcher(Configuration conf) {
-    super.dispatcher = null;
-  }
-
-  @Override
-  protected void startDispatcher() {
-    // do nothing
-  }
-  
-  @Override
-  protected void stopDispatcher() {
-    // do nothing
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d62e907/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.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/MockRM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
index 5794b43..06c6b32 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
@@ -59,7 +59,7 @@ import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
 import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher;
-import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.MemoryRMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
@@ -115,7 +115,7 @@ public class MockRM extends ResourceManager {
   
   @Override
   protected RMNodeLabelsManager createNodeLabelManager() {
-    RMNodeLabelsManager mgr = new MemoryRMNodeLabelsManager();
+    RMNodeLabelsManager mgr = new NullRMNodeLabelsManager();
     mgr.init(getConfig());
     return mgr;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d62e907/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 bd6fdba..fc419ac 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
@@ -2073,6 +2073,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     
     MemoryRMStateStore memStore = new MemoryRMStateStore();
     memStore.init(conf);
+    conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, true);
     MockRM rm1 = new MockRM(conf, memStore) {
       @Override
       protected RMNodeLabelsManager createNodeLabelManager() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d62e907/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NullRMNodeLabelsManager.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/nodelabels/NullRMNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NullRMNodeLabelsManager.java
new file mode 100644
index 0000000..b1be525
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NullRMNodeLabelsManager.java
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.nodelabels;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.nodelabels.NodeLabelsStore;
+
+public class NullRMNodeLabelsManager extends RMNodeLabelsManager {
+  Map<NodeId, Set<String>> lastNodeToLabels = null;
+  Collection<String> lastAddedlabels = null;
+  Collection<String> lastRemovedlabels = null;
+
+  @Override
+  public void initNodeLabelStore(Configuration conf) {
+    this.store = new NodeLabelsStore(this) {
+
+      @Override
+      public void recover() throws IOException {
+        // do nothing
+      }
+
+      @Override
+      public void removeClusterNodeLabels(Collection<String> labels)
+          throws IOException {
+        // do nothing
+      }
+
+      @Override
+      public void updateNodeToLabelsMappings(
+          Map<NodeId, Set<String>> nodeToLabels) throws IOException {
+        // do nothing
+      }
+
+      @Override
+      public void storeNewClusterNodeLabels(Set<String> label) throws IOException {
+        // do nothing
+      }
+
+      @Override
+      public void close() throws IOException {
+        // do nothing
+      }
+    };
+  }
+
+  @Override
+  protected void initDispatcher(Configuration conf) {
+    super.dispatcher = null;
+  }
+
+  @Override
+  protected void startDispatcher() {
+    // do nothing
+  }
+  
+  @Override
+  protected void stopDispatcher() {
+    // do nothing
+  }
+  
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+    // always enable node labels while using MemoryRMNodeLabelsManager
+    conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, true);
+    super.serviceInit(conf);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d62e907/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.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/nodelabels/TestRMNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java
index e4cdc71..4b8fe76 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java
@@ -27,6 +27,7 @@ import java.util.Set;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 import org.apache.hadoop.yarn.nodelabels.NodeLabel;
 import org.apache.hadoop.yarn.nodelabels.NodeLabelTestBase;
@@ -44,12 +45,14 @@ public class TestRMNodeLabelsManager extends NodeLabelTestBase {
   private final Resource SMALL_RESOURCE = Resource.newInstance(100, 0);
   private final Resource LARGE_NODE = Resource.newInstance(1000, 0);
   
-  MemoryRMNodeLabelsManager mgr = null;
+  NullRMNodeLabelsManager mgr = null;
 
   @Before
   public void before() {
-    mgr = new MemoryRMNodeLabelsManager();
-    mgr.init(new Configuration());
+    mgr = new NullRMNodeLabelsManager();
+    Configuration conf = new Configuration();
+    conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, true);
+    mgr.init(conf);
     mgr.start();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d62e907/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
index 2aa57a0..85696a1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
@@ -84,7 +84,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.Task;
 import org.apache.hadoop.yarn.server.resourcemanager.TestAMAuthorization.MockRMWithAMS;
 import org.apache.hadoop.yarn.server.resourcemanager.TestAMAuthorization.MyContainerManager;
-import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.MemoryRMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppMetrics;
@@ -154,7 +154,7 @@ public class TestCapacityScheduler {
     resourceManager = new ResourceManager() {
       @Override
       protected RMNodeLabelsManager createNodeLabelManager() {
-        RMNodeLabelsManager mgr = new MemoryRMNodeLabelsManager();
+        RMNodeLabelsManager mgr = new NullRMNodeLabelsManager();
         mgr.init(getConfig());
         return mgr;
       }
@@ -1485,7 +1485,7 @@ public class TestCapacityScheduler {
     resourceManager = new ResourceManager() {
        @Override
         protected RMNodeLabelsManager createNodeLabelManager() {
-          RMNodeLabelsManager mgr = new MemoryRMNodeLabelsManager();
+          RMNodeLabelsManager mgr = new NullRMNodeLabelsManager();
           mgr.init(getConfig());
           return mgr;
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d62e907/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerNodeLabelUpdate.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/TestCapacitySchedulerNodeLabelUpdate.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/TestCapacitySchedulerNodeLabelUpdate.java
index 261fa01..923c0a3 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/TestCapacitySchedulerNodeLabelUpdate.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/TestCapacitySchedulerNodeLabelUpdate.java
@@ -29,7 +29,7 @@ import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
-import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.MemoryRMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
@@ -54,7 +54,7 @@ public class TestCapacitySchedulerNodeLabelUpdate {
     conf = new YarnConfiguration();
     conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
       ResourceScheduler.class);
-    mgr = new MemoryRMNodeLabelsManager();
+    mgr = new NullRMNodeLabelsManager();
     mgr.init(conf);
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d62e907/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.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/TestContainerAllocation.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/TestContainerAllocation.java
index 9a29bff..674096e 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/TestContainerAllocation.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/TestContainerAllocation.java
@@ -45,7 +45,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.RMSecretManagerService;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.TestFifoScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.MemoryRMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
@@ -81,7 +81,7 @@ public class TestContainerAllocation {
     conf = new YarnConfiguration();
     conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
       ResourceScheduler.class);
-    mgr = new MemoryRMNodeLabelsManager();
+    mgr = new NullRMNodeLabelsManager();
     mgr.init(conf);
   }
 
@@ -451,7 +451,7 @@ public class TestContainerAllocation {
   
   @Test(timeout = 300000)
   public void testContainerAllocationWithSingleUserLimits() throws Exception {
-    final RMNodeLabelsManager mgr = new MemoryRMNodeLabelsManager();
+    final RMNodeLabelsManager mgr = new NullRMNodeLabelsManager();
     mgr.init(conf);
 
     // set node -> label

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d62e907/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueParsing.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/TestQueueParsing.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/TestQueueParsing.java
index 5a9fbe1..f821e64 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/TestQueueParsing.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/TestQueueParsing.java
@@ -26,7 +26,7 @@ import org.apache.hadoop.service.ServiceOperations;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
-import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.MemoryRMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
@@ -47,7 +47,7 @@ public class TestQueueParsing {
   
   @Before
   public void setup() {
-    nodeLabelManager = new MemoryRMNodeLabelsManager();
+    nodeLabelManager = new NullRMNodeLabelsManager();
     nodeLabelManager.init(new YarnConfiguration());
     nodeLabelManager.start();
   }
@@ -566,7 +566,7 @@ public class TestQueueParsing {
             new NMTokenSecretManagerInRM(csConf),
             new ClientToAMTokenSecretManagerInRM(), null);
     
-    RMNodeLabelsManager nodeLabelsManager = new MemoryRMNodeLabelsManager();
+    RMNodeLabelsManager nodeLabelsManager = new NullRMNodeLabelsManager();
     nodeLabelsManager.init(conf);
     nodeLabelsManager.start();
     
@@ -594,7 +594,7 @@ public class TestQueueParsing {
             new NMTokenSecretManagerInRM(csConf),
             new ClientToAMTokenSecretManagerInRM(), null);
     
-    RMNodeLabelsManager nodeLabelsManager = new MemoryRMNodeLabelsManager();
+    RMNodeLabelsManager nodeLabelsManager = new NullRMNodeLabelsManager();
     nodeLabelsManager.init(conf);
     nodeLabelsManager.start();
     
@@ -622,7 +622,7 @@ public class TestQueueParsing {
             new NMTokenSecretManagerInRM(csConf),
             new ClientToAMTokenSecretManagerInRM(), null);
     
-    RMNodeLabelsManager nodeLabelsManager = new MemoryRMNodeLabelsManager();
+    RMNodeLabelsManager nodeLabelsManager = new NullRMNodeLabelsManager();
     nodeLabelsManager.init(conf);
     nodeLabelsManager.start();
     
@@ -649,7 +649,7 @@ public class TestQueueParsing {
             new NMTokenSecretManagerInRM(csConf),
             new ClientToAMTokenSecretManagerInRM(), null);
     
-    RMNodeLabelsManager nodeLabelsManager = new MemoryRMNodeLabelsManager();
+    RMNodeLabelsManager nodeLabelsManager = new NullRMNodeLabelsManager();
     nodeLabelsManager.init(conf);
     nodeLabelsManager.start();
     

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d62e907/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebApp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebApp.java
index abc701d..fb1e61d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebApp.java
@@ -39,7 +39,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.applicationsmanager.MockAsm;
-import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.MemoryRMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
@@ -179,7 +179,7 @@ public class TestRMWebApp {
          return nodesMap;
        }
      }; 
-    rmContext.setNodeLabelManager(new MemoryRMNodeLabelsManager());
+    rmContext.setNodeLabelManager(new NullRMNodeLabelsManager());
     return rmContext;
   }
 
@@ -211,7 +211,7 @@ public class TestRMWebApp {
         null, new RMContainerTokenSecretManager(conf),
         new NMTokenSecretManagerInRM(conf),
         new ClientToAMTokenSecretManagerInRM(), null);
-    rmContext.setNodeLabelManager(new MemoryRMNodeLabelsManager());
+    rmContext.setNodeLabelManager(new NullRMNodeLabelsManager());
     cs.setRMContext(rmContext);
     cs.init(conf);
     return cs;


[50/50] [abbrv] hadoop git commit: HDFS-7224. Allow reuse of NN connections via webhdfs. Contributed by Eric Payne

Posted by zh...@apache.org.
 HDFS-7224. Allow reuse of NN connections via webhdfs. Contributed by Eric Payne


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

Branch: refs/heads/HDFS-EC
Commit: 21f5c51b87ed394056b79dc5d4010b03edb97dcd
Parents: b2de93d
Author: Kihwal Lee <ki...@apache.org>
Authored: Mon Jan 26 08:14:30 2015 -0600
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:31 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      | 20 ++++----
 .../hdfs/web/TestFSMainOperationsWebHdfs.java   | 49 ++++++++++++++++++++
 3 files changed, 64 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/21f5c51b/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 a6cbf8f..39453d1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -546,6 +546,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7623. Add htrace configuration properties to core-default.xml and
     update user doc about how to enable htrace. (yliu)
 
+    HDFS-7224. Allow reuse of NN connections via webhdfs (Eric Payne via
+    kihwal)
+
   OPTIMIZATIONS
 
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21f5c51b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
index 559efdb..460e78b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
@@ -312,16 +312,20 @@ public class WebHdfsFileSystem extends FileSystem
     if (in == null) {
       throw new IOException("The " + (useErrorStream? "error": "input") + " stream is null.");
     }
-    final String contentType = c.getContentType();
-    if (contentType != null) {
-      final MediaType parsed = MediaType.valueOf(contentType);
-      if (!MediaType.APPLICATION_JSON_TYPE.isCompatible(parsed)) {
-        throw new IOException("Content-Type \"" + contentType
-            + "\" is incompatible with \"" + MediaType.APPLICATION_JSON
-            + "\" (parsed=\"" + parsed + "\")");
+    try {
+      final String contentType = c.getContentType();
+      if (contentType != null) {
+        final MediaType parsed = MediaType.valueOf(contentType);
+        if (!MediaType.APPLICATION_JSON_TYPE.isCompatible(parsed)) {
+          throw new IOException("Content-Type \"" + contentType
+              + "\" is incompatible with \"" + MediaType.APPLICATION_JSON
+              + "\" (parsed=\"" + parsed + "\")");
+        }
       }
+      return (Map<?, ?>)JSON.parse(new InputStreamReader(in, Charsets.UTF_8));
+    } finally {
+      in.close();
     }
-    return (Map<?, ?>)JSON.parse(new InputStreamReader(in, Charsets.UTF_8));
   }
 
   private static Map<?, ?> validateResponse(final HttpOpParam.Op op,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21f5c51b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestFSMainOperationsWebHdfs.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestFSMainOperationsWebHdfs.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestFSMainOperationsWebHdfs.java
index b4216f0..4975a87 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestFSMainOperationsWebHdfs.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestFSMainOperationsWebHdfs.java
@@ -17,8 +17,14 @@
  */
 package org.apache.hadoop.hdfs.web;
 
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.doReturn;
+
 import java.io.IOException;
+import java.io.InputStream;
+import java.net.HttpURLConnection;
 import java.net.URI;
+import java.net.URL;
 import java.security.PrivilegedExceptionAction;
 
 import org.apache.commons.logging.impl.Log4JLogger;
@@ -32,6 +38,8 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.web.resources.ExceptionHandler;
+import org.apache.hadoop.hdfs.web.resources.GetOpParam;
+import org.apache.hadoop.hdfs.web.resources.HttpOpParam;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.log4j.Level;
@@ -128,6 +136,47 @@ public class TestFSMainOperationsWebHdfs extends FSMainOperationsBaseTest {
     Assert.assertEquals(1024*4, fileStatus.getLen());
   }
 
+  // Test that WebHdfsFileSystem.jsonParse() closes the connection's input
+  // stream.
+  // Closing the inputstream in jsonParse will allow WebHDFS to reuse
+  // connections to the namenode rather than needing to always open new ones.
+  boolean closedInputStream = false;
+  @Test
+  public void testJsonParseClosesInputStream() throws Exception {
+    final WebHdfsFileSystem webhdfs = (WebHdfsFileSystem)fileSystem;
+    Path file = getTestRootPath(fSys, "test/hadoop/file");
+    createFile(file);
+    final HttpOpParam.Op op = GetOpParam.Op.GETHOMEDIRECTORY;
+    final URL url = webhdfs.toUrl(op, file);
+    final HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+    conn.setRequestMethod(op.getType().toString());
+    conn.connect();
+
+    InputStream myIn = new InputStream(){
+      private HttpURLConnection localConn = conn;
+      @Override
+      public void close() throws IOException {
+        closedInputStream = true;
+        localConn.getInputStream().close();
+      }
+      @Override
+      public int read() throws IOException {
+        return localConn.getInputStream().read();
+      }
+    };
+    final HttpURLConnection spyConn = spy(conn);
+    doReturn(myIn).when(spyConn).getInputStream();
+
+    try {
+      Assert.assertFalse(closedInputStream);
+      WebHdfsFileSystem.jsonParse(spyConn, false);
+      Assert.assertTrue(closedInputStream);
+    } catch(IOException ioe) {
+      junit.framework.TestCase.fail();
+    }
+    conn.disconnect();
+  }
+
   @Override
   @Test
   public void testMkdirsFailsForSubdirectoryOfExistingFile() throws Exception {


[42/50] [abbrv] hadoop git commit: HDFS-7320. The appearance of hadoop-hdfs-httpfs site docs is inconsistent (Masatake Iwasaki via aw)

Posted by zh...@apache.org.
HDFS-7320. The appearance of hadoop-hdfs-httpfs site docs is inconsistent (Masatake Iwasaki via aw)


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

Branch: refs/heads/HDFS-EC
Commit: f8b1ce9bdc64d3afca0e64ee3a50cf27471b2fda
Parents: cc9ed52
Author: Allen Wittenauer <aw...@apache.org>
Authored: Fri Jan 23 14:21:55 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:30 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml | 2 +-
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt    | 3 +++
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f8b1ce9b/hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml b/hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml
index 0bb6d4b..4c42ef9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml
@@ -319,7 +319,7 @@
             <goals>
               <goal>dependencies</goal>
             </goals>
-            <phase>site</phase>
+            <phase>package</phase>
           </execution>
         </executions>
       </plugin>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f8b1ce9b/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 053b2eb..08f705a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -295,6 +295,9 @@ Trunk (Unreleased)
 
     HDFS-3750. API docs don't include HDFS (Jolly Chen via aw)
 
+    HDFS-7320. The appearance of hadoop-hdfs-httpfs site docs is inconsistent 
+    (Masatake Iwasaki via aw)
+
 Release 2.7.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES


[32/50] [abbrv] hadoop git commit: HADOOP-11482. Use correct UGI when KMSClientProvider is called by a proxy user. Contributed by Arun Suresh.

Posted by zh...@apache.org.
HADOOP-11482. Use correct UGI when KMSClientProvider is called by a proxy user. Contributed by Arun Suresh.


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

Branch: refs/heads/HDFS-EC
Commit: 2e02d860186572e539cad7f9f4a7723ab7b43237
Parents: 5177c14
Author: Andrew Wang <wa...@apache.org>
Authored: Fri Jan 23 12:11:15 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:29 2015 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 ++
 .../crypto/key/kms/KMSClientProvider.java       | 31 ++++++++++++++++----
 .../hadoop/crypto/key/kms/server/TestKMS.java   |  7 +++++
 3 files changed, 35 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e02d860/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 6bedd4d..6a34092 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -756,6 +756,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11507 Hadoop RPC Authentication problem with different user locale.
     (Talat UYARER via stevel)
 
+    HADOOP-11482. Use correct UGI when KMSClientProvider is called by a proxy
+    user. Contributed by Arun Suresh.
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e02d860/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
index 0464f55..97ab253 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
@@ -787,25 +787,44 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
   }
 
   @Override
-  public Token<?>[] addDelegationTokens(String renewer,
+  public Token<?>[] addDelegationTokens(final String renewer,
       Credentials credentials) throws IOException {
     Token<?>[] tokens = null;
     Text dtService = getDelegationTokenService();
     Token<?> token = credentials.getToken(dtService);
     if (token == null) {
-      URL url = createURL(null, null, null, null);
-      DelegationTokenAuthenticatedURL authUrl =
+      final URL url = createURL(null, null, null, null);
+      final DelegationTokenAuthenticatedURL authUrl =
           new DelegationTokenAuthenticatedURL(configurator);
       try {
-        token = authUrl.getDelegationToken(url, authToken, renewer);
+        // 'actualUGI' is the UGI of the user creating the client 
+        // It is possible that the creator of the KMSClientProvier
+        // calls this method on behalf of a proxyUser (the doAsUser).
+        // In which case this call has to be made as the proxy user.
+        UserGroupInformation currentUgi = UserGroupInformation.getCurrentUser();
+        final String doAsUser = (currentUgi.getAuthenticationMethod() ==
+            UserGroupInformation.AuthenticationMethod.PROXY)
+                                ? currentUgi.getShortUserName() : null;
+
+        token = actualUgi.doAs(new PrivilegedExceptionAction<Token<?>>() {
+          @Override
+          public Token<?> run() throws Exception {
+            // Not using the cached token here.. Creating a new token here
+            // everytime.
+            return authUrl.getDelegationToken(url,
+                new DelegationTokenAuthenticatedURL.Token(), renewer, doAsUser);
+          }
+        });
         if (token != null) {
           credentials.addToken(token.getService(), token);
           tokens = new Token<?>[] { token };
         } else {
           throw new IOException("Got NULL as delegation token");
         }
-      } catch (AuthenticationException ex) {
-        throw new IOException(ex);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+      } catch (Exception e) {
+        throw new IOException(e);
       }
     }
     return tokens;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e02d860/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java b/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
index f487e98..70ba95f 100644
--- a/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
+++ b/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
@@ -284,6 +284,7 @@ public class TestKMS {
       password = null;
     }
 
+    conf.set("hadoop.kms.authentication.token.validity", "1");
     if (kerberos) {
       conf.set("hadoop.kms.authentication.type", "kerberos");
       conf.set("hadoop.kms.authentication.kerberos.keytab",
@@ -337,6 +338,11 @@ public class TestKMS {
                 final KeyProvider kp = new KMSClientProvider(uri, conf);
                 // getKeys() empty
                 Assert.assertTrue(kp.getKeys().isEmpty());
+
+                Thread.sleep(4000);
+                Token<?>[] tokens = ((KMSClientProvider)kp).addDelegationTokens("myuser", new Credentials());
+                Assert.assertEquals(1, tokens.length);
+                Assert.assertEquals("kms-dt", tokens[0].getKind().toString());
                 return null;
               }
             });
@@ -346,6 +352,7 @@ public class TestKMS {
           // getKeys() empty
           Assert.assertTrue(kp.getKeys().isEmpty());
 
+          Thread.sleep(4000);
           Token<?>[] tokens = ((KMSClientProvider)kp).addDelegationTokens("myuser", new Credentials());
           Assert.assertEquals(1, tokens.length);
           Assert.assertEquals("kms-dt", tokens[0].getKind().toString());


[39/50] [abbrv] hadoop git commit: HDFS-7676. Fix TestFileTruncate to avoid bug of HDFS-7611. Contributed by Konstantin Shvachko.

Posted by zh...@apache.org.
HDFS-7676. Fix TestFileTruncate to avoid bug of HDFS-7611. Contributed by Konstantin Shvachko.


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

Branch: refs/heads/HDFS-EC
Commit: 38cbafd18cf18171ded4e4d0518d67be13145018
Parents: e461d62
Author: Konstantin V Shvachko <sh...@apache.org>
Authored: Sat Jan 24 18:03:11 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:30 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                     | 2 ++
 .../apache/hadoop/hdfs/server/namenode/TestFileTruncate.java    | 5 +++++
 2 files changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/38cbafd1/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 21c8374..a6cbf8f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -785,6 +785,8 @@ Release 2.7.0 - UNRELEASED
     HDFS-7659. truncate should check negative value of the new length.
     (Yi Liu via shv)
 
+    HDFS-7676. Fix TestFileTruncate to avoid bug of HDFS-7611. (shv)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38cbafd1/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 1612a24..e8250a2 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
@@ -496,6 +496,11 @@ public class TestFileTruncate {
    */
   @Test
   public void testTruncateEditLogLoad() throws IOException {
+    // purge previously accumulated edits
+    fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    fs.saveNamespace();
+    fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+
     int startingFileSize = 2 * BLOCK_SIZE + BLOCK_SIZE / 2;
     int toTruncate = 1;
     final String s = "/testTruncateEditLogLoad";


[04/50] [abbrv] hadoop git commit: HDFS-7496. Fix FsVolume removal race conditions on the DataNode by reference-counting the volume instances (lei via cmccabe)

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/9014305e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeListTest.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeListTest.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeListTest.java
new file mode 100644
index 0000000..f92d949
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeListTest.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.hdfs.server.datanode.fsdataset.impl;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystemTestHelper;
+import org.apache.hadoop.hdfs.StorageType;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.RoundRobinVolumeChoosingPolicy;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.VolumeChoosingPolicy;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.junit.Assert.assertNotEquals;
+import static org.mockito.Mockito.mock;
+
+public class FsVolumeListTest {
+
+  private final Configuration conf = new Configuration();
+  private VolumeChoosingPolicy<FsVolumeImpl> blockChooser =
+      new RoundRobinVolumeChoosingPolicy<>();
+  private FsDatasetImpl dataset = null;
+  private String baseDir;
+
+  @Before
+  public void setUp() {
+    dataset = mock(FsDatasetImpl.class);
+    baseDir = new FileSystemTestHelper().getTestRootDir();
+  }
+
+  @Test
+  public void testGetNextVolumeWithClosedVolume() throws IOException {
+    FsVolumeList volumeList = new FsVolumeList(0, blockChooser);
+    List<FsVolumeImpl> volumes = new ArrayList<>();
+    for (int i = 0; i < 3; i++) {
+      File curDir = new File(baseDir, "nextvolume-" + i);
+      curDir.mkdirs();
+      FsVolumeImpl volume = new FsVolumeImpl(dataset, "storage-id", curDir,
+          conf, StorageType.DEFAULT);
+      volume.setCapacityForTesting(1024 * 1024 * 1024);
+      volumes.add(volume);
+      volumeList.addVolume(volume);
+    }
+
+    // Close the second volume.
+    volumes.get(1).closeAndWait();
+    for (int i = 0; i < 10; i++) {
+      try (FsVolumeReference ref =
+          volumeList.getNextVolume(StorageType.DEFAULT, 128)) {
+        // volume No.2 will not be chosen.
+        assertNotEquals(ref.getVolume(), volumes.get(1));
+      }
+    }
+  }
+
+  @Test
+  public void testCheckDirsWithClosedVolume() throws IOException {
+    FsVolumeList volumeList = new FsVolumeList(0, blockChooser);
+    List<FsVolumeImpl> volumes = new ArrayList<>();
+    for (int i = 0; i < 3; i++) {
+      File curDir = new File(baseDir, "volume-" + i);
+      curDir.mkdirs();
+      FsVolumeImpl volume = new FsVolumeImpl(dataset, "storage-id", curDir,
+          conf, StorageType.DEFAULT);
+      volumes.add(volume);
+      volumeList.addVolume(volume);
+    }
+
+    // Close the 2nd volume.
+    volumes.get(1).closeAndWait();
+    // checkDirs() should ignore the 2nd volume since it is closed.
+    volumeList.checkDirs();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9014305e/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 aa4b68c..0120dfe 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
@@ -31,6 +31,7 @@ import org.apache.hadoop.hdfs.server.datanode.DNConf;
 import org.apache.hadoop.hdfs.server.datanode.DataBlockScanner;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataStorage;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaHandler;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.RoundRobinVolumeChoosingPolicy;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
@@ -160,14 +161,16 @@ public class TestFsDatasetImpl {
     assertEquals(actualVolumes, expectedVolumes);
   }
 
-  @Test
+  @Test(timeout = 30000)
   public void testRemoveVolumes() throws IOException {
     // Feed FsDataset with block metadata.
     final int NUM_BLOCKS = 100;
     for (int i = 0; i < NUM_BLOCKS; i++) {
       String bpid = BLOCK_POOL_IDS[NUM_BLOCKS % BLOCK_POOL_IDS.length];
       ExtendedBlock eb = new ExtendedBlock(bpid, i);
-      dataset.createRbw(StorageType.DEFAULT, eb, false);
+      try (ReplicaHandler replica =
+          dataset.createRbw(StorageType.DEFAULT, eb, false)) {
+      }
     }
     final String[] dataDirs =
         conf.get(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY).split(",");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9014305e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestWriteToReplica.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestWriteToReplica.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestWriteToReplica.java
index 60c6d03..5aafc9c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestWriteToReplica.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestWriteToReplica.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaAlreadyExistsException;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaBeingWritten;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipeline;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipelineInterface;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaUnderRecovery;
@@ -148,7 +149,8 @@ public class TestWriteToReplica {
     };
     
     ReplicaMap replicasMap = dataSet.volumeMap;
-    FsVolumeImpl vol = dataSet.volumes.getNextVolume(StorageType.DEFAULT, 0);
+    FsVolumeImpl vol = (FsVolumeImpl) dataSet.volumes
+        .getNextVolume(StorageType.DEFAULT, 0).getVolume();
     ReplicaInfo replicaInfo = new FinalizedReplica(
         blocks[FINALIZED].getLocalBlock(), vol, vol.getCurrentDir().getParentFile());
     replicasMap.add(bpid, replicaInfo);
@@ -157,10 +159,10 @@ public class TestWriteToReplica {
     
     replicasMap.add(bpid, new ReplicaInPipeline(
         blocks[TEMPORARY].getBlockId(),
-        blocks[TEMPORARY].getGenerationStamp(), vol, 
+        blocks[TEMPORARY].getGenerationStamp(), vol,
         vol.createTmpFile(bpid, blocks[TEMPORARY].getLocalBlock()).getParentFile(), 0));
     
-    replicaInfo = new ReplicaBeingWritten(blocks[RBW].getLocalBlock(), vol, 
+    replicaInfo = new ReplicaBeingWritten(blocks[RBW].getLocalBlock(), vol,
         vol.createRbwFile(bpid, blocks[RBW].getLocalBlock()).getParentFile(), null);
     replicasMap.add(bpid, replicaInfo);
     replicaInfo.getBlockFile().createNewFile();
@@ -489,8 +491,8 @@ public class TestWriteToReplica {
     long newGenStamp = blocks[NON_EXISTENT].getGenerationStamp() * 10;
     blocks[NON_EXISTENT].setGenerationStamp(newGenStamp);
     try {
-      ReplicaInPipeline replicaInfo =
-                dataSet.createTemporary(StorageType.DEFAULT, blocks[NON_EXISTENT]);
+      ReplicaInPipelineInterface replicaInfo =
+          dataSet.createTemporary(StorageType.DEFAULT, blocks[NON_EXISTENT]).getReplica();
       Assert.assertTrue(replicaInfo.getGenerationStamp() == newGenStamp);
       Assert.assertTrue(
           replicaInfo.getBlockId() == blocks[NON_EXISTENT].getBlockId());


[14/50] [abbrv] hadoop git commit: HADOOP-11256. Some site docs have inconsistent appearance (Masatake Iwasaki via aw)

Posted by zh...@apache.org.
HADOOP-11256. Some site docs have inconsistent appearance (Masatake Iwasaki via aw)


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

Branch: refs/heads/HDFS-EC
Commit: bd457d3ac613ed1a780a886774b512c638e37478
Parents: cab6f96
Author: Allen Wittenauer <aw...@apache.org>
Authored: Wed Jan 21 08:44:22 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:26 2015 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 ++
 .../src/site/resources/css/site.css             | 30 ++++++++++++++++++++
 .../src/site/resources/css/site.css             | 30 ++++++++++++++++++++
 .../src/site/resources/css/site.css             | 30 ++++++++++++++++++++
 4 files changed, 93 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bd457d3a/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 339ccfb..2951002 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -510,6 +510,9 @@ Release 2.7.0 - UNRELEASED
 
   BUG FIXES
 
+    HADOOP-11256. Some site docs have inconsistent appearance (Masatake 
+    Iwasaki via aw)
+
     HADOOP-11318. Update the document for hadoop fs -stat (aajisaka)
 
     HADOOP 11400. GraphiteSink does not reconnect to Graphite after 'broken pipe' 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bd457d3a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/site/resources/css/site.css
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/site/resources/css/site.css b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/site/resources/css/site.css
new file mode 100644
index 0000000..f830baa
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/site/resources/css/site.css
@@ -0,0 +1,30 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+#banner {
+  height: 93px;
+  background: none;
+}
+
+#bannerLeft img {
+  margin-left: 30px;
+  margin-top: 10px;
+}
+
+#bannerRight img {
+  margin: 17px;
+}
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bd457d3a/hadoop-tools/hadoop-gridmix/src/site/resources/css/site.css
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-gridmix/src/site/resources/css/site.css b/hadoop-tools/hadoop-gridmix/src/site/resources/css/site.css
new file mode 100644
index 0000000..f830baa
--- /dev/null
+++ b/hadoop-tools/hadoop-gridmix/src/site/resources/css/site.css
@@ -0,0 +1,30 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+#banner {
+  height: 93px;
+  background: none;
+}
+
+#bannerLeft img {
+  margin-left: 30px;
+  margin-top: 10px;
+}
+
+#bannerRight img {
+  margin: 17px;
+}
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bd457d3a/hadoop-tools/hadoop-rumen/src/site/resources/css/site.css
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-rumen/src/site/resources/css/site.css b/hadoop-tools/hadoop-rumen/src/site/resources/css/site.css
new file mode 100644
index 0000000..f830baa
--- /dev/null
+++ b/hadoop-tools/hadoop-rumen/src/site/resources/css/site.css
@@ -0,0 +1,30 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+#banner {
+  height: 93px;
+  background: none;
+}
+
+#bannerLeft img {
+  margin-left: 30px;
+  margin-top: 10px;
+}
+
+#bannerRight img {
+  margin: 17px;
+}
+


[03/50] [abbrv] hadoop git commit: HDFS-7610. Fix removal of dynamically added DN volumes (Lei (Eddy) Xu via Colin P. McCabe)

Posted by zh...@apache.org.
HDFS-7610. Fix removal of dynamically added DN volumes (Lei (Eddy) Xu via Colin P. McCabe)


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

Branch: refs/heads/HDFS-EC
Commit: ef4453dba91e70165da33a14bf2688b7222ae508
Parents: 8c130ae
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Tue Jan 20 20:11:09 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:25 2015 -0800

----------------------------------------------------------------------
 .../datanode/fsdataset/impl/FsDatasetImpl.java  | 16 +++++----
 .../datanode/fsdataset/impl/FsVolumeList.java   |  8 +++--
 .../fsdataset/impl/TestFsDatasetImpl.java       | 37 ++++++++++++++++++--
 3 files changed, 49 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef4453db/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
index 5347323..d8cc287 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
@@ -342,7 +342,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
 
     StorageType storageType = location.getStorageType();
     final FsVolumeImpl fsVolume = new FsVolumeImpl(
-        this, sd.getStorageUuid(), dir, this.conf, storageType);
+        this, sd.getStorageUuid(), sd.getCurrentDir(), this.conf, storageType);
     final ReplicaMap tempVolumeMap = new ReplicaMap(fsVolume);
     ArrayList<IOException> exceptions = Lists.newArrayList();
 
@@ -385,19 +385,19 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
    */
   @Override
   public synchronized void removeVolumes(Collection<StorageLocation> volumes) {
-    Set<File> volumeSet = new HashSet<File>();
+    Set<String> volumeSet = new HashSet<>();
     for (StorageLocation sl : volumes) {
-      volumeSet.add(sl.getFile());
+      volumeSet.add(sl.getFile().getAbsolutePath());
     }
     for (int idx = 0; idx < dataStorage.getNumStorageDirs(); idx++) {
       Storage.StorageDirectory sd = dataStorage.getStorageDir(idx);
-      if (volumeSet.contains(sd.getRoot())) {
-        String volume = sd.getRoot().toString();
+      String volume = sd.getRoot().getAbsolutePath();
+      if (volumeSet.contains(volume)) {
         LOG.info("Removing " + volume + " from FsDataset.");
 
         // Disable the volume from the service.
         asyncDiskService.removeVolume(sd.getCurrentDir());
-        this.volumes.removeVolume(volume);
+        this.volumes.removeVolume(sd.getRoot());
 
         // Removed all replica information for the blocks on the volume. Unlike
         // updating the volumeMap in addVolume(), this operation does not scan
@@ -407,7 +407,9 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
           for (Iterator<ReplicaInfo> it = volumeMap.replicas(bpid).iterator();
               it.hasNext(); ) {
             ReplicaInfo block = it.next();
-            if (block.getVolume().getBasePath().equals(volume)) {
+            String absBasePath =
+                  new File(block.getVolume().getBasePath()).getAbsolutePath();
+            if (absBasePath.equals(volume)) {
               invalidate(bpid, block);
               blocks.add(block);
               it.remove();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef4453db/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java
index ba19897..c837593 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
 
+import java.io.File;
 import java.io.IOException;
 import java.nio.channels.ClosedChannelException;
 import java.util.ArrayList;
@@ -322,13 +323,16 @@ class FsVolumeList {
    * Dynamically remove volume in the list.
    * @param volume the volume to be removed.
    */
-  void removeVolume(String volume) {
+  void removeVolume(File volume) {
     // Make a copy of volumes to remove one volume.
     final FsVolumeImpl[] curVolumes = volumes.get();
     final List<FsVolumeImpl> volumeList = Lists.newArrayList(curVolumes);
     for (Iterator<FsVolumeImpl> it = volumeList.iterator(); it.hasNext(); ) {
       FsVolumeImpl fsVolume = it.next();
-      if (fsVolume.getBasePath().equals(volume)) {
+      String basePath, targetPath;
+      basePath = new File(fsVolume.getBasePath()).getAbsolutePath();
+      targetPath = volume.getAbsolutePath();
+      if (basePath.equals(targetPath)) {
         // Make sure the removed volume is the one in the curVolumes.
         removeVolume(fsVolume);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef4453db/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 0120dfe..ca936b3 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
@@ -46,6 +46,7 @@ import org.mockito.stubbing.Answer;
 import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
@@ -70,6 +71,7 @@ public class TestFsDatasetImpl {
   private static final String BASE_DIR =
       new FileSystemTestHelper().getTestRootDir();
   private static final int NUM_INIT_VOLUMES = 2;
+  private static final String CLUSTER_ID = "cluser-id";
   private static final String[] BLOCK_POOL_IDS = {"bpid-0", "bpid-1"};
 
   // Use to generate storageUuid
@@ -136,10 +138,11 @@ public class TestFsDatasetImpl {
     Set<String> expectedVolumes = new HashSet<String>();
     List<NamespaceInfo> nsInfos = Lists.newArrayList();
     for (String bpid : BLOCK_POOL_IDS) {
-      nsInfos.add(new NamespaceInfo(0, "cluster-id", bpid, 1));
+      nsInfos.add(new NamespaceInfo(0, CLUSTER_ID, bpid, 1));
     }
     for (int i = 0; i < numNewVolumes; i++) {
       String path = BASE_DIR + "/newData" + i;
+      expectedVolumes.add(path);
       StorageLocation loc = StorageLocation.parse(path);
       Storage.StorageDirectory sd = createStorageDirectory(new File(path));
       DataStorage.VolumeBuilder builder =
@@ -156,7 +159,8 @@ public class TestFsDatasetImpl {
 
     Set<String> actualVolumes = new HashSet<String>();
     for (int i = 0; i < numNewVolumes; i++) {
-      dataset.getVolumes().get(numExistingVolumes + i).getBasePath();
+      actualVolumes.add(
+          dataset.getVolumes().get(numExistingVolumes + i).getBasePath());
     }
     assertEquals(actualVolumes, expectedVolumes);
   }
@@ -211,6 +215,33 @@ public class TestFsDatasetImpl {
   }
 
   @Test(timeout = 5000)
+  public void testRemoveNewlyAddedVolume() throws IOException {
+    final int numExistingVolumes = dataset.getVolumes().size();
+    List<NamespaceInfo> nsInfos = new ArrayList<>();
+    for (String bpid : BLOCK_POOL_IDS) {
+      nsInfos.add(new NamespaceInfo(0, CLUSTER_ID, bpid, 1));
+    }
+    String newVolumePath = BASE_DIR + "/newVolumeToRemoveLater";
+    StorageLocation loc = StorageLocation.parse(newVolumePath);
+
+    Storage.StorageDirectory sd = createStorageDirectory(new File(newVolumePath));
+    DataStorage.VolumeBuilder builder =
+        new DataStorage.VolumeBuilder(storage, sd);
+    when(storage.prepareVolume(eq(datanode), eq(loc.getFile()),
+        anyListOf(NamespaceInfo.class)))
+        .thenReturn(builder);
+
+    dataset.addVolume(loc, nsInfos);
+    assertEquals(numExistingVolumes + 1, dataset.getVolumes().size());
+
+    when(storage.getNumStorageDirs()).thenReturn(numExistingVolumes + 1);
+    when(storage.getStorageDir(numExistingVolumes)).thenReturn(sd);
+    List<StorageLocation> volumesToRemove = Arrays.asList(loc);
+    dataset.removeVolumes(volumesToRemove);
+    assertEquals(numExistingVolumes, dataset.getVolumes().size());
+  }
+
+  @Test(timeout = 5000)
   public void testChangeVolumeWithRunningCheckDirs() throws IOException {
     RoundRobinVolumeChoosingPolicy<FsVolumeImpl> blockChooser =
         new RoundRobinVolumeChoosingPolicy<>();
@@ -234,7 +265,7 @@ public class TestFsDatasetImpl {
       @Override
       public Object answer(InvocationOnMock invocationOnMock)
           throws Throwable {
-        volumeList.removeVolume("data4");
+        volumeList.removeVolume(new File("data4"));
         volumeList.addVolume(newVolume);
         return null;
       }


[10/50] [abbrv] hadoop git commit: HADOOP-10668. Addendum patch to fix TestZKFailoverController. Contributed by Ming Ma.

Posted by zh...@apache.org.
HADOOP-10668. Addendum patch to fix TestZKFailoverController. 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/ce169494
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ce169494
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ce169494

Branch: refs/heads/HDFS-EC
Commit: ce1694941c20bd42291c35503392e8bdcf84adeb
Parents: 8a690f9
Author: cnauroth <cn...@apache.org>
Authored: Wed Jan 21 11:52:48 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:26 2015 -0800

----------------------------------------------------------------------
 .../test/java/org/apache/hadoop/ha/MiniZKFCCluster.java   |  5 +++++
 .../org/apache/hadoop/ha/TestZKFailoverController.java    | 10 +++++-----
 2 files changed, 10 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce169494/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/MiniZKFCCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/MiniZKFCCluster.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/MiniZKFCCluster.java
index cab59a4..5aee611 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/MiniZKFCCluster.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/MiniZKFCCluster.java
@@ -155,6 +155,11 @@ public class MiniZKFCCluster {
 
   /**
    * Wait for the given HA service to enter the given HA state.
+   * This is based on the state of ZKFC, not the state of HA service.
+   * There could be difference between the two. For example,
+   * When the service becomes unhealthy, ZKFC will quit ZK election and
+   * transition to HAServiceState.INITIALIZING and remain in that state
+   * until the service becomes healthy.
    */
   public void waitForHAState(int idx, HAServiceState state)
       throws Exception {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce169494/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestZKFailoverController.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestZKFailoverController.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestZKFailoverController.java
index 83a29dd..d8271c5 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestZKFailoverController.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestZKFailoverController.java
@@ -211,8 +211,8 @@ public class TestZKFailoverController extends ClientBaseWithFixes {
       LOG.info("Faking svc0 unhealthy, should failover to svc1");
       cluster.setHealthy(0, false);
       
-      LOG.info("Waiting for svc0 to enter standby state");
-      cluster.waitForHAState(0, HAServiceState.STANDBY);
+      LOG.info("Waiting for svc0 to enter initializing state");
+      cluster.waitForHAState(0, HAServiceState.INITIALIZING);
       cluster.waitForHAState(1, HAServiceState.ACTIVE);
   
       LOG.info("Allowing svc0 to be healthy again, making svc1 unreachable " +
@@ -332,7 +332,7 @@ public class TestZKFailoverController extends ClientBaseWithFixes {
       Mockito.verify(svc1.proxy, Mockito.timeout(2000).atLeastOnce())
         .transitionToActive(Mockito.<StateChangeRequestInfo>any());
 
-      cluster.waitForHAState(0, HAServiceState.STANDBY);
+      cluster.waitForHAState(0, HAServiceState.INITIALIZING);
       cluster.waitForHAState(1, HAServiceState.STANDBY);
       
       LOG.info("Faking svc0 healthy again, should go back to svc0");
@@ -587,12 +587,12 @@ public class TestZKFailoverController extends ClientBaseWithFixes {
 
       // Failover by bad health
       cluster.setHealthy(0, false);
-      cluster.waitForHAState(0, HAServiceState.STANDBY);
+      cluster.waitForHAState(0, HAServiceState.INITIALIZING);
       cluster.waitForHAState(1, HAServiceState.ACTIVE);
       cluster.setHealthy(1, true);
       cluster.setHealthy(0, false);
       cluster.waitForHAState(1, HAServiceState.ACTIVE);
-      cluster.waitForHAState(0, HAServiceState.STANDBY);
+      cluster.waitForHAState(0, HAServiceState.INITIALIZING);
       cluster.setHealthy(0, true);
       
       cluster.waitForHealthState(0, State.SERVICE_HEALTHY);


[05/50] [abbrv] hadoop git commit: HDFS-7496. Fix FsVolume removal race conditions on the DataNode by reference-counting the volume instances (lei via cmccabe)

Posted by zh...@apache.org.
HDFS-7496. Fix FsVolume removal race conditions on the DataNode by reference-counting the volume instances (lei via cmccabe)


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

Branch: refs/heads/HDFS-EC
Commit: 9014305e0bb8cca43fd886562bf7fade545f58ac
Parents: a6d1f21
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Tue Jan 20 19:05:33 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:25 2015 -0800

----------------------------------------------------------------------
 .../hdfs/server/datanode/BlockReceiver.java     |  58 ++--
 .../hdfs/server/datanode/BlockSender.java       |  10 +
 .../hdfs/server/datanode/ReplicaHandler.java    |  49 ++++
 .../server/datanode/fsdataset/FsDatasetSpi.java |  13 +-
 .../datanode/fsdataset/FsVolumeReference.java   |  48 ++++
 .../server/datanode/fsdataset/FsVolumeSpi.java  |  10 +
 .../datanode/fsdataset/ReplicaInputStreams.java |   6 +-
 .../impl/FsDatasetAsyncDiskService.java         |  16 +-
 .../datanode/fsdataset/impl/FsDatasetImpl.java  | 284 ++++++++++++-------
 .../datanode/fsdataset/impl/FsVolumeImpl.java   | 121 +++++++-
 .../datanode/fsdataset/impl/FsVolumeList.java   |  76 ++++-
 .../impl/RamDiskAsyncLazyPersistService.java    |  19 +-
 .../src/main/proto/datatransfer.proto           |   2 +-
 .../hdfs/TestWriteBlockGetsBlockLengthHint.java |   2 +-
 .../server/datanode/SimulatedFSDataset.java     |  35 ++-
 .../hdfs/server/datanode/TestBlockRecovery.java |   2 +-
 .../datanode/TestDataNodeHotSwapVolumes.java    |  33 ++-
 .../server/datanode/TestDirectoryScanner.java   |   9 +-
 .../server/datanode/TestSimulatedFSDataset.java |   2 +-
 .../extdataset/ExternalDatasetImpl.java         |  22 +-
 .../datanode/extdataset/ExternalVolumeImpl.java |   7 +
 .../fsdataset/impl/FsVolumeListTest.java        |  94 ++++++
 .../fsdataset/impl/TestFsDatasetImpl.java       |   7 +-
 .../fsdataset/impl/TestWriteToReplica.java      |  12 +-
 24 files changed, 717 insertions(+), 220 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9014305e/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 08c96be..df8dd5c 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
@@ -26,7 +26,6 @@ import java.io.DataOutputStream;
 import java.io.File;
 import java.io.FileDescriptor;
 import java.io.FileOutputStream;
-import java.io.FileWriter;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.io.OutputStreamWriter;
@@ -49,10 +48,8 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver;
 import org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaInputStreams;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
-import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipeline;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.util.DataTransferThrottler;
 import org.apache.hadoop.io.IOUtils;
@@ -125,6 +122,8 @@ class BlockReceiver implements Closeable {
 
   private boolean syncOnClose;
   private long restartBudget;
+  /** the reference of the volume where the block receiver writes to */
+  private final ReplicaHandler replicaHandler;
 
   /**
    * for replaceBlock response
@@ -179,48 +178,50 @@ class BlockReceiver implements Closeable {
       // Open local disk out
       //
       if (isDatanode) { //replication or move
-        replicaInfo = datanode.data.createTemporary(storageType, block);
+        replicaHandler = datanode.data.createTemporary(storageType, block);
       } else {
         switch (stage) {
         case PIPELINE_SETUP_CREATE:
-          replicaInfo = datanode.data.createRbw(storageType, block, allowLazyPersist);
+          replicaHandler = datanode.data.createRbw(storageType, block, allowLazyPersist);
           datanode.notifyNamenodeReceivingBlock(
-              block, replicaInfo.getStorageUuid());
+              block, replicaHandler.getReplica().getStorageUuid());
           break;
         case PIPELINE_SETUP_STREAMING_RECOVERY:
-          replicaInfo = datanode.data.recoverRbw(
+          replicaHandler = datanode.data.recoverRbw(
               block, newGs, minBytesRcvd, maxBytesRcvd);
           block.setGenerationStamp(newGs);
           break;
         case PIPELINE_SETUP_APPEND:
-          replicaInfo = datanode.data.append(block, newGs, minBytesRcvd);
+          replicaHandler = datanode.data.append(block, newGs, minBytesRcvd);
           if (datanode.blockScanner != null) { // remove from block scanner
             datanode.blockScanner.deleteBlock(block.getBlockPoolId(),
                 block.getLocalBlock());
           }
           block.setGenerationStamp(newGs);
           datanode.notifyNamenodeReceivingBlock(
-              block, replicaInfo.getStorageUuid());
+              block, replicaHandler.getReplica().getStorageUuid());
           break;
         case PIPELINE_SETUP_APPEND_RECOVERY:
-          replicaInfo = datanode.data.recoverAppend(block, newGs, minBytesRcvd);
+          replicaHandler = datanode.data.recoverAppend(block, newGs, minBytesRcvd);
           if (datanode.blockScanner != null) { // remove from block scanner
             datanode.blockScanner.deleteBlock(block.getBlockPoolId(),
                 block.getLocalBlock());
           }
           block.setGenerationStamp(newGs);
           datanode.notifyNamenodeReceivingBlock(
-              block, replicaInfo.getStorageUuid());
+              block, replicaHandler.getReplica().getStorageUuid());
           break;
         case TRANSFER_RBW:
         case TRANSFER_FINALIZED:
           // this is a transfer destination
-          replicaInfo = datanode.data.createTemporary(storageType, block);
+          replicaHandler =
+              datanode.data.createTemporary(storageType, block);
           break;
         default: throw new IOException("Unsupported stage " + stage + 
               " while receiving block " + block + " from " + inAddr);
         }
       }
+      replicaInfo = replicaHandler.getReplica();
       this.dropCacheBehindWrites = (cachingStrategy.getDropBehind() == null) ?
         datanode.getDnConf().dropCacheBehindWrites :
           cachingStrategy.getDropBehind();
@@ -339,6 +340,9 @@ class BlockReceiver implements Closeable {
     finally{
       IOUtils.closeStream(out);
     }
+    if (replicaHandler != null) {
+      IOUtils.cleanup(null, replicaHandler);
+    }
     if (measuredFlushTime) {
       datanode.metrics.addFlushNanos(flushTotalNanos);
     }
@@ -950,15 +954,12 @@ class BlockReceiver implements Closeable {
     //
     byte[] buf = new byte[sizePartialChunk];
     byte[] crcbuf = new byte[checksumSize];
-    ReplicaInputStreams instr = null;
-    try { 
-      instr = datanode.data.getTmpInputStreams(block, blkoff, ckoff);
+    try (ReplicaInputStreams instr =
+        datanode.data.getTmpInputStreams(block, blkoff, ckoff)) {
       IOUtils.readFully(instr.getDataIn(), buf, 0, sizePartialChunk);
 
       // open meta file and read in crc value computer earlier
       IOUtils.readFully(instr.getChecksumIn(), crcbuf, 0, crcbuf.length);
-    } finally {
-      IOUtils.closeStream(instr);
     }
 
     // compute crc of partial chunk from data read in the block file.
@@ -1244,28 +1245,7 @@ class BlockReceiver implements Closeable {
 
           if (lastPacketInBlock) {
             // Finalize the block and close the block file
-            try {
-              finalizeBlock(startTime);
-            } catch (ReplicaNotFoundException e) {
-              // Verify that the exception is due to volume removal.
-              FsVolumeSpi volume;
-              synchronized (datanode.data) {
-                volume = datanode.data.getVolume(block);
-              }
-              if (volume == null) {
-                // ReplicaInfo has been removed due to the corresponding data
-                // volume has been removed. Don't need to check disk error.
-                LOG.info(myString
-                    + ": BlockReceiver is interrupted because the block pool "
-                    + block.getBlockPoolId() + " has been removed.", e);
-                sendAckUpstream(ack, expected, totalAckTimeNanos, 0,
-                    Status.OOB_INTERRUPTED);
-                running = false;
-                receiverThread.interrupt();
-                continue;
-              }
-              throw e;
-            }
+            finalizeBlock(startTime);
           }
 
           sendAckUpstream(ack, expected, totalAckTimeNanos,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9014305e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
index 27d3e5c..182b366 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.fs.ChecksumException;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
 import org.apache.hadoop.hdfs.util.DataTransferThrottler;
 import org.apache.hadoop.io.IOUtils;
@@ -143,6 +144,8 @@ class BlockSender implements java.io.Closeable {
   
   /** The file descriptor of the block being sent */
   private FileDescriptor blockInFd;
+  /** The reference to the volume where the block is located */
+  private FsVolumeReference volumeRef;
 
   // Cache-management related fields
   private final long readaheadLength;
@@ -257,6 +260,9 @@ class BlockSender implements java.io.Closeable {
       this.transferToAllowed = datanode.getDnConf().transferToAllowed &&
         (!is32Bit || length <= Integer.MAX_VALUE);
 
+      // Obtain a reference before reading data
+      this.volumeRef = datanode.data.getVolume(block).obtainReference();
+
       /* 
        * (corruptChecksumOK, meta_file_exist): operation
        * True,   True: will verify checksum  
@@ -420,6 +426,10 @@ class BlockSender implements java.io.Closeable {
       blockIn = null;
       blockInFd = null;
     }
+    if (volumeRef != null) {
+      IOUtils.cleanup(null, volumeRef);
+      volumeRef = null;
+    }
     // throw IOException if there is any
     if(ioe!= null) {
       throw ioe;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9014305e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaHandler.java
new file mode 100644
index 0000000..b563d7f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaHandler.java
@@ -0,0 +1,49 @@
+/**
+ * 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.datanode;
+
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+ * This class includes a replica being actively written and the reference to
+ * the fs volume where this replica is located.
+ */
+public class ReplicaHandler implements Closeable {
+  private final ReplicaInPipelineInterface replica;
+  private final FsVolumeReference volumeReference;
+
+  public ReplicaHandler(
+      ReplicaInPipelineInterface replica, FsVolumeReference reference) {
+    this.replica = replica;
+    this.volumeReference = reference;
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (this.volumeReference != null) {
+      volumeReference.close();
+    }
+  }
+
+  public ReplicaInPipelineInterface getReplica() {
+    return replica;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9014305e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
index da8e6d8..0d5de81 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
@@ -43,6 +43,7 @@ import org.apache.hadoop.hdfs.server.datanode.DataStorage;
 import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica;
 import org.apache.hadoop.hdfs.server.datanode.Replica;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipelineInterface;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaHandler;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
@@ -198,7 +199,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @return the meta info of the replica which is being written to
    * @throws IOException if an error occurs
    */
-  public ReplicaInPipelineInterface createTemporary(StorageType storageType,
+  public ReplicaHandler createTemporary(StorageType storageType,
       ExtendedBlock b) throws IOException;
 
   /**
@@ -208,7 +209,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @return the meta info of the replica which is being written to
    * @throws IOException if an error occurs
    */
-  public ReplicaInPipelineInterface createRbw(StorageType storageType,
+  public ReplicaHandler createRbw(StorageType storageType,
       ExtendedBlock b, boolean allowLazyPersist) throws IOException;
 
   /**
@@ -221,7 +222,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @return the meta info of the replica which is being written to
    * @throws IOException if an error occurs
    */
-  public ReplicaInPipelineInterface recoverRbw(ExtendedBlock b, 
+  public ReplicaHandler recoverRbw(ExtendedBlock b,
       long newGS, long minBytesRcvd, long maxBytesRcvd) throws IOException;
 
   /**
@@ -241,7 +242,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @return the meata info of the replica which is being written to
    * @throws IOException
    */
-  public ReplicaInPipelineInterface append(ExtendedBlock b, long newGS,
+  public ReplicaHandler append(ExtendedBlock b, long newGS,
       long expectedBlockLen) throws IOException;
 
   /**
@@ -254,8 +255,8 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @return the meta info of the replica which is being written to
    * @throws IOException
    */
-  public ReplicaInPipelineInterface recoverAppend(ExtendedBlock b, long newGS,
-      long expectedBlockLen) throws IOException;
+  public ReplicaHandler recoverAppend(
+      ExtendedBlock b, long newGS, long expectedBlockLen) throws IOException;
   
   /**
    * Recover a failed pipeline close

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9014305e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeReference.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeReference.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeReference.java
new file mode 100644
index 0000000..e61a059
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeReference.java
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode.fsdataset;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+ * This is the interface for holding reference count as AutoClosable resource.
+ * It increases the reference count by one in the constructor, and decreases
+ * the reference count by one in {@link #close()}.
+ *
+ * <pre>
+ *  {@code
+ *    try (FsVolumeReference ref = volume.obtainReference()) {
+ *      // Do IOs on the volume
+ *      volume.createRwb(...);
+ *      ...
+ *    }
+ *  }
+ * </pre>
+ */
+public interface FsVolumeReference extends Closeable {
+  /**
+   * Descrese the reference count of the volume.
+   * @throws IOException it never throws IOException.
+   */
+  @Override
+  public void close() throws IOException;
+
+  /** Returns the underlying volume object */
+  public FsVolumeSpi getVolume();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9014305e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
index 8ebf2b4..3a635b7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.datanode.fsdataset;
 
 import java.io.File;
 import java.io.IOException;
+import java.nio.channels.ClosedChannelException;
 
 import org.apache.hadoop.hdfs.StorageType;
 
@@ -26,6 +27,15 @@ import org.apache.hadoop.hdfs.StorageType;
  * This is an interface for the underlying volume.
  */
 public interface FsVolumeSpi {
+  /**
+   * Obtain a reference object that had increased 1 reference count of the
+   * volume.
+   *
+   * It is caller's responsibility to close {@link FsVolumeReference} to decrease
+   * the reference count on the volume.
+   */
+  FsVolumeReference obtainReference() throws ClosedChannelException;
+
   /** @return the StorageUuid of the volume */
   public String getStorageID();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9014305e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/ReplicaInputStreams.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/ReplicaInputStreams.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/ReplicaInputStreams.java
index 6bd7199..a8bf622 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/ReplicaInputStreams.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/ReplicaInputStreams.java
@@ -30,9 +30,12 @@ import org.apache.hadoop.io.IOUtils;
 public class ReplicaInputStreams implements Closeable {
   private final InputStream dataIn;
   private final InputStream checksumIn;
+  private final FsVolumeReference volumeRef;
 
   /** Create an object with a data input stream and a checksum input stream. */
-  public ReplicaInputStreams(FileDescriptor dataFd, FileDescriptor checksumFd) {
+  public ReplicaInputStreams(FileDescriptor dataFd, FileDescriptor checksumFd,
+      FsVolumeReference volumeRef) {
+    this.volumeRef = volumeRef;
     this.dataIn = new FileInputStream(dataFd);
     this.checksumIn = new FileInputStream(checksumFd);
   }
@@ -51,5 +54,6 @@ public class ReplicaInputStreams implements Closeable {
   public void close() {
     IOUtils.closeStream(dataIn);
     IOUtils.closeStream(checksumIn);
+    IOUtils.cleanup(null, volumeRef);
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9014305e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java
index bee7bf7..13e854f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
 
 import java.io.File;
 import java.io.FileDescriptor;
+import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.concurrent.LinkedBlockingQueue;
@@ -31,7 +32,9 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.io.nativeio.NativeIOException;
 
@@ -200,13 +203,13 @@ class FsDatasetAsyncDiskService {
    * Delete the block file and meta file from the disk asynchronously, adjust
    * dfsUsed statistics accordingly.
    */
-  void deleteAsync(FsVolumeImpl volume, File blockFile, File metaFile,
+  void deleteAsync(FsVolumeReference volumeRef, File blockFile, File metaFile,
       ExtendedBlock block, String trashDirectory) {
     LOG.info("Scheduling " + block.getLocalBlock()
         + " file " + blockFile + " for deletion");
     ReplicaFileDeleteTask deletionTask = new ReplicaFileDeleteTask(
-        volume, blockFile, metaFile, block, trashDirectory);
-    execute(volume.getCurrentDir(), deletionTask);
+        volumeRef, blockFile, metaFile, block, trashDirectory);
+    execute(((FsVolumeImpl) volumeRef.getVolume()).getCurrentDir(), deletionTask);
   }
   
   /** A task for deleting a block file and its associated meta file, as well
@@ -216,15 +219,17 @@ class FsDatasetAsyncDiskService {
    *  files are deleted immediately.
    */
   class ReplicaFileDeleteTask implements Runnable {
+    final FsVolumeReference volumeRef;
     final FsVolumeImpl volume;
     final File blockFile;
     final File metaFile;
     final ExtendedBlock block;
     final String trashDirectory;
     
-    ReplicaFileDeleteTask(FsVolumeImpl volume, File blockFile,
+    ReplicaFileDeleteTask(FsVolumeReference volumeRef, File blockFile,
         File metaFile, ExtendedBlock block, String trashDirectory) {
-      this.volume = volume;
+      this.volumeRef = volumeRef;
+      this.volume = (FsVolumeImpl) volumeRef.getVolume();
       this.blockFile = blockFile;
       this.metaFile = metaFile;
       this.block = block;
@@ -281,6 +286,7 @@ class FsDatasetAsyncDiskService {
         LOG.info("Deleted " + block.getBlockPoolId() + " "
             + block.getLocalBlock() + " file " + blockFile);
       }
+      IOUtils.cleanup(null, volumeRef);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9014305e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
index e62986f..5347323 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
@@ -29,6 +29,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.RandomAccessFile;
 import java.nio.ByteBuffer;
+import java.nio.channels.ClosedChannelException;
 import java.nio.channels.FileChannel;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -75,6 +76,7 @@ import org.apache.hadoop.hdfs.server.datanode.Replica;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaAlreadyExistsException;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaBeingWritten;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipeline;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaHandler;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaUnderRecovery;
@@ -82,6 +84,7 @@ import org.apache.hadoop.hdfs.server.datanode.ReplicaWaitingToBeRecovered;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
 import org.apache.hadoop.hdfs.server.datanode.UnexpectedReplicaStateException;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
+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.LengthInputStream;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaInputStreams;
@@ -137,22 +140,26 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
   @Override // FsDatasetSpi
   public StorageReport[] getStorageReports(String bpid)
       throws IOException {
-    StorageReport[] reports;
+    List<StorageReport> reports;
     synchronized (statsLock) {
       List<FsVolumeImpl> curVolumes = getVolumes();
-      reports = new StorageReport[curVolumes.size()];
-      int i = 0;
+      reports = new ArrayList<>(curVolumes.size());
       for (FsVolumeImpl volume : curVolumes) {
-        reports[i++] = new StorageReport(volume.toDatanodeStorage(),
-                                         false,
-                                         volume.getCapacity(),
-                                         volume.getDfsUsed(),
-                                         volume.getAvailable(),
-                                         volume.getBlockPoolUsed(bpid));
+        try (FsVolumeReference ref = volume.obtainReference()) {
+          StorageReport sr = new StorageReport(volume.toDatanodeStorage(),
+              false,
+              volume.getCapacity(),
+              volume.getDfsUsed(),
+              volume.getAvailable(),
+              volume.getBlockPoolUsed(bpid));
+          reports.add(sr);
+        } catch (ClosedChannelException e) {
+          continue;
+        }
       }
     }
 
-    return reports;
+    return reports.toArray(new StorageReport[reports.size()]);
   }
 
   @Override
@@ -622,17 +629,24 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
   public synchronized ReplicaInputStreams getTmpInputStreams(ExtendedBlock b, 
                           long blkOffset, long ckoff) throws IOException {
     ReplicaInfo info = getReplicaInfo(b);
-    File blockFile = info.getBlockFile();
-    RandomAccessFile blockInFile = new RandomAccessFile(blockFile, "r");
-    if (blkOffset > 0) {
-      blockInFile.seek(blkOffset);
-    }
-    File metaFile = info.getMetaFile();
-    RandomAccessFile metaInFile = new RandomAccessFile(metaFile, "r");
-    if (ckoff > 0) {
-      metaInFile.seek(ckoff);
+    FsVolumeReference ref = info.getVolume().obtainReference();
+    try {
+      File blockFile = info.getBlockFile();
+      RandomAccessFile blockInFile = new RandomAccessFile(blockFile, "r");
+      if (blkOffset > 0) {
+        blockInFile.seek(blkOffset);
+      }
+      File metaFile = info.getMetaFile();
+      RandomAccessFile metaInFile = new RandomAccessFile(metaFile, "r");
+      if (ckoff > 0) {
+        metaInFile.seek(ckoff);
+      }
+      return new ReplicaInputStreams(
+          blockInFile.getFD(), metaInFile.getFD(), ref);
+    } catch (IOException e) {
+      IOUtils.cleanup(null, ref);
+      throw e;
     }
-    return new ReplicaInputStreams(blockInFile.getFD(), metaInFile.getFD());
   }
 
   static File moveBlockFiles(Block b, File srcfile, File destdir)
@@ -732,26 +746,27 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
           + replicaInfo.getVolume().getStorageType());
     }
 
-    FsVolumeImpl targetVolume = volumes.getNextVolume(targetStorageType,
-        block.getNumBytes());
-    File oldBlockFile = replicaInfo.getBlockFile();
-    File oldMetaFile = replicaInfo.getMetaFile();
+    try (FsVolumeReference volumeRef = volumes.getNextVolume(
+        targetStorageType, block.getNumBytes())) {
+      File oldBlockFile = replicaInfo.getBlockFile();
+      File oldMetaFile = replicaInfo.getMetaFile();
+      FsVolumeImpl targetVolume = (FsVolumeImpl) volumeRef.getVolume();
+      // Copy files to temp dir first
+      File[] blockFiles = copyBlockFiles(block.getBlockId(),
+          block.getGenerationStamp(), oldMetaFile, oldBlockFile,
+          targetVolume.getTmpDir(block.getBlockPoolId()),
+          replicaInfo.isOnTransientStorage());
 
-    // Copy files to temp dir first
-    File[] blockFiles = copyBlockFiles(block.getBlockId(),
-        block.getGenerationStamp(), oldMetaFile, oldBlockFile,
-        targetVolume.getTmpDir(block.getBlockPoolId()),
-        replicaInfo.isOnTransientStorage());
+      ReplicaInfo newReplicaInfo = new ReplicaInPipeline(
+          replicaInfo.getBlockId(), replicaInfo.getGenerationStamp(),
+          targetVolume, blockFiles[0].getParentFile(), 0);
+      newReplicaInfo.setNumBytes(blockFiles[1].length());
+      // Finalize the copied files
+      newReplicaInfo = finalizeReplica(block.getBlockPoolId(), newReplicaInfo);
 
-    ReplicaInfo newReplicaInfo = new ReplicaInPipeline(
-        replicaInfo.getBlockId(), replicaInfo.getGenerationStamp(),
-        targetVolume, blockFiles[0].getParentFile(), 0);
-    newReplicaInfo.setNumBytes(blockFiles[1].length());
-    // Finalize the copied files
-    newReplicaInfo = finalizeReplica(block.getBlockPoolId(), newReplicaInfo);
-
-    removeOldReplica(replicaInfo, newReplicaInfo, oldBlockFile, oldMetaFile,
-        oldBlockFile.length(), oldMetaFile.length(), block.getBlockPoolId());
+      removeOldReplica(replicaInfo, newReplicaInfo, oldBlockFile, oldMetaFile,
+          oldBlockFile.length(), oldMetaFile.length(), block.getBlockPoolId());
+    }
 
     // Replace the old block if any to reschedule the scanning.
     datanode.getBlockScanner().addBlock(block);
@@ -870,7 +885,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
 
 
   @Override  // FsDatasetSpi
-  public synchronized ReplicaInPipeline append(ExtendedBlock b,
+  public synchronized ReplicaHandler append(ExtendedBlock b,
       long newGS, long expectedBlockLen) throws IOException {
     // If the block was successfully finalized because all packets
     // were successfully processed at the Datanode but the ack for
@@ -895,8 +910,16 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
           " expected length is " + expectedBlockLen);
     }
 
-    return append(b.getBlockPoolId(), (FinalizedReplica)replicaInfo, newGS,
-        b.getNumBytes());
+    FsVolumeReference ref = replicaInfo.getVolume().obtainReference();
+    ReplicaBeingWritten replica = null;
+    try {
+      replica = append(b.getBlockPoolId(), (FinalizedReplica)replicaInfo, newGS,
+          b.getNumBytes());
+    } catch (IOException e) {
+      IOUtils.cleanup(null, ref);
+      throw e;
+    }
+    return new ReplicaHandler(replica, ref);
   }
   
   /** Append to a finalized replica
@@ -1017,22 +1040,30 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     
     return replicaInfo;
   }
-  
+
   @Override  // FsDatasetSpi
-  public synchronized ReplicaInPipeline recoverAppend(ExtendedBlock b,
-      long newGS, long expectedBlockLen) throws IOException {
+  public synchronized ReplicaHandler recoverAppend(
+      ExtendedBlock b, long newGS, long expectedBlockLen) throws IOException {
     LOG.info("Recover failed append to " + b);
 
     ReplicaInfo replicaInfo = recoverCheck(b, newGS, expectedBlockLen);
 
-    // change the replica's state/gs etc.
-    if (replicaInfo.getState() == ReplicaState.FINALIZED ) {
-      return append(b.getBlockPoolId(), (FinalizedReplica) replicaInfo, newGS, 
-          b.getNumBytes());
-    } else { //RBW
-      bumpReplicaGS(replicaInfo, newGS);
-      return (ReplicaBeingWritten)replicaInfo;
+    FsVolumeReference ref = replicaInfo.getVolume().obtainReference();
+    ReplicaBeingWritten replica;
+    try {
+      // change the replica's state/gs etc.
+      if (replicaInfo.getState() == ReplicaState.FINALIZED) {
+        replica = append(b.getBlockPoolId(), (FinalizedReplica) replicaInfo,
+                         newGS, b.getNumBytes());
+      } else { //RBW
+        bumpReplicaGS(replicaInfo, newGS);
+        replica = (ReplicaBeingWritten) replicaInfo;
+      }
+    } catch (IOException e) {
+      IOUtils.cleanup(null, ref);
+      throw e;
     }
+    return new ReplicaHandler(replica, ref);
   }
 
   @Override // FsDatasetSpi
@@ -1080,8 +1111,9 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
   }
 
   @Override // FsDatasetSpi
-  public synchronized ReplicaInPipeline createRbw(StorageType storageType,
-      ExtendedBlock b, boolean allowLazyPersist) throws IOException {
+  public synchronized ReplicaHandler createRbw(
+      StorageType storageType, ExtendedBlock b, boolean allowLazyPersist)
+      throws IOException {
     ReplicaInfo replicaInfo = volumeMap.get(b.getBlockPoolId(),
         b.getBlockId());
     if (replicaInfo != null) {
@@ -1090,15 +1122,15 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
       " and thus cannot be created.");
     }
     // create a new block
-    FsVolumeImpl v;
+    FsVolumeReference ref;
     while (true) {
       try {
         if (allowLazyPersist) {
           // First try to place the block on a transient volume.
-          v = volumes.getNextTransientVolume(b.getNumBytes());
+          ref = volumes.getNextTransientVolume(b.getNumBytes());
           datanode.getMetrics().incrRamDiskBlocksWrite();
         } else {
-          v = volumes.getNextVolume(storageType, b.getNumBytes());
+          ref = volumes.getNextVolume(storageType, b.getNumBytes());
         }
       } catch (DiskOutOfSpaceException de) {
         if (allowLazyPersist) {
@@ -1110,18 +1142,25 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
       }
       break;
     }
+    FsVolumeImpl v = (FsVolumeImpl) ref.getVolume();
     // create an rbw file to hold block in the designated volume
-    File f = v.createRbwFile(b.getBlockPoolId(), b.getLocalBlock());
+    File f;
+    try {
+      f = v.createRbwFile(b.getBlockPoolId(), b.getLocalBlock());
+    } catch (IOException e) {
+      IOUtils.cleanup(null, ref);
+      throw e;
+    }
+
     ReplicaBeingWritten newReplicaInfo = new ReplicaBeingWritten(b.getBlockId(), 
         b.getGenerationStamp(), v, f.getParentFile(), b.getNumBytes());
     volumeMap.add(b.getBlockPoolId(), newReplicaInfo);
-
-    return newReplicaInfo;
+    return new ReplicaHandler(newReplicaInfo, ref);
   }
-  
+
   @Override // FsDatasetSpi
-  public synchronized ReplicaInPipeline recoverRbw(ExtendedBlock b,
-      long newGS, long minBytesRcvd, long maxBytesRcvd)
+  public synchronized ReplicaHandler recoverRbw(
+      ExtendedBlock b, long newGS, long minBytesRcvd, long maxBytesRcvd)
       throws IOException {
     LOG.info("Recover RBW replica " + b);
 
@@ -1160,20 +1199,25 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
           minBytesRcvd + ", " + maxBytesRcvd + "].");
     }
 
-    // Truncate the potentially corrupt portion.
-    // If the source was client and the last node in the pipeline was lost,
-    // any corrupt data written after the acked length can go unnoticed. 
-    if (numBytes > bytesAcked) {
-      final File replicafile = rbw.getBlockFile();
-      truncateBlock(replicafile, rbw.getMetaFile(), numBytes, bytesAcked);
-      rbw.setNumBytes(bytesAcked);
-      rbw.setLastChecksumAndDataLen(bytesAcked, null);
-    }
+    FsVolumeReference ref = rbw.getVolume().obtainReference();
+    try {
+      // Truncate the potentially corrupt portion.
+      // If the source was client and the last node in the pipeline was lost,
+      // any corrupt data written after the acked length can go unnoticed.
+      if (numBytes > bytesAcked) {
+        final File replicafile = rbw.getBlockFile();
+        truncateBlock(replicafile, rbw.getMetaFile(), numBytes, bytesAcked);
+        rbw.setNumBytes(bytesAcked);
+        rbw.setLastChecksumAndDataLen(bytesAcked, null);
+      }
 
-    // bump the replica's generation stamp to newGS
-    bumpReplicaGS(rbw, newGS);
-    
-    return rbw;
+      // bump the replica's generation stamp to newGS
+      bumpReplicaGS(rbw, newGS);
+    } catch (IOException e) {
+      IOUtils.cleanup(null, ref);
+      throw e;
+    }
+    return new ReplicaHandler(rbw, ref);
   }
   
   @Override // FsDatasetSpi
@@ -1238,8 +1282,8 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
   }
 
   @Override // FsDatasetSpi
-  public synchronized ReplicaInPipeline createTemporary(StorageType storageType,
-      ExtendedBlock b) throws IOException {
+  public synchronized ReplicaHandler createTemporary(
+      StorageType storageType, ExtendedBlock b) throws IOException {
     ReplicaInfo replicaInfo = volumeMap.get(b.getBlockPoolId(), b.getBlockId());
     if (replicaInfo != null) {
       if (replicaInfo.getGenerationStamp() < b.getGenerationStamp()
@@ -1254,14 +1298,22 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
             " and thus cannot be created.");
       }
     }
-    
-    FsVolumeImpl v = volumes.getNextVolume(storageType, b.getNumBytes());
+
+    FsVolumeReference ref = volumes.getNextVolume(storageType, b.getNumBytes());
+    FsVolumeImpl v = (FsVolumeImpl) ref.getVolume();
     // create a temporary file to hold block in the designated volume
-    File f = v.createTmpFile(b.getBlockPoolId(), b.getLocalBlock());
+    File f;
+    try {
+      f = v.createTmpFile(b.getBlockPoolId(), b.getLocalBlock());
+    } catch (IOException e) {
+      IOUtils.cleanup(null, ref);
+      throw e;
+    }
+
     ReplicaInPipeline newReplicaInfo = new ReplicaInPipeline(b.getBlockId(), 
         b.getGenerationStamp(), v, f.getParentFile(), 0);
     volumeMap.add(b.getBlockPoolId(), newReplicaInfo);
-    return newReplicaInfo;
+    return new ReplicaHandler(newReplicaInfo, ref);
   }
 
   /**
@@ -1644,10 +1696,15 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
       // Delete the block asynchronously to make sure we can do it fast enough.
       // It's ok to unlink the block file before the uncache operation
       // finishes.
-      asyncDiskService.deleteAsync(v, f,
-          FsDatasetUtil.getMetaFile(f, invalidBlks[i].getGenerationStamp()),
-          new ExtendedBlock(bpid, invalidBlks[i]),
-          dataStorage.getTrashDirectoryForBlockFile(bpid, f));
+      try {
+        asyncDiskService.deleteAsync(v.obtainReference(), f,
+            FsDatasetUtil.getMetaFile(f, invalidBlks[i].getGenerationStamp()),
+            new ExtendedBlock(bpid, invalidBlks[i]),
+            dataStorage.getTrashDirectoryForBlockFile(bpid, f));
+      } catch (ClosedChannelException e) {
+        LOG.warn("Volume " + v + " is closed, ignore the deletion task for " +
+            "block " + invalidBlks[i]);
+      }
     }
     if (!errors.isEmpty()) {
       StringBuilder b = new StringBuilder("Failed to delete ")
@@ -2282,14 +2339,17 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
       ReplicaUnderRecovery replicaInfo, String bpid, long newBlkId, long newGS)
       throws IOException {
     String blockFileName = Block.BLOCK_FILE_PREFIX + newBlkId;
-    FsVolumeImpl v = volumes.getNextVolume(
-        replicaInfo.getVolume().getStorageType(), replicaInfo.getNumBytes());
-    final File tmpDir = v.getBlockPoolSlice(bpid).getTmpDir();
-    final File destDir = DatanodeUtil.idToBlockDir(tmpDir, newBlkId);
-    final File dstBlockFile = new File(destDir, blockFileName);
-    final File dstMetaFile = FsDatasetUtil.getMetaFile(dstBlockFile, newGS);
-    return copyBlockFiles(replicaInfo.getMetaFile(), replicaInfo.getBlockFile(),
-        dstMetaFile, dstBlockFile, true);
+    try (FsVolumeReference ref = volumes.getNextVolume(
+        replicaInfo.getVolume().getStorageType(), replicaInfo.getNumBytes())) {
+      FsVolumeImpl v = (FsVolumeImpl) ref.getVolume();
+      final File tmpDir = v.getBlockPoolSlice(bpid).getTmpDir();
+      final File destDir = DatanodeUtil.idToBlockDir(tmpDir, newBlkId);
+      final File dstBlockFile = new File(destDir, blockFileName);
+      final File dstMetaFile = FsDatasetUtil.getMetaFile(dstBlockFile, newGS);
+      return copyBlockFiles(replicaInfo.getMetaFile(),
+          replicaInfo.getBlockFile(),
+          dstMetaFile, dstBlockFile, true);
+    }
   }
 
   @Override // FsDatasetSpi
@@ -2345,9 +2405,11 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     for (FsVolumeImpl volume : getVolumes()) {
       long used = 0;
       long free = 0;
-      try {
+      try (FsVolumeReference ref = volume.obtainReference()) {
         used = volume.getDfsUsed();
         free = volume.getAvailable();
+      } catch (ClosedChannelException e) {
+        continue;
       } catch (IOException e) {
         LOG.warn(e.getMessage());
         used = 0;
@@ -2379,15 +2441,23 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     List<FsVolumeImpl> curVolumes = getVolumes();
     if (!force) {
       for (FsVolumeImpl volume : curVolumes) {
-        if (!volume.isBPDirEmpty(bpid)) {
-          LOG.warn(bpid + " has some block files, cannot delete unless forced");
-          throw new IOException("Cannot delete block pool, "
-              + "it contains some block files");
+        try (FsVolumeReference ref = volume.obtainReference()) {
+          if (!volume.isBPDirEmpty(bpid)) {
+            LOG.warn(bpid + " has some block files, cannot delete unless forced");
+            throw new IOException("Cannot delete block pool, "
+                + "it contains some block files");
+          }
+        } catch (ClosedChannelException e) {
+          // ignore.
         }
       }
     }
     for (FsVolumeImpl volume : curVolumes) {
-      volume.deleteBPDirectories(bpid, force);
+      try (FsVolumeReference ref = volume.obtainReference()) {
+        volume.deleteBPDirectories(bpid, force);
+      } catch (ClosedChannelException e) {
+        // ignore.
+      }
     }
   }
   
@@ -2620,6 +2690,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
      */
     private boolean saveNextReplica() {
       RamDiskReplica block = null;
+      FsVolumeReference targetReference;
       FsVolumeImpl targetVolume;
       ReplicaInfo replicaInfo;
       boolean succeeded = false;
@@ -2637,8 +2708,9 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
             if (replicaInfo != null &&
                 replicaInfo.getVolume().isTransientStorage()) {
               // Pick a target volume to persist the block.
-              targetVolume = volumes.getNextVolume(
+              targetReference = volumes.getNextVolume(
                   StorageType.DEFAULT, replicaInfo.getNumBytes());
+              targetVolume = (FsVolumeImpl) targetReference.getVolume();
 
               ramDiskReplicaTracker.recordStartLazyPersist(
                   block.getBlockPoolId(), block.getBlockId(), targetVolume);
@@ -2654,7 +2726,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
                   block.getBlockPoolId(), block.getBlockId(),
                   replicaInfo.getGenerationStamp(), block.getCreationTime(),
                   replicaInfo.getMetaFile(), replicaInfo.getBlockFile(),
-                  targetVolume);
+                  targetReference);
             }
           }
         }
@@ -2678,9 +2750,13 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
       // Don't worry about fragmentation for now. We don't expect more than one
       // transient volume per DN.
       for (FsVolumeImpl v : getVolumes()) {
-        if (v.isTransientStorage()) {
-          capacity += v.getCapacity();
-          free += v.getAvailable();
+        try (FsVolumeReference ref = v.obtainReference()) {
+          if (v.isTransientStorage()) {
+            capacity += v.getCapacity();
+            free += v.getAvailable();
+          }
+        } catch (ClosedChannelException e) {
+          // ignore.
         }
       }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9014305e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
index 48427ad..7c8384d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
 
 import java.io.File;
 import java.io.IOException;
+import java.nio.channels.ClosedChannelException;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
@@ -31,6 +32,8 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.DF;
@@ -40,8 +43,10 @@ import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.datanode.DataStorage;
 import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
+import org.apache.hadoop.util.CloseableReferenceCount;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
@@ -62,6 +67,7 @@ public class FsVolumeImpl implements FsVolumeSpi {
   private final File currentDir;    // <StorageDirectory>/current
   private final DF usage;           
   private final long reserved;
+  private CloseableReferenceCount reference = new CloseableReferenceCount();
 
   // Disk space reserved for open blocks.
   private AtomicLong reservedForRbw;
@@ -99,6 +105,10 @@ public class FsVolumeImpl implements FsVolumeSpi {
     if (storageType.isTransient()) {
       return null;
     }
+    if (dataset.datanode == null) {
+      // FsVolumeImpl is used in test.
+      return null;
+    }
 
     final int maxNumThreads = dataset.datanode.getConf().getInt(
         DFSConfigKeys.DFS_DATANODE_FSDATASETCACHE_MAX_THREADS_PER_VOLUME_KEY,
@@ -116,7 +126,114 @@ public class FsVolumeImpl implements FsVolumeSpi {
     executor.allowCoreThreadTimeOut(true);
     return executor;
   }
-  
+
+  private void printReferenceTraceInfo(String op) {
+    StackTraceElement[] stack = Thread.currentThread().getStackTrace();
+    for (StackTraceElement ste : stack) {
+      switch (ste.getMethodName()) {
+      case "getDfsUsed":
+      case "getBlockPoolUsed":
+      case "getAvailable":
+      case "getVolumeMap":
+        return;
+      default:
+        break;
+      }
+    }
+    FsDatasetImpl.LOG.trace("Reference count: " + op + " " + this + ": " +
+        this.reference.getReferenceCount());
+    FsDatasetImpl.LOG.trace(
+        Joiner.on("\n").join(Thread.currentThread().getStackTrace()));
+  }
+
+  /**
+   * Increase the reference count. The caller must increase the reference count
+   * before issuing IOs.
+   *
+   * @throws IOException if the volume is already closed.
+   */
+  private void reference() throws ClosedChannelException {
+    this.reference.reference();
+    if (FsDatasetImpl.LOG.isTraceEnabled()) {
+      printReferenceTraceInfo("incr");
+    }
+  }
+
+  /**
+   * Decrease the reference count.
+   */
+  private void unreference() {
+    if (FsDatasetImpl.LOG.isTraceEnabled()) {
+      printReferenceTraceInfo("desc");
+    }
+    if (FsDatasetImpl.LOG.isDebugEnabled()) {
+      if (reference.getReferenceCount() <= 0) {
+        FsDatasetImpl.LOG.debug("Decrease reference count <= 0 on " + this +
+          Joiner.on("\n").join(Thread.currentThread().getStackTrace()));
+      }
+    }
+    checkReference();
+    this.reference.unreference();
+  }
+
+  private static class FsVolumeReferenceImpl implements FsVolumeReference {
+    private final FsVolumeImpl volume;
+
+    FsVolumeReferenceImpl(FsVolumeImpl volume) throws ClosedChannelException {
+      this.volume = volume;
+      volume.reference();
+    }
+
+    /**
+     * Decreases the reference count.
+     * @throws IOException it never throws IOException.
+     */
+    @Override
+    public void close() throws IOException {
+      volume.unreference();
+    }
+
+    @Override
+    public FsVolumeSpi getVolume() {
+      return this.volume;
+    }
+  }
+
+  @Override
+  public FsVolumeReference obtainReference() throws ClosedChannelException {
+    return new FsVolumeReferenceImpl(this);
+  }
+
+  private void checkReference() {
+    Preconditions.checkState(reference.getReferenceCount() > 0);
+  }
+
+  /**
+   * Close this volume and wait all other threads to release the reference count
+   * on this volume.
+   * @throws IOException if the volume is closed or the waiting is interrupted.
+   */
+  void closeAndWait() throws IOException {
+    try {
+      this.reference.setClosed();
+    } catch (ClosedChannelException e) {
+      throw new IOException("The volume has already closed.", e);
+    }
+    final int SLEEP_MILLIS = 500;
+    while (this.reference.getReferenceCount() > 0) {
+      if (FsDatasetImpl.LOG.isDebugEnabled()) {
+        FsDatasetImpl.LOG.debug(String.format(
+            "The reference count for %s is %d, wait to be 0.",
+            this, reference.getReferenceCount()));
+      }
+      try {
+        Thread.sleep(SLEEP_MILLIS);
+      } catch (InterruptedException e) {
+        throw new IOException(e);
+      }
+    }
+  }
+
   File getCurrentDir() {
     return currentDir;
   }
@@ -250,6 +367,7 @@ public class FsVolumeImpl implements FsVolumeSpi {
    * the block is finalized.
    */
   File createTmpFile(String bpid, Block b) throws IOException {
+    checkReference();
     return getBlockPoolSlice(bpid).createTmpFile(b);
   }
 
@@ -282,6 +400,7 @@ public class FsVolumeImpl implements FsVolumeSpi {
    * the block is finalized.
    */
   File createRbwFile(String bpid, Block b) throws IOException {
+    checkReference();
     reserveSpaceForRbw(b.getNumBytes());
     return getBlockPoolSlice(bpid).createRbwFile(b);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9014305e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java
index c02603d..ba19897 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
 
 import java.io.IOException;
+import java.nio.channels.ClosedChannelException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -28,6 +29,7 @@ import java.util.concurrent.atomic.AtomicReference;
 import com.google.common.collect.Lists;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.StorageType;
+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.VolumeChoosingPolicy;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
@@ -58,6 +60,21 @@ class FsVolumeList {
     return Collections.unmodifiableList(Arrays.asList(volumes.get()));
   }
 
+  private FsVolumeReference chooseVolume(List<FsVolumeImpl> list, long blockSize)
+      throws IOException {
+    while (true) {
+      FsVolumeImpl volume = blockChooser.chooseVolume(list, blockSize);
+      try {
+        return volume.obtainReference();
+      } catch (ClosedChannelException e) {
+        FsDatasetImpl.LOG.warn("Chosen a closed volume: " + volume);
+        // blockChooser.chooseVolume returns DiskOutOfSpaceException when the list
+        // is empty, indicating that all volumes are closed.
+        list.remove(volume);
+      }
+    }
+  }
+
   /** 
    * Get next volume.
    *
@@ -65,7 +82,7 @@ class FsVolumeList {
    * @param storageType the desired {@link StorageType} 
    * @return next volume to store the block in.
    */
-  FsVolumeImpl getNextVolume(StorageType storageType, long blockSize)
+  FsVolumeReference getNextVolume(StorageType storageType, long blockSize)
       throws IOException {
     // Get a snapshot of currently available volumes.
     final FsVolumeImpl[] curVolumes = volumes.get();
@@ -75,7 +92,7 @@ class FsVolumeList {
         list.add(v);
       }
     }
-    return blockChooser.chooseVolume(list, blockSize);
+    return chooseVolume(list, blockSize);
   }
 
   /**
@@ -84,7 +101,7 @@ class FsVolumeList {
    * @param blockSize free space needed on the volume
    * @return next volume to store the block in.
    */
-  FsVolumeImpl getNextTransientVolume(long blockSize) throws IOException {
+  FsVolumeReference getNextTransientVolume(long blockSize) throws IOException {
     // Get a snapshot of currently available volumes.
     final List<FsVolumeImpl> curVolumes = getVolumes();
     final List<FsVolumeImpl> list = new ArrayList<>(curVolumes.size());
@@ -93,13 +110,17 @@ class FsVolumeList {
         list.add(v);
       }
     }
-    return blockChooser.chooseVolume(list, blockSize);
+    return chooseVolume(list, blockSize);
   }
 
   long getDfsUsed() throws IOException {
     long dfsUsed = 0L;
     for (FsVolumeImpl v : volumes.get()) {
-      dfsUsed += v.getDfsUsed();
+      try(FsVolumeReference ref = v.obtainReference()) {
+        dfsUsed += v.getDfsUsed();
+      } catch (ClosedChannelException e) {
+        // ignore.
+      }
     }
     return dfsUsed;
   }
@@ -107,7 +128,11 @@ class FsVolumeList {
   long getBlockPoolUsed(String bpid) throws IOException {
     long dfsUsed = 0L;
     for (FsVolumeImpl v : volumes.get()) {
-      dfsUsed += v.getBlockPoolUsed(bpid);
+      try (FsVolumeReference ref = v.obtainReference()) {
+        dfsUsed += v.getBlockPoolUsed(bpid);
+      } catch (ClosedChannelException e) {
+        // ignore.
+      }
     }
     return dfsUsed;
   }
@@ -115,7 +140,11 @@ class FsVolumeList {
   long getCapacity() {
     long capacity = 0L;
     for (FsVolumeImpl v : volumes.get()) {
-      capacity += v.getCapacity();
+      try (FsVolumeReference ref = v.obtainReference()) {
+        capacity += v.getCapacity();
+      } catch (IOException e) {
+        // ignore.
+      }
     }
     return capacity;
   }
@@ -123,7 +152,11 @@ class FsVolumeList {
   long getRemaining() throws IOException {
     long remaining = 0L;
     for (FsVolumeSpi vol : volumes.get()) {
-      remaining += vol.getAvailable();
+      try (FsVolumeReference ref = vol.obtainReference()) {
+        remaining += vol.getAvailable();
+      } catch (ClosedChannelException e) {
+        // ignore
+      }
     }
     return remaining;
   }
@@ -139,7 +172,7 @@ class FsVolumeList {
     for (final FsVolumeImpl v : volumes.get()) {
       Thread t = new Thread() {
         public void run() {
-          try {
+          try (FsVolumeReference ref = v.obtainReference()) {
             FsDatasetImpl.LOG.info("Adding replicas to map for block pool " +
                 bpid + " on volume " + v + "...");
             long startTime = Time.monotonicNow();
@@ -147,6 +180,9 @@ class FsVolumeList {
             long timeTaken = Time.monotonicNow() - startTime;
             FsDatasetImpl.LOG.info("Time to add replicas to map for block pool"
                 + " " + bpid + " on volume " + v + ": " + timeTaken + "ms");
+          } catch (ClosedChannelException e) {
+            FsDatasetImpl.LOG.info("The volume " + v + " is closed while " +
+                "addng replicas, ignored.");
           } catch (IOException ioe) {
             FsDatasetImpl.LOG.info("Caught exception while adding replicas " +
                 "from " + v + ". Will throw later.", ioe);
@@ -189,16 +225,21 @@ class FsVolumeList {
 
       for(Iterator<FsVolumeImpl> i = volumeList.iterator(); i.hasNext(); ) {
         final FsVolumeImpl fsv = i.next();
-        try {
+        try (FsVolumeReference ref = fsv.obtainReference()) {
           fsv.checkDirs();
         } catch (DiskErrorException e) {
-          FsDatasetImpl.LOG.warn("Removing failed volume " + fsv + ": ",e);
+          FsDatasetImpl.LOG.warn("Removing failed volume " + fsv + ": ", e);
           if (removedVols == null) {
-            removedVols = new ArrayList<FsVolumeImpl>(1);
+            removedVols = new ArrayList<>(1);
           }
           removedVols.add(fsv);
           removeVolume(fsv);
           numFailedVolumes++;
+        } catch (ClosedChannelException e) {
+          FsDatasetImpl.LOG.debug("Caught exception when obtaining " +
+            "reference count on closed volume", e);
+        } catch (IOException e) {
+          FsDatasetImpl.LOG.error("Unexpected IOException", e);
         }
       }
       
@@ -221,7 +262,6 @@ class FsVolumeList {
    * @param newVolume the instance of new FsVolumeImpl.
    */
   void addVolume(FsVolumeImpl newVolume) {
-    // Make a copy of volumes to add new volumes.
     while (true) {
       final FsVolumeImpl[] curVolumes = volumes.get();
       final List<FsVolumeImpl> volumeList = Lists.newArrayList(curVolumes);
@@ -252,6 +292,12 @@ class FsVolumeList {
       if (volumeList.remove(target)) {
         if (volumes.compareAndSet(curVolumes,
             volumeList.toArray(new FsVolumeImpl[volumeList.size()]))) {
+          try {
+            target.closeAndWait();
+          } catch (IOException e) {
+            FsDatasetImpl.LOG.warn(
+                "Error occurs when waiting volume to close: " + target, e);
+          }
           target.shutdown();
           FsDatasetImpl.LOG.info("Removed volume: " + target);
           break;
@@ -298,7 +344,7 @@ class FsVolumeList {
     for (final FsVolumeImpl v : volumes.get()) {
       Thread t = new Thread() {
         public void run() {
-          try {
+          try (FsVolumeReference ref = v.obtainReference()) {
             FsDatasetImpl.LOG.info("Scanning block pool " + bpid +
                 " on volume " + v + "...");
             long startTime = Time.monotonicNow();
@@ -306,6 +352,8 @@ class FsVolumeList {
             long timeTaken = Time.monotonicNow() - startTime;
             FsDatasetImpl.LOG.info("Time taken to scan block pool " + bpid +
                 " on " + v + ": " + timeTaken + "ms");
+          } catch (ClosedChannelException e) {
+            // ignore.
           } catch (IOException ioe) {
             FsDatasetImpl.LOG.info("Caught exception while scanning " + v +
                 ". Will throw later.", ioe);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9014305e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/RamDiskAsyncLazyPersistService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/RamDiskAsyncLazyPersistService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/RamDiskAsyncLazyPersistService.java
index 5e4dd7d..cf8de0a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/RamDiskAsyncLazyPersistService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/RamDiskAsyncLazyPersistService.java
@@ -21,7 +21,9 @@ package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
 
+import javax.ws.rs.HEAD;
 import java.io.File;
 import java.io.IOException;
 import java.util.HashMap;
@@ -175,13 +177,14 @@ class RamDiskAsyncLazyPersistService {
   void submitLazyPersistTask(String bpId, long blockId,
       long genStamp, long creationTime,
       File metaFile, File blockFile,
-      FsVolumeImpl targetVolume) throws IOException {
+      FsVolumeReference target) throws IOException {
     if (LOG.isDebugEnabled()) {
       LOG.debug("LazyWriter schedule async task to persist RamDisk block pool id: "
           + bpId + " block id: " + blockId);
     }
 
-    File lazyPersistDir  = targetVolume.getLazyPersistDir(bpId);
+    FsVolumeImpl volume = (FsVolumeImpl)target.getVolume();
+    File lazyPersistDir  = volume.getLazyPersistDir(bpId);
     if (!lazyPersistDir.exists() && !lazyPersistDir.mkdirs()) {
       FsDatasetImpl.LOG.warn("LazyWriter failed to create " + lazyPersistDir);
       throw new IOException("LazyWriter fail to find or create lazy persist dir: "
@@ -190,8 +193,8 @@ class RamDiskAsyncLazyPersistService {
 
     ReplicaLazyPersistTask lazyPersistTask = new ReplicaLazyPersistTask(
         bpId, blockId, genStamp, creationTime, blockFile, metaFile,
-        targetVolume, lazyPersistDir);
-    execute(targetVolume.getCurrentDir(), lazyPersistTask);
+        target, lazyPersistDir);
+    execute(volume.getCurrentDir(), lazyPersistTask);
   }
 
   class ReplicaLazyPersistTask implements Runnable {
@@ -201,13 +204,13 @@ class RamDiskAsyncLazyPersistService {
     final long creationTime;
     final File blockFile;
     final File metaFile;
-    final FsVolumeImpl targetVolume;
+    final FsVolumeReference targetVolume;
     final File lazyPersistDir;
 
     ReplicaLazyPersistTask(String bpId, long blockId,
         long genStamp, long creationTime,
         File blockFile, File metaFile,
-        FsVolumeImpl targetVolume, File lazyPersistDir) {
+        FsVolumeReference targetVolume, File lazyPersistDir) {
       this.bpId = bpId;
       this.blockId = blockId;
       this.genStamp = genStamp;
@@ -230,14 +233,14 @@ class RamDiskAsyncLazyPersistService {
     public void run() {
       boolean succeeded = false;
       final FsDatasetImpl dataset = (FsDatasetImpl)datanode.getFSDataset();
-      try {
+      try (FsVolumeReference ref = this.targetVolume) {
         // No FsDatasetImpl lock for the file copy
         File targetFiles[] = FsDatasetImpl.copyBlockFiles(
             blockId, genStamp, metaFile, blockFile, lazyPersistDir, true);
 
         // Lock FsDataSetImpl during onCompleteLazyPersist callback
         dataset.onCompleteLazyPersist(bpId, blockId,
-                creationTime, targetFiles, targetVolume);
+                creationTime, targetFiles, (FsVolumeImpl)ref.getVolume());
         succeeded = true;
       } catch (Exception e){
         FsDatasetImpl.LOG.warn(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9014305e/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
index fd1ba8a..4bd7bda 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
@@ -220,7 +220,7 @@ enum Status {
   CHECKSUM_OK = 6;
   ERROR_UNSUPPORTED = 7;
   OOB_RESTART = 8;            // Quick restart
-  OOB_INTERRUPTED = 9;        // Interrupted
+  OOB_RESERVED1 = 9;          // Reserved
   OOB_RESERVED2 = 10;         // Reserved
   OOB_RESERVED3 = 11;         // Reserved
   IN_PROGRESS = 12;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9014305e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteBlockGetsBlockLengthHint.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteBlockGetsBlockLengthHint.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteBlockGetsBlockLengthHint.java
index b7fdccf..7f318df 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteBlockGetsBlockLengthHint.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteBlockGetsBlockLengthHint.java
@@ -97,7 +97,7 @@ public class TestWriteBlockGetsBlockLengthHint {
      * correctly propagate the hint to FsDatasetSpi.
      */
     @Override
-    public synchronized ReplicaInPipelineInterface createRbw(
+    public synchronized ReplicaHandler createRbw(
         StorageType storageType, ExtendedBlock b, boolean allowLazyPersist)
         throws IOException {
       assertThat(b.getLocalBlock().getNumBytes(), is(EXPECTED_BLOCK_LENGTH));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9014305e/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 28ba88d..16b6350 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
@@ -22,6 +22,7 @@ import java.io.FileDescriptor;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.nio.channels.ClosedChannelException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -43,8 +44,8 @@ import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
-import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
+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.LengthInputStream;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaInputStreams;
@@ -147,7 +148,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
         oStream = null;
       }
     }
-
+    
     @Override
     public String getStorageUuid() {
       return storage.getStorageUuid();
@@ -432,6 +433,11 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
     }
 
     @Override
+    public FsVolumeReference obtainReference() throws ClosedChannelException {
+      return null;
+    }
+
+    @Override
     public String getStorageID() {
       return storage.getStorageUuid();
     }
@@ -780,8 +786,8 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
   }
 
   @Override // FsDatasetSpi
-  public synchronized ReplicaInPipelineInterface append(ExtendedBlock b,
-      long newGS, long expectedBlockLen) throws IOException {
+  public synchronized ReplicaHandler append(
+      ExtendedBlock b, long newGS, long expectedBlockLen) throws IOException {
     final Map<Block, BInfo> map = getMap(b.getBlockPoolId());
     BInfo binfo = map.get(b.getLocalBlock());
     if (binfo == null || !binfo.isFinalized()) {
@@ -789,12 +795,12 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
           + " is not valid, and cannot be appended to.");
     }
     binfo.unfinalizeBlock();
-    return binfo;
+    return new ReplicaHandler(binfo, null);
   }
 
   @Override // FsDatasetSpi
-  public synchronized ReplicaInPipelineInterface recoverAppend(ExtendedBlock b,
-      long newGS, long expectedBlockLen) throws IOException {
+  public synchronized ReplicaHandler recoverAppend(
+      ExtendedBlock b, long newGS, long expectedBlockLen) throws IOException {
     final Map<Block, BInfo> map = getMap(b.getBlockPoolId());
     BInfo binfo = map.get(b.getLocalBlock());
     if (binfo == null) {
@@ -807,7 +813,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
     map.remove(b);
     binfo.theBlock.setGenerationStamp(newGS);
     map.put(binfo.theBlock, binfo);
-    return binfo;
+    return new ReplicaHandler(binfo, null);
   }
 
   @Override // FsDatasetSpi
@@ -829,8 +835,9 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
   }
   
   @Override // FsDatasetSpi
-  public synchronized ReplicaInPipelineInterface recoverRbw(ExtendedBlock b,
-      long newGS, long minBytesRcvd, long maxBytesRcvd) throws IOException {
+  public synchronized ReplicaHandler recoverRbw(
+      ExtendedBlock b, long newGS, long minBytesRcvd, long maxBytesRcvd)
+      throws IOException {
     final Map<Block, BInfo> map = getMap(b.getBlockPoolId());
     BInfo binfo = map.get(b.getLocalBlock());
     if ( binfo == null) {
@@ -844,18 +851,18 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
     map.remove(b);
     binfo.theBlock.setGenerationStamp(newGS);
     map.put(binfo.theBlock, binfo);
-    return binfo;
+    return new ReplicaHandler(binfo, null);
   }
 
   @Override // FsDatasetSpi
-  public synchronized ReplicaInPipelineInterface createRbw(
+  public synchronized ReplicaHandler createRbw(
       StorageType storageType, ExtendedBlock b,
       boolean allowLazyPersist) throws IOException {
     return createTemporary(storageType, b);
   }
 
   @Override // FsDatasetSpi
-  public synchronized ReplicaInPipelineInterface createTemporary(
+  public synchronized ReplicaHandler createTemporary(
       StorageType storageType, ExtendedBlock b) throws IOException {
     if (isValidBlock(b)) {
           throw new ReplicaAlreadyExistsException("Block " + b + 
@@ -868,7 +875,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
     final Map<Block, BInfo> map = getMap(b.getBlockPoolId());
     BInfo binfo = new BInfo(b.getBlockPoolId(), b.getLocalBlock(), true);
     map.put(binfo.theBlock, binfo);
-    return binfo;
+    return new ReplicaHandler(binfo, null);
   }
 
   synchronized InputStream getBlockInputStream(ExtendedBlock b

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9014305e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
index 9bf5e52..c7a4084 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
@@ -562,7 +562,7 @@ public class TestBlockRecovery {
       LOG.debug("Running " + GenericTestUtils.getMethodName());
     }
     ReplicaInPipelineInterface replicaInfo = dn.data.createRbw(
-        StorageType.DEFAULT, block, false);
+        StorageType.DEFAULT, block, false).getReplica();
     ReplicaOutputStreams streams = null;
     try {
       streams = replicaInfo.createStreams(true,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9014305e/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 d468493..dfcbb6d 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
@@ -56,6 +56,8 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
 import java.util.concurrent.TimeoutException;
 
 import org.apache.commons.logging.Log;
@@ -568,7 +570,7 @@ public class TestDataNodeHotSwapVolumes {
   @Test(timeout=180000)
   public void testRemoveVolumeBeingWritten()
       throws InterruptedException, TimeoutException, ReconfigurationException,
-      IOException {
+      IOException, BrokenBarrierException {
     // test against removing volumes on the different DataNode on the pipeline.
     for (int i = 0; i < 3; i++) {
       testRemoveVolumeBeingWrittenForDatanode(i);
@@ -582,7 +584,7 @@ public class TestDataNodeHotSwapVolumes {
    */
   private void testRemoveVolumeBeingWrittenForDatanode(int dataNodeIdx)
       throws IOException, ReconfigurationException, TimeoutException,
-      InterruptedException {
+      InterruptedException, BrokenBarrierException {
     // Starts DFS cluster with 3 DataNodes to form a pipeline.
     startDFSCluster(1, 3);
 
@@ -599,11 +601,27 @@ public class TestDataNodeHotSwapVolumes {
     out.write(writeBuf);
     out.hflush();
 
+    final CyclicBarrier barrier = new CyclicBarrier(2);
+
     List<String> oldDirs = getDataDirs(dn);
-    String newDirs = oldDirs.get(1);  // Remove the first volume.
-    dn.reconfigurePropertyImpl(
-        DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, newDirs);
+    final String newDirs = oldDirs.get(1);  // Remove the first volume.
+    final List<Exception> exceptions = new ArrayList<>();
+    Thread reconfigThread = new Thread() {
+      public void run() {
+        try {
+          barrier.await();
+          dn.reconfigurePropertyImpl(
+              DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, newDirs);
+        } catch (ReconfigurationException |
+            InterruptedException |
+            BrokenBarrierException e) {
+          exceptions.add(e);
+        }
+      }
+    };
+    reconfigThread.start();
 
+    barrier.await();
     rb.nextBytes(writeBuf);
     out.write(writeBuf);
     out.hflush();
@@ -614,5 +632,10 @@ public class TestDataNodeHotSwapVolumes {
     // Read the content back
     byte[] content = DFSTestUtil.readFileBuffer(fs, testFile);
     assertEquals(BLOCK_SIZE, content.length);
+
+    reconfigThread.join();
+    if (!exceptions.isEmpty()) {
+      throw new IOException(exceptions.get(0).getCause());
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9014305e/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 b7795b5..1b8f243 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
@@ -28,6 +28,7 @@ import static org.junit.Assert.assertTrue;
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
+import java.nio.channels.ClosedChannelException;
 import java.nio.channels.FileChannel;
 import java.util.LinkedList;
 import java.util.List;
@@ -44,6 +45,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
+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;
 import org.apache.hadoop.io.IOUtils;
@@ -539,7 +541,12 @@ public class TestDirectoryScanner {
     public String[] getBlockPoolList() {
       return new String[0];
     }
-    
+
+    @Override
+    public FsVolumeReference obtainReference() throws ClosedChannelException {
+      return null;
+    }
+
     @Override
     public long getAvailable() throws IOException {
       return 0;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9014305e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java
index 099a0cd..b9adce4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java
@@ -67,7 +67,7 @@ public class TestSimulatedFSDataset {
       // we pass expected len as zero, - fsdataset should use the sizeof actual
       // data written
       ReplicaInPipelineInterface bInfo = fsdataset.createRbw(
-          StorageType.DEFAULT, b, false);
+          StorageType.DEFAULT, b, false).getReplica();
       ReplicaOutputStreams out = bInfo.createStreams(true,
           DataChecksum.newDataChecksum(DataChecksum.Type.CRC32, 512));
       try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9014305e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
index 36036ff..f256ee6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
@@ -144,25 +144,25 @@ public class ExternalDatasetImpl implements FsDatasetSpi<ExternalVolumeImpl> {
   @Override
   public ReplicaInputStreams getTmpInputStreams(ExtendedBlock b, long blkoff,
       long ckoff) throws IOException {
-    return new ReplicaInputStreams(FileDescriptor.in, FileDescriptor.in);
+    return new ReplicaInputStreams(FileDescriptor.in, FileDescriptor.in, null);
   }
 
   @Override
-  public ReplicaInPipelineInterface createTemporary(StorageType t, ExtendedBlock b)
+  public ReplicaHandler createTemporary(StorageType t, ExtendedBlock b)
       throws IOException {
-    return new ExternalReplicaInPipeline();
+    return new ReplicaHandler(new ExternalReplicaInPipeline(), null);
   }
 
   @Override
-  public ReplicaInPipelineInterface createRbw(StorageType t, ExtendedBlock b, boolean tf)
+  public ReplicaHandler createRbw(StorageType t, ExtendedBlock b, boolean tf)
       throws IOException {
-    return new ExternalReplicaInPipeline();
+    return new ReplicaHandler(new ExternalReplicaInPipeline(), null);
   }
 
   @Override
-  public ReplicaInPipelineInterface recoverRbw(ExtendedBlock b, long newGS,
+  public ReplicaHandler recoverRbw(ExtendedBlock b, long newGS,
       long minBytesRcvd, long maxBytesRcvd) throws IOException {
-    return new ExternalReplicaInPipeline();
+    return new ReplicaHandler(new ExternalReplicaInPipeline(), null);
   }
 
   @Override
@@ -172,15 +172,15 @@ public class ExternalDatasetImpl implements FsDatasetSpi<ExternalVolumeImpl> {
   }
 
   @Override
-  public ReplicaInPipelineInterface append(ExtendedBlock b, long newGS,
+  public ReplicaHandler append(ExtendedBlock b, long newGS,
       long expectedBlockLen) throws IOException {
-    return new ExternalReplicaInPipeline();
+    return new ReplicaHandler(new ExternalReplicaInPipeline(), null);
   }
 
   @Override
-  public ReplicaInPipelineInterface recoverAppend(ExtendedBlock b, long newGS,
+  public ReplicaHandler recoverAppend(ExtendedBlock b, long newGS,
       long expectedBlockLen) throws IOException {
-    return new ExternalReplicaInPipeline();
+    return new ReplicaHandler(new ExternalReplicaInPipeline(), null);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9014305e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalVolumeImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalVolumeImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalVolumeImpl.java
index 8ae4c39..857e946 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalVolumeImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalVolumeImpl.java
@@ -20,10 +20,17 @@ package org.apache.hadoop.hdfs.server.datanode.extdataset;
 
 import java.io.File;
 import java.io.IOException;
+import java.nio.channels.ClosedChannelException;
+
 import org.apache.hadoop.hdfs.StorageType;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 
 public class ExternalVolumeImpl implements FsVolumeSpi {
+  @Override
+  public FsVolumeReference obtainReference() throws ClosedChannelException {
+    return null;
+  }
 
   @Override
   public String[] getBlockPoolList() {


[18/50] [abbrv] hadoop git commit: HADOOP-11490. Expose truncate API via FileSystem and shell command. Contributed by Milan Desai.

Posted by zh...@apache.org.
HADOOP-11490. Expose truncate API via FileSystem and shell command. Contributed by Milan Desai.


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

Branch: refs/heads/HDFS-EC
Commit: f2a8eca3d1031e3fb179183c13ec704b134ad51e
Parents: c1ad0a8
Author: Konstantin V Shvachko <sh...@apache.org>
Authored: Wed Jan 21 15:58:58 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:27 2015 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |   5 +-
 .../apache/hadoop/fs/ChecksumFileSystem.java    |   5 +
 .../java/org/apache/hadoop/fs/FileSystem.java   |  23 ++++
 .../org/apache/hadoop/fs/FilterFileSystem.java  |   5 +
 .../org/apache/hadoop/fs/HarFileSystem.java     |   8 ++
 .../apache/hadoop/fs/RawLocalFileSystem.java    |  25 ++++
 .../org/apache/hadoop/fs/shell/FsCommand.java   |   1 +
 .../org/apache/hadoop/fs/shell/Truncate.java    | 117 +++++++++++++++++++
 .../apache/hadoop/fs/viewfs/ViewFileSystem.java |  13 +++
 .../hadoop/hdfs/DistributedFileSystem.java      |   9 +-
 .../hdfs/server/namenode/TestFileTruncate.java  |  69 +++++++++++
 11 files changed, 271 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2a8eca3/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 c54800f..66fd138 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -18,7 +18,10 @@ Trunk (Unreleased)
     HADOOP-6590. Add a username check for hadoop sub-commands (John Smith via aw)
 
     HADOOP-11353. Add support for .hadooprc (aw)
-    
+
+    HADOOP-11490. Expose truncate API via FileSystem and shell command.
+    (Milan Desai via shv)
+
   IMPROVEMENTS
 
     HADOOP-8017. Configure hadoop-main pom to get rid of M2E plugin execution

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2a8eca3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java
index b6b865c..dddf0ce 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java
@@ -352,6 +352,11 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
     throw new IOException("Not supported");
   }
 
+  @Override
+  public boolean truncate(Path f, long newLength) throws IOException {
+    throw new IOException("Not supported");
+  }
+
   /**
    * Calculated the length of the checksum file in bytes.
    * @param size the length of the data file in bytes

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2a8eca3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
index 619f433..cfa5198 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
@@ -1317,6 +1317,29 @@ public abstract class FileSystem extends Configured implements Closeable {
       throw new IOException("rename from " + src + " to " + dst + " failed.");
     }
   }
+
+  /**
+   * Truncate the file in the indicated path to the indicated size.
+   * <ul>
+   * <li>Fails if path is a directory.
+   * <li>Fails if path does not exist.
+   * <li>Fails if path is not closed.
+   * <li>Fails if new size is greater than current size.
+   * </ul>
+   * @param f The path to the file to be truncated
+   * @param newLength The size the file is to be truncated to
+   *
+   * @return <code>true</code> if the file has been truncated to the desired
+   * <code>newLength</code> and is immediately available to be reused for
+   * write operations such as <code>append</code>, or
+   * <code>false</code> if a background process of adjusting the length of
+   * the last block has been started, and clients should wait for it to
+   * complete before proceeding with further file updates.
+   */
+  public boolean truncate(Path f, long newLength) throws IOException {
+    throw new UnsupportedOperationException("Not implemented by the " +
+        getClass().getSimpleName() + " FileSystem implementation");
+  }
   
   /**
    * Delete a file 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2a8eca3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java
index 3d5a753..d4080ad 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java
@@ -225,6 +225,11 @@ public class FilterFileSystem extends FileSystem {
   public boolean rename(Path src, Path dst) throws IOException {
     return fs.rename(src, dst);
   }
+
+  @Override
+  public boolean truncate(Path f, final long newLength) throws IOException {
+    return fs.truncate(f, newLength);
+  }
   
   /** Delete a file */
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2a8eca3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFileSystem.java
index 0fba268..e89bc49 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFileSystem.java
@@ -761,6 +761,14 @@ public class HarFileSystem extends FileSystem {
    * Not implemented.
    */
   @Override
+  public boolean truncate(Path f, long newLength) throws IOException {
+    throw new IOException("Har: truncate not allowed");
+  }
+
+  /**
+   * Not implemented.
+   */
+  @Override
   public boolean delete(Path f, boolean recursive) throws IOException { 
     throw new IOException("Har: delete not allowed");
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2a8eca3/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 75ef189..d7866b8 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
@@ -371,6 +371,31 @@ public class RawLocalFileSystem extends FileSystem {
     }
     return FileUtil.copy(this, src, this, dst, true, getConf());
   }
+
+  @Override
+  public boolean truncate(Path f, final long newLength) throws IOException {
+    FileStatus status = getFileStatus(f);
+    if(status == null) {
+      throw new FileNotFoundException("File " + f + " not found");
+    }
+    if(status.isDirectory()) {
+      throw new IOException("Cannot truncate a directory (=" + f + ")");
+    }
+    long oldLength = status.getLen();
+    if(newLength > oldLength) {
+      throw new IllegalArgumentException(
+          "Cannot truncate to a larger file size. Current size: " + oldLength +
+          ", truncate size: " + newLength + ".");
+    }
+    try (FileOutputStream out = new FileOutputStream(pathToFile(f), true)) {
+      try {
+        out.getChannel().truncate(newLength);
+      } catch(IOException e) {
+        throw new FSError(e);
+      }
+    }
+    return true;
+  }
   
   /**
    * Delete the given path to a file or directory.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2a8eca3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsCommand.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsCommand.java
index cc8fbb4..9515fde 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsCommand.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsCommand.java
@@ -60,6 +60,7 @@ abstract public class FsCommand extends Command {
     factory.registerCommands(Tail.class);
     factory.registerCommands(Test.class);
     factory.registerCommands(Touch.class);
+    factory.registerCommands(Truncate.class);
     factory.registerCommands(SnapshotCommands.class);
     factory.registerCommands(XAttrCommands.class);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2a8eca3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Truncate.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Truncate.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Truncate.java
new file mode 100644
index 0000000..9912863
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Truncate.java
@@ -0,0 +1,117 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.shell;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.PathIsDirectoryException;
+
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+
+/**
+ * Truncates a file to a new size
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class Truncate extends FsCommand {
+  public static void registerCommands(CommandFactory factory) {
+    factory.addClass(Truncate.class, "-truncate");
+  }
+
+  public static final String NAME = "truncate";
+  public static final String USAGE = "[-w] <length> <path> ...";
+  public static final String DESCRIPTION =
+      "Truncate all files that match the specified file pattern to the " +
+      "specified length.\n" +
+      "-w: Requests that the command wait for block recovery to complete, " +
+      "if necessary.";
+
+  protected long newLength = -1;
+  protected List<PathData> waitList = new LinkedList<>();
+  protected boolean waitOpt = false;
+
+  @Override
+  protected void processOptions(LinkedList<String> args) throws IOException {
+    CommandFormat cf = new CommandFormat(2, Integer.MAX_VALUE, "w");
+    cf.parse(args);
+    waitOpt = cf.getOpt("w");
+
+    try {
+      newLength = Long.parseLong(args.removeFirst());
+    } catch(NumberFormatException nfe) {
+      displayWarning("Illegal length, a non-negative integer expected");
+      throw nfe;
+    }
+    if(newLength < 0) {
+      throw new IllegalArgumentException("length must be >= 0");
+    }
+  }
+
+  @Override
+  protected void processArguments(LinkedList<PathData> args)
+      throws IOException {
+    super.processArguments(args);
+    if (waitOpt) waitForRecovery();
+  }
+
+  @Override
+  protected void processPath(PathData item) throws IOException {
+    if(item.stat.isDirectory()) {
+      throw new PathIsDirectoryException(item.toString());
+    }
+    long oldLength = item.stat.getLen();
+    if(newLength > oldLength) {
+      throw new IllegalArgumentException(
+          "Cannot truncate to a larger file size. Current size: " + oldLength +
+          ", truncate size: " + newLength + ".");
+    }
+    if(item.fs.truncate(item.path, newLength)) {
+      out.println("Truncated " + item + " to length: " + newLength);
+    }
+    else if(waitOpt) {
+      waitList.add(item);
+    }
+    else {
+      out.println("Truncating " + item + " to length: " + newLength + ". " +
+          "Wait for block recovery to complete before further updating this " +
+          "file.");
+    }
+  }
+
+  /**
+   * Wait for all files in waitList to have length equal to newLength.
+   */
+  private void waitForRecovery() throws IOException {
+    for(PathData item : waitList) {
+      out.println("Waiting for " + item + " ...");
+      out.flush();
+
+      for(;;) {
+        item.refreshStatus();
+        if(item.stat.getLen() == newLength) break;
+        try {Thread.sleep(1000);} catch(InterruptedException ignored) {}
+      }
+
+      out.println("Truncated " + item + " to length: " + newLength);
+      out.flush();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2a8eca3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
index 963289f..0f77f47 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
@@ -446,6 +446,14 @@ public class ViewFileSystem extends FileSystem {
     return resSrc.targetFileSystem.rename(resSrc.remainingPath,
         resDst.remainingPath);
   }
+
+  @Override
+  public boolean truncate(final Path f, final long newLength)
+      throws IOException {
+    InodeTree.ResolveResult<FileSystem> res =
+        fsState.resolve(getUriPath(f), true);
+    return res.targetFileSystem.truncate(f, newLength);
+  }
   
   @Override
   public void setOwner(final Path f, final String username,
@@ -834,6 +842,11 @@ public class ViewFileSystem extends FileSystem {
     }
 
     @Override
+    public boolean truncate(Path f, long newLength) throws IOException {
+      throw readOnlyMountTable("truncate", f);
+    }
+
+    @Override
     public void setOwner(Path f, String username, String groupname)
         throws AccessControlException, IOException {
       checkPathIsSlash(f);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2a8eca3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
index 6284f61..654e2f9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
@@ -627,14 +627,7 @@ public class DistributedFileSystem extends FileSystem {
     }
   }
 
-  /**
-   * Truncate the file in the indicated path to the indicated size.
-   * @param f The path to the file to be truncated
-   * @param newLength The size the file is to be truncated to
-   *
-   * @return true if and client does not need to wait for block recovery,
-   * false if client needs to wait for block recovery.
-   */
+  @Override
   public boolean truncate(Path f, final long newLength) throws IOException {
     statistics.incrementWriteOps(1);
     return dfs.truncate(getPathName(f), newLength);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2a8eca3/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 1f854d1..5498b12 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
@@ -38,6 +38,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.AppendTestUtil;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -55,6 +56,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.Time;
+import org.apache.hadoop.util.ToolRunner;
 import org.apache.log4j.Level;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -676,6 +678,73 @@ public class TestFileTruncate {
     fs.delete(parent, true);
   }
 
+  @Test
+  public void testTruncateShellCommand() throws Exception {
+    final Path parent = new Path("/test");
+    final Path src = new Path("/test/testTruncateShellCommand");
+    final int oldLength = 2*BLOCK_SIZE + 1;
+    final int newLength = BLOCK_SIZE + 1;
+
+    String[] argv =
+        new String[]{"-truncate", String.valueOf(newLength), src.toString()};
+    runTruncateShellCommand(src, oldLength, argv);
+
+    // wait for block recovery
+    checkBlockRecovery(src);
+    assertThat(fs.getFileStatus(src).getLen(), is((long) newLength));
+    fs.delete(parent, true);
+  }
+
+  @Test
+  public void testTruncateShellCommandOnBlockBoundary() throws Exception {
+    final Path parent = new Path("/test");
+    final Path src = new Path("/test/testTruncateShellCommandOnBoundary");
+    final int oldLength = 2 * BLOCK_SIZE;
+    final int newLength = BLOCK_SIZE;
+
+    String[] argv =
+        new String[]{"-truncate", String.valueOf(newLength), src.toString()};
+    runTruncateShellCommand(src, oldLength, argv);
+
+    // shouldn't need to wait for block recovery
+    assertThat(fs.getFileStatus(src).getLen(), is((long) newLength));
+    fs.delete(parent, true);
+  }
+
+  @Test
+  public void testTruncateShellCommandWithWaitOption() throws Exception {
+    final Path parent = new Path("/test");
+    final Path src = new Path("/test/testTruncateShellCommandWithWaitOption");
+    final int oldLength = 2 * BLOCK_SIZE + 1;
+    final int newLength = BLOCK_SIZE + 1;
+
+    String[] argv = new String[]{"-truncate", "-w", String.valueOf(newLength),
+        src.toString()};
+    runTruncateShellCommand(src, oldLength, argv);
+
+    // shouldn't need to wait for block recovery
+    assertThat(fs.getFileStatus(src).getLen(), is((long) newLength));
+    fs.delete(parent, true);
+  }
+
+  private void runTruncateShellCommand(Path src, int oldLength,
+                                       String[] shellOpts) throws Exception {
+    // create file and write data
+    writeContents(AppendTestUtil.initBuffer(oldLength), oldLength, src);
+    assertThat(fs.getFileStatus(src).getLen(), is((long)oldLength));
+
+    // truncate file using shell
+    FsShell shell = null;
+    try {
+      shell = new FsShell(conf);
+      assertThat(ToolRunner.run(shell, shellOpts), is(0));
+    } finally {
+      if(shell != null) {
+        shell.close();
+      }
+    }
+  }
+
   static void writeContents(byte[] contents, int fileLength, Path p)
       throws IOException {
     FSDataOutputStream out = fs.create(p, true, BLOCK_SIZE, REPLICATION,


[17/50] [abbrv] hadoop git commit: HADOOP-11466. FastByteComparisons: do not use UNSAFE_COMPARER on the SPARC architecture because it is slower there (Suman Somasundar via Colin P. McCabe)

Posted by zh...@apache.org.
HADOOP-11466. FastByteComparisons: do not use UNSAFE_COMPARER on the SPARC architecture because it is slower there (Suman Somasundar via Colin P.  McCabe)


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

Branch: refs/heads/HDFS-EC
Commit: 4ee89ce97b214bcb3e348b2cc2e86f6216d492ad
Parents: f2a8eca
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Wed Jan 21 16:33:02 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 26 09:43:27 2015 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  4 ++++
 .../apache/hadoop/io/FastByteComparisons.java   | 21 +++++++++++++++++++-
 2 files changed, 24 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ee89ce9/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 66fd138..abe699a 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -737,6 +737,10 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11327. BloomFilter#not() omits the last bit, resulting in an
     incorrect filter (Eric Payne via jlowe)
 
+    HADOOP-11466. FastByteComparisons: do not use UNSAFE_COMPARER on the SPARC
+    architecture because it is slower there (Suman Somasundar via Colin P.
+    McCabe)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ee89ce9/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/FastByteComparisons.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/FastByteComparisons.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/FastByteComparisons.java
index 3f5881b..a3fea31 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/FastByteComparisons.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/FastByteComparisons.java
@@ -24,6 +24,9 @@ import java.security.PrivilegedAction;
 
 import sun.misc.Unsafe;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
 import com.google.common.primitives.Longs;
 import com.google.common.primitives.UnsignedBytes;
 
@@ -33,6 +36,7 @@ import com.google.common.primitives.UnsignedBytes;
  * class to be able to compare arrays that start at non-zero offsets.
  */
 abstract class FastByteComparisons {
+  static final Log LOG = LogFactory.getLog(FastByteComparisons.class);
 
   /**
    * Lexicographically compare two byte arrays.
@@ -71,6 +75,13 @@ abstract class FastByteComparisons {
      * implementation if unable to do so.
      */
     static Comparer<byte[]> getBestComparer() {
+      if (System.getProperty("os.arch").equals("sparc")) {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("Lexicographical comparer selected for "
+              + "byte aligned system architecture");
+        }
+        return lexicographicalComparerJavaImpl();
+      }
       try {
         Class<?> theClass = Class.forName(UNSAFE_COMPARER_NAME);
 
@@ -78,8 +89,16 @@ abstract class FastByteComparisons {
         @SuppressWarnings("unchecked")
         Comparer<byte[]> comparer =
           (Comparer<byte[]>) theClass.getEnumConstants()[0];
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("Unsafe comparer selected for "
+              + "byte unaligned system architecture");
+        }
         return comparer;
       } catch (Throwable t) { // ensure we really catch *everything*
+        if (LOG.isTraceEnabled()) {
+          LOG.trace(t.getMessage());
+          LOG.trace("Lexicographical comparer selected");
+        }
         return lexicographicalComparerJavaImpl();
       }
     }
@@ -234,4 +253,4 @@ abstract class FastByteComparisons {
       }
     }
   }
-}
\ No newline at end of file
+}