Return-Path: X-Original-To: apmail-accumulo-commits-archive@www.apache.org Delivered-To: apmail-accumulo-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 3E4BB17B88 for ; Fri, 7 Nov 2014 23:31:16 +0000 (UTC) Received: (qmail 37412 invoked by uid 500); 7 Nov 2014 23:31:16 -0000 Delivered-To: apmail-accumulo-commits-archive@accumulo.apache.org Received: (qmail 37313 invoked by uid 500); 7 Nov 2014 23:31:16 -0000 Mailing-List: contact commits-help@accumulo.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@accumulo.apache.org Delivered-To: mailing list commits@accumulo.apache.org Received: (qmail 37120 invoked by uid 99); 7 Nov 2014 23:31:16 -0000 Received: from tyr.zones.apache.org (HELO tyr.zones.apache.org) (140.211.11.114) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 07 Nov 2014 23:31:16 +0000 Received: by tyr.zones.apache.org (Postfix, from userid 65534) id CB86099E29F; Fri, 7 Nov 2014 23:31:15 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: elserj@apache.org To: commits@accumulo.apache.org Date: Fri, 07 Nov 2014 23:31:19 -0000 Message-Id: In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [5/6] accumulo git commit: Merge branch '1.5' into 1.6 Merge branch '1.5' into 1.6 Conflicts: core/src/main/java/org/apache/accumulo/core/conf/Property.java server/base/src/main/java/org/apache/accumulo/server/Accumulo.java server/src/main/java/org/apache/accumulo/server/util/EmbeddedWebServer.java Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/9ea8f09a Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/9ea8f09a Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/9ea8f09a Branch: refs/heads/1.6 Commit: 9ea8f09a8d7ba28cb88f663aaaf008b469026af9 Parents: 4b87abd 7789b20 Author: Josh Elser Authored: Fri Nov 7 18:10:10 2014 -0500 Committer: Josh Elser Committed: Fri Nov 7 18:10:10 2014 -0500 ---------------------------------------------------------------------- .../java/org/apache/accumulo/core/conf/Property.java | 1 + .../main/java/org/apache/accumulo/server/Accumulo.java | 11 +++++++++++ .../org/apache/accumulo/monitor/EmbeddedWebServer.java | 5 +++++ 3 files changed, 17 insertions(+) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/accumulo/blob/9ea8f09a/core/src/main/java/org/apache/accumulo/core/conf/Property.java ---------------------------------------------------------------------- diff --cc core/src/main/java/org/apache/accumulo/core/conf/Property.java index 56f3d9c,1c0de78..45ef384 --- a/core/src/main/java/org/apache/accumulo/core/conf/Property.java +++ b/core/src/main/java/org/apache/accumulo/core/conf/Property.java @@@ -299,18 -234,12 +299,19 @@@ public enum Property MONITOR_BANNER_COLOR("monitor.banner.color", "#c4c4c4", PropertyType.STRING, "The color of the banner text displayed on the monitor page."), MONITOR_BANNER_BACKGROUND("monitor.banner.background", "#304065", PropertyType.STRING, "The background color of the banner text displayed on the monitor page."), - MONITOR_SSL_KEYSTORE("monitor.ssl.keyStore", "", PropertyType.PATH, "The keystore for enabling monitor SSL.", true, false), - MONITOR_SSL_KEYSTOREPASS("monitor.ssl.keyStorePassword", "", PropertyType.STRING, "The keystore password for enabling monitor SSL.", true, false), - MONITOR_SSL_TRUSTSTORE("monitor.ssl.trustStore", "", PropertyType.PATH, "The truststore for enabling monitor SSL.", true, false), - MONITOR_SSL_TRUSTSTOREPASS("monitor.ssl.trustStorePassword", "", PropertyType.STRING, "The truststore password for enabling monitor SSL.", true, false), + + MONITOR_SSL_KEYSTORE("monitor.ssl.keyStore", "", PropertyType.PATH, "The keystore for enabling monitor SSL."), + @Sensitive + MONITOR_SSL_KEYSTOREPASS("monitor.ssl.keyStorePassword", "", PropertyType.STRING, "The keystore password for enabling monitor SSL."), + MONITOR_SSL_TRUSTSTORE("monitor.ssl.trustStore", "", PropertyType.PATH, "The truststore for enabling monitor SSL."), + @Sensitive + MONITOR_SSL_TRUSTSTOREPASS("monitor.ssl.trustStorePassword", "", PropertyType.STRING, "The truststore password for enabling monitor SSL."), + MONITOR_SSL_INCLUDE_CIPHERS("monitor.ssl.include.ciphers", "", PropertyType.STRING, "A comma-separated list of allows SSL Ciphers, see monitor.ssl.exclude.ciphers to disallow ciphers"), + MONITOR_SSL_EXCLUDE_CIPHERS("monitor.ssl.exclude.ciphers", "", PropertyType.STRING, "A comma-separated list of disallowed SSL Ciphers, see mmonitor.ssl.include.ciphers to allow ciphers"), + MONITOR_SSL_INCLUDE_PROTOCOLS("monitor.ssl.include.protocols", "TLSv1,TLSv1.1,TLSv1.2", PropertyType.STRING, "A comma-separate list of allowed SSL protocols"), - MONITOR_LOCK_CHECK_INTERVAL("monitor.lock.check.interval", "5s", PropertyType.TIMEDURATION, "The amount of time to sleep between checking for the Montior ZooKeeper lock"), + + MONITOR_LOCK_CHECK_INTERVAL("monitor.lock.check.interval", "5s", PropertyType.TIMEDURATION, + "The amount of time to sleep between checking for the Montior ZooKeeper lock"), TRACE_PREFIX("trace.", null, PropertyType.PREFIX, "Properties in this category affect the behavior of distributed tracing."), TRACE_PORT("trace.port.client", "12234", PropertyType.PORT, "The listening port for the trace server"), http://git-wip-us.apache.org/repos/asf/accumulo/blob/9ea8f09a/server/base/src/main/java/org/apache/accumulo/server/Accumulo.java ---------------------------------------------------------------------- diff --cc server/base/src/main/java/org/apache/accumulo/server/Accumulo.java index 6c40094,0000000..a1581c4 mode 100644,000000..100644 --- a/server/base/src/main/java/org/apache/accumulo/server/Accumulo.java +++ b/server/base/src/main/java/org/apache/accumulo/server/Accumulo.java @@@ -1,331 -1,0 +1,342 @@@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.accumulo.server; + +import static com.google.common.base.Charsets.UTF_8; + +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.net.InetAddress; +import java.net.UnknownHostException; ++import java.util.Arrays; +import java.util.Map.Entry; +import java.util.TreeMap; + +import org.apache.accumulo.core.Constants; +import org.apache.accumulo.core.client.AccumuloException; +import org.apache.accumulo.core.client.Instance; +import org.apache.accumulo.core.conf.AccumuloConfiguration; +import org.apache.accumulo.core.conf.Property; +import org.apache.accumulo.core.trace.DistributedTrace; +import org.apache.accumulo.core.util.AddressUtil; +import org.apache.accumulo.core.util.UtilWaitThread; +import org.apache.accumulo.core.util.Version; +import org.apache.accumulo.core.volume.Volume; +import org.apache.accumulo.core.zookeeper.ZooUtil; +import org.apache.accumulo.fate.ReadOnlyStore; +import org.apache.accumulo.fate.ReadOnlyTStore; +import org.apache.accumulo.fate.ZooStore; +import org.apache.accumulo.server.client.HdfsZooInstance; +import org.apache.accumulo.server.conf.ServerConfiguration; +import org.apache.accumulo.server.fs.VolumeManager; +import org.apache.accumulo.server.util.time.SimpleTimer; +import org.apache.accumulo.server.watcher.Log4jConfiguration; +import org.apache.accumulo.server.watcher.MonitorLog4jWatcher; +import org.apache.accumulo.server.zookeeper.ZooReaderWriter; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.Logger; +import org.apache.log4j.helpers.LogLog; +import org.apache.log4j.xml.DOMConfigurator; +import org.apache.zookeeper.KeeperException; + +public class Accumulo { + + private static final Logger log = Logger.getLogger(Accumulo.class); + + public static synchronized void updateAccumuloVersion(VolumeManager fs, int oldVersion) { + for (Volume volume : fs.getVolumes()) { + try { + if (getAccumuloPersistentVersion(fs) == oldVersion) { + log.debug("Attempting to upgrade " + volume); + Path dataVersionLocation = ServerConstants.getDataVersionLocation(volume); + fs.create(new Path(dataVersionLocation, Integer.toString(ServerConstants.DATA_VERSION))).close(); + // TODO document failure mode & recovery if FS permissions cause above to work and below to fail ACCUMULO-2596 + Path prevDataVersionLoc = new Path(dataVersionLocation, Integer.toString(oldVersion)); + if (!fs.delete(prevDataVersionLoc)) { + throw new RuntimeException("Could not delete previous data version location (" + prevDataVersionLoc + ") for " + volume); + } + } + } catch (IOException e) { + throw new RuntimeException("Unable to set accumulo version: an error occurred.", e); + } + } + } + + public static synchronized int getAccumuloPersistentVersion(FileSystem fs, Path path) { + int dataVersion; + try { + FileStatus[] files = fs.listStatus(path); + if (files == null || files.length == 0) { + dataVersion = -1; // assume it is 0.5 or earlier + } else { + dataVersion = Integer.parseInt(files[0].getPath().getName()); + } + return dataVersion; + } catch (IOException e) { + throw new RuntimeException("Unable to read accumulo version: an error occurred.", e); + } + } + + public static synchronized int getAccumuloPersistentVersion(VolumeManager fs) { + // It doesn't matter which Volume is used as they should all have the data version stored + Volume v = fs.getVolumes().iterator().next(); + Path path = ServerConstants.getDataVersionLocation(v); + return getAccumuloPersistentVersion(v.getFileSystem(), path); + } + + public static synchronized Path getAccumuloInstanceIdPath(VolumeManager fs) { + // It doesn't matter which Volume is used as they should all have the instance ID stored + Volume v = fs.getVolumes().iterator().next(); + return ServerConstants.getInstanceIdLocation(v); + } + + public static void enableTracing(String address, String application) { + try { + DistributedTrace.enable(HdfsZooInstance.getInstance(), ZooReaderWriter.getInstance(), application, address); + } catch (Exception ex) { + log.error("creating remote sink for trace spans", ex); + } + } + + /** + * Finds the best log4j configuration file. A generic file is used only if an + * application-specific file is not available. An XML file is preferred over + * a properties file, if possible. + * + * @param confDir directory where configuration files should reside + * @param application application name for configuration file name + * @return configuration file name + */ + static String locateLogConfig(String confDir, String application) { + String explicitConfigFile = System.getProperty("log4j.configuration"); + if (explicitConfigFile != null) { + return explicitConfigFile; + } + String[] configFiles = { + String.format("%s/%s_logger.xml", confDir, application), + String.format("%s/%s_logger.properties", confDir, application), + String.format("%s/generic_logger.xml", confDir), + String.format("%s/generic_logger.properties", confDir) + }; + String defaultConfigFile = configFiles[2]; // generic_logger.xml + for (String f : configFiles) { + if (new File(f).exists()) { + return f; + } + } + return defaultConfigFile; + } + + public static void setupLogging(String application) throws UnknownHostException { + System.setProperty("org.apache.accumulo.core.application", application); + + if (System.getenv("ACCUMULO_LOG_DIR") != null) + System.setProperty("org.apache.accumulo.core.dir.log", System.getenv("ACCUMULO_LOG_DIR")); + else + System.setProperty("org.apache.accumulo.core.dir.log", System.getenv("ACCUMULO_HOME") + "/logs/"); + + String localhost = InetAddress.getLocalHost().getHostName(); + System.setProperty("org.apache.accumulo.core.ip.localhost.hostname", localhost); + + // Use a specific log config, if it exists + String logConfigFile = locateLogConfig(System.getenv("ACCUMULO_CONF_DIR"), application); + // Turn off messages about not being able to reach the remote logger... we protect against that. + LogLog.setQuietMode(true); + + // Read the auditing config + String auditConfig = String.format("%s/auditLog.xml", System.getenv("ACCUMULO_CONF_DIR")); + + // Set up local file-based logging right away + Log4jConfiguration logConf = new Log4jConfiguration(logConfigFile); + logConf.resetLogger(); + + // Watch the auditLog.xml for the future updates + DOMConfigurator.configureAndWatch(auditConfig, 5000); + } + + public static void init(VolumeManager fs, ServerConfiguration serverConfig, String application) throws IOException { + final AccumuloConfiguration conf = serverConfig.getConfiguration(); + final Instance instance = serverConfig.getInstance(); + + // Use a specific log config, if it exists + final String logConfigFile = locateLogConfig(System.getenv("ACCUMULO_CONF_DIR"), application); + + // Set up polling log4j updates and log-forwarding using information advertised in zookeeper by the monitor + MonitorLog4jWatcher logConfigWatcher = new MonitorLog4jWatcher(instance.getInstanceID(), logConfigFile); + logConfigWatcher.setDelay(5000L); + logConfigWatcher.start(); + + // Makes sure the log-forwarding to the monitor is configured + int logPort = conf.getPort(Property.MONITOR_LOG4J_PORT); + System.setProperty("org.apache.accumulo.core.host.log.port", Integer.toString(logPort)); + + log.info(application + " starting"); + log.info("Instance " + serverConfig.getInstance().getInstanceID()); + int dataVersion = Accumulo.getAccumuloPersistentVersion(fs); + log.info("Data Version " + dataVersion); + Accumulo.waitForZookeeperAndHdfs(fs); + + Version codeVersion = new Version(Constants.VERSION); + if (!(canUpgradeFromDataVersion(dataVersion))) { + throw new RuntimeException("This version of accumulo (" + codeVersion + ") is not compatible with files stored using data version " + dataVersion); + } + + TreeMap sortedProps = new TreeMap(); + for (Entry entry : conf) + sortedProps.put(entry.getKey(), entry.getValue()); + + for (Entry entry : sortedProps.entrySet()) { + String key = entry.getKey(); + log.info(key + " = " + (Property.isSensitive(key) ? "" : entry.getValue())); + } + + monitorSwappiness(); ++ ++ // Encourage users to configure TLS ++ final String SSL = "SSL"; ++ for (Property sslProtocolProperty : Arrays.asList(Property.MONITOR_SSL_INCLUDE_PROTOCOLS)) { ++ String value = conf.get(sslProtocolProperty); ++ if (value.contains(SSL)) { ++ log.warn("It is recommended that " + sslProtocolProperty + " only allow TLS"); ++ } ++ } ++ + } + + /** + * Sanity check that the current persistent version is allowed to upgrade to the version of Accumulo running. + * @param dataVersion the version that is persisted in the backing Volumes + */ + public static boolean canUpgradeFromDataVersion(final int dataVersion) { + return dataVersion == ServerConstants.DATA_VERSION || dataVersion == ServerConstants.PREV_DATA_VERSION || dataVersion == ServerConstants.TWO_DATA_VERSIONS_AGO; + } + + /** + * Does the data version number stored in the backing Volumes indicate we need to upgrade something? + */ + public static boolean persistentVersionNeedsUpgrade(final int accumuloPersistentVersion) { + return accumuloPersistentVersion == ServerConstants.TWO_DATA_VERSIONS_AGO || accumuloPersistentVersion == ServerConstants.PREV_DATA_VERSION; + } + + /** + * + */ + public static void monitorSwappiness() { + SimpleTimer.getInstance().schedule(new Runnable() { + @Override + public void run() { + try { + String procFile = "/proc/sys/vm/swappiness"; + File swappiness = new File(procFile); + if (swappiness.exists() && swappiness.canRead()) { + InputStream is = new FileInputStream(procFile); + try { + byte[] buffer = new byte[10]; + int bytes = is.read(buffer); + String setting = new String(buffer, 0, bytes, UTF_8); + setting = setting.trim(); + if (bytes > 0 && Integer.parseInt(setting) > 10) { + log.warn("System swappiness setting is greater than ten (" + setting + ") which can cause time-sensitive operations to be delayed. " + + " Accumulo is time sensitive because it needs to maintain distributed lock agreement."); + } + } finally { + is.close(); + } + } + } catch (Throwable t) { + log.error(t, t); + } + } + }, 1000, 10 * 60 * 1000); + } + + public static void waitForZookeeperAndHdfs(VolumeManager fs) { + log.info("Attempting to talk to zookeeper"); + while (true) { + try { + ZooReaderWriter.getInstance().getChildren(Constants.ZROOT); + break; + } catch (InterruptedException e) { + // ignored + } catch (KeeperException ex) { + log.info("Waiting for accumulo to be initialized"); + UtilWaitThread.sleep(1000); + } + } + log.info("Zookeeper connected and initialized, attemping to talk to HDFS"); + long sleep = 1000; + int unknownHostTries = 3; + while (true) { + try { + if (fs.isReady()) + break; + log.warn("Waiting for the NameNode to leave safemode"); + } catch (IOException ex) { + log.warn("Unable to connect to HDFS", ex); + } catch (IllegalArgumentException exception) { + /* Unwrap the UnknownHostException so we can deal with it directly */ + if (exception.getCause() instanceof UnknownHostException) { + if (unknownHostTries > 0) { + log.warn("Unable to connect to HDFS, will retry. cause: " + exception.getCause()); + /* We need to make sure our sleep period is long enough to avoid getting a cached failure of the host lookup. */ + sleep = Math.max(sleep, (AddressUtil.getAddressCacheNegativeTtl((UnknownHostException)(exception.getCause()))+1)*1000); + } else { + log.error("Unable to connect to HDFS and have exceeded max number of retries.", exception); + throw exception; + } + unknownHostTries--; + } else { + throw exception; + } + } + log.info("Backing off due to failure; current sleep period is " + sleep / 1000. + " seconds"); + UtilWaitThread.sleep(sleep); + /* Back off to give transient failures more time to clear. */ + sleep = Math.min(60 * 1000, sleep * 2); + } + log.info("Connected to HDFS"); + } + + /** + * Exit loudly if there are outstanding Fate operations. + * Since Fate serializes class names, we need to make sure there are no queued + * transactions from a previous version before continuing an upgrade. The status of the operations is + * irrelevant; those in SUCCESSFUL status cause the same problem as those just queued. + * + * Note that the Master should not allow write access to Fate until after all upgrade steps are complete. + * + * Should be called as a guard before performing any upgrade steps, after determining that an upgrade is needed. + * + * see ACCUMULO-2519 + */ + public static void abortIfFateTransactions() { + try { + final ReadOnlyTStore fate = new ReadOnlyStore(new ZooStore(ZooUtil.getRoot(HdfsZooInstance.getInstance()) + Constants.ZFATE, + ZooReaderWriter.getInstance())); + if (!(fate.list().isEmpty())) { + throw new AccumuloException("Aborting upgrade because there are outstanding FATE transactions from a previous Accumulo version. Please see the README document for instructions on what to do under your previous version."); + } + } catch (Exception exception) { + log.fatal("Problem verifying Fate readiness", exception); + System.exit(1); + } + } +} http://git-wip-us.apache.org/repos/asf/accumulo/blob/9ea8f09a/server/monitor/src/main/java/org/apache/accumulo/monitor/EmbeddedWebServer.java ---------------------------------------------------------------------- diff --cc server/monitor/src/main/java/org/apache/accumulo/monitor/EmbeddedWebServer.java index 82cd436,0000000..af91136 mode 100644,000000..100644 --- a/server/monitor/src/main/java/org/apache/accumulo/monitor/EmbeddedWebServer.java +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/EmbeddedWebServer.java @@@ -1,107 -1,0 +1,112 @@@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.accumulo.monitor; + +import javax.servlet.http.HttpServlet; + +import org.apache.accumulo.core.conf.AccumuloConfiguration; +import org.apache.accumulo.core.conf.Property; +import org.apache.commons.lang.StringUtils; +import org.eclipse.jetty.server.Server; +import org.eclipse.jetty.server.nio.SelectChannelConnector; +import org.eclipse.jetty.server.session.SessionHandler; +import org.eclipse.jetty.server.ssl.SslSelectChannelConnector; +import org.eclipse.jetty.servlet.ServletContextHandler; +import org.eclipse.jetty.util.ssl.SslContextFactory; + +public class EmbeddedWebServer { + private static String EMPTY = ""; + + Server server = null; + SelectChannelConnector connector = null; + ServletContextHandler handler; + boolean usingSsl; + + public EmbeddedWebServer() { + this("0.0.0.0", 0); + } + + public EmbeddedWebServer(String host, int port) { + server = new Server(); + final AccumuloConfiguration conf = Monitor.getSystemConfiguration(); + if (EMPTY.equals(conf.get(Property.MONITOR_SSL_KEYSTORE)) || EMPTY.equals(conf.get(Property.MONITOR_SSL_KEYSTOREPASS)) + || EMPTY.equals(conf.get(Property.MONITOR_SSL_TRUSTSTORE)) || EMPTY.equals(conf.get(Property.MONITOR_SSL_TRUSTSTOREPASS))) { + connector = new SelectChannelConnector(); + usingSsl = false; + } else { + SslContextFactory sslContextFactory = new SslContextFactory(); + sslContextFactory.setKeyStorePath(conf.get(Property.MONITOR_SSL_KEYSTORE)); + sslContextFactory.setKeyStorePassword(conf.get(Property.MONITOR_SSL_KEYSTOREPASS)); + sslContextFactory.setTrustStore(conf.get(Property.MONITOR_SSL_TRUSTSTORE)); + sslContextFactory.setTrustStorePassword(conf.get(Property.MONITOR_SSL_TRUSTSTOREPASS)); + + final String includedCiphers = conf.get(Property.MONITOR_SSL_INCLUDE_CIPHERS); + if (!Property.MONITOR_SSL_INCLUDE_CIPHERS.getDefaultValue().equals(includedCiphers)) { + sslContextFactory.setIncludeCipherSuites(StringUtils.split(includedCiphers, ',')); + } + + final String excludedCiphers = conf.get(Property.MONITOR_SSL_EXCLUDE_CIPHERS); + if (!Property.MONITOR_SSL_EXCLUDE_CIPHERS.getDefaultValue().equals(excludedCiphers)) { + sslContextFactory.setExcludeCipherSuites(StringUtils.split(excludedCiphers, ',')); + } + ++ final String includeProtocols = conf.get(Property.MONITOR_SSL_INCLUDE_PROTOCOLS); ++ if (null != includeProtocols && !includeProtocols.isEmpty()) { ++ sslContextFactory.setIncludeProtocols(StringUtils.split(includeProtocols, ',')); ++ } ++ + connector = new SslSelectChannelConnector(sslContextFactory); + usingSsl = true; + } + + connector.setHost(host); + connector.setPort(port); + + handler = new ServletContextHandler(server, "/", new SessionHandler(), null, null, null); + } + + public void addServlet(Class klass, String where) { + handler.addServlet(klass, where); + } + + public int getPort() { + return connector.getLocalPort(); + } + + public void start() { + try { + server.addConnector(connector); + server.setHandler(handler); + server.start(); + } catch (Exception e) { + stop(); + throw new RuntimeException(e); + } + } + + public void stop() { + try { + server.stop(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + public boolean isUsingSsl() { + return usingSsl; + } +}