Explorar o código

HADOOP-8704. add request logging to jetty/httpserver (jeagles)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1520676 13f79535-47bb-0310-9956-ffa450edef68
Jonathan Turner Eagles %!s(int64=11) %!d(string=hai) anos
pai
achega
0dd1c596f3

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

@@ -1856,6 +1856,8 @@ Release 0.23.10 - UNRELEASED
     HADOOP-9686. Easy access to final parameters in Configuration (Jason Lowe
     via jeagles)
 
+    HADOOP-8704. add request logging to jetty/httpserver (jeagles)
+
   OPTIMIZATIONS
 
   BUG FIXES

+ 25 - 0
hadoop-common-project/hadoop-common/src/main/conf/log4j.properties

@@ -240,3 +240,28 @@ log4j.appender.RMSUMMARY.layout.ConversionPattern=%d{ISO8601} %p %c{2}: %m%n
 #log4j.appender.HSAUDIT.layout.ConversionPattern=%d{ISO8601} %p %c{2}: %m%n
 #log4j.appender.HSAUDIT.DatePattern=.yyyy-MM-dd
 
+# Http Server Request Logs
+#log4j.logger.http.requests.namenode=INFO,namenoderequestlog
+#log4j.appender.namenoderequestlog=org.apache.hadoop.http.HttpRequestLogAppender
+#log4j.appender.namenoderequestlog.Filename=${hadoop.log.dir}/jetty-namenode-yyyy_mm_dd.log
+#log4j.appender.namenoderequestlog.RetainDays=3
+
+#log4j.logger.http.requests.datanode=INFO,datanoderequestlog
+#log4j.appender.datanoderequestlog=org.apache.hadoop.http.HttpRequestLogAppender
+#log4j.appender.datanoderequestlog.Filename=${hadoop.log.dir}/jetty-datanode-yyyy_mm_dd.log
+#log4j.appender.datanoderequestlog.RetainDays=3
+
+#log4j.logger.http.requests.resourcemanager=INFO,resourcemanagerrequestlog
+#log4j.appender.resourcemanagerrequestlog=org.apache.hadoop.http.HttpRequestLogAppender
+#log4j.appender.resourcemanagerrequestlog.Filename=${hadoop.log.dir}/jetty-resourcemanager-yyyy_mm_dd.log
+#log4j.appender.resourcemanagerrequestlog.RetainDays=3
+
+#log4j.logger.http.requests.jobhistory=INFO,jobhistoryrequestlog
+#log4j.appender.jobhistoryrequestlog=org.apache.hadoop.http.HttpRequestLogAppender
+#log4j.appender.jobhistoryrequestlog.Filename=${hadoop.log.dir}/jetty-jobhistory-yyyy_mm_dd.log
+#log4j.appender.jobhistoryrequestlog.RetainDays=3
+
+#log4j.logger.http.requests.nodemanager=INFO,nodemanagerrequestlog
+#log4j.appender.nodemanagerrequestlog=org.apache.hadoop.http.HttpRequestLogAppender
+#log4j.appender.nodemanagerrequestlog.Filename=${hadoop.log.dir}/jetty-nodemanager-yyyy_mm_dd.log
+#log4j.appender.nodemanagerrequestlog.RetainDays=3

+ 94 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpRequestLog.java

