diff --git a/core/src/main/java/org/apache/accumulo/core/conf/Property.java b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
index 9157fec5ffe..b205aed0961 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/Property.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
@@ -868,6 +868,8 @@ public enum Property {
+ " The resources that are used by default can be seen in"
+ " `accumulo/server/monitor/src/main/resources/templates/default.ftl`.",
"2.0.0"),
+ MONITOR_SUPPORT_HTTP2("monitor.http2.support", "false", PropertyType.BOOLEAN,
+ "If true will configure the Monitor web server to support http2 connections.", "3.1.0"),
// per table properties
TABLE_PREFIX("table.", null, PropertyType.PREFIX,
"Properties in this category affect tablet server treatment of tablets,"
diff --git a/pom.xml b/pom.xml
index 0406128385d..914e25601cb 100644
--- a/pom.xml
+++ b/pom.xml
@@ -151,6 +151,7 @@
5.5.0
2.24.1
3.4.0
+ 11.0.24
2.24.0
1.34.1
2.0.9
@@ -206,7 +207,7 @@
org.eclipse.jetty
jetty-bom
- 11.0.19
+ ${version.jetty}
pom
import
diff --git a/server/monitor/pom.xml b/server/monitor/pom.xml
index 93f11877fca..af3b744bd58 100644
--- a/server/monitor/pom.xml
+++ b/server/monitor/pom.xml
@@ -108,6 +108,10 @@
org.apache.zookeeper
zookeeper
+
+ org.eclipse.jetty
+ jetty-alpn-server
+
org.eclipse.jetty
jetty-server
@@ -120,6 +124,10 @@
org.eclipse.jetty
jetty-util
+
+ org.eclipse.jetty.http2
+ http2-server
+
org.glassfish.hk2
hk2-api
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/EmbeddedWebServer.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/EmbeddedWebServer.java
index 6784206b686..f3461411f0a 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
@@ -22,8 +22,12 @@
import org.apache.accumulo.core.conf.AccumuloConfiguration;
import org.apache.accumulo.core.conf.Property;
-import org.eclipse.jetty.server.AbstractConnectionFactory;
+import org.eclipse.jetty.alpn.server.ALPNServerConnectionFactory;
+import org.eclipse.jetty.http2.server.HTTP2CServerConnectionFactory;
+import org.eclipse.jetty.http2.server.HTTP2ServerConnectionFactory;
+import org.eclipse.jetty.server.HttpConfiguration;
import org.eclipse.jetty.server.HttpConnectionFactory;
+import org.eclipse.jetty.server.SecureRequestCustomizer;
import org.eclipse.jetty.server.Server;
import org.eclipse.jetty.server.ServerConnector;
import org.eclipse.jetty.server.SslConnectionFactory;
@@ -41,18 +45,18 @@ public class EmbeddedWebServer {
Property.MONITOR_SSL_TRUSTSTORE, Property.MONITOR_SSL_TRUSTSTOREPASS);
private final Server server;
- private final ServerConnector connector;
private final ServletContextHandler handler;
private final boolean secure;
- public EmbeddedWebServer(Monitor monitor, int port) {
+ private int actualHttpPort;
+ private ServerConnector connector;
+
+ public EmbeddedWebServer(final Monitor monitor, int httpPort) {
server = new Server();
final AccumuloConfiguration conf = monitor.getContext().getConfiguration();
secure = requireForSecure.stream().map(conf::get).allMatch(s -> s != null && !s.isEmpty());
- connector = new ServerConnector(server, getConnectionFactories(conf, secure));
- connector.setHost(monitor.getHostname());
- connector.setPort(port);
+ addConnectors(monitor, conf, httpPort, secure);
handler =
new ServletContextHandler(ServletContextHandler.SESSIONS | ServletContextHandler.SECURITY);
@@ -60,11 +64,16 @@ public EmbeddedWebServer(Monitor monitor, int port) {
handler.setContextPath("/");
}
- private static AbstractConnectionFactory[] getConnectionFactories(AccumuloConfiguration conf,
+ private void addConnectors(Monitor monitor, AccumuloConfiguration conf, int httpPort,
boolean secure) {
- HttpConnectionFactory httpFactory = new HttpConnectionFactory();
+
+ boolean configureHttp2 = conf.getBoolean(Property.MONITOR_SUPPORT_HTTP2);
+
+ final HttpConfiguration httpConfig = new HttpConfiguration();
+ httpConfig.setSendServerVersion(false);
+
if (secure) {
- LOG.debug("Configuring Jetty to use TLS");
+ LOG.debug("Configuring Jetty with TLS");
final SslContextFactory.Server sslContextFactory = new SslContextFactory.Server();
// If the key password is the same as the keystore password, we don't
// have to explicitly set it. Thus, if the user doesn't provide a key
@@ -95,12 +104,38 @@ private static AbstractConnectionFactory[] getConnectionFactories(AccumuloConfig
sslContextFactory.setIncludeProtocols(includeProtocols.split(","));
}
- SslConnectionFactory sslFactory =
- new SslConnectionFactory(sslContextFactory, httpFactory.getProtocol());
- return new AbstractConnectionFactory[] {sslFactory, httpFactory};
+ if (!configureHttp2) {
+ HttpConnectionFactory http11 = new HttpConnectionFactory(httpConfig);
+ SslConnectionFactory tls =
+ new SslConnectionFactory(sslContextFactory, http11.getProtocol());
+ connector = new ServerConnector(server, tls, http11);
+ } else {
+ LOG.debug("Enabling http2 support");
+ httpConfig.addCustomizer(new SecureRequestCustomizer());
+ HttpConnectionFactory http11 = new HttpConnectionFactory(httpConfig);
+ ALPNServerConnectionFactory alpn = new ALPNServerConnectionFactory();
+ alpn.setDefaultProtocol(http11.getProtocol());
+ HTTP2ServerConnectionFactory http2 = new HTTP2ServerConnectionFactory(httpConfig);
+ SslConnectionFactory tls = new SslConnectionFactory(sslContextFactory, alpn.getProtocol());
+ connector = new ServerConnector(server, tls, alpn, http2, http11);
+ }
+ connector.setHost(monitor.getHostname());
+ connector.setPort(httpPort);
+ server.addConnector(connector);
} else {
- LOG.debug("Not configuring Jetty to use TLS");
- return new AbstractConnectionFactory[] {httpFactory};
+ LOG.debug("Configuring Jetty without TLS");
+ if (!configureHttp2) {
+ HttpConnectionFactory http11 = new HttpConnectionFactory(httpConfig);
+ connector = new ServerConnector(server, http11);
+ } else {
+ LOG.debug("Enabling http2 support");
+ HttpConnectionFactory http11 = new HttpConnectionFactory(httpConfig);
+ HTTP2CServerConnectionFactory http2 = new HTTP2CServerConnectionFactory(httpConfig);
+ connector = new ServerConnector(server, http11, http2);
+ }
+ connector.setHost(monitor.getHostname());
+ connector.setPort(httpPort);
+ server.addConnector(connector);
}
}
@@ -108,8 +143,8 @@ public void addServlet(ServletHolder restServlet, String where) {
handler.addServlet(restServlet, where);
}
- public int getPort() {
- return connector.getLocalPort();
+ public int getHttpPort() {
+ return this.actualHttpPort;
}
public boolean isSecure() {
@@ -118,9 +153,9 @@ public boolean isSecure() {
public void start() {
try {
- server.addConnector(connector);
server.setHandler(handler);
server.start();
+ this.actualHttpPort = connector.getLocalPort();
} catch (Exception e) {
stop();
throw new RuntimeException(e);
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java
index 189d42622af..211a8841171 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java
@@ -187,7 +187,6 @@ public boolean add(Pair obj) {
+ "'accumulo compaction-coordinator'.";
private EmbeddedWebServer server;
- private int livePort = 0;
private ServiceLock monitorLock;
@@ -465,7 +464,6 @@ public void run() {
server.addServlet(getRestServlet(), "/rest/*");
server.addServlet(getViewServlet(), "/*");
server.start();
- livePort = port;
break;
} catch (Exception ex) {
log.error("Unable to start embedded web server", ex);
@@ -475,7 +473,7 @@ public void run() {
throw new RuntimeException(
"Unable to start embedded web server on ports: " + Arrays.toString(ports));
} else {
- log.debug("Monitor started on port {}", livePort);
+ log.debug("Monitor started on port {}", server.getHttpPort());
}
String advertiseHost = getHostname();
@@ -486,7 +484,8 @@ public void run() {
log.error("Unable to get hostname", e);
}
}
- HostAndPort monitorHostAndPort = HostAndPort.fromParts(advertiseHost, livePort);
+
+ HostAndPort monitorHostAndPort = HostAndPort.fromParts(advertiseHost, server.getHttpPort());
log.debug("Using {} to advertise monitor location in ZooKeeper", monitorHostAndPort);
try {
@@ -502,9 +501,10 @@ public void run() {
metricsInfo.init();
try {
- URL url = new URL(server.isSecure() ? "https" : "http", advertiseHost, server.getPort(), "/");
- final String path = context.getZooKeeperRoot() + Constants.ZMONITOR_HTTP_ADDR;
final ZooReaderWriter zoo = context.getZooReaderWriter();
+ URL url =
+ new URL(server.isSecure() ? "https" : "http", advertiseHost, server.getHttpPort(), "/");
+ final String path = context.getZooKeeperRoot() + Constants.ZMONITOR_HTTP_ADDR;
// Delete before we try to re-create in case the previous session hasn't yet expired
zoo.delete(path);
zoo.putEphemeralData(path, url.toString().getBytes(UTF_8));
@@ -1045,8 +1045,4 @@ public RecentLogs recentLogs() {
public Optional getCoordinatorHost() {
return coordinatorHost;
}
-
- public int getLivePort() {
- return livePort;
- }
}
diff --git a/test/pom.xml b/test/pom.xml
index 442256cadcf..41b6c872582 100644
--- a/test/pom.xml
+++ b/test/pom.xml
@@ -186,6 +186,26 @@
org.easymock
easymock
+
+ org.eclipse.jetty
+ jetty-client
+
+
+ org.eclipse.jetty
+ jetty-io
+
+
+ org.eclipse.jetty
+ jetty-util
+
+
+ org.eclipse.jetty.http2
+ http2-client
+
+
+ org.eclipse.jetty.http2
+ http2-http-client-transport
+
org.jline
jline
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/MonitorSslIT.java b/test/src/main/java/org/apache/accumulo/test/functional/MonitorSslIT.java
index ec3b2ebbff2..fedbdd7ac05 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/MonitorSslIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/MonitorSslIT.java
@@ -22,18 +22,13 @@
import static org.junit.jupiter.api.Assertions.assertTrue;
import java.io.File;
-import java.net.URL;
import java.security.KeyManagementException;
import java.security.NoSuchAlgorithmException;
import java.security.cert.X509Certificate;
-import java.time.Duration;
import java.util.Map;
-import javax.net.ssl.HostnameVerifier;
-import javax.net.ssl.HttpsURLConnection;
import javax.net.ssl.KeyManager;
import javax.net.ssl.SSLContext;
-import javax.net.ssl.SSLSession;
import javax.net.ssl.TrustManager;
import javax.net.ssl.X509TrustManager;
@@ -42,32 +37,39 @@
import org.apache.accumulo.core.clientImpl.ClientContext;
import org.apache.accumulo.core.conf.Property;
import org.apache.accumulo.core.util.MonitorUtil;
+import org.apache.accumulo.harness.AccumuloClusterHarness;
import org.apache.accumulo.minicluster.ServerType;
import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
import org.apache.hadoop.conf.Configuration;
+import org.eclipse.jetty.client.HttpClient;
+import org.eclipse.jetty.client.api.ContentResponse;
+import org.eclipse.jetty.client.http.HttpClientTransportOverHTTP;
+import org.eclipse.jetty.http2.client.HTTP2Client;
+import org.eclipse.jetty.http2.client.http.HttpClientTransportOverHTTP2;
+import org.eclipse.jetty.io.ClientConnector;
+import org.eclipse.jetty.util.ssl.SslContextFactory;
+import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
/**
* Check SSL for the Monitor
*/
-public class MonitorSslIT extends ConfigurableMacBase {
+public class MonitorSslIT extends AccumuloClusterHarness {
- @Override
- protected Duration defaultTimeout() {
- return Duration.ofMinutes(6);
- }
+ private static final Logger LOG = LoggerFactory.getLogger(MonitorSslIT.class);
+ private static SSLContext ctx;
@BeforeAll
public static void initHttps() throws NoSuchAlgorithmException, KeyManagementException {
- SSLContext ctx = SSLContext.getInstance("TLSv1.3");
+ ctx = SSLContext.getInstance("TLSv1.3");
TrustManager[] tm = {new TestTrustManager()};
ctx.init(new KeyManager[0], tm, RANDOM.get());
SSLContext.setDefault(ctx);
- HttpsURLConnection.setDefaultSSLSocketFactory(ctx.getSocketFactory());
- HttpsURLConnection.setDefaultHostnameVerifier(new TestHostnameVerifier());
}
@SuppressFBWarnings(value = "WEAK_TRUST_MANAGER",
@@ -85,52 +87,81 @@ public X509Certificate[] getAcceptedIssuers() {
}
}
- @SuppressFBWarnings(value = "WEAK_HOSTNAME_VERIFIER", justification = "okay for test")
- private static class TestHostnameVerifier implements HostnameVerifier {
- @Override
- public boolean verify(String hostname, SSLSession session) {
- return true;
- }
+ private static boolean sslEnabled = false;
+ private static boolean http2Enabled = false;
+
+ @AfterEach
+ public void after() throws Exception {
+ cluster.getClusterControl().stopAllServers(ServerType.MONITOR);
+ sslEnabled = !sslEnabled;
}
@Override
- public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
- super.configure(cfg, hadoopCoreSite);
- File baseDir = createTestDir(this.getClass().getName() + "_" + this.testName());
- configureForSsl(cfg, getSslDir(baseDir));
- Map siteConfig = cfg.getSiteConfig();
- siteConfig.put(Property.MONITOR_SSL_KEYSTORE.getKey(),
- siteConfig.get(Property.RPC_SSL_KEYSTORE_PATH.getKey()));
- siteConfig.put(Property.MONITOR_SSL_KEYSTOREPASS.getKey(),
- siteConfig.get(Property.RPC_SSL_KEYSTORE_PASSWORD.getKey()));
- if (siteConfig.containsKey(Property.RPC_SSL_KEYSTORE_TYPE.getKey())) {
- siteConfig.put(Property.MONITOR_SSL_KEYSTORETYPE.getKey(),
- siteConfig.get(Property.RPC_SSL_KEYSTORE_TYPE.getKey()));
- } else {
- siteConfig.put(Property.MONITOR_SSL_KEYSTORETYPE.getKey(),
- Property.RPC_SSL_KEYSTORE_TYPE.getDefaultValue());
- }
- siteConfig.put(Property.MONITOR_SSL_TRUSTSTORE.getKey(),
- siteConfig.get(Property.RPC_SSL_TRUSTSTORE_PATH.getKey()));
- siteConfig.put(Property.MONITOR_SSL_TRUSTSTOREPASS.getKey(),
- siteConfig.get(Property.RPC_SSL_TRUSTSTORE_PASSWORD.getKey()));
- if (siteConfig.containsKey(Property.RPC_SSL_TRUSTSTORE_TYPE.getKey())) {
- siteConfig.put(Property.MONITOR_SSL_TRUSTSTORETYPE.getKey(),
- siteConfig.get(Property.RPC_SSL_TRUSTSTORE_TYPE.getKey()));
- } else {
- siteConfig.put(Property.MONITOR_SSL_TRUSTSTORETYPE.getKey(),
- Property.RPC_SSL_TRUSTSTORE_TYPE.getDefaultValue());
+ public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
+ LOG.info("*** Configuring server, SSL enabled: {}", sslEnabled);
+ if (sslEnabled) {
+ File baseDir = createTestDir(this.getClass().getName() + "_" + this.testName());
+ ConfigurableMacBase.configureForSsl(cfg, getSslDir(baseDir));
+ Map siteConfig = cfg.getSiteConfig();
+ siteConfig.put(Property.MONITOR_SSL_KEYSTORE.getKey(),
+ siteConfig.get(Property.RPC_SSL_KEYSTORE_PATH.getKey()));
+ siteConfig.put(Property.MONITOR_SSL_KEYSTOREPASS.getKey(),
+ siteConfig.get(Property.RPC_SSL_KEYSTORE_PASSWORD.getKey()));
+ if (siteConfig.containsKey(Property.RPC_SSL_KEYSTORE_TYPE.getKey())) {
+ siteConfig.put(Property.MONITOR_SSL_KEYSTORETYPE.getKey(),
+ siteConfig.get(Property.RPC_SSL_KEYSTORE_TYPE.getKey()));
+ } else {
+ siteConfig.put(Property.MONITOR_SSL_KEYSTORETYPE.getKey(),
+ Property.RPC_SSL_KEYSTORE_TYPE.getDefaultValue());
+ }
+ siteConfig.put(Property.MONITOR_SSL_TRUSTSTORE.getKey(),
+ siteConfig.get(Property.RPC_SSL_TRUSTSTORE_PATH.getKey()));
+ siteConfig.put(Property.MONITOR_SSL_TRUSTSTOREPASS.getKey(),
+ siteConfig.get(Property.RPC_SSL_TRUSTSTORE_PASSWORD.getKey()));
+ if (siteConfig.containsKey(Property.RPC_SSL_TRUSTSTORE_TYPE.getKey())) {
+ siteConfig.put(Property.MONITOR_SSL_TRUSTSTORETYPE.getKey(),
+ siteConfig.get(Property.RPC_SSL_TRUSTSTORE_TYPE.getKey()));
+ } else {
+ siteConfig.put(Property.MONITOR_SSL_TRUSTSTORETYPE.getKey(),
+ Property.RPC_SSL_TRUSTSTORE_TYPE.getDefaultValue());
+ }
}
- cfg.setSiteConfig(siteConfig);
+ LOG.info("*** Configuring server, http2 enabled: {}", http2Enabled);
+ cfg.setProperty(Property.MONITOR_SUPPORT_HTTP2, Boolean.toString(http2Enabled));
+ }
+
+ @Test
+ public void test1() throws Exception {
+ test();
+ }
+
+ @Test
+ @SuppressFBWarnings(value = "ST_WRITE_TO_STATIC_FROM_INSTANCE_METHOD",
+ justification = "writing to static ok for testing")
+ public void test2() throws Exception {
+ test();
+ // Run the first two tests with http2 disabled. Enable for the remaining two tests
+ http2Enabled = true;
+ }
+
+ @Test
+ public void test3() throws Exception {
+ test();
}
- @SuppressFBWarnings(value = "URLCONNECTION_SSRF_FD", justification = "url provided by test")
@Test
+ public void test4() throws Exception {
+ test();
+ }
+
public void test() throws Exception {
- log.debug("Starting Monitor");
+ LOG.info("*** Running test, SSL enabled: {}, HTTP2 enabled: {}", sslEnabled, http2Enabled);
+
+ LOG.debug("Starting Monitor");
cluster.getClusterControl().startAllServers(ServerType.MONITOR);
String monitorLocation = null;
- try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) {
+ try (AccumuloClient client =
+ Accumulo.newClient().from(getCluster().getClientProperties()).build()) {
while (monitorLocation == null) {
try {
monitorLocation = MonitorUtil.getLocation((ClientContext) client);
@@ -138,16 +169,63 @@ public void test() throws Exception {
// ignored
}
if (monitorLocation == null) {
- log.debug("Could not fetch monitor HTTP address from zookeeper");
+ LOG.debug("Could not fetch monitor HTTP address from zookeeper");
Thread.sleep(2000);
}
}
}
- URL url = new URL(monitorLocation);
- log.debug("Fetching web page {}", url);
- String result = FunctionalTestUtils.readWebPage(url).body();
- assertTrue(result.length() > 100);
- assertTrue(result.indexOf("Accumulo Overview") >= 0);
+
+ SslContextFactory.Client sslContextFactory = null;
+ if (sslEnabled) {
+ sslContextFactory = new SslContextFactory.Client();
+ sslContextFactory.setSslContext(ctx);
+ // Don't validate server host name
+ sslContextFactory.setEndpointIdentificationAlgorithm(null);
+ }
+
+ HttpClient h1 = createHttp11(sslContextFactory);
+ h1.start();
+ ContentResponse body = h1.GET(monitorLocation);
+ assertTrue(body.getContentAsString().length() > 100);
+ assertTrue(body.getContentAsString().indexOf("Accumulo Overview") >= 0);
+ h1.stop();
+
+ if (http2Enabled) {
+ HttpClient h2 = createHttp2(sslContextFactory);
+ h2.start();
+ body = h2.GET(monitorLocation);
+ assertTrue(body.getContentAsString().length() > 100);
+ assertTrue(body.getContentAsString().indexOf("Accumulo Overview") >= 0);
+ h2.stop();
+ }
+
+ }
+
+ private HttpClient createHttp11(SslContextFactory.Client sslContextFactory) throws Exception {
+
+ ClientConnector connector = new ClientConnector();
+ if (sslContextFactory != null) {
+ connector.setSslContextFactory(sslContextFactory);
+ }
+
+ HttpClientTransportOverHTTP transport = new HttpClientTransportOverHTTP(connector);
+
+ return new HttpClient(transport);
+ }
+
+ private HttpClient createHttp2(SslContextFactory.Client sslContextFactory) throws Exception {
+
+ ClientConnector connector = new ClientConnector();
+ if (sslContextFactory != null) {
+ connector.setSslContextFactory(sslContextFactory);
+ }
+
+ HTTP2Client h2Client = new HTTP2Client(connector);
+
+ // Create and configure the HTTP/2 transport.
+ HttpClientTransportOverHTTP2 transport = new HttpClientTransportOverHTTP2(h2Client);
+
+ return new HttpClient(transport);
}
}