Переглянути джерело

Merging r1532911 through r1533207 from trunk to branch HDFS-2832

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2832@1533208 13f79535-47bb-0310-9956-ffa450edef68
Arpit Agarwal 11 роки тому
батько
коміт
f8d5755a69
22 змінених файлів з 912 додано та 37 видалено
  1. 6 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 3 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/SnapshotCommands.java
  3. 1 1
      hadoop-common-project/hadoop-common/src/site/apt/FileSystemShell.apt.vm
  4. 143 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextMainOperationsBaseTest.java
  5. 8 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextPermissionBase.java
  6. 5 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFSFileContextMainOperations.java
  7. 5 4
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestFcMainOperationsLocalFs.java
  8. 19 1
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  9. 3 0
      hadoop-hdfs-project/hadoop-hdfs/pom.xml
  10. 12 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs.cmd
  11. 27 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
  12. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  13. 1 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
  14. 116 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfs-dust.js
  15. 265 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.dust.html
  16. 43 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html
  17. 156 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.js
  18. 0 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dust-full-2.0.0.min.js
  19. 8 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dust-helpers-1.1.1.min.js
  20. 34 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestHDFSFileContextMainOperations.java
  21. 28 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java
  22. 28 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotRename.java

+ 6 - 0
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -360,6 +360,9 @@ Release 2.3.0 - UNRELEASED
     HADOOP-9897. Add method to get path start position without drive specifier in
     o.a.h.fs.Path. (Binglin Chang via cnauroth)
 
+    HADOOP-9078. enhance unit-test coverage of class
+    org.apache.hadoop.fs.FileContext (Ivan A. Veselovsky via jeagles)
+
   OPTIMIZATIONS
 
     HADOOP-9748. Reduce blocking on UGI.ensureInitialized (daryn)
@@ -431,6 +434,9 @@ Release 2.2.1 - UNRELEASED
     HADOOP-10040. hadoop.cmd in UNIX format and would not run by default on
     Windows. (cnauroth)
 
+    HADOOP-10055. FileSystemShell.apt.vm doc has typo "numRepicas".
+    (Akira Ajisaka via cnauroth)
+
 Release 2.2.0 - 2013-10-13
 
   INCOMPATIBLE CHANGES

+ 3 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/SnapshotCommands.java