@@ -0,0 +1,94 @@
+/**
+ * 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.http;
+
+import java.util.HashMap;
+
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogConfigurationException;
+import org.apache.commons.logging.LogFactory;
+import org.apache.log4j.Appender;
+import org.apache.log4j.Logger;
+import org.mortbay.jetty.NCSARequestLog;
+import org.mortbay.jetty.RequestLog;
+
+/**
+ * RequestLog object for use with Http
+ */
+public class HttpRequestLog {
+
+  public static final Log LOG = LogFactory.getLog(HttpRequestLog.class);
+  private static final HashMap<String, String> serverToComponent;
+
+  static {
+    serverToComponent = new HashMap<String, String>();
+    serverToComponent.put("cluster", "resourcemanager");
+    serverToComponent.put("hdfs", "namenode");
+    serverToComponent.put("node", "nodemanager");
+  }
+
+  public static RequestLog getRequestLog(String name) {
+
+    String lookup = serverToComponent.get(name);
+    if (lookup != null) {
+      name = lookup;
+    }
+    String loggerName = "http.requests." + name;
+    String appenderName = name + "requestlog";
+    Log logger = LogFactory.getLog(loggerName);
+
+    if (logger instanceof Log4JLogger) {
+      Log4JLogger httpLog4JLog = (Log4JLogger)logger;
+      Logger httpLogger = httpLog4JLog.getLogger();
+      Appender appender = null;
+
+      try {
+        appender = httpLogger.getAppender(appenderName);
+      } catch (LogConfigurationException e) {
+        LOG.warn("Http request log for " + loggerName
+            + " could not be created");
+        throw e;
+      }
+
+      if (appender == null) {
+        LOG.info("Http request log for " + loggerName
+            + " is not defined");
+        return null;
+      }
+
+      if (appender instanceof HttpRequestLogAppender) {
+        HttpRequestLogAppender requestLogAppender
+          = (HttpRequestLogAppender)appender;
+        NCSARequestLog requestLog = new NCSARequestLog();
+        requestLog.setFilename(requestLogAppender.getFilename());
+        requestLog.setRetainDays(requestLogAppender.getRetainDays());
+        return requestLog;
+      }
+      else {
+        LOG.warn("Jetty request log for " + loggerName
+            + " was of the wrong class");
+        return null;
+      }
+    }
+    else {
+      LOG.warn("Jetty request log can only be enabled using Log4j");
+      return null;
+    }
+  }
+}

+ 62 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpRequestLogAppender.java

@@ -0,0 +1,62 @@
+/**
+ * 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.http;
+
+import org.apache.log4j.spi.LoggingEvent;
+import org.apache.log4j.AppenderSkeleton;
+
+/**
+ * Log4j Appender adapter for HttpRequestLog
+ */
+public class HttpRequestLogAppender extends AppenderSkeleton {
+
+  private String filename;
+  private int retainDays;
+
+  public HttpRequestLogAppender() {
+  }
+
+  public void setRetainDays(int retainDays) {
+    this.retainDays = retainDays;
+  }
+
+  public int getRetainDays() {
+    return retainDays;
+  }
+
+  public void setFilename(String filename) {
+    this.filename = filename;
+  }
+
+  public String getFilename() {
+    return filename;
+  }
+
+  @Override
+  public void append(LoggingEvent event) {
+  }
+
+  @Override
+  public void close() {
+  }
+
+  @Override
+  public boolean requiresLayout() {
+    return false;
+  }
+}

+ 15 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java

@@ -66,9 +66,12 @@ import org.mortbay.io.Buffer;
 import org.mortbay.jetty.Connector;
 import org.mortbay.jetty.Handler;
 import org.mortbay.jetty.MimeTypes;
+import org.mortbay.jetty.RequestLog;
 import org.mortbay.jetty.Server;
 import org.mortbay.jetty.handler.ContextHandler;
 import org.mortbay.jetty.handler.ContextHandlerCollection;
+import org.mortbay.jetty.handler.RequestLogHandler;
+import org.mortbay.jetty.handler.HandlerCollection;
 import org.mortbay.jetty.nio.SelectChannelConnector;
 import org.mortbay.jetty.security.SslSocketConnector;
 import org.mortbay.jetty.servlet.Context;
