HDFS-7555. Remove the support of unmanaged connectors in HttpServer2. Contributed by Haohui Mai.
This commit is contained in:
parent
8f5522ed99
commit
2860eeb14a
|
@ -444,6 +444,9 @@ Release 2.7.0 - UNRELEASED
|
|||
HADOOP-11395. Add site documentation for Azure Storage FileSystem
|
||||
integration. (Chris Nauroth via Arpit Agarwal)
|
||||
|
||||
HDFS-7555. Remove the support of unmanaged connectors in HttpServer2.
|
||||
(wheat9)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
HADOOP-11323. WritableComparator#compare keeps reference to byte array.
|
||||
|
|
|
@ -20,9 +20,7 @@ package org.apache.hadoop.http;
|
|||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.io.OutputStream;
|
||||
import java.io.PrintStream;
|
||||
import java.io.PrintWriter;
|
||||
import java.net.BindException;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.URI;
|
||||
|
@ -125,26 +123,13 @@ public final class HttpServer2 implements FilterContainer {
|
|||
|
||||
protected final Server webServer;
|
||||
|
||||
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();
|
||||
private final List<Connector> 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>();
|
||||
new HashMap<>();
|
||||
protected final List<String> filterNames = new ArrayList<>();
|
||||
static final String STATE_DESCRIPTION_ALIVE = " - alive";
|
||||
static final String STATE_DESCRIPTION_NOT_LIVE = " - not live";
|
||||
|
||||
|
@ -153,7 +138,6 @@ public final class HttpServer2 implements FilterContainer {
|
|||
*/
|
||||
public static class Builder {
|
||||
private ArrayList<URI> endpoints = Lists.newArrayList();
|
||||
private Connector connector;
|
||||
private String name;
|
||||
private Configuration conf;
|
||||
private String[] pathSpecs;
|
||||
|
@ -245,11 +229,6 @@ public final class HttpServer2 implements FilterContainer {
|
|||
return this;
|
||||
}
|
||||
|
||||
public Builder setConnector(Connector connector) {
|
||||
this.connector = connector;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setPathSpec(String[] pathSpec) {
|
||||
this.pathSpecs = pathSpec;
|
||||
return this;
|
||||
|
@ -276,17 +255,11 @@ public final class HttpServer2 implements FilterContainer {
|
|||
}
|
||||
|
||||
public HttpServer2 build() throws IOException {
|
||||
if (this.name == null) {
|
||||
throw new HadoopIllegalArgumentException("name is not set");
|
||||
}
|
||||
|
||||
if (endpoints.size() == 0 && connector == null) {
|
||||
throw new HadoopIllegalArgumentException("No endpoints specified");
|
||||
}
|
||||
Preconditions.checkNotNull(name, "name is not set");
|
||||
Preconditions.checkState(!endpoints.isEmpty(), "No endpoints specified");
|
||||
|
||||
if (hostName == null) {
|
||||
hostName = endpoints.size() == 0 ? connector.getHost() : endpoints.get(
|
||||
0).getHost();
|
||||
hostName = endpoints.get(0).getHost();
|
||||
}
|
||||
|
||||
if (this.conf == null) {
|
||||
|
@ -299,12 +272,8 @@ public final class HttpServer2 implements FilterContainer {
|
|||
server.initSpnego(conf, hostName, usernameConfKey, keytabConfKey);
|
||||
}
|
||||
|
||||
if (connector != null) {
|
||||
server.addUnmanagedListener(connector);
|
||||
}
|
||||
|
||||
for (URI ep : endpoints) {
|
||||
Connector listener = null;
|
||||
final Connector listener;
|
||||
String scheme = ep.getScheme();
|
||||
if ("http".equals(scheme)) {
|
||||
listener = HttpServer2.createDefaultChannelConnector();
|
||||
|
@ -332,7 +301,7 @@ public final class HttpServer2 implements FilterContainer {
|
|||
}
|
||||
listener.setHost(ep.getHost());
|
||||
listener.setPort(ep.getPort() == -1 ? 0 : ep.getPort());
|
||||
server.addManagedListener(listener);
|
||||
server.addListener(listener);
|
||||
}
|
||||
server.loadListeners();
|
||||
return server;
|
||||
|
@ -350,7 +319,7 @@ public final class HttpServer2 implements FilterContainer {
|
|||
|
||||
private void initializeWebServer(String name, String hostName,
|
||||
Configuration conf, String[] pathSpecs)
|
||||
throws FileNotFoundException, IOException {
|
||||
throws IOException {
|
||||
|
||||
Preconditions.checkNotNull(webAppContext);
|
||||
|
||||
|
@ -408,12 +377,8 @@ public final class HttpServer2 implements FilterContainer {
|
|||
}
|
||||
}
|
||||
|
||||
private void addUnmanagedListener(Connector connector) {
|
||||
listeners.add(new ListenerInfo(false, connector));
|
||||
}
|
||||
|
||||
private void addManagedListener(Connector connector) {
|
||||
listeners.add(new ListenerInfo(true, connector));
|
||||
private void addListener(Connector connector) {
|
||||
listeners.add(connector);
|
||||
}
|
||||
|
||||
private static WebAppContext createWebAppContext(String name,
|
||||
|
@ -444,15 +409,6 @@ public final class HttpServer2 implements FilterContainer {
|
|||
Collections.<String, String> emptyMap(), new String[] { "/*" });
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a required listener for the Jetty instance listening on the port
|
||||
* provided. This wrapper and all subclasses must create at least one
|
||||
* listener.
|
||||
*/
|
||||
public Connector createBaseListener(Configuration conf) {
|
||||
return HttpServer2.createDefaultChannelConnector();
|
||||
}
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public static Connector createDefaultChannelConnector() {
|
||||
SelectChannelConnector ret = new SelectChannelConnector();
|
||||
|
@ -548,23 +504,6 @@ public final class HttpServer2 implements FilterContainer {
|
|||
defaultContexts.put(ctxt, isFiltered);
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a context
|
||||
* @param pathSpec The path spec for the context
|
||||
* @param dir The directory containing the context
|
||||
* @param isFiltered if true, the servlet is added to the filter path mapping
|
||||
* @throws IOException
|
||||
*/
|
||||
protected void addContext(String pathSpec, String dir, boolean isFiltered) throws IOException {
|
||||
if (0 == webServer.getHandlers().length) {
|
||||
throw new RuntimeException("Couldn't find handler");
|
||||
}
|
||||
WebAppContext webAppCtx = new WebAppContext();
|
||||
webAppCtx.setContextPath(pathSpec);
|
||||
webAppCtx.setWar(dir);
|
||||
addContext(webAppCtx, true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set a value in the webapp context. These values are available to the jsp
|
||||
* pages as "application.getAttribute(name)".
|
||||
|
@ -656,8 +595,8 @@ public final class HttpServer2 implements FilterContainer {
|
|||
|
||||
final String[] USER_FACING_URLS = { "*.html", "*.jsp" };
|
||||
defineFilter(webAppContext, name, classname, parameters, USER_FACING_URLS);
|
||||
LOG.info("Added filter " + name + " (class=" + classname
|
||||
+ ") to context " + webAppContext.getDisplayName());
|
||||
LOG.info(
|
||||
"Added filter " + name + " (class=" + classname + ") to context " + webAppContext.getDisplayName());
|
||||
final String[] ALL_URLS = { "/*" };
|
||||
for (Map.Entry<Context, Boolean> e : defaultContexts.entrySet()) {
|
||||
if (e.getValue()) {
|
||||
|
@ -784,7 +723,7 @@ public final class HttpServer2 implements FilterContainer {
|
|||
|
||||
private void initSpnego(Configuration conf, String hostName,
|
||||
String usernameConfKey, String keytabConfKey) throws IOException {
|
||||
Map<String, String> params = new HashMap<String, String>();
|
||||
Map<String, String> params = new HashMap<>();
|
||||
String principalInConf = conf.get(usernameConfKey);
|
||||
if (principalInConf != null && !principalInConf.isEmpty()) {
|
||||
params.put("kerberos.principal", SecurityUtil.getServerPrincipal(
|
||||
|
@ -817,8 +756,8 @@ public final class HttpServer2 implements FilterContainer {
|
|||
}
|
||||
// Make sure there is no handler failures.
|
||||
Handler[] handlers = webServer.getHandlers();
|
||||
for (int i = 0; i < handlers.length; i++) {
|
||||
if (handlers[i].isFailed()) {
|
||||
for (Handler handler : handlers) {
|
||||
if (handler.isFailed()) {
|
||||
throw new IOException(
|
||||
"Problem in starting http server. Server handlers failed");
|
||||
}
|
||||
|
@ -843,8 +782,8 @@ public final class HttpServer2 implements FilterContainer {
|
|||
}
|
||||
|
||||
private void loadListeners() {
|
||||
for (ListenerInfo li : listeners) {
|
||||
webServer.addConnector(li.listener);
|
||||
for (Connector c : listeners) {
|
||||
webServer.addConnector(c);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -853,9 +792,8 @@ public final class HttpServer2 implements FilterContainer {
|
|||
* @throws Exception
|
||||
*/
|
||||
void openListeners() throws Exception {
|
||||
for (ListenerInfo li : listeners) {
|
||||
Connector listener = li.listener;
|
||||
if (!li.isManaged || li.listener.getLocalPort() != -1) {
|
||||
for (Connector listener : listeners) {
|
||||
if (listener.getLocalPort() != -1) {
|
||||
// This listener is either started externally or has been bound
|
||||
continue;
|
||||
}
|
||||
|
@ -888,13 +826,9 @@ public final class HttpServer2 implements FilterContainer {
|
|||
*/
|
||||
public void stop() throws Exception {
|
||||
MultiException exception = null;
|
||||
for (ListenerInfo li : listeners) {
|
||||
if (!li.isManaged) {
|
||||
continue;
|
||||
}
|
||||
|
||||
for (Connector c : listeners) {
|
||||
try {
|
||||
li.listener.close();
|
||||
c.close();
|
||||
} catch (Exception e) {
|
||||
LOG.error(
|
||||
"Error while stopping listener for webapp"
|
||||
|
@ -947,23 +881,17 @@ public final class HttpServer2 implements FilterContainer {
|
|||
return webServer != null && webServer.isStarted();
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the host and port of the HttpServer, if live
|
||||
* @return the classname and any HTTP URL
|
||||
*/
|
||||
@Override
|
||||
public String toString() {
|
||||
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();
|
||||
Preconditions.checkState(!listeners.isEmpty());
|
||||
StringBuilder sb = new StringBuilder("HttpServer (")
|
||||
.append(isAlive() ? STATE_DESCRIPTION_ALIVE
|
||||
: STATE_DESCRIPTION_NOT_LIVE)
|
||||
.append("), listening at:");
|
||||
for (Connector l : listeners) {
|
||||
sb.append(l.getHost()).append(":").append(l.getPort()).append("/,");
|
||||
}
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -1001,8 +929,6 @@ public final class HttpServer2 implements FilterContainer {
|
|||
* Does the user sending the HttpServletRequest has the administrator ACLs? If
|
||||
* it isn't the case, response will be modified to send an error to the user.
|
||||
*
|
||||
* @param servletContext
|
||||
* @param request
|
||||
* @param response used to send the error response if user does not have admin access.
|
||||
* @return true if admin-authorized, false otherwise
|
||||
* @throws IOException
|
||||
|
@ -1141,7 +1067,7 @@ public final class HttpServer2 implements FilterContainer {
|
|||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public Map<String, String[]> getParameterMap() {
|
||||
Map<String, String[]> result = new HashMap<String,String[]>();
|
||||
Map<String, String[]> result = new HashMap<>();
|
||||
Map<String, String[]> raw = rawRequest.getParameterMap();
|
||||
for (Map.Entry<String,String[]> item: raw.entrySet()) {
|
||||
String[] rawValue = item.getValue();
|
||||
|
|
|
@ -17,6 +17,37 @@
|
|||
*/
|
||||
package org.apache.hadoop.http;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
||||
import org.apache.hadoop.http.HttpServer2.QuotingInputFilter.RequestQuoter;
|
||||
import org.apache.hadoop.http.resource.JerseyResource;
|
||||
import org.apache.hadoop.net.NetUtils;
|
||||
import org.apache.hadoop.security.Groups;
|
||||
import org.apache.hadoop.security.ShellBasedUnixGroupsMapping;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.security.authorize.AccessControlList;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.mockito.Mockito;
|
||||
import org.mockito.internal.util.reflection.Whitebox;
|
||||
import org.mortbay.jetty.Connector;
|
||||
import org.mortbay.util.ajax.JSON;
|
||||
|
||||
import javax.servlet.Filter;
|
||||
import javax.servlet.FilterChain;
|
||||
import javax.servlet.FilterConfig;
|
||||
import javax.servlet.ServletContext;
|
||||
import javax.servlet.ServletException;
|
||||
import javax.servlet.ServletRequest;
|
||||
import javax.servlet.ServletResponse;
|
||||
import javax.servlet.http.HttpServlet;
|
||||
import javax.servlet.http.HttpServletRequest;
|
||||
import javax.servlet.http.HttpServletRequestWrapper;
|
||||
import javax.servlet.http.HttpServletResponse;
|
||||
import java.io.IOException;
|
||||
import java.io.PrintWriter;
|
||||
import java.net.HttpURLConnection;
|
||||
|
@ -33,41 +64,6 @@ import java.util.concurrent.CountDownLatch;
|
|||
import java.util.concurrent.Executor;
|
||||
import java.util.concurrent.Executors;
|
||||
|
||||
import javax.servlet.Filter;
|
||||
import javax.servlet.FilterChain;
|
||||
import javax.servlet.FilterConfig;
|
||||
import javax.servlet.ServletContext;
|
||||
import javax.servlet.ServletException;
|
||||
import javax.servlet.ServletRequest;
|
||||
import javax.servlet.ServletResponse;
|
||||
import javax.servlet.http.HttpServlet;
|
||||
import javax.servlet.http.HttpServletRequest;
|
||||
import javax.servlet.http.HttpServletRequestWrapper;
|
||||
import javax.servlet.http.HttpServletResponse;
|
||||
|
||||
import org.junit.Assert;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
||||
import org.apache.hadoop.http.HttpServer2.QuotingInputFilter.RequestQuoter;
|
||||
import org.apache.hadoop.http.resource.JerseyResource;
|
||||
import org.apache.hadoop.net.NetUtils;
|
||||
import org.apache.hadoop.security.Groups;
|
||||
import org.apache.hadoop.security.ShellBasedUnixGroupsMapping;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.security.authorize.AccessControlList;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.mockito.Mockito;
|
||||
import org.mockito.internal.util.reflection.Whitebox;
|
||||
import org.mortbay.jetty.Connector;
|
||||
import org.mortbay.util.ajax.JSON;
|
||||
|
||||
import static org.mockito.Mockito.*;
|
||||
|
||||
public class TestHttpServer extends HttpServerFunctionalTest {
|
||||
static final Log LOG = LogFactory.getLog(TestHttpServer.class);
|
||||
private static HttpServer2 server;
|
||||
|
@ -426,8 +422,9 @@ public class TestHttpServer extends HttpServerFunctionalTest {
|
|||
Mockito.doReturn(null).when(request).getParameterValues("dummy");
|
||||
RequestQuoter requestQuoter = new RequestQuoter(request);
|
||||
String[] parameterValues = requestQuoter.getParameterValues("dummy");
|
||||
Assert.assertEquals("It should return null "
|
||||
+ "when there are no values for the parameter", null, parameterValues);
|
||||
Assert.assertNull(
|
||||
"It should return null " + "when there are no values for the parameter",
|
||||
parameterValues);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -547,8 +544,7 @@ public class TestHttpServer extends HttpServerFunctionalTest {
|
|||
// not bound, ephemeral should return requested port (0 for ephemeral)
|
||||
List<?> listeners = (List<?>) Whitebox.getInternalState(server,
|
||||
"listeners");
|
||||
Connector listener = (Connector) Whitebox.getInternalState(
|
||||
listeners.get(0), "listener");
|
||||
Connector listener = (Connector) listeners.get(0);
|
||||
|
||||
assertEquals(port, listener.getPort());
|
||||
// verify hostname is what was given
|
||||
|
@ -582,16 +578,4 @@ public class TestHttpServer extends HttpServerFunctionalTest {
|
|||
assertNotNull(conn.getHeaderField("Date"));
|
||||
assertEquals(conn.getHeaderField("Expires"), conn.getHeaderField("Date"));
|
||||
}
|
||||
|
||||
/**
|
||||
* HTTPServer.Builder should proceed if a external connector is available.
|
||||
*/
|
||||
@Test
|
||||
public void testHttpServerBuilderWithExternalConnector() throws Exception {
|
||||
Connector c = mock(Connector.class);
|
||||
doReturn("localhost").when(c).getHost();
|
||||
HttpServer2 s = new HttpServer2.Builder().setName("test").setConnector(c)
|
||||
.build();
|
||||
s.stop();
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue