Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -949,6 +949,15 @@ public class CommonConfigurationKeysPublic {
/** Defalt value for HADOOP_HTTP_LOGS_ENABLED */
public static final boolean HADOOP_HTTP_LOGS_ENABLED_DEFAULT = true;

/**
* @see
* <a href="{@docRoot}/../hadoop-project-dist/hadoop-common/core-default.xml">
* core-default.xml</a>
*/
public static final String HADOOP_HTTP_METRICS_ENABLED =
"hadoop.http.metrics.enabled";
public static final boolean HADOOP_HTTP_METRICS_ENABLED_DEFAULT = true;

/**
* @see
* <a href="{@docRoot}/../hadoop-project-dist/hadoop-common/core-default.xml">
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,7 @@
import javax.servlet.http.HttpServletRequestWrapper;
import javax.servlet.http.HttpServletResponse;

import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableMap;
import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
Expand Down Expand Up @@ -93,6 +94,7 @@
import org.eclipse.jetty.server.handler.ContextHandlerCollection;
import org.eclipse.jetty.server.handler.HandlerCollection;
import org.eclipse.jetty.server.handler.RequestLogHandler;
import org.eclipse.jetty.server.handler.StatisticsHandler;
import org.eclipse.jetty.server.session.SessionHandler;
import org.eclipse.jetty.servlet.FilterHolder;
import org.eclipse.jetty.servlet.FilterMapping;
Expand Down Expand Up @@ -201,6 +203,9 @@ public final class HttpServer2 implements FilterContainer {
protected static final String PROMETHEUS_SINK = "PROMETHEUS_SINK";
private PrometheusMetricsSink prometheusMetricsSink;

private StatisticsHandler statsHandler;
private HttpServer2Metrics metrics;

/**
* Class to construct instances of HTTP server with specific options.
*/
Expand Down Expand Up @@ -669,6 +674,27 @@ private void initializeWebServer(String name, String hostName,
addDefaultApps(contexts, appDir, conf);
webServer.setHandler(handlers);

if (conf.getBoolean(
CommonConfigurationKeysPublic.HADOOP_HTTP_METRICS_ENABLED,
CommonConfigurationKeysPublic.HADOOP_HTTP_METRICS_ENABLED_DEFAULT)) {
// Jetty StatisticsHandler must be inserted as the first handler.
// The tree might look like this:
//
// - StatisticsHandler (for all requests)
// - HandlerList
// - ContextHandlerCollection
// - RequestLogHandler (if enabled)
// - WebAppContext
// - SessionHandler
// - Servlets
// - Filters
// - etc..
//
// Reference: https://www.eclipse.org/lists/jetty-users/msg06273.html
statsHandler = new StatisticsHandler();
webServer.insertHandler(statsHandler);
}

Map<String, String> xFrameParams = setHeaders(conf);
addGlobalFilter("safety", QuotingInputFilter.class.getName(), xFrameParams);
final FilterInitializer[] initializers = getFilterInitializers(conf);
Expand Down Expand Up @@ -1227,6 +1253,16 @@ public void start() throws IOException {
.register("prometheus", "Hadoop metrics prometheus exporter",
prometheusMetricsSink);
}
if (statsHandler != null) {
// Create metrics source for each HttpServer2 instance.
// Use port number to make the metrics source name unique.
int port = -1;
for (ServerConnector connector : listeners) {
port = connector.getLocalPort();
break;
}
metrics = HttpServer2Metrics.create(statsHandler, port);
}
} catch (IOException ex) {
LOG.info("HttpServer.start() threw a non Bind IOException", ex);
throw ex;
Expand Down Expand Up @@ -1409,6 +1445,9 @@ public void stop() throws Exception {

try {
webServer.stop();
if (metrics != null) {
metrics.remove();
}
} catch (Exception e) {
LOG.error("Error while stopping web server for webapp "
+ webAppContext.getDisplayName(), e);
Expand Down Expand Up @@ -1789,4 +1828,10 @@ private Map<String, String> getDefaultHeaders() {
splitVal[1]);
return headers;
}

@VisibleForTesting
HttpServer2Metrics getMetrics() {
return metrics;
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,164 @@
/**
* 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.eclipse.jetty.server.handler.StatisticsHandler;

import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.metrics2.MetricsSystem;
import org.apache.hadoop.metrics2.annotation.Metric;
import org.apache.hadoop.metrics2.annotation.Metrics;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;

/**
* This class collects all the metrics of Jetty's StatisticsHandler
* and expose them as Hadoop Metrics.
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
@Metrics(name="HttpServer2", about="HttpServer2 metrics", context="http")
public class HttpServer2Metrics {

private final StatisticsHandler handler;
private final int port;

@Metric("number of requested that have been asynchronously dispatched")
public int asyncDispatches() {
return handler.getAsyncDispatches();
}
@Metric("total number of async requests")
public int asyncRequests() {
return handler.getAsyncRequests();
}
@Metric("currently waiting async requests")
public int asyncRequestsWaiting() {
return handler.getAsyncRequestsWaiting();
}
@Metric("maximum number of waiting async requests")
public int asyncRequestsWaitingMax() {
return handler.getAsyncRequestsWaitingMax();
}
@Metric("number of dispatches")
public int dispatched() {
return handler.getDispatched();
}
@Metric("number of dispatches currently active")
public int dispatchedActive() {
return handler.getDispatchedActive();
}
@Metric("maximum number of active dispatches being handled")
public int dispatchedActiveMax() {
return handler.getDispatchedActiveMax();
}
@Metric("maximum time spend in dispatch handling (in ms)")
public long dispatchedTimeMax() {
return handler.getDispatchedTimeMax();
}
@Metric("mean time spent in dispatch handling (in ms)")
public double dispatchedTimeMean() {
return handler.getDispatchedTimeMean();
}
@Metric("standard deviation for dispatch handling (in ms)")
public double dispatchedTimeStdDev() {
return handler.getDispatchedTimeStdDev();
}
@Metric("total time spent in dispatch handling (in ms)")
public long dispatchedTimeTotal() {
return handler.getDispatchedTimeTotal();
}
@Metric("number of async requests requests that have expired")
public int expires() {
return handler.getExpires();
}
@Metric("number of requests")
public int requests() {
return handler.getRequests();
}
@Metric("number of requests currently active")
public int requestsActive() {
return handler.getRequestsActive();
}
@Metric("maximum number of active requests")
public int requestsActiveMax() {
return handler.getRequestsActiveMax();
}
@Metric("maximum time spend handling requests (in ms)")
public long requestTimeMax() {
return handler.getRequestTimeMax();
}
@Metric("mean time spent handling requests (in ms)")
public double requestTimeMean() {
return handler.getRequestTimeMean();
}
@Metric("standard deviation for request handling (in ms)")
public double requestTimeStdDev() {
return handler.getRequestTimeStdDev();
}
@Metric("total time spend in all request handling (in ms)")
public long requestTimeTotal() {
return handler.getRequestTimeTotal();
}
@Metric("number of requests with 1xx response status")
public int responses1xx() {
return handler.getResponses1xx();
}
@Metric("number of requests with 2xx response status")
public int responses2xx() {
return handler.getResponses2xx();
}
@Metric("number of requests with 3xx response status")
public int responses3xx() {
return handler.getResponses3xx();
}
@Metric("number of requests with 4xx response status")
public int responses4xx() {
return handler.getResponses4xx();
}
@Metric("number of requests with 5xx response status")
public int responses5xx() {
return handler.getResponses5xx();
}
@Metric("total number of bytes across all responses")
public long responsesBytesTotal() {
return handler.getResponsesBytesTotal();
}
@Metric("time in milliseconds stats have been collected for")
public long statsOnMs() {
return handler.getStatsOnMs();
}

HttpServer2Metrics(StatisticsHandler handler, int port) {
this.handler = handler;
this.port = port;
}

static HttpServer2Metrics create(StatisticsHandler handler, int port) {
final MetricsSystem ms = DefaultMetricsSystem.instance();
final HttpServer2Metrics metrics = new HttpServer2Metrics(handler, port);
// Remove the old metrics from metrics system to avoid duplicate error
// when HttpServer2 is started twice.
metrics.remove();
// Add port number to the suffix to allow multiple instances in a host.
return ms.register("HttpServer2-" + port, "HttpServer2 metrics", metrics);
}

void remove() {
DefaultMetricsSystem.removeSourceName("HttpServer2-" + port);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -56,6 +56,15 @@
</description>
</property>

<property>
<name>hadoop.http.metrics.enabled</name>
<value>true</value>
<description>
If true, set Jetty's StatisticsHandler to HTTP server to collect
HTTP layer metrics and register them to Hadoop metrics system.
</description>
</property>

<!--- security properties -->

<property>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configuration.IntegerRanges;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.http.HttpServer2.QuotingInputFilter.RequestQuoter;
import org.apache.hadoop.http.resource.JerseyResource;
import org.apache.hadoop.net.NetUtils;
Expand All @@ -29,7 +30,10 @@
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authorize.AccessControlList;
import org.apache.hadoop.test.Whitebox;

import org.assertj.core.api.Assertions;
import org.eclipse.jetty.server.ServerConnector;
import org.eclipse.jetty.server.handler.StatisticsHandler;
import org.eclipse.jetty.util.ajax.JSON;
import org.junit.AfterClass;
import org.junit.Assert;
Expand Down Expand Up @@ -148,6 +152,8 @@ public void doGet(HttpServletRequest request,
@BeforeClass public static void setup() throws Exception {
Configuration conf = new Configuration();
conf.setInt(HttpServer2.HTTP_MAX_THREADS_KEY, MAX_THREADS);
conf.setBoolean(
CommonConfigurationKeysPublic.HADOOP_HTTP_METRICS_ENABLED, true);
server = createTestServer(conf);
server.addServlet("echo", "/echo", EchoServlet.class);
server.addServlet("echomap", "/echomap", EchoMapServlet.class);
Expand Down Expand Up @@ -272,6 +278,39 @@ public void testAcceptorSelectorConfigurability() throws Exception {
conn.getContentType());
}

@Test
public void testHttpServer2Metrics() throws Exception {
final HttpServer2Metrics metrics = server.getMetrics();
final int before = metrics.responses2xx();
final URL servletUrl = new URL(baseUrl, "/echo?echo");
final HttpURLConnection conn =
(HttpURLConnection)servletUrl.openConnection();
conn.connect();
Assertions.assertThat(conn.getResponseCode()).isEqualTo(200);
final int after = metrics.responses2xx();
Assertions.assertThat(after).isGreaterThan(before);
}

/**
* Jetty StatisticsHandler must be inserted via Server#insertHandler
* instead of Server#setHandler. The server fails to start if
* the handler is added by setHandler.
*/
@Test
public void testSetStatisticsHandler() throws Exception {
final Configuration conf = new Configuration();
// skip insert
conf.setBoolean(
CommonConfigurationKeysPublic.HADOOP_HTTP_METRICS_ENABLED, false);
final HttpServer2 testServer = createTestServer(conf);
testServer.webServer.setHandler(new StatisticsHandler());
try {
testServer.start();
fail("IOException should be thrown.");
} catch (IOException ignore) {
}
}

@Test
public void testHttpResonseContainsXFrameOptions() throws Exception {
validateXFrameOption(HttpServer2.XFrameOption.SAMEORIGIN);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.http.HttpServer2;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import org.apache.hadoop.security.AuthenticationFilterInitializer;
import org.apache.hadoop.security.authentication.server.ProxyUserAuthenticationFilterInitializer;
import org.apache.hadoop.security.authorize.AccessControlList;
Expand Down Expand Up @@ -150,6 +151,7 @@ private static void deprecateEnv(String varName, Configuration conf,
}

public void start() throws IOException {
DefaultMetricsSystem.initialize("httpfs");
httpServer.start();
}

Expand All @@ -159,6 +161,7 @@ public void join() throws InterruptedException {

public void stop() throws Exception {
httpServer.stop();
DefaultMetricsSystem.shutdown();
}

public URL getUrl() {
Expand Down