You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by to...@apache.org on 2012/04/19 01:35:39 UTC

svn commit: r1327724 [1/3] - in /hadoop/common/branches/HDFS-3042/hadoop-hdfs-project: ./ hadoop-hdfs-httpfs/ hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/ hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/ hadoop-hdfs...

Author: todd
Date: Wed Apr 18 23:35:30 2012
New Revision: 1327724

URL: http://svn.apache.org/viewvc?rev=1327724&view=rev
Log:
Merge trunk into auto-HA branch

Removed:
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/test/GenericTestUtils.java
Modified:
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/   (props changed)
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/   (props changed)
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/   (props changed)
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServerWebApp.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/FileSystemAccess.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/FileSystemAccessException.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/hadoop/FileSystemAccessService.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/resources/httpfs-default.xml
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/apt/ServerSetup.apt.vm
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFileSystem.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/hadoop/TestFileSystemAccessService.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/HadoopUsersConfTestHelper.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/   (props changed)
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/contrib/fuse-dfs/   (props changed)
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/start-balancer.sh
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/stop-balancer.sh
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/   (props changed)
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/Hdfs.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HAUtil.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSelector.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/IncorrectVersionException.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/resources/DatanodeWebHdfsMethods.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/journalservice/JournalListener.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/journalservice/JournalService.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ConfiguredFailoverProxyProvider.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeRegistration.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/native/   (props changed)
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode/   (props changed)
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/   (props changed)
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/   (props changed)
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/hdfs/   (props changed)
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeRegistration.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHftpDelegationToken.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplaceDatanodeOnFailure.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestClientProtocolWithDelegationToken.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlocksWithNotEnoughRacks.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRegister.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/journalservice/TestJournalService.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDelegationTokensWithHA.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestInitializeSharedEdits.java
    hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java

Propchange: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/
------------------------------------------------------------------------------
--- svn:ignore (original)
+++ svn:ignore Wed Apr 18 23:35:30 2012
@@ -1 +1,4 @@
 target
+.classpath
+.project
+.settings

Propchange: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs:r1324567-1327718

Propchange: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/
------------------------------------------------------------------------------
--- svn:ignore (original)
+++ svn:ignore Wed Apr 18 23:35:30 2012
@@ -1 +1,4 @@
 target
+.classpath
+.project
+.settings

