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 sh...@apache.org on 2011/10/23 22:28:31 UTC

svn commit: r1187969 - in /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs: ./ src/main/java/org/apache/hadoop/hdfs/server/datanode/ src/test/aop/org/apache/hadoop/hdfs/server/datanode/

Author: shv
Date: Sun Oct 23 20:28:30 2011
New Revision: 1187969

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

Added:
    hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/DataXceiverAspects.aj
    hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataXceiverServer.java   (with props)
Modified:
    hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
    hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
    hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java

Modified: hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt?rev=1187969&r1=1187968&r2=1187969&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt (original)
+++ hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt Sun Oct 23 20:28:30 2011
@@ -1634,6 +1634,9 @@ Release 0.22.0 - Unreleased
     HDFS-2290. Block with corrupt replica is not getting replicated.
     (Benoy Antony via shv)
 
+    HDFS-2452. OutOfMemoryError in DataXceiverServer takes down the DataNode
+    (Uma Maheswara Rao via cos)
+
 Release 0.21.1 - Unreleased
 
     HDFS-1466. TestFcHdfsSymlink relies on /tmp/test not existing. (eli)

Modified: hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java?rev=1187969&r1=1187968&r2=1187969&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java Sun Oct 23 20:28:30 2011
@@ -92,7 +92,6 @@ class DataXceiver extends Receiver imple
     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();
 
@@ -129,6 +128,7 @@ class DataXceiver extends Receiver imple
   public void run() {
     int opsProcessed = 0;
     Op op = null;
+    dataXceiverServer.childSockets.put(s, s);
     try {
       int stdTimeout = s.getSoTimeout();
 

Modified: hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java?rev=1187969&r1=1187968&r2=1187969&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java Sun Oct 23 20:28:30 2011
@@ -30,7 +30,6 @@ import java.util.Map;
 import org.apache.commons.logging.Log;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.balancer.Balancer;
 import org.apache.hadoop.hdfs.util.DataTransferThrottler;
 import org.apache.hadoop.io.IOUtils;
@@ -132,17 +131,12 @@ class DataXceiverServer implements Runna
   @Override
   public void run() {
     while (datanode.shouldRun) {
+      Socket s = null;
       try {
-        Socket s = ss.accept();
+        s = ss.accept();
         s.setTcpNoDelay(true);
-        final DataXceiver exciver;
-        try {
-          exciver = new DataXceiver(s, datanode, this);
-        } catch(IOException e) {
-          IOUtils.closeSocket(s);
-          throw e;
-        }
-        new Daemon(datanode.threadGroup, exciver).start();
+        new Daemon(datanode.threadGroup, new DataXceiver(s, datanode, this))
+            .start();
       } catch (SocketTimeoutException ignored) {
         // wake up to see if should continue to run
       } catch (AsynchronousCloseException ace) {
@@ -152,7 +146,19 @@ class DataXceiverServer implements Runna
           LOG.warn(datanode.getMachineName() + ":DataXceiverServer: ", ace);
         }
       } catch (IOException ie) {
+        IOUtils.closeSocket(s);
         LOG.warn(datanode.getMachineName() + ":DataXceiverServer: ", 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.getMachineName()
             + ":DataXceiverServer: Exiting due to: ", te);

Added: hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/DataXceiverAspects.aj
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/DataXceiverAspects.aj?rev=1187969&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/DataXceiverAspects.aj (added)
+++ hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/DataXceiverAspects.aj Sun Oct 23 20:28:30 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);
+    }
+  }
+}
\ No newline at end of file

Added: hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataXceiverServer.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataXceiverServer.java?rev=1187969&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataXceiverServer.java (added)
+++ hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataXceiverServer.java Sun Oct 23 20:28:30 2011
@@ -0,0 +1,97 @@
+/**
+ * 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.io.InputStream;
+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();
+          }
+          
+          @Override
+          public InputStream getInputStream() throws IOException {
+            return null;
+          }
+        };
+      }
+    };
+    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();
+    }
+  }
+}
\ No newline at end of file

Propchange: hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataXceiverServer.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain