|
@@ -19,11 +19,13 @@ package org.apache.hadoop.http;
|
|
|
|
|
|
import java.io.FileNotFoundException;
|
|
|
import java.io.IOException;
|
|
|
+import java.io.InterruptedIOException;
|
|
|
import java.io.PrintWriter;
|
|
|
import java.net.BindException;
|
|
|
import java.net.InetSocketAddress;
|
|
|
+import java.net.URI;
|
|
|
+import java.net.URISyntaxException;
|
|
|
import java.net.URL;
|
|
|
-import java.security.GeneralSecurityException;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Collections;
|
|
|
import java.util.Enumeration;
|
|
@@ -31,7 +33,6 @@ import java.util.HashMap;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
|
|
|
-import javax.net.ssl.SSLServerSocketFactory;
|
|
|
import javax.servlet.Filter;
|
|
|
import javax.servlet.FilterChain;
|
|
|
import javax.servlet.FilterConfig;
|
|
@@ -46,6 +47,7 @@ import javax.servlet.http.HttpServletResponse;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
+import org.apache.hadoop.HadoopIllegalArgumentException;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.classification.InterfaceStability;
|
|
|
import org.apache.hadoop.conf.ConfServlet;
|
|
@@ -58,16 +60,18 @@ import org.apache.hadoop.security.SecurityUtil;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
|
|
|
import org.apache.hadoop.security.authorize.AccessControlList;
|
|
|
-import org.apache.hadoop.security.ssl.SSLFactory;
|
|
|
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.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.SslSocketConnector;
|
|
|
import org.mortbay.jetty.servlet.Context;
|
|
@@ -80,6 +84,8 @@ import org.mortbay.jetty.webapp.WebAppContext;
|
|
|
import org.mortbay.thread.QueuedThreadPool;
|
|
|
import org.mortbay.util.MultiException;
|
|
|
|
|
|
+import com.google.common.base.Preconditions;
|
|
|
+import com.google.common.collect.Lists;
|
|
|
import com.sun.jersey.spi.container.servlet.ServletContainer;
|
|
|
|
|
|
/**
|
|
@@ -108,28 +114,264 @@ public class HttpServer implements FilterContainer {
|
|
|
|
|
|
public static final String BIND_ADDRESS = "bind.address";
|
|
|
|
|
|
- private AccessControlList adminsAcl;
|
|
|
+ private final AccessControlList adminsAcl;
|
|
|
|
|
|
- private SSLFactory sslFactory;
|
|
|
protected final Server webServer;
|
|
|
- protected final Connector listener;
|
|
|
+
|
|
|
+ private static class ListenerInfo {
|
|
|
+ /**
|
|
|
+ * Boolean flag to determine whether the HTTP server should clean up the
|
|
|
+ * listener in stop().
|
|
|
+ */
|
|
|
+ private final boolean isManaged;
|
|
|
+ private final Connector listener;
|
|
|
+ private ListenerInfo(boolean isManaged, Connector listener) {
|
|
|
+ this.isManaged = isManaged;
|
|
|
+ this.listener = listener;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private final List<ListenerInfo> listeners = Lists.newArrayList();
|
|
|
+
|
|
|
protected final WebAppContext webAppContext;
|
|
|
protected final boolean findPort;
|
|
|
protected final Map<Context, Boolean> defaultContexts =
|
|
|
new HashMap<Context, Boolean>();
|
|
|
protected final List<String> filterNames = new ArrayList<String>();
|
|
|
- private static final int MAX_RETRIES = 10;
|
|
|
static final String STATE_DESCRIPTION_ALIVE = " - alive";
|
|
|
static final String STATE_DESCRIPTION_NOT_LIVE = " - not live";
|
|
|
|
|
|
- private final boolean listenerStartedExternally;
|
|
|
+ /**
|
|
|
+ * Class to construct instances of HTTP server with specific options.
|
|
|
+ */
|
|
|
+ public static class Builder {
|
|
|
+ private ArrayList<URI> endpoints = Lists.newArrayList();
|
|
|
+ private Connector connector;
|
|
|
+ private String name;
|
|
|
+ private Configuration conf;
|
|
|
+ private String[] pathSpecs;
|
|
|
+ private AccessControlList adminsAcl;
|
|
|
+ private boolean securityEnabled = false;
|
|
|
+ private String usernameConfKey;
|
|
|
+ private String keytabConfKey;
|
|
|
+ private boolean needsClientAuth;
|
|
|
+ private String trustStore;
|
|
|
+ private String trustStorePassword;
|
|
|
+ private String trustStoreType;
|
|
|
+
|
|
|
+ private String keyStore;
|
|
|
+ private String keyStorePassword;
|
|
|
+ private String keyStoreType;
|
|
|
+
|
|
|
+ // The -keypass option in keytool
|
|
|
+ private String keyPassword;
|
|
|
+
|
|
|
+ @Deprecated
|
|
|
+ private String bindAddress;
|
|
|
+ @Deprecated
|
|
|
+ private int port = -1;
|
|
|
+
|
|
|
+ private boolean findPort;
|
|
|
+
|
|
|
+ private String hostName;
|
|
|
+
|
|
|
+ public Builder setName(String name){
|
|
|
+ this.name = name;
|
|
|
+ return this;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Add an endpoint that the HTTP server should listen to.
|
|
|
+ *
|
|
|
+ * @param endpoint
|
|
|
+ * the endpoint of that the HTTP server should listen to. The
|
|
|
+ * scheme specifies the protocol (i.e. HTTP / HTTPS), the host
|
|
|
+ * specifies the binding address, and the port specifies the
|
|
|
+ * listening port. Unspecified or zero port means that the server
|
|
|
+ * can listen to any port.
|
|
|
+ */
|
|
|
+ public Builder addEndpoint(URI endpoint) {
|
|
|
+ endpoints.add(endpoint);
|
|
|
+ return this;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Set the hostname of the http server. The host name is used to resolve the
|
|
|
+ * _HOST field in Kerberos principals. The hostname of the first listener
|
|
|
+ * will be used if the name is unspecified.
|
|
|
+ */
|
|
|
+ public Builder hostName(String hostName) {
|
|
|
+ this.hostName = hostName;
|
|
|
+ return this;
|
|
|
+ }
|
|
|
+
|
|
|
+ public Builder trustStore(String location, String password, String type) {
|
|
|
+ this.trustStore = location;
|
|
|
+ this.trustStorePassword = password;
|
|
|
+ this.trustStoreType = type;
|
|
|
+ return this;
|
|
|
+ }
|
|
|
+
|
|
|
+ public Builder keyStore(String location, String password, String type) {
|
|
|
+ this.keyStore = location;
|
|
|
+ this.keyStorePassword = password;
|
|
|
+ this.keyStoreType = type;
|
|
|
+ return this;
|
|
|
+ }
|
|
|
+
|
|
|
+ public Builder keyPassword(String password) {
|
|
|
+ this.keyPassword = password;
|
|
|
+ return this;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Specify whether the server should authorize the client in SSL
|
|
|
+ * connections.
|
|
|
+ */
|
|
|
+ public Builder needsClientAuth(boolean value) {
|
|
|
+ this.needsClientAuth = value;
|
|
|
+ return this;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Use addEndpoint() instead.
|
|
|
+ */
|
|
|
+ @Deprecated
|
|
|
+ public Builder setBindAddress(String bindAddress){
|
|
|
+ this.bindAddress = bindAddress;
|
|
|
+ return this;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Use addEndpoint() instead.
|
|
|
+ */
|
|
|
+ @Deprecated
|
|
|
+ public Builder setPort(int port) {
|
|
|
+ this.port = port;
|
|
|
+ return this;
|
|
|
+ }
|
|
|
+
|
|
|
+ public Builder setFindPort(boolean findPort) {
|
|
|
+ this.findPort = findPort;
|
|
|
+ return this;
|
|
|
+ }
|
|
|
+
|
|
|
+ public Builder setConf(Configuration conf) {
|
|
|
+ this.conf = conf;
|
|
|
+ return this;
|
|
|
+ }
|
|
|
+
|
|
|
+ public Builder setConnector(Connector connector) {
|
|
|
+ this.connector = connector;
|
|
|
+ return this;
|
|
|
+ }
|
|
|
+
|
|
|
+ public Builder setPathSpec(String[] pathSpec) {
|
|
|
+ this.pathSpecs = pathSpec;
|
|
|
+ return this;
|
|
|
+ }
|
|
|
+
|
|
|
+ public Builder setACL(AccessControlList acl) {
|
|
|
+ this.adminsAcl = acl;
|
|
|
+ return this;
|
|
|
+ }
|
|
|
+
|
|
|
+ public Builder setSecurityEnabled(boolean securityEnabled) {
|
|
|
+ this.securityEnabled = securityEnabled;
|
|
|
+ return this;
|
|
|
+ }
|
|
|
+
|
|
|
+ public Builder setUsernameConfKey(String usernameConfKey) {
|
|
|
+ this.usernameConfKey = usernameConfKey;
|
|
|
+ return this;
|
|
|
+ }
|
|
|
+
|
|
|
+ public Builder setKeytabConfKey(String keytabConfKey) {
|
|
|
+ this.keytabConfKey = keytabConfKey;
|
|
|
+ return this;
|
|
|
+ }
|
|
|
+
|
|
|
+ public HttpServer build() throws IOException {
|
|
|
+ if (this.name == null) {
|
|
|
+ throw new HadoopIllegalArgumentException("name is not set");
|
|
|
+ }
|
|
|
+
|
|
|
+ // Make the behavior compatible with deprecated interfaces
|
|
|
+ if (bindAddress != null && port != -1) {
|
|
|
+ try {
|
|
|
+ endpoints.add(0, new URI("http", "", bindAddress, port, "", "", ""));
|
|
|
+ } catch (URISyntaxException e) {
|
|
|
+ throw new HadoopIllegalArgumentException("Invalid endpoint: "+ e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ if (endpoints.size() == 0 && connector == null) {
|
|
|
+ throw new HadoopIllegalArgumentException("No endpoints specified");
|
|
|
+ }
|
|
|
+
|
|
|
+ if (hostName == null) {
|
|
|
+ hostName = endpoints.size() == 0 ? connector.getHost() : endpoints.get(
|
|
|
+ 0).getHost();
|
|
|
+ }
|
|
|
+
|
|
|
+ if (this.conf == null) {
|
|
|
+ conf = new Configuration();
|
|
|
+ }
|
|
|
+
|
|
|
+ HttpServer server = new HttpServer(this);
|
|
|
+
|
|
|
+ if (this.securityEnabled) {
|
|
|
+ server.initSpnego(conf, hostName, usernameConfKey, keytabConfKey);
|
|
|
+ }
|
|
|
+
|
|
|
+ if (connector != null) {
|
|
|
+ server.addUnmanagedListener(connector);
|
|
|
+ }
|
|
|
+
|
|
|
+ for (URI ep : endpoints) {
|
|
|
+ Connector listener = null;
|
|
|
+ String scheme = ep.getScheme();
|
|
|
+ if ("http".equals(scheme)) {
|
|
|
+ listener = HttpServer.createDefaultChannelConnector();
|
|
|
+ } else if ("https".equals(scheme)) {
|
|
|
+ SslSocketConnector c = new SslSocketConnector();
|
|
|
+ c.setNeedClientAuth(needsClientAuth);
|
|
|
+ c.setKeyPassword(keyPassword);
|
|
|
+
|
|
|
+ if (keyStore != null) {
|
|
|
+ c.setKeystore(keyStore);
|
|
|
+ c.setKeystoreType(keyStoreType);
|
|
|
+ c.setPassword(keyStorePassword);
|
|
|
+ }
|
|
|
+
|
|
|
+ if (trustStore != null) {
|
|
|
+ c.setTruststore(trustStore);
|
|
|
+ c.setTruststoreType(trustStoreType);
|
|
|
+ c.setTrustPassword(trustStorePassword);
|
|
|
+ }
|
|
|
+ listener = c;
|
|
|
+
|
|
|
+ } else {
|
|
|
+ throw new HadoopIllegalArgumentException(
|
|
|
+ "unknown scheme for endpoint:" + ep);
|
|
|
+ }
|
|
|
+ listener.setHost(ep.getHost());
|
|
|
+ listener.setPort(ep.getPort() == -1 ? 0 : ep.getPort());
|
|
|
+ server.addManagedListener(listener);
|
|
|
+ }
|
|
|
+ server.loadListeners();
|
|
|
+ return server;
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
/** Same as this(name, bindAddress, port, findPort, null); */
|
|
|
+ @Deprecated
|
|
|
public HttpServer(String name, String bindAddress, int port, boolean findPort
|
|
|
) throws IOException {
|
|
|
this(name, bindAddress, port, findPort, new Configuration());
|
|
|
}
|
|
|
|
|
|
+ @Deprecated
|
|
|
public HttpServer(String name, String bindAddress, int port,
|
|
|
boolean findPort, Configuration conf, Connector connector) throws IOException {
|
|
|
this(name, bindAddress, port, findPort, conf, null, connector, null);
|
|
@@ -149,6 +391,7 @@ public class HttpServer implements FilterContainer {
|
|
|
* @param pathSpecs Path specifications that this httpserver will be serving.
|
|
|
* These will be added to any filters.
|
|
|
*/
|
|
|
+ @Deprecated
|
|
|
public HttpServer(String name, String bindAddress, int port,
|
|
|
boolean findPort, Configuration conf, String[] pathSpecs) throws IOException {
|
|
|
this(name, bindAddress, port, findPort, conf, null, null, pathSpecs);
|
|
@@ -163,11 +406,13 @@ public class HttpServer implements FilterContainer {
|
|
|
* increment by 1 until it finds a free port.
|
|
|
* @param conf Configuration
|
|
|
*/
|
|
|
+ @Deprecated
|
|
|
public HttpServer(String name, String bindAddress, int port,
|
|
|
boolean findPort, Configuration conf) throws IOException {
|
|
|
this(name, bindAddress, port, findPort, conf, null, null, null);
|
|
|
}
|
|
|
|
|
|
+ @Deprecated
|
|
|
public HttpServer(String name, String bindAddress, int port,
|
|
|
boolean findPort, Configuration conf, AccessControlList adminsAcl)
|
|
|
throws IOException {
|
|
@@ -185,6 +430,7 @@ public class HttpServer implements FilterContainer {
|
|
|
* @param conf Configuration
|
|
|
* @param adminsAcl {@link AccessControlList} of the admins
|
|
|
*/
|
|
|
+ @Deprecated
|
|
|
public HttpServer(String name, String bindAddress, int port,
|
|
|
boolean findPort, Configuration conf, AccessControlList adminsAcl,
|
|
|
Connector connector) throws IOException {
|
|
@@ -205,71 +451,64 @@ public class HttpServer implements FilterContainer {
|
|
|
* @param pathSpecs Path specifications that this httpserver will be serving.
|
|
|
* These will be added to any filters.
|
|
|
*/
|
|
|
+ @Deprecated
|
|
|
public HttpServer(String name, String bindAddress, int port,
|
|
|
boolean findPort, Configuration conf, AccessControlList adminsAcl,
|
|
|
Connector connector, String[] pathSpecs) throws IOException {
|
|
|
- webServer = new Server();
|
|
|
- this.findPort = findPort;
|
|
|
- this.adminsAcl = adminsAcl;
|
|
|
-
|
|
|
- if(connector == null) {
|
|
|
- listenerStartedExternally = false;
|
|
|
- if (HttpConfig.isSecure()) {
|
|
|
- sslFactory = new SSLFactory(SSLFactory.Mode.SERVER, conf);
|
|
|
- try {
|
|
|
- sslFactory.init();
|
|
|
- } catch (GeneralSecurityException ex) {
|
|
|
- throw new IOException(ex);
|
|
|
- }
|
|
|
- SslSocketConnector sslListener = new SslSocketConnector() {
|
|
|
- @Override
|
|
|
- protected SSLServerSocketFactory createFactory() throws Exception {
|
|
|
- return sslFactory.createSSLServerSocketFactory();
|
|
|
- }
|
|
|
- };
|
|
|
- listener = sslListener;
|
|
|
- } else {
|
|
|
- listener = createBaseListener(conf);
|
|
|
- }
|
|
|
- listener.setHost(bindAddress);
|
|
|
- listener.setPort(port);
|
|
|
- LOG.info("SSL is enabled on " + toString());
|
|
|
- } else {
|
|
|
- listenerStartedExternally = true;
|
|
|
- listener = connector;
|
|
|
- }
|
|
|
-
|
|
|
- webServer.addConnector(listener);
|
|
|
+ this(new Builder().setName(name).hostName(bindAddress)
|
|
|
+ .addEndpoint(URI.create("http://" + bindAddress + ":" + port))
|
|
|
+ .setFindPort(findPort).setConf(conf).setACL(adminsAcl)
|
|
|
+ .setConnector(connector).setPathSpec(pathSpecs));
|
|
|
+ }
|
|
|
+
|
|
|
+ private HttpServer(final Builder b) throws IOException {
|
|
|
+ final String appDir = getWebAppsPath(b.name);
|
|
|
+ this.webServer = new Server();
|
|
|
+ this.adminsAcl = b.adminsAcl;
|
|
|
+ this.webAppContext = createWebAppContext(b.name, b.conf, adminsAcl, appDir);
|
|
|
+ this.findPort = b.findPort;
|
|
|
+ initializeWebServer(b.name, b.hostName, b.conf, b.pathSpecs);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void initializeWebServer(String name, String hostName,
|
|
|
+ Configuration conf, String[] pathSpecs)
|
|
|
+ throws FileNotFoundException, IOException {
|
|
|
+
|
|
|
+ Preconditions.checkNotNull(webAppContext);
|
|
|
|
|
|
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 = maxThreads == -1 ? new QueuedThreadPool()
|
|
|
+ : new QueuedThreadPool(maxThreads);
|
|
|
threadPool.setDaemon(true);
|
|
|
webServer.setThreadPool(threadPool);
|
|
|
|
|
|
- final String appDir = getWebAppsPath(name);
|
|
|
ContextHandlerCollection contexts = new ContextHandlerCollection();
|
|
|
- webServer.setHandler(contexts);
|
|
|
-
|
|
|
- webAppContext = new WebAppContext();
|
|
|
- webAppContext.setDisplayName(name);
|
|
|
- webAppContext.setContextPath("/");
|
|
|
- webAppContext.setWar(appDir + "/" + name);
|
|
|
- webAppContext.getServletContext().setAttribute(CONF_CONTEXT_ATTRIBUTE, conf);
|
|
|
- webAppContext.getServletContext().setAttribute(ADMINS_ACL, adminsAcl);
|
|
|
- addNoCacheFilter(webAppContext);
|
|
|
+ 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);
|
|
|
+ }
|
|
|
+
|
|
|
+ final String appDir = getWebAppsPath(name);
|
|
|
+
|
|
|
webServer.addHandler(webAppContext);
|
|
|
|
|
|
addDefaultApps(contexts, appDir, conf);
|
|
|
-
|
|
|
+
|
|
|
addGlobalFilter("safety", QuotingInputFilter.class.getName(), null);
|
|
|
- final FilterInitializer[] initializers = getFilterInitializers(conf);
|
|
|
+ final FilterInitializer[] initializers = getFilterInitializers(conf);
|
|
|
if (initializers != null) {
|
|
|
conf = new Configuration(conf);
|
|
|
- conf.set(BIND_ADDRESS, bindAddress);
|
|
|
- for(FilterInitializer c : initializers) {
|
|
|
+ conf.set(BIND_ADDRESS, hostName);
|
|
|
+ for (FilterInitializer c : initializers) {
|
|
|
c.initFilter(this, conf);
|
|
|
}
|
|
|
}
|
|
@@ -284,10 +523,29 @@ public class HttpServer implements FilterContainer {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- @SuppressWarnings("unchecked")
|
|
|
- private void addNoCacheFilter(WebAppContext ctxt) {
|
|
|
- defineFilter(ctxt, NO_CACHE_FILTER,
|
|
|
- NoCacheFilter.class.getName(), Collections.EMPTY_MAP, new String[] { "/*"});
|
|
|
+ private void addUnmanagedListener(Connector connector) {
|
|
|
+ listeners.add(new ListenerInfo(false, connector));
|
|
|
+ }
|
|
|
+
|
|
|
+ private void addManagedListener(Connector connector) {
|
|
|
+ listeners.add(new ListenerInfo(true, connector));
|
|
|
+ }
|
|
|
+
|
|
|
+ private static WebAppContext createWebAppContext(String name,
|
|
|
+ Configuration conf, AccessControlList adminsAcl, final String appDir) {
|
|
|
+ WebAppContext ctx = new WebAppContext();
|
|
|
+ ctx.setDisplayName(name);
|
|
|
+ ctx.setContextPath("/");
|
|
|
+ ctx.setWar(appDir + "/" + name);
|
|
|
+ ctx.getServletContext().setAttribute(CONF_CONTEXT_ATTRIBUTE, conf);
|
|
|
+ ctx.getServletContext().setAttribute(ADMINS_ACL, adminsAcl);
|
|
|
+ addNoCacheFilter(ctx);
|
|
|
+ return ctx;
|
|
|
+ }
|
|
|
+
|
|
|
+ private static void addNoCacheFilter(WebAppContext ctxt) {
|
|
|
+ defineFilter(ctxt, NO_CACHE_FILTER, NoCacheFilter.class.getName(),
|
|
|
+ Collections.<String, String> emptyMap(), new String[] { "/*" });
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -352,7 +610,9 @@ public class HttpServer implements FilterContainer {
|
|
|
if (conf.getBoolean(
|
|
|
CommonConfigurationKeys.HADOOP_JETTY_LOGS_SERVE_ALIASES,
|
|
|
CommonConfigurationKeys.DEFAULT_HADOOP_JETTY_LOGS_SERVE_ALIASES)) {
|
|
|
- logContext.getInitParams().put(
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
+ Map<String, String> params = logContext.getInitParams();
|
|
|
+ params.put(
|
|
|
"org.mortbay.jetty.servlet.Default.aliases", "true");
|
|
|
}
|
|
|
logContext.setDisplayName("logs");
|
|
@@ -529,7 +789,7 @@ public class HttpServer implements FilterContainer {
|
|
|
/**
|
|
|
* Define a filter for a context and set up default url mappings.
|
|
|
*/
|
|
|
- public void defineFilter(Context ctx, String name,
|
|
|
+ public static void defineFilter(Context ctx, String name,
|
|
|
String classname, Map<String,String> parameters, String[] urls) {
|
|
|
|
|
|
FilterHolder holder = new FilterHolder();
|
|
@@ -569,7 +829,7 @@ public class HttpServer implements FilterContainer {
|
|
|
public Object getAttribute(String name) {
|
|
|
return webAppContext.getAttribute(name);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
public WebAppContext getWebAppContext(){
|
|
|
return this.webAppContext;
|
|
|
}
|
|
@@ -593,80 +853,47 @@ public class HttpServer implements FilterContainer {
|
|
|
* Get the port that the server is on
|
|
|
* @return the port
|
|
|
*/
|
|
|
+ @Deprecated
|
|
|
public int getPort() {
|
|
|
return webServer.getConnectors()[0].getLocalPort();
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Set the min, max number of worker threads (simultaneous connections).
|
|
|
+ * 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.
|
|
|
*/
|
|
|
- public void setThreads(int min, int max) {
|
|
|
- QueuedThreadPool pool = (QueuedThreadPool) webServer.getThreadPool() ;
|
|
|
- pool.setMinThreads(min);
|
|
|
- pool.setMaxThreads(max);
|
|
|
- }
|
|
|
+ public InetSocketAddress getConnectorAddress(int index) {
|
|
|
+ Preconditions.checkArgument(index >= 0);
|
|
|
+ if (index > webServer.getConnectors().length)
|
|
|
+ return null;
|
|
|
|
|
|
- /**
|
|
|
- * Configure an ssl listener on the server.
|
|
|
- * @param addr address to listen on
|
|
|
- * @param keystore location of the keystore
|
|
|
- * @param storPass password for the keystore
|
|
|
- * @param keyPass password for the key
|
|
|
- * @deprecated Use {@link #addSslListener(InetSocketAddress, Configuration, boolean)}
|
|
|
- */
|
|
|
- @Deprecated
|
|
|
- public void addSslListener(InetSocketAddress addr, String keystore,
|
|
|
- String storPass, String keyPass) throws IOException {
|
|
|
- if (webServer.isStarted()) {
|
|
|
- throw new IOException("Failed to add ssl listener");
|
|
|
+ Connector c = webServer.getConnectors()[index];
|
|
|
+ if (c.getLocalPort() == -1) {
|
|
|
+ // The connector is not bounded
|
|
|
+ return null;
|
|
|
}
|
|
|
- SslSocketConnector sslListener = new SslSocketConnector();
|
|
|
- sslListener.setHost(addr.getHostName());
|
|
|
- sslListener.setPort(addr.getPort());
|
|
|
- sslListener.setKeystore(keystore);
|
|
|
- sslListener.setPassword(storPass);
|
|
|
- sslListener.setKeyPassword(keyPass);
|
|
|
- webServer.addConnector(sslListener);
|
|
|
+
|
|
|
+ return new InetSocketAddress(c.getHost(), c.getLocalPort());
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Configure an ssl listener on the server.
|
|
|
- * @param addr address to listen on
|
|
|
- * @param sslConf conf to retrieve ssl options
|
|
|
- * @param needCertsAuth whether x509 certificate authentication is required
|
|
|
+ * Set the min, max number of worker threads (simultaneous connections).
|
|
|
*/
|
|
|
- public void addSslListener(InetSocketAddress addr, Configuration sslConf,
|
|
|
- boolean needCertsAuth) throws IOException {
|
|
|
- if (webServer.isStarted()) {
|
|
|
- throw new IOException("Failed to add ssl listener");
|
|
|
- }
|
|
|
- if (needCertsAuth) {
|
|
|
- // setting up SSL truststore for authenticating clients
|
|
|
- System.setProperty("javax.net.ssl.trustStore", sslConf.get(
|
|
|
- "ssl.server.truststore.location", ""));
|
|
|
- System.setProperty("javax.net.ssl.trustStorePassword", sslConf.get(
|
|
|
- "ssl.server.truststore.password", ""));
|
|
|
- System.setProperty("javax.net.ssl.trustStoreType", sslConf.get(
|
|
|
- "ssl.server.truststore.type", "jks"));
|
|
|
- }
|
|
|
- SslSocketConnector sslListener = new SslSocketConnector();
|
|
|
- sslListener.setHost(addr.getHostName());
|
|
|
- sslListener.setPort(addr.getPort());
|
|
|
- sslListener.setKeystore(sslConf.get("ssl.server.keystore.location"));
|
|
|
- sslListener.setPassword(sslConf.get("ssl.server.keystore.password", ""));
|
|
|
- sslListener.setKeyPassword(sslConf.get("ssl.server.keystore.keypassword", ""));
|
|
|
- sslListener.setKeystoreType(sslConf.get("ssl.server.keystore.type", "jks"));
|
|
|
- sslListener.setNeedClientAuth(needCertsAuth);
|
|
|
- webServer.addConnector(sslListener);
|
|
|
+ public void setThreads(int min, int max) {
|
|
|
+ QueuedThreadPool pool = (QueuedThreadPool) webServer.getThreadPool();
|
|
|
+ pool.setMinThreads(min);
|
|
|
+ pool.setMaxThreads(max);
|
|
|
}
|
|
|
-
|
|
|
- protected void initSpnego(Configuration conf,
|
|
|
+
|
|
|
+ private void initSpnego(Configuration conf, String hostName,
|
|
|
String usernameConfKey, String keytabConfKey) throws IOException {
|
|
|
Map<String, String> params = new HashMap<String, String>();
|
|
|
String principalInConf = conf.get(usernameConfKey);
|
|
|
if (principalInConf != null && !principalInConf.isEmpty()) {
|
|
|
- params.put("kerberos.principal",
|
|
|
- SecurityUtil.getServerPrincipal(principalInConf, listener.getHost()));
|
|
|
+ params.put("kerberos.principal", SecurityUtil.getServerPrincipal(
|
|
|
+ principalInConf, hostName));
|
|
|
}
|
|
|
String httpKeytab = conf.get(keytabConfKey);
|
|
|
if (httpKeytab != null && !httpKeytab.isEmpty()) {
|
|
@@ -684,8 +911,7 @@ public class HttpServer implements FilterContainer {
|
|
|
public void start() throws IOException {
|
|
|
try {
|
|
|
try {
|
|
|
- openListener();
|
|
|
- LOG.info("Jetty bound to port " + listener.getLocalPort());
|
|
|
+ openListeners();
|
|
|
webServer.start();
|
|
|
} catch (IOException ex) {
|
|
|
LOG.info("HttpServer.start() threw a non Bind IOException", ex);
|
|
@@ -718,73 +944,65 @@ public class HttpServer implements FilterContainer {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ private void loadListeners() {
|
|
|
+ for (ListenerInfo li : listeners) {
|
|
|
+ webServer.addConnector(li.listener);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Open the main listener for the server
|
|
|
* @throws Exception
|
|
|
*/
|
|
|
- void openListener() throws Exception {
|
|
|
- if (listener.getLocalPort() != -1) { // it's already bound
|
|
|
- return;
|
|
|
- }
|
|
|
- if (listenerStartedExternally) { // Expect that listener was started securely
|
|
|
- throw new Exception("Expected webserver's listener to be started " +
|
|
|
- "previously but wasn't");
|
|
|
- }
|
|
|
- int port = listener.getPort();
|
|
|
- while (true) {
|
|
|
- // jetty has a bug where you can't reopen a listener that previously
|
|
|
- // failed to open w/o issuing a close first, even if the port is changed
|
|
|
- try {
|
|
|
- listener.close();
|
|
|
- listener.open();
|
|
|
- break;
|
|
|
- } catch (BindException ex) {
|
|
|
- if (port == 0 || !findPort) {
|
|
|
- BindException be = new BindException(
|
|
|
- "Port in use: " + listener.getHost() + ":" + listener.getPort());
|
|
|
- be.initCause(ex);
|
|
|
- throw be;
|
|
|
+ void openListeners() throws Exception {
|
|
|
+ for (ListenerInfo li : listeners) {
|
|
|
+ Connector listener = li.listener;
|
|
|
+ if (!li.isManaged || li.listener.getLocalPort() != -1) {
|
|
|
+ // This listener is either started externally or has been bound
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+ int port = listener.getPort();
|
|
|
+ while (true) {
|
|
|
+ // jetty has a bug where you can't reopen a listener that previously
|
|
|
+ // failed to open w/o issuing a close first, even if the port is changed
|
|
|
+ try {
|
|
|
+ listener.close();
|
|
|
+ listener.open();
|
|
|
+ LOG.info("Jetty bound to port " + listener.getLocalPort());
|
|
|
+ break;
|
|
|
+ } catch (BindException ex) {
|
|
|
+ if (port == 0 || !findPort) {
|
|
|
+ BindException be = new BindException("Port in use: "
|
|
|
+ + listener.getHost() + ":" + listener.getPort());
|
|
|
+ be.initCause(ex);
|
|
|
+ throw be;
|
|
|
+ }
|
|
|
}
|
|
|
+ // try the next port number
|
|
|
+ listener.setPort(++port);
|
|
|
+ Thread.sleep(100);
|
|
|
}
|
|
|
- // try the next port number
|
|
|
- listener.setPort(++port);
|
|
|
- Thread.sleep(100);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Return the bind address of the listener.
|
|
|
- * @return InetSocketAddress of the listener
|
|
|
- */
|
|
|
- public InetSocketAddress getListenerAddress() {
|
|
|
- int port = listener.getLocalPort();
|
|
|
- if (port == -1) { // not bound, return requested port
|
|
|
- port = listener.getPort();
|
|
|
- }
|
|
|
- return new InetSocketAddress(listener.getHost(), port);
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* stop the server
|
|
|
*/
|
|
|
public void stop() throws Exception {
|
|
|
MultiException exception = null;
|
|
|
- try {
|
|
|
- listener.close();
|
|
|
- } catch (Exception e) {
|
|
|
- LOG.error("Error while stopping listener for webapp"
|
|
|
- + webAppContext.getDisplayName(), e);
|
|
|
- exception = addMultiException(exception, e);
|
|
|
- }
|
|
|
+ for (ListenerInfo li : listeners) {
|
|
|
+ if (!li.isManaged) {
|
|
|
+ continue;
|
|
|
+ }
|
|
|
|
|
|
- try {
|
|
|
- if (sslFactory != null) {
|
|
|
- sslFactory.destroy();
|
|
|
+ try {
|
|
|
+ li.listener.close();
|
|
|
+ } catch (Exception e) {
|
|
|
+ LOG.error(
|
|
|
+ "Error while stopping listener for webapp"
|
|
|
+ + webAppContext.getDisplayName(), e);
|
|
|
+ exception = addMultiException(exception, e);
|
|
|
}
|
|
|
- } catch (Exception e) {
|
|
|
- LOG.error("Error while destroying the SSLFactory"
|
|
|
- + webAppContext.getDisplayName(), e);
|
|
|
- exception = addMultiException(exception, e);
|
|
|
}
|
|
|
|
|
|
try {
|
|
@@ -796,6 +1014,7 @@ public class HttpServer implements FilterContainer {
|
|
|
+ webAppContext.getDisplayName(), e);
|
|
|
exception = addMultiException(exception, e);
|
|
|
}
|
|
|
+
|
|
|
try {
|
|
|
webServer.stop();
|
|
|
} catch (Exception e) {
|
|
@@ -836,10 +1055,17 @@ public class HttpServer implements FilterContainer {
|
|
|
*/
|
|
|
@Override
|
|
|
public String toString() {
|
|
|
- return listener != null ?
|
|
|
- ("HttpServer at http://" + listener.getHost() + ":" + listener.getLocalPort() + "/"
|
|
|
- + (isAlive() ? STATE_DESCRIPTION_ALIVE : STATE_DESCRIPTION_NOT_LIVE))
|
|
|
- : "Inactive HttpServer";
|
|
|
+ if (listeners.size() == 0) {
|
|
|
+ return "Inactive HttpServer";
|
|
|
+ } else {
|
|
|
+ StringBuilder sb = new StringBuilder("HttpServer (")
|
|
|
+ .append(isAlive() ? STATE_DESCRIPTION_ALIVE : STATE_DESCRIPTION_NOT_LIVE).append("), listening at:");
|
|
|
+ for (ListenerInfo li : listeners) {
|
|
|
+ Connector l = li.listener;
|
|
|
+ sb.append(l.getHost()).append(":").append(l.getPort()).append("/,");
|
|
|
+ }
|
|
|
+ return sb.toString();
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|