|
@@ -56,7 +56,6 @@ import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
|
|
import org.apache.hadoop.security.AuthenticationFilterInitializer;
|
|
|
import org.apache.hadoop.security.authentication.util.SignerSecretProvider;
|
|
|
-import org.apache.hadoop.security.ssl.SslSelectChannelConnectorSecure;
|
|
|
import org.apache.hadoop.jmx.JMXJsonServlet;
|
|
|
import org.apache.hadoop.log.LogLevel;
|
|
|
import org.apache.hadoop.security.SecurityUtil;
|
|
@@ -65,34 +64,42 @@ import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
|
|
|
import org.apache.hadoop.security.authorize.AccessControlList;
|
|
|
import org.apache.hadoop.util.ReflectionUtils;
|
|
|
import org.apache.hadoop.util.Shell;
|
|
|
-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.SessionManager;
|
|
|
-import org.mortbay.jetty.handler.ContextHandler;
|
|
|
-import org.mortbay.jetty.handler.ContextHandlerCollection;
|
|
|
-import org.mortbay.jetty.handler.HandlerCollection;
|
|
|
-import org.mortbay.jetty.handler.RequestLogHandler;
|
|
|
-import org.mortbay.jetty.nio.SelectChannelConnector;
|
|
|
-import org.mortbay.jetty.security.SslSelectChannelConnector;
|
|
|
-import org.mortbay.jetty.servlet.AbstractSessionManager;
|
|
|
-import org.mortbay.jetty.servlet.Context;
|
|
|
-import org.mortbay.jetty.servlet.DefaultServlet;
|
|
|
-import org.mortbay.jetty.servlet.FilterHolder;
|
|
|
-import org.mortbay.jetty.servlet.SessionHandler;
|
|
|
-import org.mortbay.jetty.servlet.FilterMapping;
|
|
|
-import org.mortbay.jetty.servlet.ServletHandler;
|
|
|
-import org.mortbay.jetty.servlet.ServletHolder;
|
|
|
-import org.mortbay.jetty.webapp.WebAppContext;
|
|
|
-import org.mortbay.thread.QueuedThreadPool;
|
|
|
-import org.mortbay.util.MultiException;
|
|
|
+import org.eclipse.jetty.http.HttpVersion;
|
|
|
+import org.eclipse.jetty.server.ConnectionFactory;
|
|
|
+import org.eclipse.jetty.server.Connector;
|
|
|
+import org.eclipse.jetty.server.Handler;
|
|
|
+import org.eclipse.jetty.server.HttpConfiguration;
|
|
|
+import org.eclipse.jetty.server.HttpConnectionFactory;
|
|
|
+import org.eclipse.jetty.server.RequestLog;
|
|
|
+import org.eclipse.jetty.server.SecureRequestCustomizer;
|
|
|
+import org.eclipse.jetty.server.Server;
|
|
|
+import org.eclipse.jetty.server.ServerConnector;
|
|
|
+import org.eclipse.jetty.server.SessionManager;
|
|
|
+import org.eclipse.jetty.server.SslConnectionFactory;
|
|
|
+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.session.AbstractSessionManager;
|
|
|
+import org.eclipse.jetty.server.session.SessionHandler;
|
|
|
+import org.eclipse.jetty.servlet.DefaultServlet;
|
|
|
+import org.eclipse.jetty.servlet.FilterHolder;
|
|
|
+import org.eclipse.jetty.servlet.FilterMapping;
|
|
|
+import org.eclipse.jetty.servlet.ServletContextHandler;
|
|
|
+import org.eclipse.jetty.servlet.ServletHandler;
|
|
|
+import org.eclipse.jetty.servlet.ServletHolder;
|
|
|
+import org.eclipse.jetty.servlet.ServletMapping;
|
|
|
+import org.eclipse.jetty.util.ArrayUtil;
|
|
|
+import org.eclipse.jetty.util.MultiException;
|
|
|
+import org.eclipse.jetty.webapp.WebAppContext;
|
|
|
+import org.eclipse.jetty.util.thread.QueuedThreadPool;
|
|
|
|
|
|
import com.google.common.base.Preconditions;
|
|
|
import com.google.common.collect.Lists;
|
|
|
import com.sun.jersey.spi.container.servlet.ServletContainer;
|
|
|
+import org.eclipse.jetty.util.ssl.SslContextFactory;
|
|
|
+
|
|
|
+import static org.apache.hadoop.fs.CommonConfigurationKeys.DEFAULT_HADOOP_HTTP_STATIC_USER;
|
|
|
+import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_HTTP_STATIC_USER;
|
|
|
|
|
|
/**
|
|
|
* Create a Jetty embedded server to answer http requests. The primary goal is
|
|
@@ -126,11 +133,13 @@ public final class HttpServer2 implements FilterContainer {
|
|
|
|
|
|
protected final Server webServer;
|
|
|
|
|
|
- private final List<Connector> listeners = Lists.newArrayList();
|
|
|
+ private final HandlerCollection handlers;
|
|
|
+
|
|
|
+ private final List<ServerConnector> listeners = Lists.newArrayList();
|
|
|
|
|
|
protected final WebAppContext webAppContext;
|
|
|
protected final boolean findPort;
|
|
|
- protected final Map<Context, Boolean> defaultContexts =
|
|
|
+ private final Map<ServletContextHandler, Boolean> defaultContexts =
|
|
|
new HashMap<>();
|
|
|
protected final List<String> filterNames = new ArrayList<>();
|
|
|
static final String STATE_DESCRIPTION_ALIVE = " - alive";
|
|
@@ -327,49 +336,59 @@ public final class HttpServer2 implements FilterContainer {
|
|
|
}
|
|
|
|
|
|
for (URI ep : endpoints) {
|
|
|
- final Connector listener;
|
|
|
+ final ServerConnector connector;
|
|
|
String scheme = ep.getScheme();
|
|
|
if ("http".equals(scheme)) {
|
|
|
- listener = HttpServer2.createDefaultChannelConnector();
|
|
|
+ connector =
|
|
|
+ HttpServer2.createDefaultChannelConnector(server.webServer);
|
|
|
} else if ("https".equals(scheme)) {
|
|
|
- listener = createHttpsChannelConnector();
|
|
|
+ connector = createHttpsChannelConnector(server.webServer);
|
|
|
|
|
|
} else {
|
|
|
throw new HadoopIllegalArgumentException(
|
|
|
"unknown scheme for endpoint:" + ep);
|
|
|
}
|
|
|
- listener.setHost(ep.getHost());
|
|
|
- listener.setPort(ep.getPort() == -1 ? 0 : ep.getPort());
|
|
|
- server.addListener(listener);
|
|
|
+ connector.setHost(ep.getHost());
|
|
|
+ connector.setPort(ep.getPort() == -1 ? 0 : ep.getPort());
|
|
|
+ server.addListener(connector);
|
|
|
}
|
|
|
server.loadListeners();
|
|
|
return server;
|
|
|
}
|
|
|
|
|
|
- private Connector createHttpsChannelConnector() {
|
|
|
- SslSelectChannelConnector c = new SslSelectChannelConnectorSecure();
|
|
|
- configureChannelConnector(c);
|
|
|
-
|
|
|
- c.setNeedClientAuth(needsClientAuth);
|
|
|
- c.setKeyPassword(keyPassword);
|
|
|
-
|
|
|
+ private ServerConnector createHttpsChannelConnector(Server server) {
|
|
|
+ ServerConnector conn = new ServerConnector(server);
|
|
|
+ HttpConfiguration httpConfig = new HttpConfiguration();
|
|
|
+ httpConfig.setRequestHeaderSize(JettyUtils.HEADER_SIZE);
|
|
|
+ httpConfig.setResponseHeaderSize(JettyUtils.HEADER_SIZE);
|
|
|
+ httpConfig.setSecureScheme("https");
|
|
|
+ httpConfig.addCustomizer(new SecureRequestCustomizer());
|
|
|
+ ConnectionFactory connFactory = new HttpConnectionFactory(httpConfig);
|
|
|
+ conn.addConnectionFactory(connFactory);
|
|
|
+ configureChannelConnector(conn);
|
|
|
+
|
|
|
+ SslContextFactory sslContextFactory = new SslContextFactory();
|
|
|
+ sslContextFactory.setNeedClientAuth(needsClientAuth);
|
|
|
+ sslContextFactory.setKeyManagerPassword(keyPassword);
|
|
|
if (keyStore != null) {
|
|
|
- c.setKeystore(keyStore);
|
|
|
- c.setKeystoreType(keyStoreType);
|
|
|
- c.setPassword(keyStorePassword);
|
|
|
+ sslContextFactory.setKeyStorePath(keyStore);
|
|
|
+ sslContextFactory.setKeyStoreType(keyStoreType);
|
|
|
+ sslContextFactory.setKeyStorePassword(keyStorePassword);
|
|
|
}
|
|
|
-
|
|
|
if (trustStore != null) {
|
|
|
- c.setTruststore(trustStore);
|
|
|
- c.setTruststoreType(trustStoreType);
|
|
|
- c.setTrustPassword(trustStorePassword);
|
|
|
+ sslContextFactory.setTrustStorePath(trustStore);
|
|
|
+ sslContextFactory.setTrustStoreType(trustStoreType);
|
|
|
+ sslContextFactory.setTrustStorePassword(trustStorePassword);
|
|
|
}
|
|
|
-
|
|
|
if(null != excludeCiphers && !excludeCiphers.isEmpty()) {
|
|
|
- c.setExcludeCipherSuites(excludeCiphers.split(","));
|
|
|
+ sslContextFactory.setExcludeCipherSuites(excludeCiphers.split(","));
|
|
|
LOG.info("Excluded Cipher List:" + excludeCiphers);
|
|
|
}
|
|
|
- return c;
|
|
|
+
|
|
|
+ conn.addFirstConnectionFactory(new SslConnectionFactory(sslContextFactory,
|
|
|
+ HttpVersion.HTTP_1_1.asString()));
|
|
|
+
|
|
|
+ return conn;
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -377,6 +396,7 @@ public final class HttpServer2 implements FilterContainer {
|
|
|
final String appDir = getWebAppsPath(b.name);
|
|
|
this.webServer = new Server();
|
|
|
this.adminsAcl = b.adminsAcl;
|
|
|
+ this.handlers = new HandlerCollection();
|
|
|
this.webAppContext = createWebAppContext(b.name, b.conf, adminsAcl, appDir);
|
|
|
this.xFrameOptionIsEnabled = b.xFrameEnabled;
|
|
|
this.xFrameOption = b.xFrameOption;
|
|
@@ -406,36 +426,33 @@ public final class HttpServer2 implements FilterContainer {
|
|
|
int maxThreads = conf.getInt(HTTP_MAX_THREADS, -1);
|
|
|
// If HTTP_MAX_THREADS is not configured, QueueThreadPool() will use the
|
|
|
// default value (currently 250).
|
|
|
- QueuedThreadPool threadPool = maxThreads == -1 ? new QueuedThreadPool()
|
|
|
- : new QueuedThreadPool(maxThreads);
|
|
|
+
|
|
|
+ QueuedThreadPool threadPool = (QueuedThreadPool) webServer.getThreadPool();
|
|
|
threadPool.setDaemon(true);
|
|
|
- webServer.setThreadPool(threadPool);
|
|
|
+ if (maxThreads != -1) {
|
|
|
+ threadPool.setMaxThreads(maxThreads);
|
|
|
+ }
|
|
|
|
|
|
SessionManager sm = webAppContext.getSessionHandler().getSessionManager();
|
|
|
if (sm instanceof AbstractSessionManager) {
|
|
|
AbstractSessionManager asm = (AbstractSessionManager)sm;
|
|
|
asm.setHttpOnly(true);
|
|
|
- asm.setSecureCookies(true);
|
|
|
+ asm.getSessionCookieConfig().setSecure(true);
|
|
|
}
|
|
|
|
|
|
ContextHandlerCollection contexts = new ContextHandlerCollection();
|
|
|
RequestLog requestLog = HttpRequestLog.getRequestLog(name);
|
|
|
|
|
|
+ handlers.addHandler(contexts);
|
|
|
if (requestLog != null) {
|
|
|
RequestLogHandler requestLogHandler = new RequestLogHandler();
|
|
|
requestLogHandler.setRequestLog(requestLog);
|
|
|
- HandlerCollection handlers = new HandlerCollection();
|
|
|
- handlers.setHandlers(new Handler[] {contexts, requestLogHandler});
|
|
|
- webServer.setHandler(handlers);
|
|
|
- } else {
|
|
|
- webServer.setHandler(contexts);
|
|
|
+ handlers.addHandler(requestLogHandler);
|
|
|
}
|
|
|
-
|
|
|
+ handlers.addHandler(webAppContext);
|
|
|
final String appDir = getWebAppsPath(name);
|
|
|
-
|
|
|
- webServer.addHandler(webAppContext);
|
|
|
-
|
|
|
addDefaultApps(contexts, appDir, conf);
|
|
|
+ webServer.setHandler(handlers);
|
|
|
|
|
|
Map<String, String> xFrameParams = new HashMap<>();
|
|
|
xFrameParams.put(X_FRAME_ENABLED,
|
|
@@ -461,7 +478,7 @@ public final class HttpServer2 implements FilterContainer {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private void addListener(Connector connector) {
|
|
|
+ private void addListener(ServerConnector connector) {
|
|
|
listeners.add(connector);
|
|
|
}
|
|
|
|
|
@@ -507,16 +524,14 @@ public final class HttpServer2 implements FilterContainer {
|
|
|
return prop;
|
|
|
}
|
|
|
|
|
|
- private static void addNoCacheFilter(WebAppContext ctxt) {
|
|
|
+ private static void addNoCacheFilter(ServletContextHandler ctxt) {
|
|
|
defineFilter(ctxt, NO_CACHE_FILTER, NoCacheFilter.class.getName(),
|
|
|
Collections.<String, String> emptyMap(), new String[] { "/*" });
|
|
|
}
|
|
|
|
|
|
- private static void configureChannelConnector(SelectChannelConnector c) {
|
|
|
- c.setLowResourceMaxIdleTime(10000);
|
|
|
+ private static void configureChannelConnector(ServerConnector c) {
|
|
|
+ c.setIdleTimeout(10000);
|
|
|
c.setAcceptQueueSize(128);
|
|
|
- c.setResolveNames(false);
|
|
|
- c.setUseDirectBuffers(false);
|
|
|
if(Shell.WINDOWS) {
|
|
|
// result of setting the SO_REUSEADDR flag is different on Windows
|
|
|
// http://msdn.microsoft.com/en-us/library/ms740621(v=vs.85).aspx
|
|
@@ -524,14 +539,18 @@ public final class HttpServer2 implements FilterContainer {
|
|
|
// the same port with indeterminate routing of incoming requests to them
|
|
|
c.setReuseAddress(false);
|
|
|
}
|
|
|
- c.setHeaderBufferSize(1024*64);
|
|
|
}
|
|
|
|
|
|
@InterfaceAudience.Private
|
|
|
- public static Connector createDefaultChannelConnector() {
|
|
|
- SelectChannelConnector ret = new SelectChannelConnector();
|
|
|
- configureChannelConnector(ret);
|
|
|
- return ret;
|
|
|
+ public static ServerConnector createDefaultChannelConnector(Server server) {
|
|
|
+ ServerConnector conn = new ServerConnector(server);
|
|
|
+ HttpConfiguration httpConfig = new HttpConfiguration();
|
|
|
+ httpConfig.setRequestHeaderSize(JettyUtils.HEADER_SIZE);
|
|
|
+ httpConfig.setResponseHeaderSize(JettyUtils.HEADER_SIZE);
|
|
|
+ ConnectionFactory connFactory = new HttpConnectionFactory(httpConfig);
|
|
|
+ conn.addConnectionFactory(connFactory);
|
|
|
+ configureChannelConnector(conn);
|
|
|
+ return conn;
|
|
|
}
|
|
|
|
|
|
/** Get an array of FilterConfiguration specified in the conf */
|
|
@@ -567,7 +586,8 @@ public final class HttpServer2 implements FilterContainer {
|
|
|
CommonConfigurationKeys.HADOOP_HTTP_LOGS_ENABLED,
|
|
|
CommonConfigurationKeys.HADOOP_HTTP_LOGS_ENABLED_DEFAULT);
|
|
|
if (logDir != null && logsEnabled) {
|
|
|
- Context logContext = new Context(parent, "/logs");
|
|
|
+ ServletContextHandler logContext =
|
|
|
+ new ServletContextHandler(parent, "/logs");
|
|
|
logContext.setResourceBase(logDir);
|
|
|
logContext.addServlet(AdminAuthorizedServlet.class, "/*");
|
|
|
if (conf.getBoolean(
|
|
@@ -575,8 +595,7 @@ public final class HttpServer2 implements FilterContainer {
|
|
|
CommonConfigurationKeys.DEFAULT_HADOOP_JETTY_LOGS_SERVE_ALIASES)) {
|
|
|
@SuppressWarnings("unchecked")
|
|
|
Map<String, String> params = logContext.getInitParams();
|
|
|
- params.put(
|
|
|
- "org.mortbay.jetty.servlet.Default.aliases", "true");
|
|
|
+ params.put("org.eclipse.jetty.servlet.Default.aliases", "true");
|
|
|
}
|
|
|
logContext.setDisplayName("logs");
|
|
|
SessionHandler handler = new SessionHandler();
|
|
@@ -584,34 +603,37 @@ public final class HttpServer2 implements FilterContainer {
|
|
|
if (sm instanceof AbstractSessionManager) {
|
|
|
AbstractSessionManager asm = (AbstractSessionManager) sm;
|
|
|
asm.setHttpOnly(true);
|
|
|
- asm.setSecureCookies(true);
|
|
|
+ asm.getSessionCookieConfig().setSecure(true);
|
|
|
}
|
|
|
logContext.setSessionHandler(handler);
|
|
|
setContextAttributes(logContext, conf);
|
|
|
- addNoCacheFilter(webAppContext);
|
|
|
+ addNoCacheFilter(logContext);
|
|
|
defaultContexts.put(logContext, true);
|
|
|
}
|
|
|
// set up the context for "/static/*"
|
|
|
- Context staticContext = new Context(parent, "/static");
|
|
|
+ ServletContextHandler staticContext =
|
|
|
+ new ServletContextHandler(parent, "/static");
|
|
|
staticContext.setResourceBase(appDir + "/static");
|
|
|
staticContext.addServlet(DefaultServlet.class, "/*");
|
|
|
staticContext.setDisplayName("static");
|
|
|
@SuppressWarnings("unchecked")
|
|
|
Map<String, String> params = staticContext.getInitParams();
|
|
|
- params.put("org.mortbay.jetty.servlet.Default.dirAllowed", "false");
|
|
|
+ params.put("org.eclipse.jetty.servlet.Default.dirAllowed", "false");
|
|
|
+ params.put("org.eclipse.jetty.servlet.Default.gzip", "true");
|
|
|
SessionHandler handler = new SessionHandler();
|
|
|
SessionManager sm = handler.getSessionManager();
|
|
|
if (sm instanceof AbstractSessionManager) {
|
|
|
AbstractSessionManager asm = (AbstractSessionManager) sm;
|
|
|
asm.setHttpOnly(true);
|
|
|
- asm.setSecureCookies(true);
|
|
|
+ asm.getSessionCookieConfig().setSecure(true);
|
|
|
}
|
|
|
staticContext.setSessionHandler(handler);
|
|
|
setContextAttributes(staticContext, conf);
|
|
|
defaultContexts.put(staticContext, true);
|
|
|
}
|
|
|
|
|
|
- private void setContextAttributes(Context context, Configuration conf) {
|
|
|
+ private void setContextAttributes(ServletContextHandler context,
|
|
|
+ Configuration conf) {
|
|
|
context.getServletContext().setAttribute(CONF_CONTEXT_ATTRIBUTE, conf);
|
|
|
context.getServletContext().setAttribute(ADMINS_ACL, adminsAcl);
|
|
|
}
|
|
@@ -627,9 +649,9 @@ public final class HttpServer2 implements FilterContainer {
|
|
|
addServlet("conf", "/conf", ConfServlet.class);
|
|
|
}
|
|
|
|
|
|
- public void addContext(Context ctxt, boolean isFiltered) {
|
|
|
- webServer.addHandler(ctxt);
|
|
|
- addNoCacheFilter(webAppContext);
|
|
|
+ public void addContext(ServletContextHandler ctxt, boolean isFiltered) {
|
|
|
+ handlers.addHandler(ctxt);
|
|
|
+ addNoCacheFilter(ctxt);
|
|
|
defaultContexts.put(ctxt, isFiltered);
|
|
|
}
|
|
|
|
|
@@ -691,7 +713,7 @@ public final class HttpServer2 implements FilterContainer {
|
|
|
* protect with Kerberos authentication.
|
|
|
* Note: This method is to be used for adding servlets that facilitate
|
|
|
* internal communication and not for user facing functionality. For
|
|
|
- + * servlets added using this method, filters (except internal Kerberos
|
|
|
+ * servlets added using this method, filters (except internal Kerberos
|
|
|
* filters) are not enabled.
|
|
|
*
|
|
|
* @param name The name of the servlet (can be passed as null)
|
|
@@ -705,19 +727,58 @@ public final class HttpServer2 implements FilterContainer {
|
|
|
if (name != null) {
|
|
|
holder.setName(name);
|
|
|
}
|
|
|
+ // Jetty doesn't like the same path spec mapping to different servlets, so
|
|
|
+ // if there's already a mapping for this pathSpec, remove it and assume that
|
|
|
+ // the newest one is the one we want
|
|
|
+ final ServletMapping[] servletMappings =
|
|
|
+ webAppContext.getServletHandler().getServletMappings();
|
|
|
+ for (int i = 0; i < servletMappings.length; i++) {
|
|
|
+ if (servletMappings[i].containsPathSpec(pathSpec)) {
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("Found existing " + servletMappings[i].getServletName() +
|
|
|
+ " servlet at path " + pathSpec + "; will replace mapping" +
|
|
|
+ " with " + holder.getName() + " servlet");
|
|
|
+ }
|
|
|
+ ServletMapping[] newServletMappings =
|
|
|
+ ArrayUtil.removeFromArray(servletMappings, servletMappings[i]);
|
|
|
+ webAppContext.getServletHandler()
|
|
|
+ .setServletMappings(newServletMappings);
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ }
|
|
|
webAppContext.addServlet(holder, pathSpec);
|
|
|
|
|
|
if(requireAuth && UserGroupInformation.isSecurityEnabled()) {
|
|
|
- LOG.info("Adding Kerberos (SPNEGO) filter to " + name);
|
|
|
- ServletHandler handler = webAppContext.getServletHandler();
|
|
|
- FilterMapping fmap = new FilterMapping();
|
|
|
- fmap.setPathSpec(pathSpec);
|
|
|
- fmap.setFilterName(SPNEGO_FILTER);
|
|
|
- fmap.setDispatches(Handler.ALL);
|
|
|
- handler.addFilterMapping(fmap);
|
|
|
+ LOG.info("Adding Kerberos (SPNEGO) filter to " + name);
|
|
|
+ ServletHandler handler = webAppContext.getServletHandler();
|
|
|
+ FilterMapping fmap = new FilterMapping();
|
|
|
+ fmap.setPathSpec(pathSpec);
|
|
|
+ fmap.setFilterName(SPNEGO_FILTER);
|
|
|
+ fmap.setDispatches(FilterMapping.ALL);
|
|
|
+ handler.addFilterMapping(fmap);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Add the given handler to the front of the list of handlers.
|
|
|
+ *
|
|
|
+ * @param handler The handler to add
|
|
|
+ */
|
|
|
+ public void addHandlerAtFront(Handler handler) {
|
|
|
+ Handler[] h = ArrayUtil.prependToArray(
|
|
|
+ handler, this.handlers.getHandlers(), Handler.class);
|
|
|
+ handlers.setHandlers(h);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Add the given handler to the end of the list of handlers.
|
|
|
+ *
|
|
|
+ * @param handler The handler to add
|
|
|
+ */
|
|
|
+ public void addHandlerAtEnd(Handler handler) {
|
|
|
+ handlers.addHandler(handler);
|
|
|
+ }
|
|
|
+
|
|
|
@Override
|
|
|
public void addFilter(String name, String classname,
|
|
|
Map<String, String> parameters) {
|
|
@@ -727,12 +788,14 @@ public final class HttpServer2 implements FilterContainer {
|
|
|
FilterMapping fmap = getFilterMapping(name, USER_FACING_URLS);
|
|
|
defineFilter(webAppContext, filterHolder, fmap);
|
|
|
LOG.info(
|
|
|
- "Added filter " + name + " (class=" + classname + ") to context " + webAppContext.getDisplayName());
|
|
|
+ "Added filter " + name + " (class=" + classname + ") to context "
|
|
|
+ + webAppContext.getDisplayName());
|
|
|
final String[] ALL_URLS = { "/*" };
|
|
|
fmap = getFilterMapping(name, ALL_URLS);
|
|
|
- for (Map.Entry<Context, Boolean> e : defaultContexts.entrySet()) {
|
|
|
+ for (Map.Entry<ServletContextHandler, Boolean> e
|
|
|
+ : defaultContexts.entrySet()) {
|
|
|
if (e.getValue()) {
|
|
|
- Context ctx = e.getKey();
|
|
|
+ ServletContextHandler ctx = e.getKey();
|
|
|
defineFilter(ctx, filterHolder, fmap);
|
|
|
LOG.info("Added filter " + name + " (class=" + classname
|
|
|
+ ") to context " + ctx.getDisplayName());
|
|
@@ -748,7 +811,7 @@ public final class HttpServer2 implements FilterContainer {
|
|
|
FilterHolder filterHolder = getFilterHolder(name, classname, parameters);
|
|
|
FilterMapping fmap = getFilterMapping(name, ALL_URLS);
|
|
|
defineFilter(webAppContext, filterHolder, fmap);
|
|
|
- for (Context ctx : defaultContexts.keySet()) {
|
|
|
+ for (ServletContextHandler ctx : defaultContexts.keySet()) {
|
|
|
defineFilter(ctx, filterHolder, fmap);
|
|
|
}
|
|
|
LOG.info("Added global filter '" + name + "' (class=" + classname + ")");
|
|
@@ -757,7 +820,7 @@ public final class HttpServer2 implements FilterContainer {
|
|
|
/**
|
|
|
* Define a filter for a context and set up default url mappings.
|
|
|
*/
|
|
|
- public static void defineFilter(Context ctx, String name,
|
|
|
+ public static void defineFilter(ServletContextHandler ctx, String name,
|
|
|
String classname, Map<String,String> parameters, String[] urls) {
|
|
|
FilterHolder filterHolder = getFilterHolder(name, classname, parameters);
|
|
|
FilterMapping fmap = getFilterMapping(name, urls);
|
|
@@ -767,8 +830,8 @@ public final class HttpServer2 implements FilterContainer {
|
|
|
/**
|
|
|
* Define a filter for a context and set up default url mappings.
|
|
|
*/
|
|
|
- private static void defineFilter(Context ctx, FilterHolder holder,
|
|
|
- FilterMapping fmap) {
|
|
|
+ private static void defineFilter(ServletContextHandler ctx,
|
|
|
+ FilterHolder holder, FilterMapping fmap) {
|
|
|
ServletHandler handler = ctx.getServletHandler();
|
|
|
handler.addFilter(holder, fmap);
|
|
|
}
|
|
@@ -776,7 +839,7 @@ public final class HttpServer2 implements FilterContainer {
|
|
|
private static FilterMapping getFilterMapping(String name, String[] urls) {
|
|
|
FilterMapping fmap = new FilterMapping();
|
|
|
fmap.setPathSpecs(urls);
|
|
|
- fmap.setDispatches(Handler.ALL);
|
|
|
+ fmap.setDispatches(FilterMapping.ALL);
|
|
|
fmap.setFilterName(name);
|
|
|
return fmap;
|
|
|
}
|
|
@@ -786,7 +849,9 @@ public final class HttpServer2 implements FilterContainer {
|
|
|
FilterHolder holder = new FilterHolder();
|
|
|
holder.setName(name);
|
|
|
holder.setClassName(classname);
|
|
|
- holder.setInitParameters(parameters);
|
|
|
+ if (parameters != null) {
|
|
|
+ holder.setInitParameters(parameters);
|
|
|
+ }
|
|
|
return holder;
|
|
|
}
|
|
|
|
|
@@ -796,13 +861,13 @@ public final class HttpServer2 implements FilterContainer {
|
|
|
* @param webAppCtx The WebApplicationContext to add to
|
|
|
*/
|
|
|
protected void addFilterPathMapping(String pathSpec,
|
|
|
- Context webAppCtx) {
|
|
|
+ ServletContextHandler webAppCtx) {
|
|
|
ServletHandler handler = webAppCtx.getServletHandler();
|
|
|
for(String name : filterNames) {
|
|
|
FilterMapping fmap = new FilterMapping();
|
|
|
fmap.setPathSpec(pathSpec);
|
|
|
fmap.setFilterName(name);
|
|
|
- fmap.setDispatches(Handler.ALL);
|
|
|
+ fmap.setDispatches(FilterMapping.ALL);
|
|
|
handler.addFilterMapping(fmap);
|
|
|
}
|
|
|
}
|
|
@@ -841,23 +906,23 @@ public final class HttpServer2 implements FilterContainer {
|
|
|
*/
|
|
|
@Deprecated
|
|
|
public int getPort() {
|
|
|
- return webServer.getConnectors()[0].getLocalPort();
|
|
|
+ return ((ServerConnector)webServer.getConnectors()[0]).getLocalPort();
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Get the address that corresponds to a particular connector.
|
|
|
*
|
|
|
* @return the corresponding address for the connector, or null if there's no
|
|
|
- * such connector or the connector is not bounded.
|
|
|
+ * such connector or the connector is not bounded or was closed.
|
|
|
*/
|
|
|
public InetSocketAddress getConnectorAddress(int index) {
|
|
|
Preconditions.checkArgument(index >= 0);
|
|
|
if (index > webServer.getConnectors().length)
|
|
|
return null;
|
|
|
|
|
|
- Connector c = webServer.getConnectors()[index];
|
|
|
- if (c.getLocalPort() == -1) {
|
|
|
- // The connector is not bounded
|
|
|
+ ServerConnector c = (ServerConnector)webServer.getConnectors()[index];
|
|
|
+ if (c.getLocalPort() == -1 || c.getLocalPort() == -2) {
|
|
|
+ // The connector is not bounded or was closed
|
|
|
return null;
|
|
|
}
|
|
|
|
|
@@ -907,8 +972,8 @@ public final class HttpServer2 implements FilterContainer {
|
|
|
throw ex;
|
|
|
}
|
|
|
// Make sure there is no handler failures.
|
|
|
- Handler[] handlers = webServer.getHandlers();
|
|
|
- for (Handler handler : handlers) {
|
|
|
+ Handler[] hs = webServer.getHandlers();
|
|
|
+ for (Handler handler : hs) {
|
|
|
if (handler.isFailed()) {
|
|
|
throw new IOException(
|
|
|
"Problem in starting http server. Server handlers failed");
|
|
@@ -944,9 +1009,10 @@ public final class HttpServer2 implements FilterContainer {
|
|
|
* @throws Exception
|
|
|
*/
|
|
|
void openListeners() throws Exception {
|
|
|
- for (Connector listener : listeners) {
|
|
|
- if (listener.getLocalPort() != -1) {
|
|
|
- // This listener is either started externally or has been bound
|
|
|
+ for (ServerConnector listener : listeners) {
|
|
|
+ if (listener.getLocalPort() != -1 && listener.getLocalPort() != -2) {
|
|
|
+ // This listener is either started externally or has been bound or was
|
|
|
+ // closed
|
|
|
continue;
|
|
|
}
|
|
|
int port = listener.getPort();
|
|
@@ -978,7 +1044,7 @@ public final class HttpServer2 implements FilterContainer {
|
|
|
*/
|
|
|
public void stop() throws Exception {
|
|
|
MultiException exception = null;
|
|
|
- for (Connector c : listeners) {
|
|
|
+ for (ServerConnector c : listeners) {
|
|
|
try {
|
|
|
c.close();
|
|
|
} catch (Exception e) {
|
|
@@ -1042,12 +1108,30 @@ public final class HttpServer2 implements FilterContainer {
|
|
|
.append(isAlive() ? STATE_DESCRIPTION_ALIVE
|
|
|
: STATE_DESCRIPTION_NOT_LIVE)
|
|
|
.append("), listening at:");
|
|
|
- for (Connector l : listeners) {
|
|
|
+ for (ServerConnector l : listeners) {
|
|
|
sb.append(l.getHost()).append(":").append(l.getPort()).append("/,");
|
|
|
}
|
|
|
return sb.toString();
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * check whether user is static and unauthenticated, if the
|
|
|
+ * answer is TRUE, that means http sever is in non-security
|
|
|
+ * environment.
|
|
|
+ * @param servletContext the servlet context.
|
|
|
+ * @param request the servlet request.
|
|
|
+ * @return TRUE/FALSE based on the logic described above.
|
|
|
+ */
|
|
|
+ public static boolean isStaticUserAndNoneAuthType(
|
|
|
+ ServletContext servletContext, HttpServletRequest request) {
|
|
|
+ Configuration conf =
|
|
|
+ (Configuration) servletContext.getAttribute(CONF_CONTEXT_ATTRIBUTE);
|
|
|
+ final String authType = request.getAuthType();
|
|
|
+ final String staticUser = conf.get(HADOOP_HTTP_STATIC_USER,
|
|
|
+ DEFAULT_HADOOP_HTTP_STATIC_USER);
|
|
|
+ return authType == null && staticUser.equals(request.getRemoteUser());
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Checks the user has privileges to access to instrumentation servlets.
|
|
|
* <p/>
|
|
@@ -1145,9 +1229,14 @@ public final class HttpServer2 implements FilterContainer {
|
|
|
|
|
|
@Override
|
|
|
public void doGet(HttpServletRequest request, HttpServletResponse response)
|
|
|
- throws ServletException, IOException {
|
|
|
- if (!HttpServer2.isInstrumentationAccessAllowed(getServletContext(),
|
|
|
- request, response)) {
|
|
|
+ throws ServletException, IOException {
|
|
|
+ // If user is a static user and auth Type is null, that means
|
|
|
+ // there is a non-security environment and no need authorization,
|
|
|
+ // otherwise, do the authorization.
|
|
|
+ final ServletContext servletContext = getServletContext();
|
|
|
+ if (!HttpServer2.isStaticUserAndNoneAuthType(servletContext, request) &&
|
|
|
+ !HttpServer2.isInstrumentationAccessAllowed(servletContext,
|
|
|
+ request, response)) {
|
|
|
return;
|
|
|
}
|
|
|
response.setContentType("text/plain; charset=UTF-8");
|
|
@@ -1300,10 +1389,10 @@ public final class HttpServer2 implements FilterContainer {
|
|
|
*/
|
|
|
private String inferMimeType(ServletRequest request) {
|
|
|
String path = ((HttpServletRequest)request).getRequestURI();
|
|
|
- ContextHandler.SContext sContext = (ContextHandler.SContext)config.getServletContext();
|
|
|
- MimeTypes mimes = sContext.getContextHandler().getMimeTypes();
|
|
|
- Buffer mimeBuffer = mimes.getMimeByExtension(path);
|
|
|
- return (mimeBuffer == null) ? null : mimeBuffer.toString();
|
|
|
+ ServletContextHandler.Context sContext =
|
|
|
+ (ServletContextHandler.Context)config.getServletContext();
|
|
|
+ String mime = sContext.getMimeType(path);
|
|
|
+ return (mime == null) ? null : mime;
|
|
|
}
|
|
|
|
|
|
}
|