@@ -354,7 +357,18 @@ public class HttpServer implements FilterContainer {
 
     final String appDir = getWebAppsPath(name);
     ContextHandlerCollection contexts = new ContextHandlerCollection();
-    webServer.setHandler(contexts);
+    RequestLog requestLog = HttpRequestLog.getRequestLog(name);
+
+    if (requestLog != null) {
+      RequestLogHandler requestLogHandler = new RequestLogHandler();
+      requestLogHandler.setRequestLog(requestLog);
+      HandlerCollection handlers = new HandlerCollection();
+      handlers.setHandlers(new Handler[] {requestLogHandler, contexts});
+      webServer.setHandler(handlers);
+    }
+    else {
+      webServer.setHandler(contexts);
+    }
 
     webAppContext = new WebAppContext();
     webAppContext.setDisplayName(name);

+ 5 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/HttpServerFunctionalTest.java

@@ -101,8 +101,12 @@ public class HttpServerFunctionalTest extends Assert {
     String webapps = System.getProperty(TEST_BUILD_WEBAPPS, BUILD_WEBAPPS_DIR);
     File testWebappDir = new File(webapps +
         File.separatorChar + TEST);
+    try {
     if (!testWebappDir.exists()) {
-      fail("Test webapp dir " + testWebappDir + " missing");
+      fail("Test webapp dir " + testWebappDir.getCanonicalPath() + " missing");
+    }
+    }
+    catch (IOException e) {
     }
   }
 

+ 47 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpRequestLog.java

@@ -0,0 +1,47 @@
+/**
+ * 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.http;
+
+import org.apache.log4j.Logger;
+import org.junit.Test;
+import org.mortbay.jetty.NCSARequestLog;
+import org.mortbay.jetty.RequestLog;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+
+public class TestHttpRequestLog {
+
+  @Test
+  public void testAppenderUndefined() {
+    RequestLog requestLog = HttpRequestLog.getRequestLog("test");
+    assertNull("RequestLog should be null", requestLog);
+  }
+
+  @Test
+  public void testAppenderDefined() {
+    HttpRequestLogAppender requestLogAppender = new HttpRequestLogAppender();
+    requestLogAppender.setName("testrequestlog");
+    Logger.getLogger("http.requests.test").addAppender(requestLogAppender);
+    RequestLog requestLog = HttpRequestLog.getRequestLog("test");
+    Logger.getLogger("http.requests.test").removeAppender(requestLogAppender);
+    assertNotNull("RequestLog should not be null", requestLog);
+    assertEquals("Class mismatch", NCSARequestLog.class, requestLog.getClass());
+  }
+}

+ 37 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpRequestLogAppender.java

@@ -0,0 +1,37 @@
+/**
+ * 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.http;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestHttpRequestLogAppender {
+
+  @Test
+  public void testParameterPropagation() {
+
+    HttpRequestLogAppender requestLogAppender = new HttpRequestLogAppender();
+    requestLogAppender.setFilename("jetty-namenode-yyyy_mm_dd.log");
+    requestLogAppender.setRetainDays(17);
+    assertEquals("Filename mismatch", "jetty-namenode-yyyy_mm_dd.log",
+        requestLogAppender.getFilename());
+    assertEquals("Retain days mismatch", 17,
+        requestLogAppender.getRetainDays());
+  }
+}

+ 22 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServerLifecycle.java

@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.http;
 
+import org.apache.log4j.Logger;
 import org.junit.Test;
 
 public class TestHttpServerLifecycle extends HttpServerFunctionalTest {
@@ -66,6 +67,27 @@ public class TestHttpServerLifecycle extends HttpServerFunctionalTest {
     stop(server);
   }
 
+  /**
+   * Test that the server with request logging enabled
+   *
+   * @throws Throwable on failure
+   */
+  @Test
+  public void testStartedServerWithRequestLog() throws Throwable {
+    HttpRequestLogAppender requestLogAppender = new HttpRequestLogAppender();
+    requestLogAppender.setName("httprequestlog");
+    requestLogAppender.setFilename(System.getProperty("test.build.data", "/tmp/")
+        + "jetty-name-yyyy_mm_dd.log");
+    Logger.getLogger(HttpServer.class.getName() + ".test").addAppender(requestLogAppender);
+    HttpServer server = null;
+    server = createTestServer();
+    assertNotLive(server);
+    server.start();
+    assertAlive(server);
+    stop(server);
+    Logger.getLogger(HttpServer.class.getName() + ".test").removeAppender(requestLogAppender);
+  }
+
   /**
    * Assert that the result of {@link HttpServer#toString()} contains the specific text
    * @param server server to examine