Ver código fonte

HDFS-5192. NameNode may fail to start when dfs.client.test.drop.namenode.response.number is set. Contributed by Jing Zhao.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1522775 13f79535-47bb-0310-9956-ffa450edef68
Jing Zhao 11 anos atrás
pai
commit
965ce2041a

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -439,6 +439,9 @@ Release 2.1.1-beta - UNRELEASED
     HDFS-5159. Secondary NameNode fails to checkpoint if error occurs
     downloading edits on first checkpoint. (atm)
 
+    HDFS-5192. NameNode may fail to start when 
+    dfs.client.test.drop.namenode.response.number is set. (jing9)
+
 Release 2.1.0-beta - 2013-08-22
 
   INCOMPATIBLE CHANGES

+ 8 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -484,14 +484,17 @@ public class DFSClient implements java.io.Closeable {
     int numResponseToDrop = conf.getInt(
         DFSConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY,
         DFSConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_DEFAULT);
+    NameNodeProxies.ProxyAndInfo<ClientProtocol> proxyInfo = null;
     if (numResponseToDrop > 0) {
       // This case is used for testing.
       LOG.warn(DFSConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY
           + " is set to " + numResponseToDrop
           + ", this hacked client will proactively drop responses");
-      NameNodeProxies.ProxyAndInfo<ClientProtocol> proxyInfo = NameNodeProxies
-          .createProxyWithLossyRetryHandler(conf, nameNodeUri,
-              ClientProtocol.class, numResponseToDrop);
+      proxyInfo = NameNodeProxies.createProxyWithLossyRetryHandler(conf,
+          nameNodeUri, ClientProtocol.class, numResponseToDrop);
+    }
+    
+    if (proxyInfo != null) {
       this.dtService = proxyInfo.getDelegationTokenService();
       this.namenode = proxyInfo.getProxy();
     } else if (rpcNamenode != null) {
@@ -502,9 +505,8 @@ public class DFSClient implements java.io.Closeable {
     } else {
       Preconditions.checkArgument(nameNodeUri != null,
           "null URI");
-      NameNodeProxies.ProxyAndInfo<ClientProtocol> proxyInfo =
-        NameNodeProxies.createProxy(conf, nameNodeUri, ClientProtocol.class);
-      
+      proxyInfo = NameNodeProxies.createProxy(conf, nameNodeUri,
+          ClientProtocol.class);
       this.dtService = proxyInfo.getDelegationTokenService();
       this.namenode = proxyInfo.getProxy();
     }

+ 6 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java

@@ -158,8 +158,8 @@ public class NameNodeProxies {
    * Generate a dummy namenode proxy instance that utilizes our hacked
    * {@link LossyRetryInvocationHandler}. Proxy instance generated using this
    * method will proactively drop RPC responses. Currently this method only
-   * support HA setup. IllegalStateException will be thrown if the given
-   * configuration is not for HA.
+   * support HA setup. null will be returned if the given configuration is not 
+   * for HA.
    * 
    * @param config the configuration containing the required IPC
    *        properties, client failover configurations, etc.
@@ -168,7 +168,8 @@ public class NameNodeProxies {
    * @param xface the IPC interface which should be created
    * @param numResponseToDrop The number of responses to drop for each RPC call
    * @return an object containing both the proxy and the associated
-   *         delegation token service it corresponds to
+   *         delegation token service it corresponds to. Will return null of the
+   *         given configuration does not support HA.
    * @throws IOException if there is an error creating the proxy
    */
   @SuppressWarnings("unchecked")
@@ -204,8 +205,9 @@ public class NameNodeProxies {
       Text dtService = HAUtil.buildTokenServiceForLogicalUri(nameNodeUri);
       return new ProxyAndInfo<T>(proxy, dtService);
     } else {
-      throw new IllegalStateException("Currently creating proxy using " +
+      LOG.warn("Currently creating proxy using " +
       		"LossyRetryInvocationHandler requires NN HA setup");
+      return null;
     }
   }
 

+ 57 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestLossyRetryInvocationHandler.java

@@ -0,0 +1,57 @@
+/**
+ * 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.ha;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.junit.Test;
+
+/**
+ * This test makes sure that when
+ * {@link DFSConfigKeys#DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY} is set,
+ * DFSClient instances can still be created within NN/DN (e.g., the fs instance
+ * used by the trash emptier thread in NN)
+ */
+public class TestLossyRetryInvocationHandler {
+  
+  @Test
+  public void testStartNNWithTrashEmptier() throws Exception {
+    MiniDFSCluster cluster = null;
+    Configuration conf = new HdfsConfiguration();
+    
+    // enable both trash emptier and dropping response
+    conf.setLong("fs.trash.interval", 360);
+    conf.setInt(DFSConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY, 2);
+    
+    try {
+      cluster = new MiniDFSCluster.Builder(conf)
+          .nnTopology(MiniDFSNNTopology.simpleHATopology()).numDataNodes(0)
+          .build();
+      cluster.waitActive();
+      cluster.transitionToActive(0);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+  
+}