Explorar o código

HADOOP-8711. Merge 1478426 and 1478429 from branch-1

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-1.2@1478432 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas %!s(int64=12) %!d(string=hai) anos
pai
achega
4f62721380

+ 3 - 0
CHANGES.txt

@@ -613,6 +613,9 @@ Release 1.2.0 - 2013.04.16
     HADOOP-9537. Backport changes to add support running Hadoop client on AIX.
     (Aaron T. Myers, backported by Arpit Agarwal via suresh)
 
+    HADOOP-8711. Provide an option for IPC server users to avoid printing stack
+    information for certain exceptions. (Brandon Li via suresh)
+
 Release 1.1.2 - 2013.01.30
 
   INCOMPATIBLE CHANGES

+ 53 - 2
src/core/org/apache/hadoop/ipc/Server.java

@@ -43,11 +43,13 @@ import java.nio.channels.WritableByteChannel;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
+import java.util.Set;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.LinkedBlockingQueue;
@@ -95,7 +97,44 @@ import org.apache.hadoop.util.StringUtils;
 public abstract class Server {
   private final boolean authorize;
   private boolean isSecurityEnabled;
-  
+  private ExceptionsHandler exceptionsHandler = new ExceptionsHandler();
+
+  public void addTerseExceptions(Class<?>... exceptionClass) {
+    exceptionsHandler.addTerseExceptions(exceptionClass);
+  }
+
+  /**
+   * ExceptionsHandler manages Exception groups for special handling e.g., terse
+   * exception group for concise logging messages
+   */
+  static class ExceptionsHandler {
+    private volatile Set<String> terseExceptions = new HashSet<String>();
+
+    /**
+     * Add exception class so server won't log its stack trace. Modifying the
+     * terseException through this method is thread safe.
+     * 
+     * @param exceptionClass
+     *          exception classes
+     */
+    void addTerseExceptions(Class<?>... exceptionClass) {
+
+      // Make a copy of terseException for performing modification
+      final HashSet<String> newSet = new HashSet<String>(terseExceptions);
+
+      // Add all class names into the HashSet
+      for (Class<?> name : exceptionClass) {
+        newSet.add(name.toString());
+      }
+      // Replace terseException set
+      terseExceptions = Collections.unmodifiableSet(newSet);
+    }
+
+    boolean isTerse(Class<?> t) {
+      return terseExceptions.contains(t.toString());
+    }
+  }
+
   /**
    * The first four bytes of Hadoop RPC connections
    */
@@ -1398,7 +1437,19 @@ public abstract class Server {
                   );
             }
           } catch (Throwable e) {
-            LOG.info(getName()+", call "+call+": error: " + e, e);
+            String logMsg = getName() + ", call " + call + ": error: " + e;
+            if (e instanceof RuntimeException || e instanceof Error) {
+              // These exception types indicate something is probably wrong
+              // on the server side, as opposed to just a normal exceptional
+              // result.
+              LOG.warn(logMsg, e);
+            } else if (exceptionsHandler.isTerse(e.getClass())) {
+              // Don't log the whole stack trace of these exceptions.
+              // Way too noisy!
+              LOG.info(logMsg);
+            } else {
+              LOG.info(logMsg, e);
+            }
             errorClass = e.getClass().getName();
             error = StringUtils.stringifyException(e);
           }

+ 42 - 0
src/test/org/apache/hadoop/ipc/TestServer.java

@@ -0,0 +1,42 @@
+/**
+ * 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.ipc;
+
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+
+import org.junit.Test;
+
+/**
+ * This is intended to be a set of unit tests for the 
+ * org.apache.hadoop.ipc.Server class.
+ */
+public class TestServer {
+
+  @Test
+  public void testExceptionsHandler() throws IOException {
+    Server.ExceptionsHandler handler = new Server.ExceptionsHandler();
+    handler.addTerseExceptions(IOException.class);
+    handler.addTerseExceptions(RemoteException.class);
+
+    assertTrue(handler.isTerse(IOException.class));
+    assertTrue(handler.isTerse(RemoteException.class));
+  }
+}