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 co...@apache.org on 2011/10/21 04:27:16 UTC

svn commit: r1187168 - in /hadoop/common/branches/branch-0.22/hdfs: ./ src/java/org/apache/hadoop/hdfs/server/datanode/ src/test/aop/org/apache/hadoop/hdfs/server/datanode/

Author: cos
Date: Fri Oct 21 02:27:15 2011
New Revision: 1187168

URL: http://svn.apache.org/viewvc?rev=1187168&view=rev
Log:
HDFS-2452. OutOfMemoryError in DataXceiverServer takes down the DataNode. Contributed by Uma Maheswara Rao.

Added:
    hadoop/common/branches/branch-0.22/hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/DataXceiverAspects.aj
    hadoop/common/branches/branch-0.22/hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataXceiverServer.java
Modified:
    hadoop/common/branches/branch-0.22/hdfs/CHANGES.txt
    hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
    hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java

Modified: hadoop/common/branches/branch-0.22/hdfs/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.22/hdfs/CHANGES.txt?rev=1187168&r1=1187167&r2=1187168&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.22/hdfs/CHANGES.txt (original)
+++ hadoop/common/branches/branch-0.22/hdfs/CHANGES.txt Fri Oct 21 02:27:15 2011
@@ -650,6 +650,9 @@ Release 0.22.0 - Unreleased
 
     HDFS-2451. TestNodeCount.testNodeCount failes with NPE (cos)
 
+    HDFS-2452. OutOfMemoryError in DataXceiverServer takes down the DataNode
+    (Uma Maheswara Rao via cos)
+
 Release 0.21.1 - Unreleased
 
   IMPROVEMENTS

Modified: hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java?rev=1187168&r1=1187167&r2=1187168&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java (original)
+++ hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java Fri Oct 21 02:27:15 2011
@@ -83,7 +83,6 @@ class DataXceiver extends DataTransferPr
     this.isLocal = s.getInetAddress().equals(s.getLocalAddress());
     this.datanode = datanode;
     this.dataXceiverServer = dataXceiverServer;
-    dataXceiverServer.childSockets.put(s, s);
     remoteAddress = s.getRemoteSocketAddress().toString();
     localAddress = s.getLocalSocketAddress().toString();
 
@@ -118,6 +117,7 @@ class DataXceiver extends DataTransferPr
    * Read/write data from/to the DataXceiveServer.
    */
   public void run() {
+    dataXceiverServer.childSockets.put(s, s);
     updateCurrentThreadName("Waiting for operation");
 
     DataInputStream in=null; 

Modified: hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java?rev=1187168&r1=1187167&r2=1187168&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java (original)
+++ hadoop/common/branches/branch-0.22/hdfs/src/java/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java Fri Oct 21 02:27:15 2011
@@ -35,6 +35,7 @@ import org.apache.hadoop.hdfs.util.DataT
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.io.IOUtils;
 
 
 /**
@@ -132,8 +133,9 @@ class DataXceiverServer implements Runna
    */
   public void run() {
     while (datanode.shouldRun) {
+      Socket s = null;
       try {
-        Socket s = ss.accept();
+        s = ss.accept();
         s.setTcpNoDelay(true);
         new Daemon(datanode.threadGroup, 
             new DataXceiver(s, datanode, this)).start();
@@ -147,8 +149,19 @@ class DataXceiverServer implements Runna
           LOG.warn(datanode.dnRegistration + ":DataXceiverServer: ", ace);
         }
       } catch (IOException ie) {
+        IOUtils.closeSocket(s);
         LOG.warn(datanode.dnRegistration + ":DataXceiveServer: " 
                                 + StringUtils.stringifyException(ie));
+      } catch (OutOfMemoryError ie) {
+        IOUtils.closeSocket(s);
+        // DataNode can run out of memory if there is too many transfers.
+        // Log the event, Sleep for 30 seconds, other transfers may complete by then.
+        LOG.warn("DataNode is out of memory. Will retry in 30 seconds.", ie);
+        try {
+          Thread.sleep(30 * 1000);
+        } catch (InterruptedException e) {
+          // ignore
+        }
       } catch (Throwable te) {
         LOG.error(datanode.dnRegistration + ":DataXceiveServer: Exiting due to:" 
                                  + StringUtils.stringifyException(te));

Added: hadoop/common/branches/branch-0.22/hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/DataXceiverAspects.aj
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.22/hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/DataXceiverAspects.aj?rev=1187168&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.22/hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/DataXceiverAspects.aj (added)
+++ hadoop/common/branches/branch-0.22/hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/DataXceiverAspects.aj Fri Oct 21 02:27:15 2011
@@ -0,0 +1,41 @@
+/**
+ * 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 org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * This aspect takes care about faults injected into datanode.DataXceiver 
+ * class 
+ */
+privileged public aspect DataXceiverAspects {
+  public static final Log LOG = LogFactory.getLog(DataXceiverAspects.class);
+
+  pointcut runXceiverThread(DataXceiver xceiver) :
+    execution (* run(..)) && target(xceiver);
+
+  void around (DataXceiver xceiver) : runXceiverThread(xceiver) {
+    if ("true".equals(System.getProperty("fi.enabledOOM"))) {
+      LOG.info("fi.enabledOOM is enabled");
+      throw new OutOfMemoryError("Pretend there's no more memory");
+    } else {
+    	proceed(xceiver);
+    }
+  }
+}

Added: hadoop/common/branches/branch-0.22/hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataXceiverServer.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.22/hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataXceiverServer.java?rev=1187168&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.22/hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataXceiverServer.java (added)
+++ hadoop/common/branches/branch-0.22/hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataXceiverServer.java Fri Oct 21 02:27:15 2011
@@ -0,0 +1,91 @@
+/**
+ * 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 static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.ServerSocket;
+import java.net.Socket;
+import java.net.SocketAddress;
+import java.util.concurrent.CountDownLatch;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+/**
+ * This is a test for DataXceiverServer when DataXceiver thread spawning is
+ * failed due to OutOfMemoryError. Expected behavior is that DataXceiverServer
+ * should not be exited. It should retry again after 30 seconds
+ */
+public class TestFiDataXceiverServer {
+
+  @Test(timeout = 30000)
+  public void testOutOfMemoryErrorInDataXceiverServerRun() throws Exception {
+    final CountDownLatch latch = new CountDownLatch(1);
+    ServerSocket sock = new ServerSocket() {
+      @Override
+      public Socket accept() throws IOException {
+        return new Socket() {
+          @Override
+          public InetAddress getInetAddress() {
+            return super.getLocalAddress();
+          }
+
+          @Override
+          public SocketAddress getRemoteSocketAddress() {
+            return new InetSocketAddress(8080);
+          }
+
+          @Override
+          public SocketAddress getLocalSocketAddress() {
+            return new InetSocketAddress(0);
+          }
+
+          @Override
+          public synchronized void close() throws IOException {
+            latch.countDown();
+            super.close();
+          }
+        };
+      }
+    };
+    Thread thread = null;
+    System.setProperty("fi.enabledOOM", "true");
+    DataNode dn = Mockito.mock(DataNode.class);
+    try {
+      Configuration conf = new Configuration();
+      Mockito.doReturn(conf).when(dn).getConf();
+      dn.shouldRun = true;
+      DataXceiverServer server = new DataXceiverServer(sock, conf, dn);
+      thread = new Thread(server);
+      thread.start();
+      latch.await();
+      assertTrue("Not running the thread", thread.isAlive());
+    } finally {
+      System.setProperty("fi.enabledOOM", "false");
+      dn.shouldRun = false;
+      if (null != thread)
+        thread.interrupt();
+      sock.close();
+    }
+  }
+}