Modified: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java?rev=1327724&r1=1327723&r2=1327724&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java (original)
+++ hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java Wed Apr 18 23:35:30 2012
@@ -170,7 +170,7 @@ public class HttpFSServer {
     throws IOException, FileSystemAccessException {
     String hadoopUser = getEffectiveUser(user, doAs);
     FileSystemAccess fsAccess = HttpFSServerWebApp.get().get(FileSystemAccess.class);
-    Configuration conf = HttpFSServerWebApp.get().get(FileSystemAccess.class).getDefaultConfiguration();
+    Configuration conf = HttpFSServerWebApp.get().get(FileSystemAccess.class).getFileSystemConfiguration();
     return fsAccess.execute(hadoopUser, conf, executor);
   }
 
@@ -194,7 +194,7 @@ public class HttpFSServer {
   private FileSystem createFileSystem(Principal user, String doAs) throws IOException, FileSystemAccessException {
     String hadoopUser = getEffectiveUser(user, doAs);
     FileSystemAccess fsAccess = HttpFSServerWebApp.get().get(FileSystemAccess.class);
-    Configuration conf = HttpFSServerWebApp.get().get(FileSystemAccess.class).getDefaultConfiguration();
+    Configuration conf = HttpFSServerWebApp.get().get(FileSystemAccess.class).getFileSystemConfiguration();
     FileSystem fs = fsAccess.createFileSystem(hadoopUser, conf);
     FileSystemReleaseFilter.setFileSystem(fs);
     return fs;

Modified: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServerWebApp.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServerWebApp.java?rev=1327724&r1=1327723&r2=1327724&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServerWebApp.java (original)
+++ hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServerWebApp.java Wed Apr 18 23:35:30 2012
@@ -19,6 +19,7 @@
 package org.apache.hadoop.fs.http.server;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.lib.server.ServerException;
 import org.apache.hadoop.lib.service.FileSystemAccess;
 import org.apache.hadoop.lib.servlet.ServerWebApp;
@@ -29,8 +30,9 @@ import java.io.IOException;
 
 /**
  * Bootstrap class that manages the initialization and destruction of the
- * HttpFSServer server, it is a <code>javax.servlet.ServletContextListener</code>
- * implementation that is wired in HttpFSServer's WAR <code>WEB-INF/web.xml</code>.
+ * HttpFSServer server, it is a <code>javax.servlet.ServletContextListener
+ * </code> implementation that is wired in HttpFSServer's WAR
+ * <code>WEB-INF/web.xml</code>.
  * <p/>
  * It provides acces to the server context via the singleton {@link #get}.
  * <p/>
@@ -38,7 +40,8 @@ import java.io.IOException;
  * with <code>httpfs.</code>.
  */
 public class HttpFSServerWebApp extends ServerWebApp {
-  private static final Logger LOG = LoggerFactory.getLogger(HttpFSServerWebApp.class);
+  private static final Logger LOG =
+    LoggerFactory.getLogger(HttpFSServerWebApp.class);
 
   /**
    * Server name and prefix for all configuration properties.
@@ -67,8 +70,8 @@ public class HttpFSServerWebApp extends 
   /**
    * Constructor used for testing purposes.
    */
-  protected HttpFSServerWebApp(String homeDir, String configDir, String logDir, String tempDir,
-                               Configuration config) {
+  protected HttpFSServerWebApp(String homeDir, String configDir, String logDir,
+                               String tempDir, Configuration config) {
     super(NAME, homeDir, configDir, logDir, tempDir, config);
   }
 
@@ -80,9 +83,11 @@ public class HttpFSServerWebApp extends 
   }
 
   /**
-   * Initializes the HttpFSServer server, loads configuration and required services.
+   * Initializes the HttpFSServer server, loads configuration and required
+   * services.
    *
-   * @throws ServerException thrown if HttpFSServer server could not be initialized.
+   * @throws ServerException thrown if HttpFSServer server could not be
+   * initialized.
    */
   @Override
   public void init() throws ServerException {
@@ -93,7 +98,8 @@ public class HttpFSServerWebApp extends 
     SERVER = this;
     adminGroup = getConfig().get(getPrefixedName(CONF_ADMIN_GROUP), "admin");
     LOG.info("Connects to Namenode [{}]",
-             get().get(FileSystemAccess.class).getDefaultConfiguration().get("fs.default.name"));
+             get().get(FileSystemAccess.class).getFileSystemConfiguration().
+               get(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY));
   }
 
   /**
@@ -106,7 +112,8 @@ public class HttpFSServerWebApp extends 
   }
 
   /**
-   * Returns HttpFSServer server singleton, configuration and services are accessible through it.
+   * Returns HttpFSServer server singleton, configuration and services are
+   * accessible through it.
    *
    * @return the HttpFSServer server singleton.
    */

Modified: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/FileSystemAccess.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/FileSystemAccess.java?rev=1327724&r1=1327723&r2=1327724&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/FileSystemAccess.java (original)
+++ hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/FileSystemAccess.java Wed Apr 18 23:35:30 2012
@@ -37,6 +37,6 @@ public interface FileSystemAccess {
 
   public void releaseFileSystem(FileSystem fs) throws IOException;
 
-  public Configuration getDefaultConfiguration();
+  public Configuration getFileSystemConfiguration();
 
 }

Modified: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/FileSystemAccessException.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/FileSystemAccessException.java?rev=1327724&r1=1327723&r2=1327724&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/FileSystemAccessException.java (original)
+++ hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/FileSystemAccessException.java Wed Apr 18 23:35:30 2012
@@ -26,12 +26,14 @@ public class FileSystemAccessException e
     H01("Service property [{0}] not defined"),
     H02("Kerberos initialization failed, {0}"),
     H03("FileSystemExecutor error, {0}"),
-    H04("JobClientExecutor error, {0}"),
+    H04("Invalid configuration, it has not be created by the FileSystemAccessService"),
     H05("[{0}] validation failed, {1}"),
     H06("Property [{0}] not defined in configuration object"),
     H07("[{0}] not healthy, {1}"),
-    H08(""),
-    H09("Invalid FileSystemAccess security mode [{0}]");
+    H08("{0}"),
+    H09("Invalid FileSystemAccess security mode [{0}]"),
+    H10("Hadoop config directory not found [{0}]"),
+    H11("Could not load Hadoop config files, {0}");
 
     private String template;
 

Modified: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/hadoop/FileSystemAccessService.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/hadoop/FileSystemAccessService.java?rev=1327724&r1=1327723&r2=1327724&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/hadoop/FileSystemAccessService.java (original)
+++ hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/hadoop/FileSystemAccessService.java Wed Apr 18 23:35:30 2012
@@ -19,7 +19,9 @@
 package org.apache.hadoop.lib.service.hadoop;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.lib.server.BaseService;
 import org.apache.hadoop.lib.server.ServiceException;
 import org.apache.hadoop.lib.service.FileSystemAccess;
@@ -32,6 +34,7 @@ import org.apache.hadoop.util.VersionInf
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.File;
 import java.io.IOException;
 import java.net.URI;
 import java.security.PrivilegedExceptionAction;
@@ -54,9 +57,11 @@ public class FileSystemAccessService ext
 
   public static final String NAME_NODE_WHITELIST = "name.node.whitelist";
 
-  private static final String HADOOP_CONF_PREFIX = "conf:";
+  public static final String HADOOP_CONF_DIR = "config.dir";
 
-  private static final String NAME_NODE_PROPERTY = "fs.default.name";
+  private static final String[] HADOOP_CONF_FILES = {"core-site.xml", "hdfs-site.xml"};
+
+  private static final String FILE_SYSTEM_SERVICE_CREATED = "FileSystemAccessService.created";
 
   public FileSystemAccessService() {
     super(PREFIX);
@@ -102,26 +107,40 @@ public class FileSystemAccessService ext
       throw new ServiceException(FileSystemAccessException.ERROR.H09, security);
     }
 
-    serviceHadoopConf = new Configuration(false);
-    for (Map.Entry entry : getServiceConfig()) {
-      String name = (String) entry.getKey();
-      if (name.startsWith(HADOOP_CONF_PREFIX)) {
-        name = name.substring(HADOOP_CONF_PREFIX.length());
-        String value = (String) entry.getValue();
-        serviceHadoopConf.set(name, value);
-
-      }
+    String hadoopConfDirProp = getServiceConfig().get(HADOOP_CONF_DIR, getServer().getConfigDir());
+    File hadoopConfDir = new File(hadoopConfDirProp).getAbsoluteFile();
+    if (hadoopConfDir == null) {
+      hadoopConfDir = new File(getServer().getConfigDir()).getAbsoluteFile();
+    }
+    if (!hadoopConfDir.exists()) {
+      throw new ServiceException(FileSystemAccessException.ERROR.H10, hadoopConfDir);
+    }
+    try {
+      serviceHadoopConf = loadHadoopConf(hadoopConfDir);
+    } catch (IOException ex) {
+      throw new ServiceException(FileSystemAccessException.ERROR.H11, ex.toString(), ex);
     }
-    setRequiredServiceHadoopConf(serviceHadoopConf);
 
-    LOG.debug("FileSystemAccess default configuration:");
+    LOG.debug("FileSystemAccess FileSystem configuration:");
     for (Map.Entry entry : serviceHadoopConf) {
       LOG.debug("  {} = {}", entry.getKey(), entry.getValue());
     }
+    setRequiredServiceHadoopConf(serviceHadoopConf);
 
     nameNodeWhitelist = toLowerCase(getServiceConfig().getTrimmedStringCollection(NAME_NODE_WHITELIST));
   }
 
+  private Configuration loadHadoopConf(File dir) throws IOException {
+    Configuration hadoopConf = new Configuration(false);
+    for (String file : HADOOP_CONF_FILES) {
+      File f = new File(dir, file);
+      if (f.exists()) {
+        hadoopConf.addResource(new Path(f.getAbsolutePath()));
+      }
+    }
+    return hadoopConf;
+  }
+
   @Override
   public void postInit() throws ServiceException {
     super.postInit();
@@ -166,17 +185,6 @@ public class FileSystemAccessService ext
     conf.set("fs.hdfs.impl.disable.cache", "true");
   }
 
-  protected Configuration createHadoopConf(Configuration conf) {
-    Configuration hadoopConf = new Configuration();
-    ConfigurationUtils.copy(serviceHadoopConf, hadoopConf);
-    ConfigurationUtils.copy(conf, hadoopConf);
-    return hadoopConf;
-  }
-
-  protected Configuration createNameNodeConf(Configuration conf) {
-    return createHadoopConf(conf);
-  }
-
   protected FileSystem createFileSystem(Configuration namenodeConf) throws IOException {
     return FileSystem.get(namenodeConf);
   }
@@ -202,16 +210,22 @@ public class FileSystemAccessService ext
     Check.notEmpty(user, "user");
     Check.notNull(conf, "conf");
     Check.notNull(executor, "executor");
-    if (conf.get(NAME_NODE_PROPERTY) == null || conf.getTrimmed(NAME_NODE_PROPERTY).length() == 0) {
-      throw new FileSystemAccessException(FileSystemAccessException.ERROR.H06, NAME_NODE_PROPERTY);
+    if (!conf.getBoolean(FILE_SYSTEM_SERVICE_CREATED, false)) {
+      throw new FileSystemAccessException(FileSystemAccessException.ERROR.H04);
+    }
+    if (conf.get(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY) == null ||
+        conf.getTrimmed(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY).length() == 0) {
+      throw new FileSystemAccessException(FileSystemAccessException.ERROR.H06,
+                                          CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY);
     }
     try {
-      validateNamenode(new URI(conf.get(NAME_NODE_PROPERTY)).getAuthority());
+      validateNamenode(
+        new URI(conf.get(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY)).
+          getAuthority());
       UserGroupInformation ugi = getUGI(user);
       return ugi.doAs(new PrivilegedExceptionAction<T>() {
         public T run() throws Exception {
-          Configuration namenodeConf = createNameNodeConf(conf);
-          FileSystem fs = createFileSystem(namenodeConf);
+          FileSystem fs = createFileSystem(conf);
           Instrumentation instrumentation = getServer().get(Instrumentation.class);
           Instrumentation.Cron cron = instrumentation.createCron();
           try {
@@ -236,13 +250,16 @@ public class FileSystemAccessService ext
     throws IOException, FileSystemAccessException {
     Check.notEmpty(user, "user");
     Check.notNull(conf, "conf");
+    if (!conf.getBoolean(FILE_SYSTEM_SERVICE_CREATED, false)) {
+      throw new FileSystemAccessException(FileSystemAccessException.ERROR.H04);
+    }
     try {
-      validateNamenode(new URI(conf.get(NAME_NODE_PROPERTY)).getAuthority());
+      validateNamenode(
+        new URI(conf.get(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY)).getAuthority());
       UserGroupInformation ugi = getUGI(user);
       return ugi.doAs(new PrivilegedExceptionAction<FileSystem>() {
         public FileSystem run() throws Exception {
-          Configuration namenodeConf = createNameNodeConf(conf);
-          return createFileSystem(namenodeConf);
+          return createFileSystem(conf);
         }
       });
     } catch (IOException ex) {
@@ -267,11 +284,11 @@ public class FileSystemAccessService ext
     closeFileSystem(fs);
   }
 
-
   @Override
-  public Configuration getDefaultConfiguration() {
-    Configuration conf = new Configuration(false);
+  public Configuration getFileSystemConfiguration() {
+    Configuration conf = new Configuration(true);
     ConfigurationUtils.copy(serviceHadoopConf, conf);
+    conf.setBoolean(FILE_SYSTEM_SERVICE_CREATED, true);
     return conf;
   }
 

Modified: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/resources/httpfs-default.xml
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/resources/httpfs-default.xml?rev=1327724&r1=1327723&r2=1327724&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/resources/httpfs-default.xml (original)
+++ hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/resources/httpfs-default.xml Wed Apr 18 23:35:30 2012
@@ -153,29 +153,6 @@
     </description>
   </property>
 
-  <!-- FileSystemAccess Namenode Configuration -->
-
-  <property>
-    <name>namenode.hostname</name>
-    <value>localhost</value>
-    <description>
-      The HDFS Namenode host the httpfs server connects to perform file
-      system operations.
-
-      This property is only used to resolve other properties within this
-      configuration file.
-    </description>
-  </property>
-
-  <property>
-    <name>httpfs.hadoop.conf:fs.default.name</name>
-    <value>hdfs://${namenode.hostname}:8020</value>
-    <description>
-      The HDFS Namenode URI the httpfs server connects to perform file
-      system operations.
-    </description>
-  </property>
-
   <!-- FileSystemAccess Namenode Security Configuration -->
 
   <property>
@@ -206,12 +183,4 @@
     </description>
   </property>
 
-  <property>
-    <name>httpfs.hadoop.conf:dfs.namenode.kerberos.principal</name>
-    <value>hdfs/${namenode.hostname}@${kerberos.realm}</value>
-    <description>
-      The HDFS Namenode Kerberos principal.
-    </description>
-  </property>
-
 </configuration>

Modified: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/apt/ServerSetup.apt.vm
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/apt/ServerSetup.apt.vm?rev=1327724&r1=1327723&r2=1327724&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/apt/ServerSetup.apt.vm (original)
+++ hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/apt/ServerSetup.apt.vm Wed Apr 18 23:35:30 2012
@@ -37,13 +37,13 @@ Hadoop HDFS over HTTP ${project.version}
 
 * Configure HttpFS
 
-  Edit the <<<httpfs-${project.version}/conf/httpfs-site.xml>>> file and
-  set the <<<httpfs.fsAccess.conf:fs.default.name>>> property to the HDFS
-  Namenode URI. For example:
-
-+---+
-httpfs.fsAccess.conf:fs.default.name=hdfs://localhost:8021
-+---+
+  By default, HttpFS assumes that Hadoop configuration files
+  (<<<core-site.xml & hdfs-site.xml>>>) are in the HttpFS
+  configuration directory.
+
+  If this is not the case, add to the <<<httpfs-site.xml>>> file the
+  <<<httpfs.hadoop.config.dir>>> property set to the location
+  of the Hadoop configuration directory.
 
 * Configure Hadoop
 
@@ -53,11 +53,11 @@ httpfs.fsAccess.conf:fs.default.name=hdf
 +---+
   ...
   <property>
-    <name>fsAccess.proxyuser.#HTTPFSUSER#.hosts</name>
+    <name>hadoop.proxyuser.#HTTPFSUSER#.hosts</name>
     <value>httpfs-host.foo.com</value>
   </property>
   <property>
-    <name>fsAccess.proxyuser.#HTTPFSUSER#.groups</name>
+    <name>hadoop.proxyuser.#HTTPFSUSER#.groups</name>
     <value>*</value>
   </property>
   ...

Modified: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFileSystem.java?rev=1327724&r1=1327723&r2=1327724&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFileSystem.java (original)
+++ hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFileSystem.java Wed Apr 18 23:35:30 2012
@@ -19,6 +19,7 @@
 package org.apache.hadoop.fs.http.client;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.fs.FileStatus;
@@ -70,16 +71,24 @@ public class TestHttpFSFileSystem extend
     w.write("secret");
     w.close();
 
-    String fsDefaultName = TestHdfsHelper.getHdfsConf().get("fs.default.name");
+    //HDFS configuration
+    String fsDefaultName = TestHdfsHelper.getHdfsConf().get(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY);
     Configuration conf = new Configuration(false);
-    conf.set("httpfs.hadoop.conf:fs.default.name", fsDefaultName);
-    conf.set("httpfs.proxyuser." + HadoopUsersConfTestHelper.getHadoopProxyUser() + ".groups", HadoopUsersConfTestHelper
-      .getHadoopProxyUserGroups());
-    conf.set("httpfs.proxyuser." + HadoopUsersConfTestHelper.getHadoopProxyUser() + ".hosts", HadoopUsersConfTestHelper
-      .getHadoopProxyUserHosts());
+    conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, fsDefaultName);
+    File hdfsSite = new File(new File(homeDir, "conf"), "hdfs-site.xml");
+    OutputStream os = new FileOutputStream(hdfsSite);
+    conf.writeXml(os);
+    os.close();
+
+    //HTTPFS configuration
+    conf = new Configuration(false);
+    conf.set("httpfs.proxyuser." + HadoopUsersConfTestHelper.getHadoopProxyUser() + ".groups",
+             HadoopUsersConfTestHelper.getHadoopProxyUserGroups());
+    conf.set("httpfs.proxyuser." + HadoopUsersConfTestHelper.getHadoopProxyUser() + ".hosts",
+             HadoopUsersConfTestHelper.getHadoopProxyUserHosts());
     conf.set("httpfs.authentication.signature.secret.file", secretFile.getAbsolutePath());
-    File hoopSite = new File(new File(homeDir, "conf"), "httpfs-site.xml");
-    OutputStream os = new FileOutputStream(hoopSite);
+    File httpfsSite = new File(new File(homeDir, "conf"), "httpfs-site.xml");
+    os = new FileOutputStream(httpfsSite);
     conf.writeXml(os);
     os.close();
 

Modified: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java?rev=1327724&r1=1327723&r2=1327724&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java (original)
+++ hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java Wed Apr 18 23:35:30 2012
@@ -20,10 +20,12 @@ package org.apache.hadoop.fs.http.server
 
 import junit.framework.Assert;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.lib.service.security.DummyGroupMapping;
+import org.apache.hadoop.lib.server.Service;
+import org.apache.hadoop.lib.server.ServiceException;
+import org.apache.hadoop.lib.service.Groups;
 import org.apache.hadoop.test.HFSTestCase;
 import org.apache.hadoop.test.HadoopUsersConfTestHelper;
 import org.apache.hadoop.test.TestDir;
@@ -40,12 +42,15 @@ import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.FileWriter;
+import java.io.IOException;
 import java.io.InputStreamReader;
 import java.io.OutputStream;
 import java.io.Writer;
 import java.net.HttpURLConnection;
 import java.net.URL;
 import java.text.MessageFormat;
+import java.util.Arrays;
+import java.util.List;
 
 public class TestHttpFSServer extends HFSTestCase {
 
@@ -54,12 +59,48 @@ public class TestHttpFSServer extends HF
   @TestJetty
   public void server() throws Exception {
     String dir = TestDirHelper.getTestDir().getAbsolutePath();
-    Configuration hoopConf = new Configuration(false);
-    HttpFSServerWebApp server = new HttpFSServerWebApp(dir, dir, dir, dir, hoopConf);
+
+    Configuration httpfsConf = new Configuration(false);
+    HttpFSServerWebApp server = new HttpFSServerWebApp(dir, dir, dir, dir, httpfsConf);
     server.init();
     server.destroy();
   }
 
+  public static class MockGroups implements Service,Groups {
+
+    @Override
+    public void init(org.apache.hadoop.lib.server.Server server) throws ServiceException {
+    }
+
+    @Override
+    public void postInit() throws ServiceException {
+    }
+
+    @Override
+    public void destroy() {
+    }
+
+    @Override
+    public Class[] getServiceDependencies() {
+      return new Class[0];
+    }
+
+    @Override
+    public Class getInterface() {
+      return Groups.class;
+    }
+
+    @Override
+    public void serverStatusChange(org.apache.hadoop.lib.server.Server.Status oldStatus,
+                                   org.apache.hadoop.lib.server.Server.Status newStatus) throws ServiceException {
+    }
+
+    @Override
+    public List<String> getGroups(String user) throws IOException {
+      return Arrays.asList(HadoopUsersConfTestHelper.getHadoopUserGroups(user));
+    }
+
+  }
   private void createHttpFSServer() throws Exception {
     File homeDir = TestDirHelper.getTestDir();
     Assert.assertTrue(new File(homeDir, "conf").mkdir());
@@ -72,13 +113,29 @@ public class TestHttpFSServer extends HF
     w.write("secret");
     w.close();
 
-    String fsDefaultName = TestHdfsHelper.getHdfsConf().get("fs.default.name");
+    //HDFS configuration
+    File hadoopConfDir = new File(new File(homeDir, "conf"), "hadoop-conf");
+    hadoopConfDir.mkdirs();
+    String fsDefaultName = TestHdfsHelper.getHdfsConf().get(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY);
     Configuration conf = new Configuration(false);
-    conf.set("httpfs.hadoop.conf:fs.default.name", fsDefaultName);
-    conf.set("httpfs.groups." + CommonConfigurationKeys.HADOOP_SECURITY_GROUP_MAPPING, DummyGroupMapping.class.getName());
+    conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, fsDefaultName);
+    File hdfsSite = new File(hadoopConfDir, "hdfs-site.xml");
+    OutputStream os = new FileOutputStream(hdfsSite);
+    conf.writeXml(os);
+    os.close();
+
+    //HTTPFS configuration
+    conf = new Configuration(false);
+    conf.set("httpfs.services.ext", MockGroups.class.getName());
+    conf.set("httpfs.admin.group", HadoopUsersConfTestHelper.
+      getHadoopUserGroups(HadoopUsersConfTestHelper.getHadoopUsers()[0])[0]);
+    conf.set("httpfs.proxyuser." + HadoopUsersConfTestHelper.getHadoopProxyUser() + ".groups",
+             HadoopUsersConfTestHelper.getHadoopProxyUserGroups());
+    conf.set("httpfs.proxyuser." + HadoopUsersConfTestHelper.getHadoopProxyUser() + ".hosts",
+             HadoopUsersConfTestHelper.getHadoopProxyUserHosts());
     conf.set("httpfs.authentication.signature.secret.file", secretFile.getAbsolutePath());
-    File hoopSite = new File(new File(homeDir, "conf"), "httpfs-site.xml");
-    OutputStream os = new FileOutputStream(hoopSite);
+    File httpfsSite = new File(new File(homeDir, "conf"), "httpfs-site.xml");
+    os = new FileOutputStream(httpfsSite);
     conf.writeXml(os);
     os.close();
 
@@ -103,7 +160,8 @@ public class TestHttpFSServer extends HF
     Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_UNAUTHORIZED);
 
     url = new URL(TestJettyHelper.getJettyURL(),
-                  MessageFormat.format("/webhdfs/v1?user.name={0}&op=instrumentation", "root"));
+                  MessageFormat.format("/webhdfs/v1?user.name={0}&op=instrumentation",
+                                       HadoopUsersConfTestHelper.getHadoopUsers()[0]));
     conn = (HttpURLConnection) url.openConnection();
     Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK);
     BufferedReader reader = new BufferedReader(new InputStreamReader(conn.getInputStream()));
@@ -112,7 +170,8 @@ public class TestHttpFSServer extends HF
     Assert.assertTrue(line.contains("\"counters\":{"));
 
     url = new URL(TestJettyHelper.getJettyURL(),
-                  MessageFormat.format("/webhdfs/v1/foo?user.name={0}&op=instrumentation", "root"));
+                  MessageFormat.format("/webhdfs/v1/foo?user.name={0}&op=instrumentation",
+                                       HadoopUsersConfTestHelper.getHadoopUsers()[0]));
     conn = (HttpURLConnection) url.openConnection();
     Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_BAD_REQUEST);
   }

Modified: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/hadoop/TestFileSystemAccessService.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/hadoop/TestFileSystemAccessService.java?rev=1327724&r1=1327723&r2=1327724&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/hadoop/TestFileSystemAccessService.java (original)
+++ hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/hadoop/TestFileSystemAccessService.java Wed Apr 18 23:35:30 2012
@@ -20,6 +20,7 @@ package org.apache.hadoop.lib.service.ha
 
 import junit.framework.Assert;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.lib.server.Server;
@@ -34,13 +35,32 @@ import org.apache.hadoop.test.TestExcept
 import org.apache.hadoop.test.TestHdfs;
 import org.apache.hadoop.test.TestHdfsHelper;
 import org.apache.hadoop.util.StringUtils;
+import org.junit.Before;
 import org.junit.Test;
 
+import java.io.File;
+import java.io.FileOutputStream;
 import java.io.IOException;
+import java.io.OutputStream;
 import java.util.Arrays;
 
 public class TestFileSystemAccessService extends HFSTestCase {
 
+  private void createHadoopConf(Configuration hadoopConf) throws Exception {
+    String dir = TestDirHelper.getTestDir().getAbsolutePath();
+    File hdfsSite = new File(dir, "hdfs-site.xml");
+    OutputStream os = new FileOutputStream(hdfsSite);
+    hadoopConf.writeXml(os);
+    os.close();
+  }
+
+  @Before
+  public void createHadoopConf() throws Exception {
+    Configuration hadoopConf = new Configuration(false);
+    hadoopConf.set("foo", "FOO");
+    createHadoopConf(hadoopConf);
+  }
+
   @Test
   @TestDir
   public void simpleSecurity() throws Exception {
@@ -124,7 +144,7 @@ public class TestFileSystemAccessService
                                                           FileSystemAccessService.class.getName()));
     Configuration conf = new Configuration(false);
     conf.set("server.services", services);
-    conf.set("server.hadoop.conf:foo", "FOO");
+
     Server server = new Server("server", dir, dir, dir, dir, conf);
     server.init();
     FileSystemAccessService fsAccess = (FileSystemAccessService) server.get(FileSystemAccess.class);
@@ -134,6 +154,32 @@ public class TestFileSystemAccessService
 
   @Test
   @TestDir
+  public void serviceHadoopConfCustomDir() throws Exception {
+    String dir = TestDirHelper.getTestDir().getAbsolutePath();
+    String hadoopConfDir = new File(dir, "confx").getAbsolutePath();
+    new File(hadoopConfDir).mkdirs();
+    String services = StringUtils.join(",", Arrays.asList(InstrumentationService.class.getName(),
+                                                          FileSystemAccessService.class.getName()));
+    Configuration conf = new Configuration(false);
+    conf.set("server.services", services);
+    conf.set("server.hadoop.config.dir", hadoopConfDir);
+
+    File hdfsSite = new File(hadoopConfDir, "hdfs-site.xml");
+    OutputStream os = new FileOutputStream(hdfsSite);
+    Configuration hadoopConf = new Configuration(false);
+    hadoopConf.set("foo", "BAR");
+    hadoopConf.writeXml(os);
+    os.close();
+
+    Server server = new Server("server", dir, dir, dir, dir, conf);
+    server.init();
+    FileSystemAccessService fsAccess = (FileSystemAccessService) server.get(FileSystemAccess.class);
+    Assert.assertEquals(fsAccess.serviceHadoopConf.get("foo"), "BAR");
+    server.destroy();
+  }
+
+  @Test
+  @TestDir
   public void inWhitelists() throws Exception {
     String dir = TestDirHelper.getTestDir().getAbsolutePath();
     String services = StringUtils.join(",", Arrays.asList(InstrumentationService.class.getName(),
@@ -188,12 +234,17 @@ public class TestFileSystemAccessService
     String dir = TestDirHelper.getTestDir().getAbsolutePath();
     String services = StringUtils.join(",", Arrays.asList(InstrumentationService.class.getName(),
                                                           FileSystemAccessService.class.getName()));
+
+    Configuration hadoopConf = new Configuration(false);
+    hadoopConf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, TestHdfsHelper.getHdfsConf().get(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY));
+    createHadoopConf(hadoopConf);
+
     Configuration conf = new Configuration(false);
     conf.set("server.services", services);
     Server server = new Server("server", dir, dir, dir, dir, conf);
     server.init();
     FileSystemAccess hadoop = server.get(FileSystemAccess.class);
-    FileSystem fs = hadoop.createFileSystem("u", TestHdfsHelper.getHdfsConf());
+    FileSystem fs = hadoop.createFileSystem("u", hadoop.getFileSystemConfiguration());
     Assert.assertNotNull(fs);
     fs.mkdirs(new Path("/tmp/foo"));
     hadoop.releaseFileSystem(fs);
@@ -214,6 +265,11 @@ public class TestFileSystemAccessService
     String dir = TestDirHelper.getTestDir().getAbsolutePath();
     String services = StringUtils.join(",", Arrays.asList(InstrumentationService.class.getName(),
                                                           FileSystemAccessService.class.getName()));
+
+    Configuration hadoopConf = new Configuration(false);
+    hadoopConf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, TestHdfsHelper.getHdfsConf().get(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY));
+    createHadoopConf(hadoopConf);
+
     Configuration conf = new Configuration(false);
     conf.set("server.services", services);
     Server server = new Server("server", dir, dir, dir, dir, conf);
@@ -222,7 +278,7 @@ public class TestFileSystemAccessService
 
     final FileSystem fsa[] = new FileSystem[1];
 
-    hadoop.execute("u", TestHdfsHelper.getHdfsConf(), new FileSystemAccess.FileSystemExecutor<Void>() {
+    hadoop.execute("u", hadoop.getFileSystemConfiguration(), new FileSystemAccess.FileSystemExecutor<Void>() {
       @Override
       public Void execute(FileSystem fs) throws IOException {
         fs.mkdirs(new Path("/tmp/foo"));
@@ -248,14 +304,18 @@ public class TestFileSystemAccessService
     String dir = TestDirHelper.getTestDir().getAbsolutePath();
     String services = StringUtils.join(",", Arrays.asList(InstrumentationService.class.getName(),
                                                           FileSystemAccessService.class.getName()));
+    Configuration hadoopConf = new Configuration(false);
+    hadoopConf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, TestHdfsHelper.getHdfsConf().get(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY));
+    createHadoopConf(hadoopConf);
+
     Configuration conf = new Configuration(false);
     conf.set("server.services", services);
     Server server = new Server("server", dir, dir, dir, dir, conf);
     server.init();
     FileSystemAccess fsAccess = server.get(FileSystemAccess.class);
 
-    Configuration hdfsConf = TestHdfsHelper.getHdfsConf();
-    hdfsConf.set("fs.default.name", "");
+    Configuration hdfsConf = fsAccess.getFileSystemConfiguration();
+    hdfsConf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, "");
     fsAccess.execute("u", hdfsConf, new FileSystemAccess.FileSystemExecutor<Void>() {
       @Override
       public Void execute(FileSystem fs) throws IOException {
@@ -271,6 +331,11 @@ public class TestFileSystemAccessService
     String dir = TestDirHelper.getTestDir().getAbsolutePath();
     String services = StringUtils.join(",", Arrays.asList(InstrumentationService.class.getName(),
                                                           FileSystemAccessService.class.getName()));
+
+    Configuration hadoopConf = new Configuration(false);
+    hadoopConf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, TestHdfsHelper.getHdfsConf().get(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY));
+    createHadoopConf(hadoopConf);
+
     Configuration conf = new Configuration(false);
     conf.set("server.services", services);
     Server server = new Server("server", dir, dir, dir, dir, conf);
@@ -279,7 +344,7 @@ public class TestFileSystemAccessService
 
     final FileSystem fsa[] = new FileSystem[1];
     try {
-      hadoop.execute("u", TestHdfsHelper.getHdfsConf(), new FileSystemAccess.FileSystemExecutor<Void>() {
+      hadoop.execute("u", hadoop.getFileSystemConfiguration(), new FileSystemAccess.FileSystemExecutor<Void>() {
         @Override
         public Void execute(FileSystem fs) throws IOException {
           fsa[0] = fs;

Modified: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/HadoopUsersConfTestHelper.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/HadoopUsersConfTestHelper.java?rev=1327724&r1=1327723&r2=1327724&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/HadoopUsersConfTestHelper.java (original)
+++ hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/HadoopUsersConfTestHelper.java Wed Apr 18 23:35:30 2012
@@ -145,7 +145,12 @@ public class HadoopUsersConfTestHelper {
    */
   public static String[] getHadoopUserGroups(String user) {
     if (getHadoopUsers() == DEFAULT_USERS) {
-      return DEFAULT_USERS_GROUP;
+      for (String defaultUser : DEFAULT_USERS) {
+        if (defaultUser.equals(user)) {
+          return DEFAULT_USERS_GROUP;
+        }
+      }
+      return new String[0];
     } else {
       String groups = System.getProperty(HADOOP_USER_PREFIX + user);
       return (groups != null) ? groups.split(",") : new String[0];

Modified: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt?rev=1327724&r1=1327723&r2=1327724&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt (original)
+++ hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt Wed Apr 18 23:35:30 2012
@@ -62,6 +62,12 @@ Trunk (unreleased changes)
     HDFS-3178. Add states and state handler for journal synchronization in
     JournalService.  (szetszwo)
 
+    HDFS-3273. Refactor BackupImage and FSEditLog, and rename
+    JournalListener.rollLogs(..) to startLogSegment(..).  (szetszwo)
+
+    HDFS-3292. Remove the deprecated DiskStatus, getDiskStatus(), getRawCapacity() and
+    getRawUsed() from DistributedFileSystem.  (Arpit Gupta via szetszwo)
+
   OPTIMIZATIONS
 
     HDFS-2834. Add a ByteBuffer-based read API to DFSInputStream.
@@ -91,8 +97,6 @@ Trunk (unreleased changes)
     HDFS-2373. Commands using WebHDFS and hftp print unnecessary debug 
     info on the console with security enabled. (Arpit Gupta via suresh)
 
-    HDFS-2765. TestNameEditsConfigs is incorrectly swallowing IOE. (atm)
-
     HDFS-2776. Missing interface annotation on JournalSet. 
     (Brandon Li via jitendra)
 
@@ -367,6 +371,25 @@ Release 2.0.0 - UNRELEASED 
 
     HDFS-3249. Use ToolRunner.confirmPrompt in NameNode (todd)
 
+    HDFS-3179.  Improve the exception message thrown by DataStreamer when 
+    it failed to add a datanode.  (szetszwo)
+
+    HDFS-2983. Relax the build version check to permit rolling upgrades within
+    a release. (atm)
+
+    HDFS-3259. NameNode#initializeSharedEdits should populate shared edits dir
+    with edit log segments. (atm)
+
+    HDFS-2708. Stats for the # of blocks per DN. (atm)
+
+    HDFS-3279. Move the FSEditLog constructor with @VisibleForTesting to
+    TestEditLog.  (Arpit Gupta via szetszwo)
+
+    HDFS-3294. Fix code indentation in NamenodeWebHdfsMethods and
+    DatanodeWebHdfsMethods.  (szetszwo)
+
+    HDFS-3263. HttpFS should read HDFS config from Hadoop site.xml files (tucu)
+
   OPTIMIZATIONS
 
     HDFS-3024. Improve performance of stringification in addStoredBlock (todd)
@@ -492,6 +515,26 @@ Release 2.0.0 - UNRELEASED 
 
     HDFS-2696. Fix the fuse-fds build. (Bruno Mahé via eli)
 
+    HDFS-3260. TestDatanodeRegistration should set minimum DN version in
+    addition to minimum NN version. (atm)
+
+    HDFS-3255. HA DFS returns wrong token service (Daryn Sharp via todd)
+
+    HDFS-3256. HDFS considers blocks under-replicated if topology script is
+    configured with only 1 rack. (atm)
+
+    HDFS-2799. Trim fs.checkpoint.dir values. (Amith D K via eli)
+
+    HDFS-2765. TestNameEditsConfigs is incorrectly swallowing IOE. (atm)
+
+    HDFS-3268. FileContext API mishandles token service and incompatible with
+    HA (Daryn Sharp via todd)
+
+    HDFS-3284. bootstrapStandby fails in secure cluster (todd)
+
+    HDFS-3165. HDFS Balancer scripts are refering to wrong path of
+    hadoop-daemon.sh (Amith D K via eli)
+
   BREAKDOWN OF HDFS-1623 SUBTASKS
 
     HDFS-2179. Add fencing framework and mechanisms for NameNode HA. (todd)
@@ -825,6 +868,9 @@ Release 0.23.3 - UNRELEASED
     HDFS-3176. Use MD5MD5CRC32FileChecksum.readFields() in JsonUtil .  (Kihwal
     Lee via szetszwo)
 
+    HDFS-2652. Add support for host-based delegation tokens.  (Daryn Sharp via
+    szetszwo)
+
 Release 0.23.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

Propchange: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/
------------------------------------------------------------------------------
--- svn:ignore (original)
+++ svn:ignore Wed Apr 18 23:35:30 2012
@@ -1 +1,4 @@
 target
+.classpath
+.project
+.settings

Propchange: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/contrib/fuse-dfs/
------------------------------------------------------------------------------
--- svn:ignore (added)
+++ svn:ignore Wed Apr 18 23:35:30 2012
@@ -0,0 +1,4 @@
+target
+.classpath
+.project
+.settings

Modified: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/start-balancer.sh
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/start-balancer.sh?rev=1327724&r1=1327723&r2=1327724&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/start-balancer.sh (original)
+++ hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/start-balancer.sh Wed Apr 18 23:35:30 2012
@@ -24,4 +24,4 @@ HADOOP_LIBEXEC_DIR=${HADOOP_LIBEXEC_DIR:
 
 # Start balancer daemon.
 
-"$HADOOP_PREFIX"/bin/hadoop-daemon.sh --config $HADOOP_CONF_DIR --script "$bin"/hdfs start balancer $@
+"$HADOOP_PREFIX"/sbin/hadoop-daemon.sh --config $HADOOP_CONF_DIR --script "$bin"/hdfs start balancer $@

Modified: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/stop-balancer.sh
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/stop-balancer.sh?rev=1327724&r1=1327723&r2=1327724&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/stop-balancer.sh (original)
+++ hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/stop-balancer.sh Wed Apr 18 23:35:30 2012
@@ -25,4 +25,4 @@ HADOOP_LIBEXEC_DIR=${HADOOP_LIBEXEC_DIR:
 # Stop balancer daemon.
 # Run this on the machine where the balancer is running
 
-"$HADOOP_PREFIX"/bin/hadoop-daemon.sh --config $HADOOP_CONF_DIR --script "$bin"/hdfs stop balancer
+"$HADOOP_PREFIX"/sbin/hadoop-daemon.sh --config $HADOOP_CONF_DIR --script "$bin"/hdfs stop balancer

Propchange: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java:r1324567-1327718

Modified: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/Hdfs.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/Hdfs.java?rev=1327724&r1=1327723&r2=1327724&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/Hdfs.java (original)
+++ hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/Hdfs.java Wed Apr 18 23:35:30 2012
@@ -20,7 +20,6 @@ package org.apache.hadoop.fs;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.ArrayList;
@@ -391,11 +390,15 @@ public class Hdfs extends AbstractFileSy
     return new Path(dfs.getLinkTarget(getUriPath(p)));
   }
   
+  @Override
+  public String getCanonicalServiceName() {
+    return dfs.getCanonicalServiceName();
+  }
+
   @Override //AbstractFileSystem
   public List<Token<?>> getDelegationTokens(String renewer) throws IOException {
     Token<DelegationTokenIdentifier> result = dfs
         .getDelegationToken(renewer == null ? null : new Text(renewer));
-    result.setService(new Text(this.getCanonicalServiceName()));
     List<Token<?>> tokenList = new ArrayList<Token<?>>();
     tokenList.add(result);
     return tokenList;

Modified: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java?rev=1327724&r1=1327723&r2=1327724&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java (original)
+++ hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java Wed Apr 18 23:35:30 2012
@@ -638,6 +638,16 @@ public class DFSClient implements java.i
   }
   
   /**
+   * Get a canonical token service name for this client's tokens.  Null should
+   * be returned if the client is not using tokens.
+   * @return the token service for the client
+   */
+  @InterfaceAudience.LimitedPrivate( { "HDFS" }) 
+  public String getCanonicalServiceName() {
+    return (dtService != null) ? dtService.toString() : null;
+  }
+  
+  /**
    * @see ClientProtocol#getDelegationToken(Text)
    */
   public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)

Modified: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java?rev=1327724&r1=1327723&r2=1327724&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java (original)
+++ hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java Wed Apr 18 23:35:30 2012
@@ -146,6 +146,8 @@ public class DFSConfigKeys extends Commo
   public static final int     DFS_NAMENODE_NUM_CHECKPOINTS_RETAINED_DEFAULT = 2;
   public static final String  DFS_NAMENODE_NUM_EXTRA_EDITS_RETAINED_KEY = "dfs.namenode.num.extra.edits.retained";
   public static final int     DFS_NAMENODE_NUM_EXTRA_EDITS_RETAINED_DEFAULT = 1000000; //1M
+  public static final String  DFS_NAMENODE_MIN_SUPPORTED_DATANODE_VERSION_KEY = "dfs.namenode.min.supported.datanode.version";
+  public static final String  DFS_NAMENODE_MIN_SUPPORTED_DATANODE_VERSION_DEFAULT = "3.0.0";
 
   public static final String  DFS_NAMENODE_EDITS_DIR_MINIMUM_KEY = "dfs.namenode.edits.dir.minimum";
   public static final int     DFS_NAMENODE_EDITS_DIR_MINIMUM_DEFAULT = 1;
@@ -262,6 +264,8 @@ public class DFSConfigKeys extends Commo
   public static final String  DFS_DATANODE_IPC_ADDRESS_KEY = "dfs.datanode.ipc.address";
   public static final int     DFS_DATANODE_IPC_DEFAULT_PORT = 50020;
   public static final String  DFS_DATANODE_IPC_ADDRESS_DEFAULT = "0.0.0.0" + DFS_DATANODE_IPC_DEFAULT_PORT;
+  public static final String  DFS_DATANODE_MIN_SUPPORTED_NAMENODE_VERSION_KEY = "dfs.datanode.min.supported.namenode.version";
+  public static final String  DFS_DATANODE_MIN_SUPPORTED_NAMENODE_VERSION_DEFAULT = "3.0.0";
 
   public static final String  DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY = "dfs.block.access.token.enable";
   public static final boolean DFS_BLOCK_ACCESS_TOKEN_ENABLE_DEFAULT = false;

Modified: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java?rev=1327724&r1=1327723&r2=1327724&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java (original)
+++ hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java Wed Apr 18 23:35:30 2012
@@ -128,7 +128,7 @@ class DFSOutputStream extends FSOutputSu
   private volatile boolean appendChunk = false;   // appending to existing partial block
   private long initialFileSize = 0; // at time of file open
   private Progressable progress;
-  private short blockReplication; // replication factor of file
+  private final short blockReplication; // replication factor of file
   
   private class Packet {
     long    seqno;               // sequencenumber of buffer in block
@@ -775,9 +775,13 @@ class DFSOutputStream extends FSOutputSu
     private int findNewDatanode(final DatanodeInfo[] original
         ) throws IOException {
       if (nodes.length != original.length + 1) {
-        throw new IOException("Failed to add a datanode:"
-            + " nodes.length != original.length + 1, nodes="
-            + Arrays.asList(nodes) + ", original=" + Arrays.asList(original));
+        throw new IOException("Failed to add a datanode.  "
+            + "User may turn off this feature by setting "
+            + DFSConfigKeys.DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_POLICY_KEY
+            + " in configuration, where the current policy is "
+            + dfsClient.dtpReplaceDatanodeOnFailure
+            + ".  (Nodes: current=" + Arrays.asList(nodes)
+            + ", original=" + Arrays.asList(original) + ")");
       }
       for(int i = 0; i < nodes.length; i++) {
         int j = 0;

Modified: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java?rev=1327724&r1=1327723&r2=1327724&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java (original)
+++ hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java Wed Apr 18 23:35:30 2012
@@ -499,56 +499,12 @@ public class DistributedFileSystem exten
     return dfs;
   }        
   
-  /** @deprecated Use {@link org.apache.hadoop.fs.FsStatus} instead */
-  @InterfaceAudience.Private
-  @Deprecated
-  public static class DiskStatus extends FsStatus {
-    public DiskStatus(FsStatus stats) {
-      super(stats.getCapacity(), stats.getUsed(), stats.getRemaining());
-    }
-
-    public DiskStatus(long capacity, long dfsUsed, long remaining) {
-      super(capacity, dfsUsed, remaining);
-    }
-
-    public long getDfsUsed() {
-      return super.getUsed();
-    }
-  }
-  
   @Override
   public FsStatus getStatus(Path p) throws IOException {
     statistics.incrementReadOps(1);
     return dfs.getDiskStatus();
   }
 
-  /** Return the disk usage of the filesystem, including total capacity,
-   * used space, and remaining space 
-   * @deprecated Use {@link org.apache.hadoop.fs.FileSystem#getStatus()} 
-   * instead */
-   @Deprecated
-  public DiskStatus getDiskStatus() throws IOException {
-    return new DiskStatus(dfs.getDiskStatus());
-  }
-  
-  /** Return the total raw capacity of the filesystem, disregarding
-   * replication.
-   * @deprecated Use {@link org.apache.hadoop.fs.FileSystem#getStatus()} 
-   * instead */
-   @Deprecated
-  public long getRawCapacity() throws IOException{
-    return dfs.getDiskStatus().getCapacity();
-  }
-
-  /** Return the total raw used space in the filesystem, disregarding
-   * replication.
-   * @deprecated Use {@link org.apache.hadoop.fs.FileSystem#getStatus()} 
-   * instead */
-   @Deprecated
-  public long getRawUsed() throws IOException{
-    return dfs.getDiskStatus().getUsed();
-  }
-   
   /**
    * Returns count of blocks with no good replicas left. Normally should be
    * zero.
@@ -848,11 +804,7 @@ public class DistributedFileSystem exten
    */
   @Override
   public String getCanonicalServiceName() {
-    if (HAUtil.isLogicalUri(getConf(), getUri())) {
-      return getUri().getHost();
-    } else {
-      return super.getCanonicalServiceName();
-    }
+    return dfs.getCanonicalServiceName();
   }
 
   /**

Modified: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HAUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HAUtil.java?rev=1327724&r1=1327723&r2=1327724&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HAUtil.java (original)
+++ hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HAUtil.java Wed Apr 18 23:35:30 2012
@@ -52,6 +52,9 @@ public class HAUtil {
   private static final Log LOG = 
     LogFactory.getLog(HAUtil.class);
   
+  private static final DelegationTokenSelector tokenSelector =
+      new DelegationTokenSelector();
+
   private HAUtil() { /* Hidden constructor */ }
 
   /**
@@ -241,25 +244,28 @@ public class HAUtil {
    * one is found, clone it to also represent the underlying namenode address.
    * @param ugi the UGI to modify
    * @param haUri the logical URI for the cluster
-   * @param singleNNAddr one of the NNs in the cluster to which the token
+   * @param nnAddrs collection of NNs in the cluster to which the token
    * applies
    */
   public static void cloneDelegationTokenForLogicalUri(
       UserGroupInformation ugi, URI haUri,
-      InetSocketAddress singleNNAddr) {
-    Text haService = buildTokenServiceForLogicalUri(haUri);
+      Collection<InetSocketAddress> nnAddrs) {
+    Text haService = HAUtil.buildTokenServiceForLogicalUri(haUri);
     Token<DelegationTokenIdentifier> haToken =
-        DelegationTokenSelector.selectHdfsDelegationToken(haService, ugi);
-    if (haToken == null) {
-      // no token
-      return;
-    }
-    Token<DelegationTokenIdentifier> specificToken =
-        new Token<DelegationTokenIdentifier>(haToken);
-    specificToken.setService(SecurityUtil.buildTokenService(singleNNAddr));
-    ugi.addToken(specificToken);
-    LOG.debug("Mapped HA service delegation token for logical URI " +
-        haUri + " to namenode " + singleNNAddr);
+        tokenSelector.selectToken(haService, ugi.getTokens());
+    if (haToken != null) {
+      for (InetSocketAddress singleNNAddr : nnAddrs) {
+        Token<DelegationTokenIdentifier> specificToken =
+            new Token<DelegationTokenIdentifier>(haToken);
+        SecurityUtil.setTokenService(specificToken, singleNNAddr);
+        ugi.addToken(specificToken);
+        LOG.debug("Mapped HA service delegation token for logical URI " +
+            haUri + " to namenode " + singleNNAddr);
+      }
+    } else {
+      LOG.debug("No HA service delegation token found for logical URI " +
+          haUri);
+    }
   }
 
   /**

Modified: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java?rev=1327724&r1=1327723&r2=1327724&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java (original)
+++ hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java Wed Apr 18 23:35:30 2012
@@ -30,6 +30,7 @@ import java.security.PrivilegedException
 import java.text.ParseException;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.TimeZone;
 
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -48,7 +49,6 @@ import org.apache.hadoop.hdfs.security.t
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenRenewer;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.tools.DelegationTokenFetcher;
 import org.apache.hadoop.hdfs.web.URLUtils;
 import org.apache.hadoop.io.Text;
@@ -168,10 +168,7 @@ public class HftpFileSystem extends File
 
   protected void initDelegationToken() throws IOException {
     // look for hftp token, then try hdfs
-    Token<?> token = selectHftpDelegationToken();
-    if (token == null) {
-      token = selectHdfsDelegationToken();
-    }  
+    Token<?> token = selectDelegationToken();
 
     // if we don't already have a token, go get one over https
     boolean createdToken = false;
@@ -192,14 +189,8 @@ public class HftpFileSystem extends File
     }
   }
 
-  protected Token<DelegationTokenIdentifier> selectHftpDelegationToken() {
-    Text serviceName = SecurityUtil.buildTokenService(nnSecureAddr);
-    return hftpTokenSelector.selectToken(serviceName, ugi.getTokens());
-  }
-
-  protected Token<DelegationTokenIdentifier> selectHdfsDelegationToken() {
-    return  DelegationTokenSelector.selectHdfsDelegationToken(
-        nnAddr, ugi, getConf());
+  protected Token<DelegationTokenIdentifier> selectDelegationToken() {
+  	return hftpTokenSelector.selectToken(getUri(), ugi.getTokens(), getConf());
   }
   
 
@@ -699,9 +690,22 @@ public class HftpFileSystem extends File
   
   private static class HftpDelegationTokenSelector
   extends AbstractDelegationTokenSelector<DelegationTokenIdentifier> {
+    private static final DelegationTokenSelector hdfsTokenSelector =
+        new DelegationTokenSelector();
 
     public HftpDelegationTokenSelector() {
       super(TOKEN_KIND);
     }
+    
+    Token<DelegationTokenIdentifier> selectToken(URI nnUri,
+        Collection<Token<?>> tokens, Configuration conf) {
+      Token<DelegationTokenIdentifier> token =
+          selectToken(SecurityUtil.buildTokenService(nnUri), tokens);
+      if (token == null) {
+        // try to get a HDFS token
+        token = hdfsTokenSelector.selectToken(nnUri, tokens, conf); 
+      }
+      return token;
+    }
   }
 }

Modified: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java?rev=1327724&r1=1327723&r2=1327724&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java (original)
+++ hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java Wed Apr 18 23:35:30 2012
@@ -386,7 +386,7 @@ public class PBHelper {
     StorageInfoProto storage = info.getStorageInfo();
     return new NamespaceInfo(storage.getNamespceID(), storage.getClusterID(),
         info.getBlockPoolID(), storage.getCTime(), info.getDistUpgradeVersion(),
-        info.getBuildVersion());
+        info.getBuildVersion(), info.getSoftwareVersion());
   }
 
   public static NamenodeCommand convert(NamenodeCommandProto cmd) {
@@ -612,13 +612,14 @@ public class PBHelper {
         .newBuilder();
     return builder.setDatanodeID(PBHelper.convert((DatanodeID) registration))
         .setStorageInfo(PBHelper.convert(registration.getStorageInfo()))
-        .setKeys(PBHelper.convert(registration.getExportedKeys())).build();
+        .setKeys(PBHelper.convert(registration.getExportedKeys()))
+        .setSoftwareVersion(registration.getSoftwareVersion()).build();
   }
 
   public static DatanodeRegistration convert(DatanodeRegistrationProto proto) {
     return new DatanodeRegistration(PBHelper.convert(proto.getDatanodeID()),
         PBHelper.convert(proto.getStorageInfo()), PBHelper.convert(proto
-            .getKeys()));
+            .getKeys()), proto.getSoftwareVersion());
   }
 
   public static DatanodeCommand convert(DatanodeCommandProto proto) {
@@ -894,7 +895,8 @@ public class PBHelper {
         .setBlockPoolID(info.getBlockPoolID())
         .setBuildVersion(info.getBuildVersion())
         .setDistUpgradeVersion(info.getDistributedUpgradeVersion())
-        .setStorageInfo(PBHelper.convert((StorageInfo)info)).build();
+        .setStorageInfo(PBHelper.convert((StorageInfo)info))
+        .setSoftwareVersion(info.getSoftwareVersion()).build();
   }
   
   // Located Block Arrays and Lists

Modified: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSelector.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSelector.java?rev=1327724&r1=1327723&r2=1327724&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSelector.java (original)
+++ hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSelector.java Wed Apr 18 23:35:30 2012
@@ -17,7 +17,8 @@
  */
 package org.apache.hadoop.hdfs.security.token.delegation;
 
-import java.net.InetSocketAddress;
+import java.net.URI;
+import java.util.Collection;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
@@ -25,7 +26,6 @@ import org.apache.hadoop.hdfs.server.nam
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.SecurityUtil;
-import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSelector;
 
@@ -37,32 +37,35 @@ public class DelegationTokenSelector
     extends AbstractDelegationTokenSelector<DelegationTokenIdentifier>{
   public static final String SERVICE_NAME_KEY = "hdfs.service.host_";
 
-  private static final DelegationTokenSelector INSTANCE = new DelegationTokenSelector();
-
-  /** Select the delegation token for hdfs from the ugi. */
-  public static Token<DelegationTokenIdentifier> selectHdfsDelegationToken(
-      final InetSocketAddress nnAddr, final UserGroupInformation ugi,
+  /**
+   * Select the delegation token for hdfs.  The port will be rewritten to
+   * the port of hdfs.service.host_$nnAddr, or the default rpc namenode port. 
+   * This method should only be called by non-hdfs filesystems that do not
+   * use the rpc port to acquire tokens.  Ex. webhdfs, hftp 
+   * @param nnUri of the remote namenode
+   * @param tokens as a collection
+   * @param conf hadoop configuration
+   * @return Token
+   */
+  public Token<DelegationTokenIdentifier> selectToken(
+      final URI nnUri, Collection<Token<?>> tokens,
       final Configuration conf) {
     // this guesses the remote cluster's rpc service port.
     // the current token design assumes it's the same as the local cluster's
     // rpc port unless a config key is set.  there should be a way to automatic
     // and correctly determine the value
-    final String key = SERVICE_NAME_KEY + SecurityUtil.buildTokenService(nnAddr);
-    final String nnServiceName = conf.get(key);
+    Text serviceName = SecurityUtil.buildTokenService(nnUri);
+    final String nnServiceName = conf.get(SERVICE_NAME_KEY + serviceName);
     
     int nnRpcPort = NameNode.DEFAULT_PORT;
     if (nnServiceName != null) {
       nnRpcPort = NetUtils.createSocketAddr(nnServiceName, nnRpcPort).getPort(); 
     }
+    // use original hostname from the uri to avoid unintentional host resolving
+    serviceName = SecurityUtil.buildTokenService(
+    		NetUtils.createSocketAddrForHost(nnUri.getHost(), nnRpcPort));
     
-    final Text serviceName = SecurityUtil.buildTokenService(
-        new InetSocketAddress(nnAddr.getHostName(), nnRpcPort));
-    return INSTANCE.selectToken(serviceName, ugi.getTokens());
-  }
-  
-  public static Token<DelegationTokenIdentifier> selectHdfsDelegationToken(
-      Text serviceName, UserGroupInformation ugi) {
-    return INSTANCE.selectToken(serviceName, ugi.getTokens());
+    return selectToken(serviceName, tokens);
   }
 
   public DelegationTokenSelector() {

Modified: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java?rev=1327724&r1=1327723&r2=1327724&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java (original)
+++ hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java Wed Apr 18 23:35:30 2012
@@ -247,8 +247,7 @@ public class BlockManager {
 
     this.maxReplicationStreams = conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY,
                                              DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_DEFAULT);
-    this.shouldCheckForEnoughRacks = conf.get(DFSConfigKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY) == null ? false
-                                                                             : true;
+    this.shouldCheckForEnoughRacks = conf.get(DFSConfigKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY) != null;
     
     this.replicationRecheckInterval = 
       conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 
@@ -2829,7 +2828,9 @@ assert storedBlock.findDatanode(dn) < 0 
       DatanodeDescriptor cur = it.next();
       if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) {
         if ((corruptNodes == null ) || !corruptNodes.contains(cur)) {
-          if (numExpectedReplicas == 1) {
+          if (numExpectedReplicas == 1 ||
+              (numExpectedReplicas > 1 &&
+                  !datanodeManager.hasClusterEverBeenMultiRack())) {
             enoughRacks = true;
             break;
           }

Modified: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java?rev=1327724&r1=1327723&r2=1327724&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java (original)
+++ hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java Wed Apr 18 23:35:30 2012
@@ -71,6 +71,7 @@ import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.HostsFileReader;
 import org.apache.hadoop.util.ReflectionUtils;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.net.InetAddresses;
 
 /**
@@ -126,6 +127,12 @@ public class DatanodeManager {
   /** Ask Datanode only up to this many blocks to delete. */
   final int blockInvalidateLimit;
   
+  /**
+   * Whether or not this cluster has ever consisted of more than 1 rack,
+   * according to the NetworkTopology.
+   */
+  private boolean hasClusterEverBeenMultiRack = false;
+  
   DatanodeManager(final BlockManager blockManager,
       final Namesystem namesystem, final Configuration conf
       ) throws IOException {
@@ -331,6 +338,7 @@ public class DatanodeManager {
 
     host2DatanodeMap.add(node);
     networktopology.add(node);
+    checkIfClusterIsNowMultiRack(node);
 
     if (LOG.isDebugEnabled()) {
       LOG.debug(getClass().getSimpleName() + ".addDatanode: "
@@ -769,6 +777,42 @@ public class DatanodeManager {
   }
 
   /**
+   * @return true if this cluster has ever consisted of multiple racks, even if
+   *         it is not now a multi-rack cluster.
+   */
+  boolean hasClusterEverBeenMultiRack() {
+    return hasClusterEverBeenMultiRack;
+  }
+
+  /**
+   * Check if the cluster now consists of multiple racks. If it does, and this
+   * is the first time it's consisted of multiple racks, then process blocks
+   * that may now be misreplicated.
+   * 
+   * @param node DN which caused cluster to become multi-rack. Used for logging.
+   */
+  @VisibleForTesting
+  void checkIfClusterIsNowMultiRack(DatanodeDescriptor node) {
+    if (!hasClusterEverBeenMultiRack && networktopology.getNumOfRacks() > 1) {
+      String message = "DN " + node + " joining cluster has expanded a formerly " +
+          "single-rack cluster to be multi-rack. ";
+      if (namesystem.isPopulatingReplQueues()) {
+        message += "Re-checking all blocks for replication, since they should " +
+            "now be replicated cross-rack";
+        LOG.info(message);
+      } else {
+        message += "Not checking for mis-replicated blocks because this NN is " +
+            "not yet processing repl queues.";
+        LOG.debug(message);
+      }
+      hasClusterEverBeenMultiRack = true;
+      if (namesystem.isPopulatingReplQueues()) {
+        blockManager.processMisReplicatedBlocks();
+      }
+    }
+  }
+
+  /**
    * Parse a DatanodeID from a hosts file entry
    * @param hostLine of form [hostname|ip][:port]?
    * @return DatanodeID constructed from the given string

Modified: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/IncorrectVersionException.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/IncorrectVersionException.java?rev=1327724&r1=1327723&r2=1327724&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/IncorrectVersionException.java (original)
+++ hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/IncorrectVersionException.java Wed Apr 18 23:35:30 2012
@@ -32,7 +32,19 @@ import org.apache.hadoop.hdfs.protocol.H
 @InterfaceStability.Evolving
 public class IncorrectVersionException extends IOException {
   private static final long serialVersionUID = 1L;
+  
+  public IncorrectVersionException(String message) {
+    super(message);
+  }
 
+  public IncorrectVersionException(String minimumVersion, String reportedVersion,
+      String remoteDaemon, String thisDaemon) {
+    this("The reported " + remoteDaemon + " version is too low to communicate" +
+        " with this " + thisDaemon + ". " + remoteDaemon + " version: '" +
+        reportedVersion + "' Minimum " + remoteDaemon + " version: '" +
+        minimumVersion + "'");
+  }
+  
   public IncorrectVersionException(int versionReported, String ofWhat) {
     this(versionReported, ofWhat, HdfsConstants.LAYOUT_VERSION);
   }
@@ -40,16 +52,9 @@ public class IncorrectVersionException e
   public IncorrectVersionException(int versionReported,
                                    String ofWhat,
                                    int versionExpected) {
-    super("Unexpected version " 
-          + (ofWhat==null ? "" : "of " + ofWhat) + ". Reported: "
-          + versionReported + ". Expecting = " + versionExpected + ".");
+    this("Unexpected version " 
+        + (ofWhat==null ? "" : "of " + ofWhat) + ". Reported: "
+        + versionReported + ". Expecting = " + versionExpected + ".");
   }
 
-  public IncorrectVersionException(String versionReported,
-                                   String ofWhat,
-                                   String versionExpected) {
-    super("Unexpected version " 
-          + (ofWhat==null ? "" : "of " + ofWhat) + ". Reported: "
-          + versionReported + ". Expecting = " + versionExpected + ".");
-  }
 }