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 zj...@apache.org on 2015/01/16 19:31:20 UTC

[07/25] hadoop git commit: HDFS-2219. Change fsck to support fully qualified paths so that a particular namenode in a federated cluster with multiple namenodes can be specified in the path parameter.

HDFS-2219. Change fsck to support fully qualified paths so that a particular namenode in a federated cluster with multiple namenodes can be specified in the path parameter.


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

Branch: refs/heads/YARN-2928
Commit: 7fe0f25ad21f006eb41b832a181eb2a812a6f7b7
Parents: 446545c
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Wed Jan 14 11:31:18 2015 -0800
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Wed Jan 14 11:31:18 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   4 +
 .../org/apache/hadoop/hdfs/tools/DFSck.java     |  32 +++--
 .../namenode/TestFsckWithMultipleNameNodes.java | 141 +++++++++++++++++++
 3 files changed, 163 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7fe0f25a/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 421d2f0..b879f62d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -502,6 +502,10 @@ Release 2.7.0 - UNRELEASED
 
     HDFS-7600. Refine hdfs admin classes to reuse common code. (jing9)
 
+    HDFS-2219. Change fsck to support fully qualified paths so that a 
+    particular namenode in a federated cluster with multiple namenodes
+    can be specified in the path parameter.  (szetszwo)
+
   OPTIMIZATIONS
 
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7fe0f25a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java
index 0d73b43..98f2030 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HAUtil;
@@ -229,14 +230,14 @@ public class DFSck extends Configured implements Tool {
    * @return Returns http address or null if failure.
    * @throws IOException if we can't determine the active NN address
    */
-  private URI getCurrentNamenodeAddress() throws IOException {
+  private URI getCurrentNamenodeAddress(Path target) throws IOException {
     //String nnAddress = null;
     Configuration conf = getConf();
 
     //get the filesystem object to verify it is an HDFS system
-    FileSystem fs;
+    final FileSystem fs;
     try {
-      fs = FileSystem.get(conf);
+      fs = target.getFileSystem(conf);
     } catch (IOException ioe) {
       System.err.println("FileSystem is inaccessible due to:\n"
           + StringUtils.stringifyException(ioe));
@@ -254,16 +255,6 @@ public class DFSck extends Configured implements Tool {
   private int doWork(final String[] args) throws IOException {
     final StringBuilder url = new StringBuilder();
     
-    URI namenodeAddress = getCurrentNamenodeAddress();
-    if (namenodeAddress == null) {
-      //Error message already output in {@link #getCurrentNamenodeAddress()}
-      System.err.println("DFSck exiting.");
-      return 0;
-    }
-
-    url.append(namenodeAddress.toString());
-    System.err.println("Connecting to namenode via " + url.toString());
-    
     url.append("/fsck?ugi=").append(ugi.getShortUserName());
     String dir = null;
     boolean doListCorruptFileBlocks = false;
@@ -309,7 +300,20 @@ public class DFSck extends Configured implements Tool {
     if (null == dir) {
       dir = "/";
     }
-    url.append("&path=").append(URLEncoder.encode(dir, "UTF-8"));
+
+    final Path dirpath = new Path(dir);
+    final URI namenodeAddress = getCurrentNamenodeAddress(dirpath);
+    if (namenodeAddress == null) {
+      //Error message already output in {@link #getCurrentNamenodeAddress()}
+      System.err.println("DFSck exiting.");
+      return 0;
+    }
+
+    url.insert(0, namenodeAddress.toString());
+    url.append("&path=").append(URLEncoder.encode(
+        Path.getPathWithoutSchemeAndAuthority(dirpath).toString(), "UTF-8"));
+    System.err.println("Connecting to namenode via " + url.toString());
+
     if (doListCorruptFileBlocks) {
       return listCorruptFileBlocks(dir, url.toString());
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7fe0f25a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsckWithMultipleNameNodes.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsckWithMultipleNameNodes.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsckWithMultipleNameNodes.java
new file mode 100644
index 0000000..518a003
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsckWithMultipleNameNodes.java
@@ -0,0 +1,141 @@
+/**
+ * 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.namenode;
+
+import java.io.IOException;
+import java.util.Random;
+import java.util.concurrent.TimeoutException;
+
+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.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.server.balancer.TestBalancer;
+import org.apache.log4j.Level;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test fsck with multiple NameNodes
+ */
+public class TestFsckWithMultipleNameNodes {
+  static final Log LOG = LogFactory.getLog(TestFsckWithMultipleNameNodes.class);
+  {
+    ((Log4JLogger)NameNode.stateChangeLog).getLogger().setLevel(Level.OFF);
+    ((Log4JLogger)LeaseManager.LOG).getLogger().setLevel(Level.OFF);
+    ((Log4JLogger)LogFactory.getLog(FSNamesystem.class)).getLogger().setLevel(Level.OFF);
+  }
+
+  
+  private static final String FILE_NAME = "/tmp.txt";
+  private static final Path FILE_PATH = new Path(FILE_NAME);
+  
+  private static final Random RANDOM = new Random();
+
+  static {
+    TestBalancer.initTestSetup();
+  }
+
+  /** Common objects used in various methods. */
+  private static class Suite {
+    final MiniDFSCluster cluster;
+    final ClientProtocol[] clients;
+    final short replication;
+    
+    Suite(MiniDFSCluster cluster, final int nNameNodes, final int nDataNodes)
+        throws IOException {
+      this.cluster = cluster;
+      clients = new ClientProtocol[nNameNodes];
+      for(int i = 0; i < nNameNodes; i++) {
+        clients[i] = cluster.getNameNode(i).getRpcServer();
+      }
+      replication = (short)Math.max(1, nDataNodes - 1);
+    }
+
+    /** create a file with a length of <code>fileLen</code> */
+    private void createFile(int index, long len
+        ) throws IOException, InterruptedException, TimeoutException {
+      final FileSystem fs = cluster.getFileSystem(index);
+      DFSTestUtil.createFile(fs, FILE_PATH, len, replication, RANDOM.nextLong());
+      DFSTestUtil.waitReplication(fs, FILE_PATH, replication);
+    }
+
+  }
+
+  private static Configuration createConf() {
+    final Configuration conf = new HdfsConfiguration();
+    conf.setLong(DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY, 1L);
+    conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 10000L);
+    return conf;
+  }
+
+  private void runTest(final int nNameNodes, final int nDataNodes,
+      Configuration conf) throws Exception {
+    LOG.info("nNameNodes=" + nNameNodes + ", nDataNodes=" + nDataNodes);
+
+    LOG.info("RUN_TEST -1");
+    final MiniDFSCluster cluster = new MiniDFSCluster
+        .Builder(conf)
+        .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(nNameNodes))
+        .numDataNodes(nDataNodes)
+        .build();
+    LOG.info("RUN_TEST 0");
+    DFSTestUtil.setFederatedConfiguration(cluster, conf);
+
+    try {
+      cluster.waitActive();
+      LOG.info("RUN_TEST 1");
+      final Suite s = new Suite(cluster, nNameNodes, nDataNodes);
+      for(int i = 0; i < nNameNodes; i++) {
+        s.createFile(i, 1024);
+      }
+
+      LOG.info("RUN_TEST 2");
+      final String[] urls = new String[nNameNodes];
+      for(int i = 0; i < urls.length; i++) {
+        urls[i] = cluster.getFileSystem(i).getUri() + FILE_NAME;
+        LOG.info("urls[" + i + "]=" + urls[i]);
+        final String result = TestFsck.runFsck(conf, 0, false, urls[i]);
+        LOG.info("result=" + result);
+        Assert.assertTrue(result.contains("Status: HEALTHY"));
+      }
+    } finally {
+      cluster.shutdown();
+    }
+    LOG.info("RUN_TEST 6");
+  }
+  
+  /** Test a cluster with even distribution, 
+   * then a new empty node is added to the cluster
+   */
+  @Test
+  public void testFsck() throws Exception {
+    final Configuration conf = createConf();
+    runTest(3, 1, conf);
+  }
+
+}
+