Ver código fonte

HADOOP-4679. Datanode prints tons of log messages because of a shutdown deadlock. Contributed by Hairong Kuang.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/trunk@723013 13f79535-47bb-0310-9956-ffa450edef68
Hairong Kuang 16 anos atrás
pai
commit
00189265b3

+ 3 - 0
CHANGES.txt

@@ -1297,6 +1297,9 @@ Release 0.18.3 - Unreleased
     HADOOP-4726. Fix documentation typos "the the". (Edward J. Yoon via
     szetszwo)
 
+    HADOOP-4679. Datanode prints tons of log messages: waiting for threadgroup
+    to exit, active threads is XX. (hairong)
+
 Release 0.18.2 - 2008-11-03
 
   BUG FIXES

+ 5 - 1
src/hdfs/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java

@@ -23,7 +23,6 @@ import java.io.DataOutputStream;
 import java.io.EOFException;
 import java.io.IOException;
 import java.io.OutputStream;
-import java.net.InetAddress;
 import java.nio.ByteBuffer;
 import java.util.LinkedList;
 import java.util.zip.CRC32;
@@ -110,6 +109,11 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
       }
     } catch(IOException ioe) {
       IOUtils.closeStream(this);
+      IOException cause = FSDataset.getCauseIfDiskError(ioe);
+      if (cause != null) { // possible disk error
+        ioe = cause;
+        datanode.checkDiskError(ioe);
+      }
       throw ioe;
     }
   }

+ 13 - 3
src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -297,6 +297,7 @@ public class DataNode extends Configured
           ServerSocketChannel.open().socket() : new ServerSocket();
     Server.bind(ss, socAddr, 0);
     ss.setReceiveBufferSize(DEFAULT_DATA_SOCKET_SIZE); 
