瀏覽代碼

HDFS-7734. Class cast exception in NameNode#main. Contributed by Yi Liu.

(cherry picked from commit 9175105eeaecf0a1d60b57989b73ce45cee4689b)
Andrew Wang 10 年之前
父節點
當前提交
696266e99c

+ 74 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LogAdapter.java

@@ -0,0 +1,74 @@
+/**
+ * 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.util;
+
+import org.apache.commons.logging.Log;
+import org.slf4j.Logger;
+
+class LogAdapter {
+  private Log LOG;
+  private Logger LOGGER;
+
+  private LogAdapter(Log LOG) {
+    this.LOG = LOG;
+  }
+
+  private LogAdapter(Logger LOGGER) {
+    this.LOGGER = LOGGER;
+  }
+
+  public static LogAdapter create(Log LOG) {
+    return new LogAdapter(LOG);
+  }
+
+  public static LogAdapter create(Logger LOGGER) {
+    return new LogAdapter(LOGGER);
+  }
+
+  public void info(String msg) {
+    if (LOG != null) {
+      LOG.info(msg);
+    } else if (LOGGER != null) {
+      LOGGER.info(msg);
+    }
+  }
+
+  public void warn(String msg, Throwable t) {
+    if (LOG != null) {
+      LOG.warn(msg, t);
+    } else if (LOGGER != null) {
+      LOGGER.warn(msg, t);
+    }
+  }
+
+  public void debug(Throwable t) {
+    if (LOG != null) {
+      LOG.debug(t);
+    } else if (LOGGER != null) {
+      LOGGER.debug("", t);
+    }
+  }
+
+  public void error(String msg) {
+    if (LOG != null) {
+      LOG.error(msg);
+    } else if (LOGGER != null) {
+      LOGGER.error(msg);
+    }
+  }
+}

+ 6 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SignalLogger.java

@@ -42,10 +42,10 @@ public enum SignalLogger {
    * Our signal handler.
    */
   private static class Handler implements SignalHandler {
-    final private org.apache.commons.logging.Log LOG;
+    final private LogAdapter LOG;
     final private SignalHandler prevHandler;
 
-    Handler(String name, Log LOG) {
+    Handler(String name, LogAdapter LOG) {
       this.LOG = LOG;
       prevHandler = Signal.handle(new Signal(name), this);
     }
@@ -69,6 +69,10 @@ public enum SignalLogger {
    * @param LOG        The log4j logfile to use in the signal handlers.
    */
   public void register(final Log LOG) {
+    register(LogAdapter.create(LOG));
+  }
+
+  void register(final LogAdapter LOG) {
     if (registered) {
       throw new IllegalStateException("Can't re-install the signal handlers.");
     }

+ 16 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java

@@ -628,6 +628,22 @@ public class StringUtils {
    */
   public static void startupShutdownMessage(Class<?> clazz, String[] args,
                                      final org.apache.commons.logging.Log LOG) {
+    startupShutdownMessage(clazz, args, LogAdapter.create(LOG));
+  }
+
+  /**
+   * Print a log message for starting up and shutting down
+   * @param clazz the class of the server
+   * @param args arguments
+   * @param LOG the target log object
+   */
+  public static void startupShutdownMessage(Class<?> clazz, String[] args,
+                                     final org.slf4j.Logger LOG) {
+    startupShutdownMessage(clazz, args, LogAdapter.create(LOG));
+  }
+
+  static void startupShutdownMessage(Class<?> clazz, String[] args,
+                                     final LogAdapter LOG) { 
     final String hostname = NetUtils.getHostname();
     final String classname = clazz.getSimpleName();
     LOG.info(

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

@@ -553,6 +553,8 @@ Release 2.7.0 - UNRELEASED
     HDFS-7707. Edit log corruption due to delayed block removal again.
     (Yongjun Zhang via kihwal)
 
+    HDFS-7734. Class cast exception in NameNode#main. (yliu via wang)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -37,7 +37,6 @@ import java.util.TreeMap;
 import java.util.TreeSet;
 import java.util.concurrent.atomic.AtomicLong;
 
-import org.apache.commons.logging.Log;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java

@@ -1549,8 +1549,7 @@ public class NameNode implements NameNodeStatusMXBean {
     }
 
     try {
-      StringUtils.startupShutdownMessage(NameNode.class, argv,
-          (org.apache.commons.logging.Log) LogManager.getLogger(LOG.getName()));
+      StringUtils.startupShutdownMessage(NameNode.class, argv, LOG);
       NameNode namenode = createNameNode(argv, null);
       if (namenode != null) {
         namenode.join();