@@ -68,7 +68,7 @@ class SnapshotCommands extends FsCommand {
         throw new IllegalArgumentException("<snapshotDir> is missing.");
       } 
       if (args.size() > 2) {
-        throw new IllegalArgumentException("Too many arguements.");
+        throw new IllegalArgumentException("Too many arguments.");
       }
       if (args.size() == 2) {
         snapshotName = args.removeLast();
@@ -110,7 +110,7 @@ class SnapshotCommands extends FsCommand {
     @Override
     protected void processOptions(LinkedList<String> args) throws IOException {
       if (args.size() != 2) {
-        throw new IOException("args number not 2: " + args.size());
+        throw new IllegalArgumentException("Incorrect number of arguments.");
       }
       snapshotName = args.removeLast();
     }
@@ -150,7 +150,7 @@ class SnapshotCommands extends FsCommand {
     @Override
     protected void processOptions(LinkedList<String> args) throws IOException {
       if (args.size() != 3) {
-        throw new IOException("args number not 3: " + args.size());
+        throw new IllegalArgumentException("Incorrect number of arguments.");
       }
       newName = args.removeLast();
       oldName = args.removeLast();

+ 1 - 1
hadoop-common-project/hadoop-common/src/site/apt/FileSystemShell.apt.vm

@@ -381,7 +381,7 @@ rmr
 
 setrep
 
-   Usage: <<<hdfs dfs -setrep [-R] [-w] <numRepicas> <path> >>>
+   Usage: <<<hdfs dfs -setrep [-R] [-w] <numReplicas> <path> >>>
 
    Changes the replication factor of a file. If <path> is a directory then
    the command recursively changes the replication factor of all files under

+ 143 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextMainOperationsBaseTest.java

@@ -22,6 +22,7 @@ import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.EnumSet;
+import java.util.NoSuchElementException;
 
 import org.apache.commons.lang.RandomStringUtils;
 import org.apache.hadoop.HadoopIllegalArgumentException;
@@ -30,6 +31,7 @@ import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.junit.After;
 import org.junit.Assert;
+import static org.junit.Assert.*;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -92,7 +94,7 @@ public abstract class FileContextMainOperationsBaseTest  {
     }     
   };
   
-  private static byte[] data = getFileData(numBlocks,
+  private static final byte[] data = getFileData(numBlocks,
       getDefaultBlockSize());
   
   @Before
@@ -107,7 +109,8 @@ public abstract class FileContextMainOperationsBaseTest  {
   
   @After
   public void tearDown() throws Exception {
-    fc.delete(new Path(fileContextTestHelper.getAbsoluteTestRootPath(fc), new Path("test")), true);
+    boolean del = fc.delete(new Path(fileContextTestHelper.getAbsoluteTestRootPath(fc), new Path("test")), true);
+    assertTrue(del);
     fc.delete(localFsRootPath, true);
   }
   
@@ -194,6 +197,14 @@ public abstract class FileContextMainOperationsBaseTest  {
     fc.setWorkingDirectory(absoluteDir);
     Assert.assertEquals(absoluteDir, fc.getWorkingDirectory());
 
+    Path aRegularFile = new Path("aRegularFile");
+    createFile(aRegularFile);
+    try {
+      fc.setWorkingDirectory(aRegularFile);
+      fail("An IOException expected.");
+    } catch (IOException ioe) {
+      // okay
+    }
   }
   
   @Test
@@ -1195,6 +1206,136 @@ public abstract class FileContextMainOperationsBaseTest  {
         return true;
       }
     return false;
+ }
+
+  @Test
+  public void testOpen2() throws IOException {
+    final Path rootPath = getTestRootPath(fc, "test");
+    //final Path rootPath = getAbsoluteTestRootPath(fc);
+    final Path path = new Path(rootPath, "zoo");
+    createFile(path);
+    final long length = fc.getFileStatus(path).getLen();
+    FSDataInputStream fsdis = fc.open(path, 2048);
+    try {
+      byte[] bb = new byte[(int)length];
+      fsdis.readFully(bb);
+      assertArrayEquals(data, bb);
+    } finally {
+      fsdis.close();
+    }
+  }
+
+  @Test
+  public void testSetVerifyChecksum() throws IOException {
+    final Path rootPath = getTestRootPath(fc, "test");
+    final Path path = new Path(rootPath, "zoo");
+
+    FSDataOutputStream out = fc.create(path, EnumSet.of(CREATE),
+        Options.CreateOpts.createParent());
+    try {
+      // instruct FS to verify checksum through the FileContext:
+      fc.setVerifyChecksum(true, path);
+      out.write(data, 0, data.length);
+    } finally {
+      out.close();
+    }
+
+    // NB: underlying FS may be different (this is an abstract test),
+    // so we cannot assert .zoo.crc existence.
+    // Instead, we check that the file is read correctly:
+    FileStatus fileStatus = fc.getFileStatus(path);
+    final long len = fileStatus.getLen();
+    assertTrue(len == data.length);
+    byte[] bb = new byte[(int)len];
+    FSDataInputStream fsdis = fc.open(path);
+    try {
+      fsdis.read(bb);
+    } finally {
+      fsdis.close();
+    }
+    assertArrayEquals(data, bb);
+  }
+
+  @Test
+  public void testListCorruptFileBlocks() throws IOException {
+    final Path rootPath = getTestRootPath(fc, "test");
+    final Path path = new Path(rootPath, "zoo");
+    createFile(path);
+    try {
+      final RemoteIterator<Path> remoteIterator = fc
+          .listCorruptFileBlocks(path);
+      if (listCorruptedBlocksSupported()) {
+        assertTrue(remoteIterator != null);
+        Path p;
+        while (remoteIterator.hasNext()) {
+          p = remoteIterator.next();
+          System.out.println("corrupted block: " + p);
+        }
+        try {
+          remoteIterator.next();
+          fail();
+        } catch (NoSuchElementException nsee) {
+          // okay
+        }
+      } else {
+        fail();
+      }
+    } catch (UnsupportedOperationException uoe) {
+      if (listCorruptedBlocksSupported()) {
+        fail(uoe.toString());
+      } else {
+        // okay
+      }
+    }
+  }
+
+  protected abstract boolean listCorruptedBlocksSupported();
+
+  @Test
+  public void testDeleteOnExitUnexisting() throws IOException {
+    final Path rootPath = getTestRootPath(fc, "test");
+    final Path path = new Path(rootPath, "zoo");
+    boolean registered = fc.deleteOnExit(path);
+    // because "zoo" does not exist:
+    assertTrue(!registered);
+  }
+
+  @Test
+  public void testFileContextStatistics() throws IOException {
+    FileContext.clearStatistics();
+
+    final Path rootPath = getTestRootPath(fc, "test");
+    final Path path = new Path(rootPath, "zoo");
+    createFile(path);
+    byte[] bb = new byte[data.length];
+    FSDataInputStream fsdis = fc.open(path);
+    try {
+      fsdis.read(bb);
+    } finally {
+      fsdis.close();
+    }
+    assertArrayEquals(data, bb);
+
+    FileContext.printStatistics();
+  }
+
+  @Test
+  /*
+   * Test method
+   *  org.apache.hadoop.fs.FileContext.getFileContext(AbstractFileSystem)
+   */
+  public void testGetFileContext1() throws IOException {
+    final Path rootPath = getTestRootPath(fc, "test");
+    AbstractFileSystem asf = fc.getDefaultFileSystem();
+    // create FileContext using the protected #getFileContext(1) method:
+    FileContext fc2 = FileContext.getFileContext(asf);
+    // Now just check that this context can do something reasonable:
+    final Path path = new Path(rootPath, "zoo");
+    FSDataOutputStream out = fc2.create(path, EnumSet.of(CREATE),
+        Options.CreateOpts.createParent());
+    out.close();
+    Path pathResolved = fc2.resolvePath(path);
+    assertEquals(pathResolved.toUri().getPath(), path.toUri().getPath());
   }
   
   private Path getTestRootPath(FileContext fc, String pathString) {

+ 8 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextPermissionBase.java

@@ -35,6 +35,7 @@ import org.junit.Test;
 
 import static org.apache.hadoop.fs.FileContextTestHelper.*;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
 
 /**
  * <p>
@@ -174,6 +175,13 @@ public abstract class FileContextPermissionBase {
         System.out.println("Not testing changing the group since user " +
                            "belongs to only one group.");
       }
+      
+      try {
+        fc.setOwner(f, null, null);
+        fail("Exception expected.");
+      } catch (IllegalArgumentException iae) {
+        // okay
+      }
     } 
     finally {cleanupFile(fc, f);}
   }

+ 5 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFSFileContextMainOperations.java

@@ -49,6 +49,11 @@ public class TestLocalFSFileContextMainOperations extends FileContextMainOperati
     FileContext fc1 = FileContext.getLocalFSFileContext();
     Assert.assertTrue(fc1 != fc);
   }
+  
+  @Override
+  protected boolean listCorruptedBlocksSupported() {
+    return false;
+  }
 
   @Test
   public void testDefaultFilePermission() throws IOException {

+ 5 - 4
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestFcMainOperationsLocalFs.java

@@ -18,13 +18,9 @@
 package org.apache.hadoop.fs.viewfs;
 
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.FileContextMainOperationsBaseTest;
-import org.apache.hadoop.fs.FileContextTestHelper;
-import org.apache.hadoop.fs.FsConstants;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.viewfs.ConfigUtil;
 
 import org.junit.After;
 import org.junit.Before;
@@ -49,4 +45,9 @@ public class TestFcMainOperationsLocalFs  extends
     super.tearDown();
     ViewFsTestSetup.tearDownForViewFsLocalFs(fileContextTestHelper);
   }
+  
+  @Override
+  protected boolean listCorruptedBlocksSupported() {
+    return false;
+  }
 }

+ 19 - 1
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -255,7 +255,12 @@ Release 2.3.0 - UNRELEASED
 
     HDFS-5342. Provide more information in the FSNamesystem JMX interfaces.
     (Haohui Mai via jing9)
- 
+
+    HDFS-5334. Implement dfshealth.jsp in HTML pages. (Haohui Mai via jing9)
+
+    HDFS-5379. Update links to datanode information in dfshealth.html. (Haohui
+    Mai via jing9)
+
   IMPROVEMENTS
 
     HDFS-5267. Remove volatile from LightWeightHashSet. (Junping Du via llu)
@@ -352,6 +357,13 @@ Release 2.3.0 - UNRELEASED
     HDFS-5283. Under construction blocks only inside snapshots should not be
     counted in safemode threshhold.  (Vinay via szetszwo)
 
+    HDFS-4376. Fix race conditions in Balancer.  (Junping Du via szetszwo)
+
+    HDFS-5375. hdfs.cmd does not expose several snapshot commands. (cnauroth)
+
+    HDFS-5336. DataNode should not output 'StartupProgress' metrics.
+    (Akira Ajisaka via cnauroth)
+
 Release 2.2.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -360,6 +372,9 @@ Release 2.2.1 - UNRELEASED
 
   IMPROVEMENTS
 
+    HDFS-5360. Improvement of usage message of renameSnapshot and
+    deleteSnapshot. (Shinichi Yamashita via wang)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -3674,6 +3689,9 @@ Release 0.23.10 - UNRELEASED
     HDFS-5010. Reduce the frequency of getCurrentUser() calls from namenode
     (kihwal)
 
+    HDFS-5346. Avoid unnecessary call to getNumLiveDataNodes() for each block 
+    during IBR processing (Ravi Prakash via kihwal)
+
   OPTIMIZATIONS
 
   BUG FIXES

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/pom.xml

@@ -542,6 +542,9 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
             <exclude>src/main/docs/releasenotes.html</exclude>
             <exclude>src/contrib/**</exclude>
             <exclude>src/site/resources/images/*</exclude>
+            <exclude>src/main/webapps/static/dust-full-2.0.0.min.js</exclude>
+            <exclude>src/main/webapps/static/dust-helpers-1.1.1.min.js</exclude>
+            <exclude>src/main/webapps/hdfs/dfshealth.dust.html</exclude>
           </excludes>
         </configuration>
       </plugin>

+ 12 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs.cmd

@@ -123,6 +123,14 @@ goto :eof
   set CLASS=org.apache.hadoop.hdfs.tools.GetGroups
   goto :eof
 
+:snapshotDiff
+  set CLASS=org.apache.hadoop.hdfs.tools.snapshot.SnapshotDiff
+  goto :eof
+
+:lsSnapshottableDir
+  set CLASS=org.apache.hadoop.hdfs.tools.snapshot.LsSnapshottableDir
+  goto :eof
+
 @rem This changes %1, %2 etc. Hence those cannot be used after calling this.
 :make_command_arguments
   if "%1" == "--config" (
@@ -164,7 +172,10 @@ goto :eof
   @echo   fetchdt              fetch a delegation token from the NameNode
   @echo   getconf              get config values from configuration
   @echo   groups               get the groups which users belong to
-  @echo                        Use -help to see options
+  @echo   snapshotDiff         diff two snapshots of a directory or diff the
+  @echo                        current directory contents with a snapshot
+  @echo   lsSnapshottableDir   list all snapshottable dirs owned by the current user
+  @echo 						Use -help to see options
   @echo.
   @echo Most commands print help when invoked w/o parameters.
 

+ 27 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java

@@ -506,7 +506,7 @@ public class Balancer {
     final DatanodeInfo datanode;
     final double utilization;
     final long maxSize2Move;
-    protected long scheduledSize = 0L;
+    private long scheduledSize = 0L;
     //  blocks being moved but not confirmed yet
     private List<PendingBlockMove> pendingBlocks = 
       new ArrayList<PendingBlockMove>(MAX_NUM_CONCURRENT_MOVES); 
@@ -555,20 +555,35 @@ public class Balancer {
     }
     
     /** Decide if still need to move more bytes */
-    protected boolean hasSpaceForScheduling() {
+    protected synchronized boolean hasSpaceForScheduling() {
       return scheduledSize<maxSize2Move;
     }
 
     /** Return the total number of bytes that need to be moved */
-    protected long availableSizeToMove() {
+    protected synchronized long availableSizeToMove() {
       return maxSize2Move-scheduledSize;
     }
     
-    /* increment scheduled size */
-    protected void incScheduledSize(long size) {
+    /** increment scheduled size */
+    protected synchronized void incScheduledSize(long size) {
       scheduledSize += size;
     }
     
+    /** decrement scheduled size */
+    protected synchronized void decScheduledSize(long size) {
+      scheduledSize -= size;
+    }
+    
+    /** get scheduled size */
+    protected synchronized long getScheduledSize(){
+      return scheduledSize;
+    }
+    
+    /** get scheduled size */
+    protected synchronized void setScheduledSize(long size){
+      scheduledSize = size;
+    }
+    
     /* Check if the node can schedule more blocks to move */
     synchronized private boolean isPendingQNotFull() {
       if ( pendingBlocks.size() < MAX_NUM_CONCURRENT_MOVES ) {
@@ -702,8 +717,8 @@ public class Balancer {
           pendingBlock.source = this;
           pendingBlock.target = target;
           if ( pendingBlock.chooseBlockAndProxy() ) {
-            long blockSize = pendingBlock.block.getNumBytes(); 
-            scheduledSize -= blockSize;
+            long blockSize = pendingBlock.block.getNumBytes();
+            decScheduledSize(blockSize);
             task.size -= blockSize;
             if (task.size == 0) {
               tasks.remove();
@@ -747,10 +762,11 @@ public class Balancer {
     private static final long MAX_ITERATION_TIME = 20*60*1000L; //20 mins
     private void dispatchBlocks() {
       long startTime = Time.now();
+      long scheduledSize = getScheduledSize();
       this.blocksToReceive = 2*scheduledSize;
       boolean isTimeUp = false;
       int noPendingBlockIteration = 0;
-      while(!isTimeUp && scheduledSize>0 &&
+      while(!isTimeUp && getScheduledSize()>0 &&
           (!srcBlockList.isEmpty() || blocksToReceive>0)) {
         PendingBlockMove pendingBlock = chooseNextBlockToMove();
         if (pendingBlock != null) {
@@ -779,7 +795,7 @@ public class Balancer {
           // in case no blocks can be moved for source node's task,
           // jump out of while-loop after 5 iterations.
           if (noPendingBlockIteration >= MAX_NO_PENDING_BLOCK_ITERATIONS) {
-            scheduledSize = 0;
+            setScheduledSize(0);
           }
         }
         
@@ -981,7 +997,7 @@ public class Balancer {
 
     long bytesToMove = 0L;
     for (Source src : sources) {
-      bytesToMove += src.scheduledSize;
+      bytesToMove += src.getScheduledSize();
     }
     return bytesToMove;
   }
@@ -1082,7 +1098,7 @@ public class Balancer {
       bytesMoved += bytes;
     }
 
-    private long get() {
+    private synchronized long get() {
       return bytesMoved;
     }
   };

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -4613,7 +4613,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
      */
     private boolean needEnter() {
       return (threshold != 0 && blockSafe < blockThreshold) ||
-        (getNumLiveDataNodes() < datanodeThreshold) ||
+        (datanodeThreshold != 0 && getNumLiveDataNodes() < datanodeThreshold) ||
         (!nameNodeHasResourcesAvailable());
     }
       

+ 1 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java

@@ -270,10 +270,6 @@ public class NameNode implements NameNodeStatusMXBean {
 
   static NameNodeMetrics metrics;
   private static final StartupProgress startupProgress = new StartupProgress();
-  static {
-    StartupProgressMetrics.register(startupProgress);
-  }
-
   /** Return the {@link FSNamesystem} object.
    * @return {@link FSNamesystem} object.
    */
@@ -485,6 +481,7 @@ public class NameNode implements NameNodeStatusMXBean {
     loginAsNameNodeUser(conf);
 
     NameNode.initMetrics(conf, this.getRole());
+    StartupProgressMetrics.register(startupProgress);
 
     if (NamenodeRole.NAMENODE == role) {
       startHttpServer(conf);

+ 116 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfs-dust.js

@@ -0,0 +1,116 @@
+/**
+ * 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.
+ */
+(function ($, dust, exports) {
+  "use strict";
+
+  var filters = {
+    'fmt_bytes': function (v) {
+      var UNITS = ['B', 'KB', 'MB', 'GB', 'TB', 'PB', 'ZB'];
+      var prev = 0, i = 0;
+      while (Math.floor(v) > 0 && i < UNITS.length) {
+        prev = v;
+        v /= 1024;
+        i += 1;
+      }
+
+      if (i > 0 && i < UNITS.length) {
+        v = prev;
+        i -= 1;
+      }
+      return Math.round(v * 100) / 100 + ' ' + UNITS[i];
+    },
+
+    'fmt_percentage': function (v) {
+      return Math.round(v * 100) / 100 + '%';
+    },
+
+    'fmt_time': function (v) {
+      var s = Math.floor(v / 1000), h = Math.floor(s / 3600);
+      s -= h * 3600;
+      var m = Math.floor(s / 60);
+      s -= m * 60;
+
+      var res = s + " sec";
+      if (m !== 0) {
+        res = m + " mins, " + res;
+      }
+
+      if (h !== 0) {
+        res = h + " hrs, " + res;
+      }
+
+      return res;
+    }
+  };
+  $.extend(dust.filters, filters);
+
+  /**
+   * Load templates from external sources in sequential orders, and
+   * compile them. The loading order is important to resolve dependency.
+   *
+   * The code compile the templates on the client sides, which should be
+   * precompiled once we introduce the infrastructure in the building
+   * system.
+   *
+   * templates is an array of tuples in the format of {url, name}.
+   */
+  function load_templates(dust, templates, success_cb, error_cb) {
+    if (templates.length === 0) {
+      success_cb();
+      return;
+    }
+
+    var t = templates.shift();
+    $.get(t.url, function (tmpl) {
+      var c = dust.compile(tmpl, t.name);
+      dust.loadSource(c);
+      load_templates(dust, templates, success_cb, error_cb);
+    }).error(function (jqxhr, text, err) {
+      error_cb(t.url, jqxhr, text, err);
+    });
+  }
+
+  /**
+   * Load a sequence of JSON.
+   *
+   * beans is an array of tuples in the format of {url, name}.
+   */
+  function load_json(beans, success_cb, error_cb) {
+    var data = {}, error = false, to_be_completed = beans.length;
+
+    $.each(beans, function(idx, b) {
+      if (error) {
+        return false;
+      }
+      $.get(b.url, function (resp) {
+        data[b.name] = resp;
+        to_be_completed -= 1;
+        if (to_be_completed === 0) {
+          success_cb(data);
+        }
+      }).error(function (jqxhr, text, err) {
+        error = true;
+        error_cb(b.url, jqxhr, text, err);
+      });
+    });
+  }
+
+  exports.load_templates = load_templates;
+  exports.load_json = load_json;
+
+}($, dust, window));

+ 265 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.dust.html

@@ -0,0 +1,265 @@
+<div class="page-header">
+  {#nnstat}
+  <h1>NameNode '{HostAndPort}' ({State})</h1>
+  {/nnstat}
+</div>
+
+{#nn}
+{@if cond="{DistinctVersionCount} > 1"}
+<div class="alert alert-dismissable alert-success">
+  <button type="button" class="close" data-dismiss="alert" aria-hidden="true">&times;</button>
+
+  There are {DistinctVersionCount} versions of datanodes currently live: 
+  {#DistinctVersions}
+  {key} ({value}) {@sep},{/sep}
+  {/DistinctVersions}
+</div>
+{/if}
+
+{@if cond="{NumberOfMissingBlocks} > 0"}
+<div class="alert alert-dismissable alert-warning">
+  <button type="button" class="close" data-dismiss="alert" aria-hidden="true">&times;</button>
+
+  <p>There are {NumberOfMissingBlocks} missing blocks. The following files may be corrupted:</p>
+  <br/>
+  <div class="well">
+    {#CorruptFiles}
+    {.}<br/>
+    {/CorruptFiles}
+  </div>
+  <p>Please check the logs or run fsck in order to identify the missing blocks. See the Hadoop FAQ for common causes and potential solutions.</p>
+</div>
+{/if}
+{/nn}
+
+<div class="panel panel-primary">
+  <div class="panel-heading">Overview</div>
+  <div class="panel-body">
+    {#nn}
+    <table class="table table-bordered">
+      <tr><th>Started:</th><td>{NNStarted}</td></tr>
+      <tr><th>Version:</th><td>{Version}</td></tr>
+      <tr><th>Compiled:</th><td>{CompileInfo}</td></tr>
+      <tr><th>Cluster ID:</th><td>{ClusterId}</td></tr>
+      <tr><th>Block Pool ID:</th><td>{BlockPoolId}</td></tr>
+    </table>
+    {/nn}
+  </div>
+</div>
+
+<a id="browse-dir-first" style="cursor:pointer">Browse the filesystem</a>  <a href="/logs/">NameNode Logs</a>
+
+<hr/>
+
+<div class="panel panel-primary">
+  <div class="panel-heading">Cluster Summary</div>
+  <div class="panel-body">
+
+    <p>
+      Security is {#nnstat}{#SecurityModeEnabled}on{:else}off{/SecurityModeEnabled}{/nnstat}.</p>
+    <p>{#nn}{#Safemode}{.}{:else}Safemode is off.{/Safemode}{/nn}</p>
+
+    <p>
+      {#fs}
+      {TotalLoad} files and directories, {BlocksTotal} blocks = {FilesTotal} total filesystem object(s).
+      {#helper_fs_max_objects/}
+      {/fs}
+    </p>
+    {#mem.HeapMemoryUsage}
+    <p>Heap Memory used {used|fmt_bytes} of {committed|fmt_bytes} Heap Memory. Max Heap Memory is {max|fmt_bytes}. </p>
+    {/mem.HeapMemoryUsage}
+
+    {#mem.NonHeapMemoryUsage}
+    <p>Non Heap Memory used {used|fmt_bytes} of {committed|fmt_bytes} Commited Non Heap Memory. Max Non Heap Memory is {max|fmt_bytes}. </p>
+    {/mem.NonHeapMemoryUsage}
+
+    {#nn}
+    <table class="table table-bordered table-striped">
+      <tr><th> Configured Capacity:</th><td>{Total|fmt_bytes}</td></tr>
+      <tr><th> DFS Used:</th><td>{Used|fmt_bytes}</td></tr>
+      <tr><th> Non DFS Used:</th><td>{NonDfsUsedSpace|fmt_bytes}</td></tr>
+      <tr><th> DFS Remaining:</th><td>{Free|fmt_bytes}</td></tr>
+      <tr><th> DFS Used%:</th><td>{PercentUsed|fmt_percentage}</td></tr>
+      <tr><th> DFS Remaining%:</th><td>{PercentRemaining|fmt_percentage}</td></tr>
+      <tr><th> Block Pool Used:</th><td>{BlockPoolUsedSpace|fmt_bytes}</td></tr>
+      <tr><th> Block Pool Used%:</th><td>{PercentBlockPoolUsed|fmt_percentage}</td></tr>
+      <tr><th> DataNodes usages% (Min/Median/Max/stdDev): </th>
+	<td>{#NodeUsage.nodeUsage}{min} / {median} / {max} / {stdDev}{/NodeUsage.nodeUsage}</td></tr>
+      {/nn}
+
+      {#fs}
+      <tr><th><a href="#nodelist-operation">Live Nodes</a></th><td>{NumLiveDataNodes} (Decommissioned: {NumDecomLiveDataNodes})</td></tr>
+      <tr><th><a href="#nodelist-operation">Dead Nodes</a></th><td>{NumDeadDataNodes} (Decommissioned: {NumDecomDeadDataNodes})</td></tr>
+      <tr><th><a href="#nodelist-decom">Decommissioning Nodes</a></th><td>{NumDecommissioningDataNodes}</td></tr>
+      <tr><th title="Excludes missing blocks.">Number of Under-Replicated Blocks</th><td>{UnderReplicatedBlocks}</td></tr>
+      {/fs}
+    </table>
+  </div>
+</div>
+
+<hr/>
+<div class="panel panel-primary">
+  <div class="panel-heading">NameNode Journal Status</div>
+  <div class="panel-body">
+    <p><b>Current transaction ID:</b> {nn.JournalTransactionInfo.LastAppliedOrWrittenTxId}</p>
+    <table class="table" title="NameNode Journals">
+      <thead>
+	<tr><th>Journal Manager</th><th>State</th></tr>
+      </thead>
+      <tbody>
+	{#nn.NameJournalStatus}
+	<tr><td>{manager}</td><td>{stream}</td></tr>
+	{/nn.NameJournalStatus}
+      </tbody>
+    </table>
+  </div>
+</div>
+
+<hr/>
+<div class="panel panel-primary">
+  <div class="panel-heading">NameNode Storage</div>
+  <div class="panel-body">
+    <table class="table" title="NameNode Storage">
+      <thead><tr><td><b>Storage Directory</b></td><td><b>Type</b></td><td><b>State</b></td></tr></thead>
+      {#nn.NameDirStatuses}
+      {#active}{#helper_dir_status type="Active"/}{/active}
+      {#failed}{#helper_dir_status type="Failed"/}{/failed}
+      {/nn.NameDirStatuses}
+    </table>
+  </div>
+</div>
+<hr/>
+
+<div class="panel panel-primary">
+  <div class="panel-heading">Snapshot Summary</div>
+  <div class="panel-body">
+    {#fs.SnapshotStats}
+    <table class="table" title="Snapshot Summary">
+      <thead><tr><td><b>Snapshottable directories</b></td>
+	  <td><b>Snapshotted directories</b></td></tr>
+      </thead>
+      <tbody>
+	<tr>
+	  <td>{SnapshottableDirectories}</td>
+	  <td>{Snapshots}</td>
+	</tr>
+      </tbody>
+    </table>
+    {/fs.SnapshotStats}
+  </div>
+</div>
+<hr/>
+
+{#startup}
+<div class="panel panel-primary">
+  <div class="panel-heading">Startup Progress</div>
+  <div class="panel-body">
+    <p>Elapsed Time: {elapsedTime|fmt_time}, Percent Complete: {percentComplete|fmt_percentage}</p>
+    <table class="table">
+      <thead>
+	<tr>
+	  <th>Phase</th>
+	  <th>Completion</th>
+	  <th>Elapsed Time</th>
+	</tr>
+      </thead>
+      <tbody>
+	{#phases}
+	<tr class="phase">
+	  <td class="startupdesc">{desc} {file} {size|fmt_bytes}</td>
+	  <td>{percentComplete|fmt_percentage}</td>
+	  <td>{elapsedTime|fmt_time}</td>
+	</tr>
+	{#steps root_file=file}
+	<tr class="step">
+	  <td class="startupdesc">{stepDesc} {stepFile} {stepSize|fmt_bytes} ({count}/{total})</td>
+	  <td>{percentComplete|fmt_percentage}</td>
+	  <td></td>
+	</tr>
+	{/steps}
+	{/phases}
+    </table>
+  </div>
+</div>
+{/startup}
+
+<hr/>
+<div class="panel panel-primary">
+  <div class="panel-heading">Datanode Information</div>
+  <div class="panel-body">
+    <div class="panel panel-default" id="nodelist-operation">
+      <div class="panel-heading">Nodes in operation</div>
+      <div class="panel-body">
+        <table class="table">
+          <thead>
+            <tr>
+              <th>Node</th>
+              <th>Last contact</th>
+              <th>Admin State</th>
+              <th>Capacity</th>
+              <th>Used</th>
+              <th>Non DFS Used</th>
+              <th>Remaining</th>
+              <th>Blocks</th>
+              <th>Block pool used</th>
+              <th>Failed Volumes</th>
+            </tr>
+          </thead>
+          {#nn.LiveNodes}
+          <tr>
+            <td><a class="browse-dir-links" info-http-addr="{infoAddr}" info-https-addr="{infoSecureAddr}">{name}</a> ({xferaddr})</td>
+            <td>{lastContact}</td>
+            <td>{adminState}</td>
+            <td>{capacity|fmt_bytes}</td>
+            <td>{used|fmt_bytes}</td>
+            <td>{nonDfsUsedSpace|fmt_bytes}</td>
+            <td>{remaining|fmt_bytes}</td>
+            <td>{numBlocks}</td>
+            <td>{blockPoolUsed|fmt_bytes} ({blockPoolUsedPercent|fmt_percentage})</td>
+            <td>{volfails}</td>
+          </tr>
+          {/nn.LiveNodes}
+          {#nn.DeadNodes}
+          <tr class="danger">
+            <td>{name} ({xferaddr})</td>
+            <td>{lastContact}</td>
+            <td>Dead{?decomissioned}, Decomissioned{/decomissioned}</td>
+            <td>-</td>
+            <td>-</td>
+            <td>-</td>
+            <td>-</td>
+            <td>-</td>
+            <td>-</td>
+            <td>-</td>
+          </tr>
+          {/nn.DeadNodes}
+        </table>
+      </div>
+    </div>
+    <div class="panel panel-default" id="nodelist-decom">
+      <div class="panel-heading">Nodes being decomissioned</div>
+      <div class="panel-body">
+        <table class="table">
+          <thead>
+            <tr>
+              <th>Node</th>
+              <th>Last contact</th>
+              <th>Under replicated blocks</th>
+              <th>Blocks with no live replicas</th>
+              <th>Under Replicated Blocks <br/>In files under construction</th>
+            </tr>
+          </thead>
+          {#nn.DecomNodes}
+          <tr>
+            <td>{name} ({xferaddr})</td>
+            <td>{lastContact}</td>
+            <td>{underReplicatedBlocks}</td>
+            <td>{decommissionOnlyReplicas}</td>
+	    <td>{underReplicateInOpenFiles}</td>
+	  </tr>
+	  {/nn.DecomNodes}
+	</table>
+      </div>
+    </div>
+  </div>
+</div>

+ 43 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html

@@ -0,0 +1,43 @@
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+<!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Strict//EN"
+    "http://www.w3.org/TR/xhtml1/DTD/xhtml1-strict.dtd">
+<html xmlns="http://www.w3.org/1999/xhtml">
+<head>
+<link rel="stylesheet" type="text/css" href="//netdna.bootstrapcdn.com/bootstrap/3.0.0/css/bootstrap.min.css" />
+<link rel="stylesheet" type="text/css" href="/static/hadoop.css" />
+<title>Namenode information</title>
+</head>
+<body>
+<div class="container">
+<div class="alert alert-danger" id="alert-panel" style="display:none">
+<button type="button" class="close" onclick="$('#alert-panel').hide();">&times;</button>
+<div class="alert-body" id="alert-panel-body"></div>
+</div>
+<div id="panel"></div>
+</div>
+<script type="text/javascript" src="//ajax.googleapis.com/ajax/libs/jquery/2.0.3/jquery.min.js">
+</script><script type="text/javascript" src="//netdna.bootstrapcdn.com/bootstrap/3.0.0/js/bootstrap.min.js">
+</script><script type="text/javascript" src="/static/dust-full-2.0.0.min.js">
+</script><script type="text/javascript" src="/static/dust-helpers-1.1.1.min.js">
+</script><script type="text/javascript" src="dfs-dust.js">
+</script><script type="text/javascript" src="dfshealth.js">
+</script>
+<hr />
+<p><a href="http://hadoop.apache.org/core">Hadoop</a>, 2013.</p>
+</body>
+</html>

+ 156 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.js

@@ -0,0 +1,156 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+(function () {
+  "use strict";
+
+  var data = {};
+  function generate_browse_dn_link(info_http_addr, info_https_addr) {
+    var is_https = window.location.protocol === 'https:';
+    var authority = is_https ? info_https_addr : info_http_addr;
+
+    var nn_info_port = window.location.port;
+    if (nn_info_port === "") {
+      nn_info_port = is_https ? 443 : 80;
+    }
+
+    var l = '//' + authority + '/browseDirectory.jsp?dir=%2F&namenodeInfoPort=' +
+      nn_info_port + '&nnaddr=' + data.nnstat.HostAndPort;
+    return l;
+  }
+
+  function render() {
+    var helpers = {
+      'helper_fs_max_objects': function (chunk, ctx, bodies, params) {
+        var o = ctx.current();
+        if (o.MaxObjects > 0) {
+          chunk.write('(' + Math.round((o.FilesTotal + o.BlockTotal) / o.MaxObjects * 100) * 100 + ')%');
+        }
+      },
+
+      'helper_dir_status': function (chunk, ctx, bodies, params) {
+        var j = ctx.current();
+        for (var i in j) {
+          chunk.write('<tr><td>' + i + '</td><td>' + j[i] + '</td><td>' + params.type + '</td></tr>');
+        }
+      }
+    };
+
+    var base = dust.makeBase(helpers);
+
+    var TEMPLATES = [ { 'name': 'dfshealth', 'url': 'dfshealth.dust.html' } ];
+
+    load_templates(dust, TEMPLATES, function() {
+      dust.render('dfshealth', base.push(data), function(err, out) {
+
+        $('#panel').append(out);
+
+        $('#browse-dir-first').click(function () {
+          var len = data.nn.LiveNodes.length;
+          if (len < 1) {
+            show_err_msg('Cannot browse the DFS since there are no live nodes available.');
+            return false;
+          }
+
+          var dn = data.nn.LiveNodes[Math.floor(Math.random() * len)];
+          window.location.href = generate_browse_dn_link(dn.infoAddr, dn.infoSecureAddr);
+        });
+
+        $('.browse-dir-links').click(function () {
+          var http_addr = $(this).attr('info-http-addr'), https_addr = $(this).attr('info-https-addr');
+          window.location.href = generate_browse_dn_link(http_addr, https_addr);
+        });
+      });
+    }, function () {
+      show_err_msg('Failed to load the page.');
+    });
+  }
+
+  var BEANS = [
+    {"name": "nn",      "url": "/jmx?qry=Hadoop:service=NameNode,name=NameNodeInfo"},
+    {"name": "nnstat",  "url": "/jmx?qry=Hadoop:service=NameNode,name=NameNodeStatus"},
+    {"name": "fs",      "url": "/jmx?qry=Hadoop:service=NameNode,name=FSNamesystemState"},
+    {"name": "mem",     "url": "/jmx?qry=java.lang:type=Memory"},
+    {"name": "startup", "url": "/startupProgress"}
+  ];
+
+  // Workarounds for the fact that JMXJsonServlet returns non-standard JSON strings
+  function data_workaround(d) {
+    function node_map_to_array(nodes) {
+      var res = [];
+      for (var n in nodes) {
+        var p = nodes[n];
+        p.name = n;
+        res.push(p);
+      }
+      return res;
+    }
+
+    function startup_progress_workaround(r) {
+      function rename_property(o, s, d) {
+        if (o[s] !== undefined) {
+          o[d] = o[s];
+          delete o[s];
+        }
+      }
+      r.percentComplete *= 100;
+      $.each(r.phases, function (idx, p) {
+        p.percentComplete *= 100;
+        $.each(p.steps, function (idx2, s) {
+          s.percentComplete *= 100;
+          // dust.js is confused by these optional keys in nested
+          // structure, rename them
+          rename_property(s, "desc", "stepDesc");
+          rename_property(s, "file", "stepFile");
+          rename_property(s, "size", "stepSize");
+        });
+      });
+      return r;
+    }
+
+    d.nn.JournalTransactionInfo = JSON.parse(d.nn.JournalTransactionInfo);
+    d.nn.NameJournalStatus = JSON.parse(d.nn.NameJournalStatus);
+    d.nn.NameDirStatuses = JSON.parse(d.nn.NameDirStatuses);
+    d.nn.NodeUsage = JSON.parse(d.nn.NodeUsage);
+    d.nn.LiveNodes = node_map_to_array(JSON.parse(d.nn.LiveNodes));
+    d.nn.DeadNodes = node_map_to_array(JSON.parse(d.nn.DeadNodes));
+    d.nn.DecomNodes = node_map_to_array(JSON.parse(d.nn.DecomNodes));
+    d.nn.CorruptFiles = JSON.parse(d.nn.CorruptFiles);
+
+    d.fs.SnapshotStats = JSON.parse(d.fs.SnapshotStats);
+    d.startup = startup_progress_workaround(d.startup);
+    return d;
+  }
+
+  function show_err_msg(msg) {
+    $('#alert-panel-body').html(msg);
+    $('#alert-panel').show();
+  }
+
+  load_json(
+    BEANS,
+    function(d) {
+      for (var k in d) {
+        data[k] = k === "startup" ? d[k] : d[k].beans[0];
+      }
+      data = data_workaround(data);
+      render();
+    },
+    function (url, jqxhr, text, err) {
+      show_err_msg('<p>Failed to retrieve data from ' + url + ', cause: ' + err + '</p>');
+    });
+})();

Різницю між файлами не показано, бо вона завелика
+ 0 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dust-full-2.0.0.min.js


+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dust-helpers-1.1.1.min.js

@@ -0,0 +1,8 @@
+(function(k){function n(b){b=b.f();return"object"===typeof b&&!0===b.h}function p(b,c){return"function"===typeof c?c.toString():c}function l(b,c,d,a,e){a=a||{};var m=d.a,g,f,l=a.d||"";if("undefined"!==typeof a.key)g=k.b.c(a.key,b,c);else if(n(c))g=c.f().i,c.f().g&&(e=function(){return!1});else return h.log("No key specified for filter in:"+l+" helper "),b;f=k.b.c(a.value,b,c);if(e(q(f,a.type,c),q(g,a.type,c))){n(c)&&(c.f().g=!0);if(m)return b.e(m,c);h.log("Missing body block in the "+l+" helper ")}else if(d["else"])return b.e(d["else"],
+c);return b}function q(b,c,d){if(b)switch(c||typeof b){case "number":return+b;case "string":return String(b);case "boolean":return Boolean("false"===b?!1:b);case "date":return new Date(b);case "context":return d.get(b)}return b}var h="undefined"!==typeof console?console:{log:function(){}};k.b={tap:function(b,c,d){var a=b;"function"===typeof b&&(!0===b.l?a=b():(a="",c.c(function(b){a+=b;return""}).e(b,d).p(),""===a&&(a=!1)));return a},sep:function(b,c,d){return c.stack.index===c.stack.m-1?b:d.a?d.a(b,
+c):b},idx:function(b,c,d){return d.a?d.a(b,c.push(c.stack.index)):b},contextDump:function(b,c,d,a){a=a||{};d=a.o||"output";a=a.key||"current";d=k.b.c(d,b,c);a=k.b.c(a,b,c);c="full"===a?JSON.stringify(c.stack,p,2):JSON.stringify(c.stack.head,p,2);return"console"===d?(h.log(c),b):b.write(c)},"if":function(b,c,d,a){var e=d.a,m=d["else"];if(a&&a.j){a=a.j;a=k.b.c(a,b,c);if(eval(a)){if(e)return b.e(d.a,c);h.log("Missing body block in the if helper!");return b}if(m)return b.e(d["else"],c)}else h.log("No condition given in the if helper!");
+return b},math:function(b,c,d,a){if(a&&"undefined"!==typeof a.key&&a.method){var e=a.key,m=a.method,g=a.n;a=a.round;var f=null,e=k.b.c(e,b,c),g=k.b.c(g,b,c);switch(m){case "mod":0!==g&&-0!==g||h.log("operand for divide operation is 0/-0: expect Nan!");f=parseFloat(e)%parseFloat(g);break;case "add":f=parseFloat(e)+parseFloat(g);break;case "subtract":f=parseFloat(e)-parseFloat(g);break;case "multiply":f=parseFloat(e)*parseFloat(g);break;case "divide":0!==g&&-0!==g||h.log("operand for divide operation is 0/-0: expect Nan/Infinity!");
+f=parseFloat(e)/parseFloat(g);break;case "ceil":f=Math.ceil(parseFloat(e));break;case "floor":f=Math.floor(parseFloat(e));break;case "round":f=Math.round(parseFloat(e));break;case "abs":f=Math.abs(parseFloat(e));break;default:h.log("method passed is not supported")}if(null!==f)return a&&(f=Math.round(f)),d&&d.a?b.e(d.a,c.push({h:!0,g:!1,i:f})):b.write(f)}else h.log("Key is a required parameter for math helper along with method/operand!");return b},select:function(b,c,d,a){var e=d.a;if(a&&"undefined"!==
+typeof a.key){a=k.b.c(a.key,b,c);if(e)return b.e(d.a,c.push({h:!0,g:!1,i:a}));h.log("Missing body block in the select helper ")}else h.log("No key given in the select helper!");return b},eq:function(b,c,d,a){a&&(a.d="eq");return l(b,c,d,a,function(a,b){return b===a})},ne:function(b,c,d,a){return a?(a.d="ne",l(b,c,d,a,function(a,b){return b!==a})):b},lt:function(b,c,d,a){if(a)return a.d="lt",l(b,c,d,a,function(a,b){return b<a})},lte:function(b,c,d,a){return a?(a.d="lte",l(b,c,d,a,function(a,b){return b<=
+a})):b},gt:function(b,c,d,a){return a?(a.d="gt",l(b,c,d,a,function(a,b){return b>a})):b},gte:function(b,c,d,a){return a?(a.d="gte",l(b,c,d,a,function(a,b){return b>=a})):b},"default":function(b,c,d,a){a&&(a.d="default");return l(b,c,d,a,function(){return!0})},size:function(b,c,d,a){c=0;var e;a=a||{};if((a=a.key)&&!0!==a)if(k.isArray(a))c=a.length;else if(!isNaN(parseFloat(a))&&isFinite(a))c=a;else if("object"===typeof a)for(e in c=0,a)Object.hasOwnProperty.call(a,e)&&c++;else c=(a+"").length;else c=
+0;return b.write(c)}}})("undefined"!==typeof exports?module.k=require("dustjs-linkedin"):dust);

+ 34 - 9
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestHDFSFileContextMainOperations.java

@@ -19,8 +19,10 @@
 package org.apache.hadoop.fs;
 
 import static org.apache.hadoop.fs.FileContextTestHelper.exists;
+import static org.junit.Assert.fail;
 
 import java.io.IOException;
+import java.net.URI;
 import java.net.URISyntaxException;
 
 import javax.security.auth.login.LoginException;
@@ -55,7 +57,8 @@ public class TestHDFSFileContextMainOperations extends
       LoginException, URISyntaxException {
     cluster = new MiniDFSCluster.Builder(CONF).numDataNodes(2).build();
     cluster.waitClusterUp();
-    fc = FileContext.getFileContext(cluster.getURI(0), CONF);
+    URI uri0 = cluster.getURI(0);
+    fc = FileContext.getFileContext(uri0, CONF);
     defaultWorkingDirectory = fc.makeQualified( new Path("/user/" + 
         UserGroupInformation.getCurrentUser().getShortUserName()));
     fc.mkdir(defaultWorkingDirectory, FileContext.DEFAULT_PERM, true);
@@ -77,7 +80,10 @@ public class TestHDFSFileContextMainOperations extends
       
   @AfterClass
   public static void ClusterShutdownAtEnd() throws Exception {
-    cluster.shutdown();   
+    if (cluster != null) {
+      cluster.shutdown();
+      cluster = null;
+    }    
   }
   
   @Override
@@ -111,7 +117,7 @@ public class TestHDFSFileContextMainOperations extends
   
   @Test
   public void testOldRenameWithQuota() throws Exception {
-    DistributedFileSystem fs = (DistributedFileSystem) cluster.getFileSystem();
+    DistributedFileSystem fs = cluster.getFileSystem();
     Path src1 = getTestRootPath(fc, "test/testOldRenameWithQuota/srcdir/src1");
     Path src2 = getTestRootPath(fc, "test/testOldRenameWithQuota/srcdir/src2");
     Path dst1 = getTestRootPath(fc, "test/testOldRenameWithQuota/dstdir/dst1");
@@ -146,7 +152,7 @@ public class TestHDFSFileContextMainOperations extends
   
   @Test
   public void testRenameWithQuota() throws Exception {
-    DistributedFileSystem fs = (DistributedFileSystem) cluster.getFileSystem();
+    DistributedFileSystem fs = cluster.getFileSystem();
     Path src1 = getTestRootPath(fc, "test/testRenameWithQuota/srcdir/src1");
     Path src2 = getTestRootPath(fc, "test/testRenameWithQuota/srcdir/src2");
     Path dst1 = getTestRootPath(fc, "test/testRenameWithQuota/dstdir/dst1");
@@ -210,7 +216,7 @@ public class TestHDFSFileContextMainOperations extends
    */
   @Test
   public void testEditsLogOldRename() throws Exception {
-    DistributedFileSystem fs = (DistributedFileSystem) cluster.getFileSystem();
+    DistributedFileSystem fs = cluster.getFileSystem();
     Path src1 = getTestRootPath(fc, "testEditsLogOldRename/srcdir/src1");
     Path dst1 = getTestRootPath(fc, "testEditsLogOldRename/dstdir/dst1");
     createFile(src1);
@@ -226,7 +232,7 @@ public class TestHDFSFileContextMainOperations extends
     // Restart the cluster and ensure the above operations can be
     // loaded from the edits log
     restartCluster();
-    fs = (DistributedFileSystem)cluster.getFileSystem();
+    fs = cluster.getFileSystem();
     src1 = getTestRootPath(fc, "testEditsLogOldRename/srcdir/src1");
     dst1 = getTestRootPath(fc, "testEditsLogOldRename/dstdir/dst1");
     Assert.assertFalse(fs.exists(src1));   // ensure src1 is already renamed
@@ -239,7 +245,7 @@ public class TestHDFSFileContextMainOperations extends
    */
   @Test
   public void testEditsLogRename() throws Exception {
-    DistributedFileSystem fs = (DistributedFileSystem) cluster.getFileSystem();
+    DistributedFileSystem fs = cluster.getFileSystem();
     Path src1 = getTestRootPath(fc, "testEditsLogRename/srcdir/src1");
     Path dst1 = getTestRootPath(fc, "testEditsLogRename/dstdir/dst1");
     createFile(src1);
@@ -255,7 +261,7 @@ public class TestHDFSFileContextMainOperations extends
     // Restart the cluster and ensure the above operations can be
     // loaded from the edits log
     restartCluster();
-    fs = (DistributedFileSystem)cluster.getFileSystem();
+    fs = cluster.getFileSystem();
     src1 = getTestRootPath(fc, "testEditsLogRename/srcdir/src1");
     dst1 = getTestRootPath(fc, "testEditsLogRename/dstdir/dst1");
     Assert.assertFalse(fs.exists(src1));   // ensure src1 is already renamed
@@ -279,7 +285,7 @@ public class TestHDFSFileContextMainOperations extends
 
   private void oldRename(Path src, Path dst, boolean renameSucceeds,
       boolean exception) throws Exception {
-    DistributedFileSystem fs = (DistributedFileSystem) cluster.getFileSystem();
+    DistributedFileSystem fs = cluster.getFileSystem();
     try {
       Assert.assertEquals(renameSucceeds, fs.rename(src, dst));
     } catch (Exception ex) {
@@ -301,4 +307,23 @@ public class TestHDFSFileContextMainOperations extends
     Assert.assertEquals(renameSucceeds, !exists(fc, src));
     Assert.assertEquals((dstExists||renameSucceeds), exists(fc, dst));
   }
+  
+  @Override
+  protected boolean listCorruptedBlocksSupported() {
+    return true;
+  }
+  
+  @Test
+  public void testCrossFileSystemRename() throws IOException {
+    try {
+      fc.rename(
+        new Path("hdfs://127.0.0.1/aaa/bbb/Foo"), 
+        new Path("file://aaa/bbb/Moo"), 
+        Options.Rename.OVERWRITE);
+      fail("IOexception expected.");
+    } catch (IOException ioe) {
+      // okay
+    }
+  }
+  
 }

+ 28 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java

@@ -23,12 +23,15 @@ import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import java.io.ByteArrayOutputStream;
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.io.PrintStream;
 import java.security.PrivilegedAction;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
@@ -921,4 +924,29 @@ public class TestSnapshotDeletion {
     subFile1Status = hdfs.getFileStatus(subFile1SCopy);
     assertEquals(REPLICATION_1, subFile1Status.getReplication());
   }
+  
+  @Test
+  public void testDeleteSnapshotCommandWithIllegalArguments() throws Exception {
+    ByteArrayOutputStream out = new ByteArrayOutputStream();
+    PrintStream psOut = new PrintStream(out);
+    System.setOut(psOut);
+    System.setErr(psOut);
+    FsShell shell = new FsShell();
+    shell.setConf(conf);
+    
+    String[] argv1 = {"-deleteSnapshot", "/tmp"};
+    int val = shell.run(argv1);
+    assertTrue(val == -1);
+    assertTrue(out.toString().contains(
+        argv1[0] + ": Incorrect number of arguments."));
+    out.reset();
+    
+    String[] argv2 = {"-deleteSnapshot", "/tmp", "s1", "s2"};
+    val = shell.run(argv2);
+    assertTrue(val == -1);
+    assertTrue(out.toString().contains(
+        argv2[0] + ": Incorrect number of arguments."));
+    psOut.close();
+    out.close();
+  }
 }

+ 28 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotRename.java

@@ -22,10 +22,13 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import java.io.ByteArrayOutputStream;
+import java.io.PrintStream;
 import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
@@ -226,4 +229,29 @@ public class TestSnapshotRename {
       }
     }
   }
+  
+  @Test
+  public void testRenameSnapshotCommandWithIllegalArguments() throws Exception {
+    ByteArrayOutputStream out = new ByteArrayOutputStream();
+    PrintStream psOut = new PrintStream(out);
+    System.setOut(psOut);
+    System.setErr(psOut);
+    FsShell shell = new FsShell();
+    shell.setConf(conf);
+    
+    String[] argv1 = {"-renameSnapshot", "/tmp", "s1"};
+    int val = shell.run(argv1);
+    assertTrue(val == -1);
+    assertTrue(out.toString().contains(
+        argv1[0] + ": Incorrect number of arguments."));
+    out.reset();
+    
+    String[] argv2 = {"-renameSnapshot", "/tmp", "s1", "s2", "s3"};
+    val = shell.run(argv2);
+    assertTrue(val == -1);
+    assertTrue(out.toString().contains(
+        argv2[0] + ": Incorrect number of arguments."));
+    psOut.close();
+    out.close();
+  }
 }

Деякі файли не було показано, через те що забагато файлів було змінено