+    ss.setSoTimeout(conf.getInt("dfs.dataXceiver.timeoutInMS", 30000)); //30s
     // adjust machine name with the actual port
     tmpPort = ss.getLocalPort();
     selfAddr = new InetSocketAddress(ss.getInetAddress().getHostAddress(),
@@ -538,6 +539,8 @@ public class DataNode extends Configured
   /**
    * Shut down this instance of the datanode.
    * Returns only after shutdown is complete.
+   * This method can only be called by the offerService thread.
+   * Otherwise, deadlock might occur.
    */
   public void shutdown() {
     if (infoServer != null) {
@@ -605,7 +608,8 @@ public class DataNode extends Configured
    *  when IOException occurs. 
    * If so, handle the error */
   protected void checkDiskError( IOException e ) throws IOException {
-    if (e.getMessage().startsWith("No space left on device")) {
+    if (e.getMessage() != null && 
+        e.getMessage().startsWith("No space left on device")) {
       throw new DiskOutOfSpaceException("No space left on device");
     } else {
       checkDiskError();
@@ -623,12 +627,12 @@ public class DataNode extends Configured
   
   private void handleDiskError(String errMsgr) {
     LOG.warn("DataNode is shutting down.\n" + errMsgr);
+    shouldRun = false;
     try {
       namenode.errorReport(
                            dnRegistration, DatanodeProtocol.DISK_ERROR, errMsgr);
     } catch(IOException ignored) {              
     }
-    shutdown();
   }
     
   /** Number of concurrent xceivers per node. */
@@ -818,7 +822,9 @@ public class DataNode extends Configured
     case DatanodeProtocol.DNA_REGISTER:
       // namenode requested a registration - at start or if NN lost contact
       LOG.info("DatanodeCommand action: DNA_REGISTER");
-      register();
+      if (shouldRun) {
+        register();
+      }
       break;
     case DatanodeProtocol.DNA_FINALIZE:
       storage.finalizeUpgrade();
@@ -1140,6 +1146,10 @@ public class DataNode extends Configured
       dn.dataNodeThread.start();
     }
   }
+  
+  static boolean isDatanodeUp(DataNode dn) {
+    return dn.dataNodeThread != null && dn.dataNodeThread.isAlive();
+  }
 
   /** Instantiate a single datanode object. This must be run by invoking
    *  {@link DataNode#runDatanodeDaemon(DataNode)} subsequently. 

+ 3 - 0
src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.datanode;
 import java.io.IOException;
 import java.net.ServerSocket;
 import java.net.Socket;
+import java.net.SocketTimeoutException;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -130,6 +131,8 @@ class DataXceiverServer implements Runnable, FSConstants {
         s.setTcpNoDelay(true);
         new Daemon(datanode.threadGroup, 
             new DataXceiver(s, datanode, this)).start();
+      } catch (SocketTimeoutException ignored) {
+        // wake up to see if should continue to run
       } catch (IOException ie) {
         LOG.warn(datanode.dnRegistration + ":DataXceiveServer: " 
                                 + StringUtils.stringifyException(ie));

+ 21 - 1
src/hdfs/org/apache/hadoop/hdfs/server/datanode/FSDataset.java

@@ -395,7 +395,13 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
       }
       // Create the zero-length temp file
       //
-      if (!f.createNewFile()) {
+      boolean fileCreated = false;
+      try {
+        fileCreated = f.createNewFile();
+      } catch (IOException ioe) {
+        throw (IOException)new IOException(DISK_ERROR +f).initCause(ioe);
+      }
+      if (!fileCreated) {
         throw new IOException("Unexpected problem in creating temporary file for "+
                               b + ".  File " + f + " should be creatable, but is already present.");
       }
@@ -942,6 +948,20 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
     }
   }
 
+  private final static String DISK_ERROR = "Possible disk error on file creation: ";
+  /** Get the cause of an I/O exception if caused by a possible disk error
+   * @param ioe an I/O exception
+   * @return cause if the I/O exception is caused by a possible disk error;
+   *         null otherwise.
+   */ 
+  static IOException getCauseIfDiskError(IOException ioe) {
+    if (ioe.getMessage()!=null && ioe.getMessage().startsWith(DISK_ERROR)) {
+      return (IOException)ioe.getCause();
+    } else {
+      return null;
+    }
+  }
+
   /**
    * Start writing to a block file
    * If isRecovery is true and the block pre-exists, then we kill all

+ 63 - 0
src/test/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java

@@ -0,0 +1,63 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode;
+
+import java.io.File;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+
+import junit.framework.TestCase;
+
+/** Test if a datanode can handle disk error correctly*/
+public class TestDiskError extends TestCase {
+  public void testShutdown() throws Exception {
+    // bring up a cluster of 3
+    Configuration conf = new Configuration();
+    conf.setLong("dfs.block.size", 512L);
+    conf.setInt("dfs.dataXceiver.timeoutInMS", 1000);
+    MiniDFSCluster cluster = new MiniDFSCluster(conf, 3, true, null);
+    cluster.waitActive();
+    FileSystem fs = cluster.getFileSystem();
+    final int dnIndex = 0;
+    String dataDir = cluster.getDataDirectory();
+    File dir1 = new File(new File(dataDir, "data"+(2*dnIndex+1)), "tmp");
+    File dir2 = new File(new File(dataDir, "data"+(2*dnIndex+2)), "tmp");
+    try {
+      // make the data directory of the first datanode to be readonly
+      assertTrue(dir1.setReadOnly());
+      assertTrue(dir2.setReadOnly());
+
+      // create files and make sure that first datanode will be down
+      DataNode dn = cluster.getDataNodes().get(dnIndex);
+      for (int i=0; DataNode.isDatanodeUp(dn); i++) {
+        Path fileName = new Path("/test.txt"+i);
+        DFSTestUtil.createFile(fs, fileName, 1024, (short)2, 1L);
+        DFSTestUtil.waitReplication(fs, fileName, (short)2);
+      }
+    } finally {
+      // restore its old permission
+      dir1.setWritable(true);
+      dir2.setWritable(true);
+      cluster.shutdown();
+    }
+  }
+}