001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.http;
019
020import java.io.FileNotFoundException;
021import java.io.IOException;
022import java.io.InterruptedIOException;
023import java.io.PrintStream;
024import java.net.BindException;
025import java.net.InetSocketAddress;
026import java.net.URI;
027import java.net.URISyntaxException;
028import java.net.URL;
029import java.nio.file.Files;
030import java.nio.file.Path;
031import java.nio.file.Paths;
032import java.util.ArrayList;
033import java.util.Collections;
034import java.util.Enumeration;
035import java.util.HashMap;
036import java.util.List;
037import java.util.Map;
038import java.util.stream.Collectors;
039import javax.servlet.Filter;
040import javax.servlet.FilterChain;
041import javax.servlet.FilterConfig;
042import javax.servlet.ServletContext;
043import javax.servlet.ServletException;
044import javax.servlet.ServletRequest;
045import javax.servlet.ServletResponse;
046import javax.servlet.http.HttpServlet;
047import javax.servlet.http.HttpServletRequest;
048import javax.servlet.http.HttpServletRequestWrapper;
049import javax.servlet.http.HttpServletResponse;
050import org.apache.hadoop.HadoopIllegalArgumentException;
051import org.apache.hadoop.conf.Configuration;
052import org.apache.hadoop.fs.CommonConfigurationKeys;
053import org.apache.hadoop.hbase.HBaseInterfaceAudience;
054import org.apache.hadoop.hbase.http.conf.ConfServlet;
055import org.apache.hadoop.hbase.http.jmx.JMXJsonServlet;
056import org.apache.hadoop.hbase.http.log.LogLevel;
057import org.apache.hadoop.hbase.util.ReflectionUtils;
058import org.apache.hadoop.hbase.util.Threads;
059import org.apache.hadoop.security.SecurityUtil;
060import org.apache.hadoop.security.UserGroupInformation;
061import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
062import org.apache.hadoop.security.authorize.AccessControlList;
063import org.apache.hadoop.util.Shell;
064import org.apache.yetus.audience.InterfaceAudience;
065import org.apache.yetus.audience.InterfaceStability;
066import org.eclipse.jetty.http.HttpVersion;
067import org.eclipse.jetty.server.Handler;
068import org.eclipse.jetty.server.HttpConfiguration;
069import org.eclipse.jetty.server.HttpConnectionFactory;
070import org.eclipse.jetty.server.RequestLog;
071import org.eclipse.jetty.server.SecureRequestCustomizer;
072import org.eclipse.jetty.server.Server;
073import org.eclipse.jetty.server.ServerConnector;
074import org.eclipse.jetty.server.SslConnectionFactory;
075import org.eclipse.jetty.server.handler.ContextHandlerCollection;
076import org.eclipse.jetty.server.handler.HandlerCollection;
077import org.eclipse.jetty.server.handler.RequestLogHandler;
078import org.eclipse.jetty.servlet.DefaultServlet;
079import org.eclipse.jetty.servlet.FilterHolder;
080import org.eclipse.jetty.servlet.FilterMapping;
081import org.eclipse.jetty.servlet.ServletContextHandler;
082import org.eclipse.jetty.servlet.ServletHandler;
083import org.eclipse.jetty.servlet.ServletHolder;
084import org.eclipse.jetty.util.MultiException;
085import org.eclipse.jetty.util.ssl.SslContextFactory;
086import org.eclipse.jetty.util.thread.QueuedThreadPool;
087import org.eclipse.jetty.webapp.WebAppContext;
088import org.glassfish.jersey.server.ResourceConfig;
089import org.glassfish.jersey.servlet.ServletContainer;
090import org.slf4j.Logger;
091import org.slf4j.LoggerFactory;
092
093import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
094import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
095import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
096
097/**
098 * Create a Jetty embedded server to answer http requests. The primary goal
099 * is to serve up status information for the server.
100 * There are three contexts:
101 *   "/logs/" -> points to the log directory
102 *   "/static/" -> points to common static files (src/webapps/static)
103 *   "/" -> the jsp server code from (src/webapps/<name>)
104 */
105@InterfaceAudience.Private
106@InterfaceStability.Evolving
107public class HttpServer implements FilterContainer {
108  private static final Logger LOG = LoggerFactory.getLogger(HttpServer.class);
109  private static final String EMPTY_STRING = "";
110
111  private static final int DEFAULT_MAX_HEADER_SIZE = 64 * 1024; // 64K
112
113  static final String FILTER_INITIALIZERS_PROPERTY
114      = "hbase.http.filter.initializers";
115  static final String HTTP_MAX_THREADS = "hbase.http.max.threads";
116
117  public static final String HTTP_UI_AUTHENTICATION = "hbase.security.authentication.ui";
118  static final String HTTP_AUTHENTICATION_PREFIX = "hbase.security.authentication.";
119  static final String HTTP_SPNEGO_AUTHENTICATION_PREFIX = HTTP_AUTHENTICATION_PREFIX
120      + "spnego.";
121  static final String HTTP_SPNEGO_AUTHENTICATION_PRINCIPAL_SUFFIX = "kerberos.principal";
122  public static final String HTTP_SPNEGO_AUTHENTICATION_PRINCIPAL_KEY =
123      HTTP_SPNEGO_AUTHENTICATION_PREFIX + HTTP_SPNEGO_AUTHENTICATION_PRINCIPAL_SUFFIX;
124  static final String HTTP_SPNEGO_AUTHENTICATION_KEYTAB_SUFFIX = "kerberos.keytab";
125  public static final String HTTP_SPNEGO_AUTHENTICATION_KEYTAB_KEY =
126      HTTP_SPNEGO_AUTHENTICATION_PREFIX + HTTP_SPNEGO_AUTHENTICATION_KEYTAB_SUFFIX;
127  static final String HTTP_SPNEGO_AUTHENTICATION_KRB_NAME_SUFFIX = "kerberos.name.rules";
128  public static final String HTTP_SPNEGO_AUTHENTICATION_KRB_NAME_KEY =
129      HTTP_SPNEGO_AUTHENTICATION_PREFIX + HTTP_SPNEGO_AUTHENTICATION_KRB_NAME_SUFFIX;
130  static final String HTTP_AUTHENTICATION_SIGNATURE_SECRET_FILE_SUFFIX =
131      "signature.secret.file";
132  public static final String HTTP_AUTHENTICATION_SIGNATURE_SECRET_FILE_KEY =
133      HTTP_AUTHENTICATION_PREFIX + HTTP_AUTHENTICATION_SIGNATURE_SECRET_FILE_SUFFIX;
134
135  // The ServletContext attribute where the daemon Configuration
136  // gets stored.
137  public static final String CONF_CONTEXT_ATTRIBUTE = "hbase.conf";
138  public static final String ADMINS_ACL = "admins.acl";
139  public static final String BIND_ADDRESS = "bind.address";
140  public static final String SPNEGO_FILTER = "SpnegoFilter";
141  public static final String NO_CACHE_FILTER = "NoCacheFilter";
142  public static final String APP_DIR = "webapps";
143
144  private final AccessControlList adminsAcl;
145
146  protected final Server webServer;
147  protected String appDir;
148  protected String logDir;
149
150  private static class ListenerInfo {
151    /**
152     * Boolean flag to determine whether the HTTP server should clean up the
153     * listener in stop().
154     */
155    private final boolean isManaged;
156    private final ServerConnector listener;
157    private ListenerInfo(boolean isManaged, ServerConnector listener) {
158      this.isManaged = isManaged;
159      this.listener = listener;
160    }
161  }
162
163  private final List<ListenerInfo> listeners = Lists.newArrayList();
164
165  @VisibleForTesting
166  public List<ServerConnector> getServerConnectors() {
167    return listeners.stream().map(info -> info.listener).collect(Collectors.toList());
168  }
169
170  protected final WebAppContext webAppContext;
171  protected final boolean findPort;
172  protected final Map<ServletContextHandler, Boolean> defaultContexts = new HashMap<>();
173  protected final List<String> filterNames = new ArrayList<>();
174  static final String STATE_DESCRIPTION_ALIVE = " - alive";
175  static final String STATE_DESCRIPTION_NOT_LIVE = " - not live";
176
177  /**
178   * Class to construct instances of HTTP server with specific options.
179   */
180  public static class Builder {
181    private ArrayList<URI> endpoints = Lists.newArrayList();
182    private Configuration conf;
183    private String[] pathSpecs;
184    private AccessControlList adminsAcl;
185    private boolean securityEnabled = false;
186    private String usernameConfKey;
187    private String keytabConfKey;
188    private boolean needsClientAuth;
189
190    private String hostName;
191    private String appDir = APP_DIR;
192    private String logDir;
193    private boolean findPort;
194
195    private String trustStore;
196    private String trustStorePassword;
197    private String trustStoreType;
198
199    private String keyStore;
200    private String keyStorePassword;
201    private String keyStoreType;
202
203    // The -keypass option in keytool
204    private String keyPassword;
205
206    private String kerberosNameRulesKey;
207    private String signatureSecretFileKey;
208
209    @Deprecated
210    private String name;
211    @Deprecated
212    private String bindAddress;
213    @Deprecated
214    private int port = -1;
215
216    /**
217     * Add an endpoint that the HTTP server should listen to.
218     *
219     * @param endpoint
220     *          the endpoint of that the HTTP server should listen to. The
221     *          scheme specifies the protocol (i.e. HTTP / HTTPS), the host
222     *          specifies the binding address, and the port specifies the
223     *          listening port. Unspecified or zero port means that the server
224     *          can listen to any port.
225     */
226    public Builder addEndpoint(URI endpoint) {
227      endpoints.add(endpoint);
228      return this;
229    }
230
231    /**
232     * Set the hostname of the http server. The host name is used to resolve the
233     * _HOST field in Kerberos principals. The hostname of the first listener
234     * will be used if the name is unspecified.
235     */
236    public Builder hostName(String hostName) {
237      this.hostName = hostName;
238      return this;
239    }
240
241    public Builder trustStore(String location, String password, String type) {
242      this.trustStore = location;
243      this.trustStorePassword = password;
244      this.trustStoreType = type;
245      return this;
246    }
247
248    public Builder keyStore(String location, String password, String type) {
249      this.keyStore = location;
250      this.keyStorePassword = password;
251      this.keyStoreType = type;
252      return this;
253    }
254
255    public Builder keyPassword(String password) {
256      this.keyPassword = password;
257      return this;
258    }
259
260    /**
261     * Specify whether the server should authorize the client in SSL
262     * connections.
263     */
264    public Builder needsClientAuth(boolean value) {
265      this.needsClientAuth = value;
266      return this;
267    }
268
269    /**
270     * Use setAppDir() instead.
271     */
272    @Deprecated
273    public Builder setName(String name){
274      this.name = name;
275      return this;
276    }
277
278    /**
279     * Use addEndpoint() instead.
280     */
281    @Deprecated
282    public Builder setBindAddress(String bindAddress){
283      this.bindAddress = bindAddress;
284      return this;
285    }
286
287    /**
288     * Use addEndpoint() instead.
289     */
290    @Deprecated
291    public Builder setPort(int port) {
292      this.port = port;
293      return this;
294    }
295
296    public Builder setFindPort(boolean findPort) {
297      this.findPort = findPort;
298      return this;
299    }
300
301    public Builder setConf(Configuration conf) {
302      this.conf = conf;
303      return this;
304    }
305
306    public Builder setPathSpec(String[] pathSpec) {
307      this.pathSpecs = pathSpec;
308      return this;
309    }
310
311    public Builder setACL(AccessControlList acl) {
312      this.adminsAcl = acl;
313      return this;
314    }
315
316    public Builder setSecurityEnabled(boolean securityEnabled) {
317      this.securityEnabled = securityEnabled;
318      return this;
319    }
320
321    public Builder setUsernameConfKey(String usernameConfKey) {
322      this.usernameConfKey = usernameConfKey;
323      return this;
324    }
325
326    public Builder setKeytabConfKey(String keytabConfKey) {
327      this.keytabConfKey = keytabConfKey;
328      return this;
329    }
330
331    public Builder setKerberosNameRulesKey(String kerberosNameRulesKey) {
332      this.kerberosNameRulesKey = kerberosNameRulesKey;
333      return this;
334    }
335
336    public Builder setSignatureSecretFileKey(String signatureSecretFileKey) {
337      this.signatureSecretFileKey = signatureSecretFileKey;
338      return this;
339    }
340
341    public Builder setAppDir(String appDir) {
342        this.appDir = appDir;
343        return this;
344      }
345
346    public Builder setLogDir(String logDir) {
347        this.logDir = logDir;
348        return this;
349      }
350
351    public HttpServer build() throws IOException {
352
353      // Do we still need to assert this non null name if it is deprecated?
354      if (this.name == null) {
355        throw new HadoopIllegalArgumentException("name is not set");
356      }
357
358      // Make the behavior compatible with deprecated interfaces
359      if (bindAddress != null && port != -1) {
360        try {
361          endpoints.add(0, new URI("http", "", bindAddress, port, "", "", ""));
362        } catch (URISyntaxException e) {
363          throw new HadoopIllegalArgumentException("Invalid endpoint: "+ e); }
364      }
365
366      if (endpoints.isEmpty()) {
367        throw new HadoopIllegalArgumentException("No endpoints specified");
368      }
369
370      if (hostName == null) {
371        hostName = endpoints.get(0).getHost();
372      }
373
374      if (this.conf == null) {
375        conf = new Configuration();
376      }
377
378      HttpServer server = new HttpServer(this);
379
380      if (this.securityEnabled) {
381        server.initSpnego(conf, hostName, usernameConfKey, keytabConfKey, kerberosNameRulesKey,
382            signatureSecretFileKey);
383      }
384
385      for (URI ep : endpoints) {
386        ServerConnector listener = null;
387        String scheme = ep.getScheme();
388        HttpConfiguration httpConfig = new HttpConfiguration();
389        httpConfig.setSecureScheme("https");
390        httpConfig.setHeaderCacheSize(DEFAULT_MAX_HEADER_SIZE);
391        httpConfig.setResponseHeaderSize(DEFAULT_MAX_HEADER_SIZE);
392        httpConfig.setRequestHeaderSize(DEFAULT_MAX_HEADER_SIZE);
393
394        if ("http".equals(scheme)) {
395          listener = new ServerConnector(server.webServer, new HttpConnectionFactory(httpConfig));
396        } else if ("https".equals(scheme)) {
397          HttpConfiguration httpsConfig = new HttpConfiguration(httpConfig);
398          httpsConfig.addCustomizer(new SecureRequestCustomizer());
399          SslContextFactory sslCtxFactory = new SslContextFactory();
400          sslCtxFactory.setNeedClientAuth(needsClientAuth);
401          sslCtxFactory.setKeyManagerPassword(keyPassword);
402
403          if (keyStore != null) {
404            sslCtxFactory.setKeyStorePath(keyStore);
405            sslCtxFactory.setKeyStoreType(keyStoreType);
406            sslCtxFactory.setKeyStorePassword(keyStorePassword);
407          }
408
409          if (trustStore != null) {
410            sslCtxFactory.setTrustStorePath(trustStore);
411            sslCtxFactory.setTrustStoreType(trustStoreType);
412            sslCtxFactory.setTrustStorePassword(trustStorePassword);
413
414          }
415          listener = new ServerConnector(server.webServer, new SslConnectionFactory(sslCtxFactory,
416              HttpVersion.HTTP_1_1.toString()), new HttpConnectionFactory(httpsConfig));
417        } else {
418          throw new HadoopIllegalArgumentException(
419              "unknown scheme for endpoint:" + ep);
420        }
421
422        // default settings for connector
423        listener.setAcceptQueueSize(128);
424        if (Shell.WINDOWS) {
425          // result of setting the SO_REUSEADDR flag is different on Windows
426          // http://msdn.microsoft.com/en-us/library/ms740621(v=vs.85).aspx
427          // without this 2 NN's can start on the same machine and listen on
428          // the same port with indeterminate routing of incoming requests to them
429          listener.setReuseAddress(false);
430        }
431
432        listener.setHost(ep.getHost());
433        listener.setPort(ep.getPort() == -1 ? 0 : ep.getPort());
434        server.addManagedListener(listener);
435      }
436
437      server.loadListeners();
438      return server;
439
440    }
441
442  }
443
444  /** Same as this(name, bindAddress, port, findPort, null); */
445  @Deprecated
446  public HttpServer(String name, String bindAddress, int port, boolean findPort
447      ) throws IOException {
448    this(name, bindAddress, port, findPort, new Configuration());
449  }
450
451  /**
452   * Create a status server on the given port. Allows you to specify the
453   * path specifications that this server will be serving so that they will be
454   * added to the filters properly.
455   *
456   * @param name The name of the server
457   * @param bindAddress The address for this server
458   * @param port The port to use on the server
459   * @param findPort whether the server should start at the given port and
460   *        increment by 1 until it finds a free port.
461   * @param conf Configuration
462   * @param pathSpecs Path specifications that this httpserver will be serving.
463   *        These will be added to any filters.
464   */
465  @Deprecated
466  public HttpServer(String name, String bindAddress, int port,
467      boolean findPort, Configuration conf, String[] pathSpecs) throws IOException {
468    this(name, bindAddress, port, findPort, conf, null, pathSpecs);
469  }
470
471  /**
472   * Create a status server on the given port.
473   * The jsp scripts are taken from src/webapps/&lt;name&gt;.
474   * @param name The name of the server
475   * @param port The port to use on the server
476   * @param findPort whether the server should start at the given port and
477   *        increment by 1 until it finds a free port.
478   * @param conf Configuration
479   */
480  @Deprecated
481  public HttpServer(String name, String bindAddress, int port,
482      boolean findPort, Configuration conf) throws IOException {
483    this(name, bindAddress, port, findPort, conf, null, null);
484  }
485
486  @Deprecated
487  public HttpServer(String name, String bindAddress, int port,
488      boolean findPort, Configuration conf, AccessControlList adminsAcl)
489      throws IOException {
490    this(name, bindAddress, port, findPort, conf, adminsAcl, null);
491  }
492
493  /**
494   * Create a status server on the given port.
495   * The jsp scripts are taken from src/webapps/&lt;name&gt;.
496   * @param name The name of the server
497   * @param bindAddress The address for this server
498   * @param port The port to use on the server
499   * @param findPort whether the server should start at the given port and
500   *        increment by 1 until it finds a free port.
501   * @param conf Configuration
502   * @param adminsAcl {@link AccessControlList} of the admins
503   * @param pathSpecs Path specifications that this httpserver will be serving.
504   *        These will be added to any filters.
505   */
506  @Deprecated
507  public HttpServer(String name, String bindAddress, int port,
508      boolean findPort, Configuration conf, AccessControlList adminsAcl,
509      String[] pathSpecs) throws IOException {
510    this(new Builder().setName(name)
511        .addEndpoint(URI.create("http://" + bindAddress + ":" + port))
512        .setFindPort(findPort).setConf(conf).setACL(adminsAcl)
513        .setPathSpec(pathSpecs));
514  }
515
516  private HttpServer(final Builder b) throws IOException {
517    this.appDir = b.appDir;
518    this.logDir = b.logDir;
519    final String appDir = getWebAppsPath(b.name);
520
521
522    int maxThreads = b.conf.getInt(HTTP_MAX_THREADS, 16);
523    // If HTTP_MAX_THREADS is less than or equal to 0, QueueThreadPool() will use the
524    // default value (currently 200).
525    QueuedThreadPool threadPool = maxThreads <= 0 ? new QueuedThreadPool()
526        : new QueuedThreadPool(maxThreads);
527    threadPool.setDaemon(true);
528    this.webServer = new Server(threadPool);
529
530    this.adminsAcl = b.adminsAcl;
531    this.webAppContext = createWebAppContext(b.name, b.conf, adminsAcl, appDir);
532    this.findPort = b.findPort;
533    initializeWebServer(b.name, b.hostName, b.conf, b.pathSpecs);
534  }
535
536  private void initializeWebServer(String name, String hostName,
537      Configuration conf, String[] pathSpecs)
538      throws FileNotFoundException, IOException {
539
540    Preconditions.checkNotNull(webAppContext);
541
542    HandlerCollection handlerCollection = new HandlerCollection();
543
544    ContextHandlerCollection contexts = new ContextHandlerCollection();
545    RequestLog requestLog = HttpRequestLog.getRequestLog(name);
546
547    if (requestLog != null) {
548      RequestLogHandler requestLogHandler = new RequestLogHandler();
549      requestLogHandler.setRequestLog(requestLog);
550      handlerCollection.addHandler(requestLogHandler);
551    }
552
553    final String appDir = getWebAppsPath(name);
554
555    handlerCollection.addHandler(contexts);
556    handlerCollection.addHandler(webAppContext);
557
558    webServer.setHandler(handlerCollection);
559
560    addDefaultApps(contexts, appDir, conf);
561
562    addGlobalFilter("safety", QuotingInputFilter.class.getName(), null);
563
564    addGlobalFilter("clickjackingprevention",
565        ClickjackingPreventionFilter.class.getName(),
566        ClickjackingPreventionFilter.getDefaultParameters(conf));
567
568    addGlobalFilter("securityheaders",
569        SecurityHeadersFilter.class.getName(),
570        SecurityHeadersFilter.getDefaultParameters(conf));
571
572    final FilterInitializer[] initializers = getFilterInitializers(conf);
573    if (initializers != null) {
574      conf = new Configuration(conf);
575      conf.set(BIND_ADDRESS, hostName);
576      for (FilterInitializer c : initializers) {
577        c.initFilter(this, conf);
578      }
579    }
580
581    addDefaultServlets(contexts);
582
583    if (pathSpecs != null) {
584      for (String path : pathSpecs) {
585        LOG.info("adding path spec: " + path);
586        addFilterPathMapping(path, webAppContext);
587      }
588    }
589  }
590
591  private void addManagedListener(ServerConnector connector) {
592    listeners.add(new ListenerInfo(true, connector));
593  }
594
595  private static WebAppContext createWebAppContext(String name,
596      Configuration conf, AccessControlList adminsAcl, final String appDir) {
597    WebAppContext ctx = new WebAppContext();
598    ctx.setDisplayName(name);
599    ctx.setContextPath("/");
600    ctx.setWar(appDir + "/" + name);
601    ctx.getServletContext().setAttribute(CONF_CONTEXT_ATTRIBUTE, conf);
602    // for org.apache.hadoop.metrics.MetricsServlet
603    ctx.getServletContext().setAttribute(
604      org.apache.hadoop.http.HttpServer2.CONF_CONTEXT_ATTRIBUTE, conf);
605    ctx.getServletContext().setAttribute(ADMINS_ACL, adminsAcl);
606    addNoCacheFilter(ctx);
607    return ctx;
608  }
609
610  private static void addNoCacheFilter(WebAppContext ctxt) {
611    defineFilter(ctxt, NO_CACHE_FILTER, NoCacheFilter.class.getName(),
612        Collections.<String, String> emptyMap(), new String[] { "/*" });
613  }
614
615  /** Get an array of FilterConfiguration specified in the conf */
616  private static FilterInitializer[] getFilterInitializers(Configuration conf) {
617    if (conf == null) {
618      return null;
619    }
620
621    Class<?>[] classes = conf.getClasses(FILTER_INITIALIZERS_PROPERTY);
622    if (classes == null) {
623      return null;
624    }
625
626    FilterInitializer[] initializers = new FilterInitializer[classes.length];
627    for(int i = 0; i < classes.length; i++) {
628      initializers[i] = (FilterInitializer)ReflectionUtils.newInstance(classes[i]);
629    }
630    return initializers;
631  }
632
633  /**
634   * Add default apps.
635   * @param appDir The application directory
636   * @throws IOException
637   */
638  protected void addDefaultApps(ContextHandlerCollection parent,
639      final String appDir, Configuration conf) throws IOException {
640    // set up the context for "/logs/" if "hadoop.log.dir" property is defined.
641    String logDir = this.logDir;
642    if (logDir == null) {
643        logDir = System.getProperty("hadoop.log.dir");
644    }
645    if (logDir != null) {
646      ServletContextHandler logContext = new ServletContextHandler(parent, "/logs");
647      logContext.addServlet(AdminAuthorizedServlet.class, "/*");
648      logContext.setResourceBase(logDir);
649
650      if (conf.getBoolean(
651          ServerConfigurationKeys.HBASE_JETTY_LOGS_SERVE_ALIASES,
652          ServerConfigurationKeys.DEFAULT_HBASE_JETTY_LOGS_SERVE_ALIASES)) {
653        Map<String, String> params = logContext.getInitParams();
654        params.put(
655            "org.mortbay.jetty.servlet.Default.aliases", "true");
656      }
657      logContext.setDisplayName("logs");
658      setContextAttributes(logContext, conf);
659      addNoCacheFilter(webAppContext);
660      defaultContexts.put(logContext, true);
661    }
662    // set up the context for "/static/*"
663    ServletContextHandler staticContext = new ServletContextHandler(parent, "/static");
664    staticContext.setResourceBase(appDir + "/static");
665    staticContext.addServlet(DefaultServlet.class, "/*");
666    staticContext.setDisplayName("static");
667    setContextAttributes(staticContext, conf);
668    defaultContexts.put(staticContext, true);
669  }
670
671  private void setContextAttributes(ServletContextHandler context, Configuration conf) {
672    context.getServletContext().setAttribute(CONF_CONTEXT_ATTRIBUTE, conf);
673    context.getServletContext().setAttribute(ADMINS_ACL, adminsAcl);
674  }
675
676  /**
677   * Add default servlets.
678   */
679  protected void addDefaultServlets(ContextHandlerCollection contexts) throws IOException {
680    // set up default servlets
681    addServlet("stacks", "/stacks", StackServlet.class);
682    addServlet("logLevel", "/logLevel", LogLevel.Servlet.class);
683    // Hadoop3 has moved completely to metrics2, and  dropped support for Metrics v1's
684    // MetricsServlet (see HADOOP-12504).  We'll using reflection to load if against hadoop2.
685    // Remove when we drop support for hbase on hadoop2.x.
686    try {
687      Class clz = Class.forName("org.apache.hadoop.metrics.MetricsServlet");
688      addServlet("metrics", "/metrics", clz);
689    } catch (Exception e) {
690      // do nothing
691    }
692    addServlet("jmx", "/jmx", JMXJsonServlet.class);
693    addServlet("conf", "/conf", ConfServlet.class);
694    final String asyncProfilerHome = ProfileServlet.getAsyncProfilerHome();
695    if (asyncProfilerHome != null && !asyncProfilerHome.trim().isEmpty()) {
696      addServlet("prof", "/prof", ProfileServlet.class);
697      Path tmpDir = Paths.get(ProfileServlet.OUTPUT_DIR);
698      if (Files.notExists(tmpDir)) {
699        Files.createDirectories(tmpDir);
700      }
701      ServletContextHandler genCtx = new ServletContextHandler(contexts, "/prof-output");
702      genCtx.addServlet(ProfileOutputServlet.class, "/*");
703      genCtx.setResourceBase(tmpDir.toAbsolutePath().toString());
704      genCtx.setDisplayName("prof-output");
705    } else {
706      addServlet("prof", "/prof", ProfileServlet.DisabledServlet.class);
707      LOG.info("ASYNC_PROFILER_HOME environment variable and async.profiler.home system property " +
708        "not specified. Disabling /prof endpoint.");
709    }
710  }
711
712  /**
713   * Set a value in the webapp context. These values are available to the jsp
714   * pages as "application.getAttribute(name)".
715   * @param name The name of the attribute
716   * @param value The value of the attribute
717   */
718  public void setAttribute(String name, Object value) {
719    webAppContext.setAttribute(name, value);
720  }
721
722  /**
723   * Add a Jersey resource package.
724   * @param packageName The Java package name containing the Jersey resource.
725   * @param pathSpec The path spec for the servlet
726   */
727  public void addJerseyResourcePackage(final String packageName,
728      final String pathSpec) {
729    LOG.info("addJerseyResourcePackage: packageName=" + packageName
730        + ", pathSpec=" + pathSpec);
731
732    ResourceConfig application = new ResourceConfig().packages(packageName);
733    final ServletHolder sh = new ServletHolder(new ServletContainer(application));
734    webAppContext.addServlet(sh, pathSpec);
735  }
736
737  /**
738   * Add a servlet in the server.
739   * @param name The name of the servlet (can be passed as null)
740   * @param pathSpec The path spec for the servlet
741   * @param clazz The servlet class
742   */
743  public void addServlet(String name, String pathSpec,
744      Class<? extends HttpServlet> clazz) {
745    addInternalServlet(name, pathSpec, clazz, false);
746    addFilterPathMapping(pathSpec, webAppContext);
747  }
748
749  /**
750   * Add an internal servlet in the server.
751   * Note: This method is to be used for adding servlets that facilitate
752   * internal communication and not for user facing functionality. For
753   * servlets added using this method, filters are not enabled.
754   *
755   * @param name The name of the servlet (can be passed as null)
756   * @param pathSpec The path spec for the servlet
757   * @param clazz The servlet class
758   */
759  public void addInternalServlet(String name, String pathSpec,
760      Class<? extends HttpServlet> clazz) {
761    addInternalServlet(name, pathSpec, clazz, false);
762  }
763
764  /**
765   * Add an internal servlet in the server, specifying whether or not to
766   * protect with Kerberos authentication.
767   * Note: This method is to be used for adding servlets that facilitate
768   * internal communication and not for user facing functionality. For
769   +   * servlets added using this method, filters (except internal Kerberos
770   * filters) are not enabled.
771   *
772   * @param name The name of the servlet (can be passed as null)
773   * @param pathSpec The path spec for the servlet
774   * @param clazz The servlet class
775   * @param requireAuth Require Kerberos authenticate to access servlet
776   */
777  public void addInternalServlet(String name, String pathSpec,
778      Class<? extends HttpServlet> clazz, boolean requireAuth) {
779    ServletHolder holder = new ServletHolder(clazz);
780    if (name != null) {
781      holder.setName(name);
782    }
783    webAppContext.addServlet(holder, pathSpec);
784
785    if(requireAuth && UserGroupInformation.isSecurityEnabled()) {
786       LOG.info("Adding Kerberos (SPNEGO) filter to " + name);
787       ServletHandler handler = webAppContext.getServletHandler();
788       FilterMapping fmap = new FilterMapping();
789       fmap.setPathSpec(pathSpec);
790       fmap.setFilterName(SPNEGO_FILTER);
791       fmap.setDispatches(FilterMapping.ALL);
792       handler.addFilterMapping(fmap);
793    }
794  }
795
796  @Override
797  public void addFilter(String name, String classname,
798      Map<String, String> parameters) {
799
800    final String[] USER_FACING_URLS = { "*.html", "*.jsp" };
801    defineFilter(webAppContext, name, classname, parameters, USER_FACING_URLS);
802    LOG.info("Added filter " + name + " (class=" + classname
803        + ") to context " + webAppContext.getDisplayName());
804    final String[] ALL_URLS = { "/*" };
805    for (Map.Entry<ServletContextHandler, Boolean> e : defaultContexts.entrySet()) {
806      if (e.getValue()) {
807        ServletContextHandler handler = e.getKey();
808        defineFilter(handler, name, classname, parameters, ALL_URLS);
809        LOG.info("Added filter " + name + " (class=" + classname
810            + ") to context " + handler.getDisplayName());
811      }
812    }
813    filterNames.add(name);
814  }
815
816  @Override
817  public void addGlobalFilter(String name, String classname,
818      Map<String, String> parameters) {
819    final String[] ALL_URLS = { "/*" };
820    defineFilter(webAppContext, name, classname, parameters, ALL_URLS);
821    for (ServletContextHandler ctx : defaultContexts.keySet()) {
822      defineFilter(ctx, name, classname, parameters, ALL_URLS);
823    }
824    LOG.info("Added global filter '" + name + "' (class=" + classname + ")");
825  }
826
827  /**
828   * Define a filter for a context and set up default url mappings.
829   */
830  public static void defineFilter(ServletContextHandler handler, String name,
831      String classname, Map<String,String> parameters, String[] urls) {
832
833    FilterHolder holder = new FilterHolder();
834    holder.setName(name);
835    holder.setClassName(classname);
836    if (parameters != null) {
837      holder.setInitParameters(parameters);
838    }
839    FilterMapping fmap = new FilterMapping();
840    fmap.setPathSpecs(urls);
841    fmap.setDispatches(FilterMapping.ALL);
842    fmap.setFilterName(name);
843    handler.getServletHandler().addFilter(holder, fmap);
844  }
845
846  /**
847   * Add the path spec to the filter path mapping.
848   * @param pathSpec The path spec
849   * @param webAppCtx The WebApplicationContext to add to
850   */
851  protected void addFilterPathMapping(String pathSpec,
852      WebAppContext webAppCtx) {
853    for(String name : filterNames) {
854      FilterMapping fmap = new FilterMapping();
855      fmap.setPathSpec(pathSpec);
856      fmap.setFilterName(name);
857      fmap.setDispatches(FilterMapping.ALL);
858      webAppCtx.getServletHandler().addFilterMapping(fmap);
859    }
860  }
861
862  /**
863   * Get the value in the webapp context.
864   * @param name The name of the attribute
865   * @return The value of the attribute
866   */
867  public Object getAttribute(String name) {
868    return webAppContext.getAttribute(name);
869  }
870
871  public WebAppContext getWebAppContext(){
872    return this.webAppContext;
873  }
874
875  public String getWebAppsPath(String appName) throws FileNotFoundException {
876      return getWebAppsPath(this.appDir, appName);
877  }
878
879  /**
880   * Get the pathname to the webapps files.
881   * @param appName eg "secondary" or "datanode"
882   * @return the pathname as a URL
883   * @throws FileNotFoundException if 'webapps' directory cannot be found on CLASSPATH.
884   */
885  protected String getWebAppsPath(String webapps, String appName) throws FileNotFoundException {
886    URL url = getClass().getClassLoader().getResource(webapps + "/" + appName);
887    if (url == null)
888      throw new FileNotFoundException(webapps + "/" + appName
889          + " not found in CLASSPATH");
890    String urlString = url.toString();
891    return urlString.substring(0, urlString.lastIndexOf('/'));
892  }
893
894  /**
895   * Get the port that the server is on
896   * @return the port
897   */
898  @Deprecated
899  public int getPort() {
900    return ((ServerConnector)webServer.getConnectors()[0]).getLocalPort();
901  }
902
903  /**
904   * Get the address that corresponds to a particular connector.
905   *
906   * @return the corresponding address for the connector, or null if there's no
907   *         such connector or the connector is not bounded.
908   */
909  public InetSocketAddress getConnectorAddress(int index) {
910    Preconditions.checkArgument(index >= 0);
911    if (index > webServer.getConnectors().length)
912      return null;
913
914    ServerConnector c = (ServerConnector)webServer.getConnectors()[index];
915    if (c.getLocalPort() == -1 || c.getLocalPort() == -2) {
916      // -1 if the connector has not been opened
917      // -2 if it has been closed
918      return null;
919    }
920
921    return new InetSocketAddress(c.getHost(), c.getLocalPort());
922  }
923
924  /**
925   * Set the min, max number of worker threads (simultaneous connections).
926   */
927  public void setThreads(int min, int max) {
928    QueuedThreadPool pool = (QueuedThreadPool) webServer.getThreadPool();
929    pool.setMinThreads(min);
930    pool.setMaxThreads(max);
931  }
932
933  private void initSpnego(Configuration conf, String hostName,
934      String usernameConfKey, String keytabConfKey, String kerberosNameRuleKey,
935      String signatureSecretKeyFileKey) throws IOException {
936    Map<String, String> params = new HashMap<>();
937    String principalInConf = getOrEmptyString(conf, usernameConfKey);
938    if (!principalInConf.isEmpty()) {
939      params.put(HTTP_SPNEGO_AUTHENTICATION_PRINCIPAL_SUFFIX, SecurityUtil.getServerPrincipal(
940          principalInConf, hostName));
941    }
942    String httpKeytab = getOrEmptyString(conf, keytabConfKey);
943    if (!httpKeytab.isEmpty()) {
944      params.put(HTTP_SPNEGO_AUTHENTICATION_KEYTAB_SUFFIX, httpKeytab);
945    }
946    String kerberosNameRule = getOrEmptyString(conf, kerberosNameRuleKey);
947    if (!kerberosNameRule.isEmpty()) {
948      params.put(HTTP_SPNEGO_AUTHENTICATION_KRB_NAME_SUFFIX, kerberosNameRule);
949    }
950    String signatureSecretKeyFile = getOrEmptyString(conf, signatureSecretKeyFileKey);
951    if (!signatureSecretKeyFile.isEmpty()) {
952      params.put(HTTP_AUTHENTICATION_SIGNATURE_SECRET_FILE_SUFFIX,
953          signatureSecretKeyFile);
954    }
955    params.put(AuthenticationFilter.AUTH_TYPE, "kerberos");
956
957    // Verify that the required options were provided
958    if (isMissing(params.get(HTTP_SPNEGO_AUTHENTICATION_PRINCIPAL_SUFFIX)) ||
959            isMissing(params.get(HTTP_SPNEGO_AUTHENTICATION_KEYTAB_SUFFIX))) {
960      throw new IllegalArgumentException(usernameConfKey + " and "
961          + keytabConfKey + " are both required in the configuration "
962          + "to enable SPNEGO/Kerberos authentication for the Web UI");
963    }
964
965    addGlobalFilter(SPNEGO_FILTER, AuthenticationFilter.class.getName(), params);
966  }
967
968  /**
969   * Returns true if the argument is non-null and not whitespace
970   */
971  private boolean isMissing(String value) {
972    if (null == value) {
973      return true;
974    }
975    return value.trim().isEmpty();
976  }
977
978  /**
979   * Extracts the value for the given key from the configuration of returns a string of
980   * zero length.
981   */
982  private String getOrEmptyString(Configuration conf, String key) {
983    if (null == key) {
984      return EMPTY_STRING;
985    }
986    final String value = conf.get(key.trim());
987    return null == value ? EMPTY_STRING : value;
988  }
989
990  /**
991   * Start the server. Does not wait for the server to start.
992   */
993  public void start() throws IOException {
994    try {
995      try {
996        openListeners();
997        webServer.start();
998      } catch (IOException ex) {
999        LOG.info("HttpServer.start() threw a non Bind IOException", ex);
1000        throw ex;
1001      } catch (MultiException ex) {
1002        LOG.info("HttpServer.start() threw a MultiException", ex);
1003        throw ex;
1004      }
1005      // Make sure there is no handler failures.
1006      Handler[] handlers = webServer.getHandlers();
1007      for (int i = 0; i < handlers.length; i++) {
1008        if (handlers[i].isFailed()) {
1009          throw new IOException(
1010              "Problem in starting http server. Server handlers failed");
1011        }
1012      }
1013      // Make sure there are no errors initializing the context.
1014      Throwable unavailableException = webAppContext.getUnavailableException();
1015      if (unavailableException != null) {
1016        // Have to stop the webserver, or else its non-daemon threads
1017        // will hang forever.
1018        webServer.stop();
1019        throw new IOException("Unable to initialize WebAppContext",
1020            unavailableException);
1021      }
1022    } catch (IOException e) {
1023      throw e;
1024    } catch (InterruptedException e) {
1025      throw (IOException) new InterruptedIOException(
1026          "Interrupted while starting HTTP server").initCause(e);
1027    } catch (Exception e) {
1028      throw new IOException("Problem starting http server", e);
1029    }
1030  }
1031
1032  private void loadListeners() {
1033    for (ListenerInfo li : listeners) {
1034      webServer.addConnector(li.listener);
1035    }
1036  }
1037
1038  /**
1039   * Open the main listener for the server
1040   * @throws Exception
1041   */
1042  @VisibleForTesting
1043  void openListeners() throws Exception {
1044    for (ListenerInfo li : listeners) {
1045      ServerConnector listener = li.listener;
1046      if (!li.isManaged || (li.listener.getLocalPort() != -1 && li.listener.getLocalPort() != -2)) {
1047        // This listener is either started externally, or has not been opened, or has been closed
1048        continue;
1049      }
1050      int port = listener.getPort();
1051      while (true) {
1052        // jetty has a bug where you can't reopen a listener that previously
1053        // failed to open w/o issuing a close first, even if the port is changed
1054        try {
1055          listener.close();
1056          listener.open();
1057          LOG.info("Jetty bound to port " + listener.getLocalPort());
1058          break;
1059        } catch (BindException ex) {
1060          if (port == 0 || !findPort) {
1061            BindException be = new BindException("Port in use: "
1062                + listener.getHost() + ":" + listener.getPort());
1063            be.initCause(ex);
1064            throw be;
1065          }
1066        }
1067        // try the next port number
1068        listener.setPort(++port);
1069        Thread.sleep(100);
1070      }
1071    }
1072  }
1073
1074  /**
1075   * stop the server
1076   */
1077  public void stop() throws Exception {
1078    MultiException exception = null;
1079    for (ListenerInfo li : listeners) {
1080      if (!li.isManaged) {
1081        continue;
1082      }
1083
1084      try {
1085        li.listener.close();
1086      } catch (Exception e) {
1087        LOG.error(
1088            "Error while stopping listener for webapp"
1089                + webAppContext.getDisplayName(), e);
1090        exception = addMultiException(exception, e);
1091      }
1092    }
1093
1094    try {
1095      // clear & stop webAppContext attributes to avoid memory leaks.
1096      webAppContext.clearAttributes();
1097      webAppContext.stop();
1098    } catch (Exception e) {
1099      LOG.error("Error while stopping web app context for webapp "
1100          + webAppContext.getDisplayName(), e);
1101      exception = addMultiException(exception, e);
1102    }
1103
1104    try {
1105      webServer.stop();
1106    } catch (Exception e) {
1107      LOG.error("Error while stopping web server for webapp "
1108          + webAppContext.getDisplayName(), e);
1109      exception = addMultiException(exception, e);
1110    }
1111
1112    if (exception != null) {
1113      exception.ifExceptionThrow();
1114    }
1115
1116  }
1117
1118  private MultiException addMultiException(MultiException exception, Exception e) {
1119    if(exception == null){
1120      exception = new MultiException();
1121    }
1122    exception.add(e);
1123    return exception;
1124  }
1125
1126  public void join() throws InterruptedException {
1127    webServer.join();
1128  }
1129
1130  /**
1131   * Test for the availability of the web server
1132   * @return true if the web server is started, false otherwise
1133   */
1134  public boolean isAlive() {
1135    return webServer != null && webServer.isStarted();
1136  }
1137
1138  /**
1139   * Return the host and port of the HttpServer, if live
1140   * @return the classname and any HTTP URL
1141   */
1142  @Override
1143  public String toString() {
1144    if (listeners.isEmpty()) {
1145      return "Inactive HttpServer";
1146    } else {
1147      StringBuilder sb = new StringBuilder("HttpServer (")
1148        .append(isAlive() ? STATE_DESCRIPTION_ALIVE : STATE_DESCRIPTION_NOT_LIVE).append("), listening at:");
1149      for (ListenerInfo li : listeners) {
1150        ServerConnector l = li.listener;
1151        sb.append(l.getHost()).append(":").append(l.getPort()).append("/,");
1152      }
1153      return sb.toString();
1154    }
1155  }
1156
1157  /**
1158   * Checks the user has privileges to access to instrumentation servlets.
1159   * <p>
1160   * If <code>hadoop.security.instrumentation.requires.admin</code> is set to FALSE
1161   * (default value) it always returns TRUE.
1162   * </p><p>
1163   * If <code>hadoop.security.instrumentation.requires.admin</code> is set to TRUE
1164   * it will check that if the current user is in the admin ACLS. If the user is
1165   * in the admin ACLs it returns TRUE, otherwise it returns FALSE.
1166   * </p>
1167   *
1168   * @param servletContext the servlet context.
1169   * @param request the servlet request.
1170   * @param response the servlet response.
1171   * @return TRUE/FALSE based on the logic decribed above.
1172   */
1173  public static boolean isInstrumentationAccessAllowed(
1174    ServletContext servletContext, HttpServletRequest request,
1175    HttpServletResponse response) throws IOException {
1176    Configuration conf =
1177      (Configuration) servletContext.getAttribute(CONF_CONTEXT_ATTRIBUTE);
1178
1179    boolean access = true;
1180    boolean adminAccess = conf.getBoolean(
1181      CommonConfigurationKeys.HADOOP_SECURITY_INSTRUMENTATION_REQUIRES_ADMIN,
1182      false);
1183    if (adminAccess) {
1184      access = hasAdministratorAccess(servletContext, request, response);
1185    }
1186    return access;
1187  }
1188
1189  /**
1190   * Does the user sending the HttpServletRequest has the administrator ACLs? If
1191   * it isn't the case, response will be modified to send an error to the user.
1192   *
1193   * @param servletContext
1194   * @param request
1195   * @param response used to send the error response if user does not have admin access.
1196   * @return true if admin-authorized, false otherwise
1197   * @throws IOException
1198   */
1199  public static boolean hasAdministratorAccess(
1200      ServletContext servletContext, HttpServletRequest request,
1201      HttpServletResponse response) throws IOException {
1202    Configuration conf =
1203        (Configuration) servletContext.getAttribute(CONF_CONTEXT_ATTRIBUTE);
1204    // If there is no authorization, anybody has administrator access.
1205    if (!conf.getBoolean(
1206        CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, false)) {
1207      return true;
1208    }
1209
1210    String remoteUser = request.getRemoteUser();
1211    if (remoteUser == null) {
1212      response.sendError(HttpServletResponse.SC_UNAUTHORIZED,
1213                         "Unauthenticated users are not " +
1214                         "authorized to access this page.");
1215      return false;
1216    }
1217
1218    if (servletContext.getAttribute(ADMINS_ACL) != null &&
1219        !userHasAdministratorAccess(servletContext, remoteUser)) {
1220      response.sendError(HttpServletResponse.SC_UNAUTHORIZED, "User "
1221          + remoteUser + " is unauthorized to access this page.");
1222      return false;
1223    }
1224
1225    return true;
1226  }
1227
1228  /**
1229   * Get the admin ACLs from the given ServletContext and check if the given
1230   * user is in the ACL.
1231   *
1232   * @param servletContext the context containing the admin ACL.
1233   * @param remoteUser the remote user to check for.
1234   * @return true if the user is present in the ACL, false if no ACL is set or
1235   *         the user is not present
1236   */
1237  public static boolean userHasAdministratorAccess(ServletContext servletContext,
1238      String remoteUser) {
1239    AccessControlList adminsAcl = (AccessControlList) servletContext
1240        .getAttribute(ADMINS_ACL);
1241    UserGroupInformation remoteUserUGI =
1242        UserGroupInformation.createRemoteUser(remoteUser);
1243    return adminsAcl != null && adminsAcl.isUserAllowed(remoteUserUGI);
1244  }
1245
1246  /**
1247   * A very simple servlet to serve up a text representation of the current
1248   * stack traces. It both returns the stacks to the caller and logs them.
1249   * Currently the stack traces are done sequentially rather than exactly the
1250   * same data.
1251   */
1252  public static class StackServlet extends HttpServlet {
1253    private static final long serialVersionUID = -6284183679759467039L;
1254
1255    @Override
1256    public void doGet(HttpServletRequest request, HttpServletResponse response)
1257      throws ServletException, IOException {
1258      if (!HttpServer.isInstrumentationAccessAllowed(getServletContext(),
1259                                                     request, response)) {
1260        return;
1261      }
1262      response.setContentType("text/plain; charset=UTF-8");
1263      try (PrintStream out = new PrintStream(
1264        response.getOutputStream(), false, "UTF-8")) {
1265        Threads.printThreadInfo(out, "");
1266        out.flush();
1267      }
1268      ReflectionUtils.logThreadInfo(LOG, "jsp requested", 1);
1269    }
1270  }
1271
1272  /**
1273   * A Servlet input filter that quotes all HTML active characters in the
1274   * parameter names and values. The goal is to quote the characters to make
1275   * all of the servlets resistant to cross-site scripting attacks.
1276   */
1277  @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
1278  public static class QuotingInputFilter implements Filter {
1279    private FilterConfig config;
1280
1281    public static class RequestQuoter extends HttpServletRequestWrapper {
1282      private final HttpServletRequest rawRequest;
1283      public RequestQuoter(HttpServletRequest rawRequest) {
1284        super(rawRequest);
1285        this.rawRequest = rawRequest;
1286      }
1287
1288      /**
1289       * Return the set of parameter names, quoting each name.
1290       */
1291      @Override
1292      public Enumeration<String> getParameterNames() {
1293        return new Enumeration<String>() {
1294          private Enumeration<String> rawIterator =
1295            rawRequest.getParameterNames();
1296          @Override
1297          public boolean hasMoreElements() {
1298            return rawIterator.hasMoreElements();
1299          }
1300
1301          @Override
1302          public String nextElement() {
1303            return HtmlQuoting.quoteHtmlChars(rawIterator.nextElement());
1304          }
1305        };
1306      }
1307
1308      /**
1309       * Unquote the name and quote the value.
1310       */
1311      @Override
1312      public String getParameter(String name) {
1313        return HtmlQuoting.quoteHtmlChars(rawRequest.getParameter
1314                                     (HtmlQuoting.unquoteHtmlChars(name)));
1315      }
1316
1317      @Override
1318      public String[] getParameterValues(String name) {
1319        String unquoteName = HtmlQuoting.unquoteHtmlChars(name);
1320        String[] unquoteValue = rawRequest.getParameterValues(unquoteName);
1321        if (unquoteValue == null) {
1322          return null;
1323        }
1324        String[] result = new String[unquoteValue.length];
1325        for(int i=0; i < result.length; ++i) {
1326          result[i] = HtmlQuoting.quoteHtmlChars(unquoteValue[i]);
1327        }
1328        return result;
1329      }
1330
1331      @Override
1332      public Map<String, String[]> getParameterMap() {
1333        Map<String, String[]> result = new HashMap<>();
1334        Map<String, String[]> raw = rawRequest.getParameterMap();
1335        for (Map.Entry<String,String[]> item: raw.entrySet()) {
1336          String[] rawValue = item.getValue();
1337          String[] cookedValue = new String[rawValue.length];
1338          for(int i=0; i< rawValue.length; ++i) {
1339            cookedValue[i] = HtmlQuoting.quoteHtmlChars(rawValue[i]);
1340          }
1341          result.put(HtmlQuoting.quoteHtmlChars(item.getKey()), cookedValue);
1342        }
1343        return result;
1344      }
1345
1346      /**
1347       * Quote the url so that users specifying the HOST HTTP header
1348       * can't inject attacks.
1349       */
1350      @Override
1351      public StringBuffer getRequestURL(){
1352        String url = rawRequest.getRequestURL().toString();
1353        return new StringBuffer(HtmlQuoting.quoteHtmlChars(url));
1354      }
1355
1356      /**
1357       * Quote the server name so that users specifying the HOST HTTP header
1358       * can't inject attacks.
1359       */
1360      @Override
1361      public String getServerName() {
1362        return HtmlQuoting.quoteHtmlChars(rawRequest.getServerName());
1363      }
1364    }
1365
1366    @Override
1367    public void init(FilterConfig config) throws ServletException {
1368      this.config = config;
1369    }
1370
1371    @Override
1372    public void destroy() {
1373    }
1374
1375    @Override
1376    public void doFilter(ServletRequest request,
1377                         ServletResponse response,
1378                         FilterChain chain
1379                         ) throws IOException, ServletException {
1380      HttpServletRequestWrapper quoted =
1381        new RequestQuoter((HttpServletRequest) request);
1382      HttpServletResponse httpResponse = (HttpServletResponse) response;
1383
1384      String mime = inferMimeType(request);
1385      if (mime == null) {
1386        httpResponse.setContentType("text/plain; charset=utf-8");
1387      } else if (mime.startsWith("text/html")) {
1388        // HTML with unspecified encoding, we want to
1389        // force HTML with utf-8 encoding
1390        // This is to avoid the following security issue:
1391        // http://openmya.hacker.jp/hasegawa/security/utf7cs.html
1392        httpResponse.setContentType("text/html; charset=utf-8");
1393      } else if (mime.startsWith("application/xml")) {
1394        httpResponse.setContentType("text/xml; charset=utf-8");
1395      }
1396      chain.doFilter(quoted, httpResponse);
1397    }
1398
1399    /**
1400     * Infer the mime type for the response based on the extension of the request
1401     * URI. Returns null if unknown.
1402     */
1403    private String inferMimeType(ServletRequest request) {
1404      String path = ((HttpServletRequest)request).getRequestURI();
1405      ServletContext context = config.getServletContext();
1406      return context.getMimeType(path);
1407    }
1408  }
1409}