diff --git a/nifi-registry/nifi-registry-core/nifi-registry-jetty/src/main/java/org/apache/nifi/registry/jetty/JettyServer.java b/nifi-registry/nifi-registry-core/nifi-registry-jetty/src/main/java/org/apache/nifi/registry/jetty/JettyServer.java
index f868a39270..fa97a678d7 100644
--- a/nifi-registry/nifi-registry-core/nifi-registry-jetty/src/main/java/org/apache/nifi/registry/jetty/JettyServer.java
+++ b/nifi-registry/nifi-registry-core/nifi-registry-jetty/src/main/java/org/apache/nifi/registry/jetty/JettyServer.java
@@ -19,78 +19,54 @@ package org.apache.nifi.registry.jetty;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.jetty.configuration.connector.ServerConnectorFactory;
import org.apache.nifi.registry.jetty.connector.ApplicationServerConnectorFactory;
-import org.apache.nifi.registry.jetty.headers.ContentSecurityPolicyFilter;
-import org.apache.nifi.registry.jetty.headers.StrictTransportSecurityFilter;
-import org.apache.nifi.registry.jetty.headers.XFrameOptionsFilter;
-import org.apache.nifi.registry.jetty.headers.XSSProtectionFilter;
+import org.apache.nifi.registry.jetty.handler.HandlerProvider;
import org.apache.nifi.registry.properties.NiFiRegistryProperties;
-import org.apache.nifi.registry.security.crypto.CryptoKeyProvider;
import org.eclipse.jetty.annotations.AnnotationConfiguration;
-import org.eclipse.jetty.server.Connector;
import org.eclipse.jetty.server.Handler;
import org.eclipse.jetty.server.Server;
import org.eclipse.jetty.server.ServerConnector;
-import org.eclipse.jetty.server.handler.HandlerCollection;
-import org.eclipse.jetty.servlet.DefaultServlet;
-import org.eclipse.jetty.servlet.FilterHolder;
-import org.eclipse.jetty.servlet.ServletHolder;
+import org.eclipse.jetty.server.SslConnectionFactory;
import org.eclipse.jetty.util.thread.QueuedThreadPool;
import org.eclipse.jetty.webapp.Configuration;
import org.eclipse.jetty.webapp.JettyWebXmlConfiguration;
-import org.eclipse.jetty.webapp.WebAppClassLoader;
import org.eclipse.jetty.webapp.WebAppContext;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import javax.servlet.DispatcherType;
-import javax.servlet.Filter;
-import java.io.File;
-import java.io.FileFilter;
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.MalformedURLException;
-import java.net.NetworkInterface;
-import java.net.SocketException;
-import java.net.URL;
-import java.net.URLClassLoader;
-import java.nio.file.Paths;
-import java.util.ArrayList;
+import java.net.URI;
import java.util.Arrays;
-import java.util.Collections;
-import java.util.EnumSet;
-import java.util.Enumeration;
-import java.util.HashSet;
-import java.util.LinkedList;
import java.util.List;
-import java.util.Set;
-
+import java.util.Objects;
+import java.util.Optional;
+import java.util.stream.Collectors;
+/**
+ * Registry Jetty Server
+ */
public class JettyServer {
private static final Logger logger = LoggerFactory.getLogger(JettyServer.class);
- private static final String WEB_DEFAULTS_XML = "org/apache/nifi-registry/web/webdefault.xml";
- private static final String ALL_PATHS = "/*";
+ private static final String APPLICATION_URL_FORMAT = "%s://%s:%d/nifi-registry";
- private static final FileFilter WAR_FILTER = pathname -> {
- final String nameToTest = pathname.getName().toLowerCase();
- return nameToTest.endsWith(".war") && pathname.isFile();
- };
+ private static final String HTTPS_SCHEME = "https";
+
+ private static final String HTTP_SCHEME = "http";
+
+ private static final String HOST_UNSPECIFIED = "0.0.0.0";
private final NiFiRegistryProperties properties;
- private final CryptoKeyProvider masterKeyProvider;
- private final String docsLocation;
+
private final Server server;
- private WebAppContext webApiContext;
+ public JettyServer(final NiFiRegistryProperties properties, final HandlerProvider handlerProvider) {
+ Objects.requireNonNull(properties, "Properties required");
+ Objects.requireNonNull(handlerProvider, "Handler Provider required");
- public JettyServer(final NiFiRegistryProperties properties, final CryptoKeyProvider cryptoKeyProvider, final String docsLocation) {
final QueuedThreadPool threadPool = new QueuedThreadPool(properties.getWebThreads());
threadPool.setName("NiFi Registry Web Server");
this.properties = properties;
- this.masterKeyProvider = cryptoKeyProvider;
- this.docsLocation = docsLocation;
this.server = new Server(threadPool);
// enable the annotation based configuration to ensure the jsp container is initialized properly
@@ -99,356 +75,73 @@ public class JettyServer {
try {
configureConnectors();
- loadWars();
+ final Handler handler = handlerProvider.getHandler(properties);
+ server.setHandler(handler);
} catch (final Throwable t) {
- startUpFailure(t);
+ shutdown(t);
}
}
- /**
- * Instantiates this object but does not perform any configuration. Used for unit testing.
- */
- JettyServer(Server server, NiFiRegistryProperties properties) {
- this.server = server;
- this.properties = properties;
- this.masterKeyProvider = null;
- this.docsLocation = null;
- }
-
- /**
- * Returns a File object for the directory containing NIFI documentation.
- *
- * Formerly, if the docsDirectory did not exist NIFI would fail to start
- * with an IllegalStateException and a rather unhelpful log message.
- * NIFI-2184 updates the process such that if the docsDirectory does not
- * exist an attempt will be made to create the directory. If that is
- * successful NIFI will no longer fail and will start successfully barring
- * any other errors. The side effect of the docsDirectory not being present
- * is that the documentation links under the 'General' portion of the help
- * page will not be accessible, but at least the process will be running.
- *
- * @param docsDirectory Name of documentation directory in installation directory.
- * @return A File object to the documentation directory; else startUpFailure called.
- */
- private File getDocsDir(final String docsDirectory) {
- File docsDir;
- try {
- docsDir = Paths.get(docsDirectory).toRealPath().toFile();
- } catch (IOException ex) {
- logger.info("Directory '" + docsDirectory + "' is missing. Some documentation will be unavailable.");
- docsDir = new File(docsDirectory).getAbsoluteFile();
- final boolean made = docsDir.mkdirs();
- if (!made) {
- logger.error("Failed to create 'docs' directory!");
- startUpFailure(new IOException(docsDir.getAbsolutePath() + " could not be created"));
- }
- }
- return docsDir;
- }
-
private void configureConnectors() {
final ServerConnectorFactory serverConnectorFactory = new ApplicationServerConnectorFactory(server, properties);
final ServerConnector serverConnector = serverConnectorFactory.getServerConnector();
server.addConnector(serverConnector);
}
- private void loadWars() throws IOException {
- final File warDirectory = properties.getWarLibDirectory();
- final File[] wars = warDirectory.listFiles(WAR_FILTER);
-
- if (wars == null) {
- throw new RuntimeException("Unable to access war lib directory: " + warDirectory);
- }
-
- File webUiWar = null;
- File webApiWar = null;
- File webDocsWar = null;
- for (final File war : wars) {
- if (war.getName().startsWith("nifi-registry-web-ui")) {
- webUiWar = war;
- } else if (war.getName().startsWith("nifi-registry-web-api")) {
- webApiWar = war;
- } else if (war.getName().startsWith("nifi-registry-web-docs")) {
- webDocsWar = war;
- }
- }
-
- if (webUiWar == null) {
- throw new IllegalStateException("Unable to locate NiFi Registry Web UI");
- } else if (webApiWar == null) {
- throw new IllegalStateException("Unable to locate NiFi Registry Web API");
- } else if (webDocsWar == null) {
- throw new IllegalStateException("Unable to locate NiFi Registry Web Docs");
- }
-
- WebAppContext webUiContext = loadWar(webUiWar, "/nifi-registry");
- webUiContext.getInitParams().put("oidc-supported", String.valueOf(properties.isOidcEnabled()));
-
- webApiContext = loadWar(webApiWar, "/nifi-registry-api", getWebApiAdditionalClasspath());
- logger.info("Adding {} object to ServletContext with key 'nifi-registry.properties'", properties.getClass().getSimpleName());
- webApiContext.setAttribute("nifi-registry.properties", properties);
- logger.info("Adding {} object to ServletContext with key 'nifi-registry.key'", masterKeyProvider.getClass().getSimpleName());
- webApiContext.setAttribute("nifi-registry.key", masterKeyProvider);
-
- // there is an issue scanning the asm repackaged jar so narrow down what we are scanning
- webApiContext.setAttribute("org.eclipse.jetty.server.webapp.WebInfIncludeJarPattern", ".*/spring-[^/]*\\.jar$");
-
- final String docsContextPath = "/nifi-registry-docs";
- WebAppContext webDocsContext = loadWar(webDocsWar, docsContextPath);
- addDocsServlets(webDocsContext);
-
- final HandlerCollection handlers = new HandlerCollection();
- handlers.addHandler(webUiContext);
- handlers.addHandler(webApiContext);
- handlers.addHandler(webDocsContext);
- server.setHandler(handlers);
- }
-
- private WebAppContext loadWar(final File warFile, final String contextPath)
- throws IOException {
- return loadWar(warFile, contextPath, new URL[0]);
- }
-
- private WebAppContext loadWar(final File warFile, final String contextPath, final URL[] additionalResources)
- throws IOException {
- final WebAppContext webappContext = new WebAppContext(warFile.getPath(), contextPath);
- webappContext.setContextPath(contextPath);
- webappContext.setDisplayName(contextPath);
-
- // remove slf4j server class to allow WAR files to have slf4j dependencies in WEB-INF/lib
- List serverClasses = new ArrayList<>(Arrays.asList(webappContext.getServerClasses()));
- serverClasses.remove("org.slf4j.");
- webappContext.setServerClasses(serverClasses.toArray(new String[0]));
- webappContext.setDefaultsDescriptor(WEB_DEFAULTS_XML);
-
- // get the temp directory for this webapp
- final File webWorkingDirectory = properties.getWebWorkingDirectory();
- final File tempDir = new File(webWorkingDirectory, warFile.getName());
- if (tempDir.exists() && !tempDir.isDirectory()) {
- throw new RuntimeException(tempDir.getAbsolutePath() + " is not a directory");
- } else if (!tempDir.exists()) {
- final boolean made = tempDir.mkdirs();
- if (!made) {
- throw new RuntimeException(tempDir.getAbsolutePath() + " could not be created");
- }
- }
- if (!(tempDir.canRead() && tempDir.canWrite())) {
- throw new RuntimeException(tempDir.getAbsolutePath() + " directory does not have read/write privilege");
- }
-
- // configure the temp dir
- webappContext.setTempDirectory(tempDir);
-
- // configure the max form size (3x the default)
- webappContext.setMaxFormContentSize(600000);
-
- // add HTTP security headers to all responses
- ArrayList> filters = new ArrayList<>(Arrays.asList(XFrameOptionsFilter.class, ContentSecurityPolicyFilter.class, XSSProtectionFilter.class));
- if (properties.isHTTPSConfigured()) {
- filters.add(StrictTransportSecurityFilter.class);
- }
-
- filters.forEach( (filter) -> addFilters(filter, webappContext));
-
- // start out assuming the system ClassLoader will be the parent, but if additional resources were specified then
- // inject a new ClassLoader in between the system and webapp ClassLoaders that contains the additional resources
- ClassLoader parentClassLoader = ClassLoader.getSystemClassLoader();
- if (additionalResources != null && additionalResources.length > 0) {
- parentClassLoader = new URLClassLoader(additionalResources, ClassLoader.getSystemClassLoader());
- }
-
- webappContext.setClassLoader(new WebAppClassLoader(parentClassLoader, webappContext));
-
- logger.info("Loading WAR: " + warFile.getAbsolutePath() + " with context path set to " + contextPath);
- return webappContext;
- }
-
- private void addFilters(Class extends Filter> clazz, final WebAppContext webappContext) {
- FilterHolder holder = new FilterHolder(clazz);
- holder.setName(clazz.getSimpleName());
- webappContext.addFilter(holder, ALL_PATHS, EnumSet.allOf(DispatcherType.class));
- }
-
- private URL[] getWebApiAdditionalClasspath() {
- final String dbDriverDir = properties.getDatabaseDriverDirectory();
-
- if (StringUtils.isBlank(dbDriverDir)) {
- logger.info("No database driver directory was specified");
- return new URL[0];
- }
-
- final File dirFile = new File(dbDriverDir);
-
- if (!dirFile.exists()) {
- logger.warn("Skipping database driver directory that does not exist: " + dbDriverDir);
- return new URL[0];
- }
-
- if (!dirFile.canRead()) {
- logger.warn("Skipping database driver directory that can not be read: " + dbDriverDir);
- return new URL[0];
- }
-
- final List resources = new LinkedList<>();
- try {
- resources.add(dirFile.toURI().toURL());
- } catch (final MalformedURLException mfe) {
- logger.warn("Unable to add {} to classpath due to {}", new Object[]{ dirFile.getAbsolutePath(), mfe.getMessage()}, mfe);
- }
-
- if (dirFile.isDirectory()) {
- final File[] files = dirFile.listFiles();
- if (files != null) {
- for (final File resource : files) {
- if (resource.isDirectory()) {
- logger.warn("Recursive directories are not supported, skipping " + resource.getAbsolutePath());
- } else {
- try {
- resources.add(resource.toURI().toURL());
- } catch (final MalformedURLException mfe) {
- logger.warn("Unable to add {} to classpath due to {}", new Object[]{ resource.getAbsolutePath(), mfe.getMessage()}, mfe);
- }
- }
- }
- }
- }
-
- if (!resources.isEmpty()) {
- logger.info("Added additional resources to nifi-registry-api classpath: [");
- for (URL resource : resources) {
- logger.info(" " + resource.toString());
- }
- logger.info("]");
- }
-
- return resources.toArray(new URL[0]);
- }
-
- private void addDocsServlets(WebAppContext docsContext) {
- try {
- // Load the nifi-registry/docs directory
- final File docsDir = getDocsDir(docsLocation);
-
- // Create the servlet which will serve the static resources
- ServletHolder defaultHolder = new ServletHolder("default", DefaultServlet.class);
- defaultHolder.setInitParameter("dirAllowed", "false");
-
- ServletHolder docs = new ServletHolder("docs", DefaultServlet.class);
- docs.setInitParameter("resourceBase", docsDir.getPath());
- docs.setInitParameter("dirAllowed", "false");
-
- docsContext.addServlet(docs, "/html/*");
- docsContext.addServlet(defaultHolder, "/");
-
- // load the rest documentation
- final File webApiDocsDir = new File(webApiContext.getTempDirectory(), "webapp/docs");
- if (!webApiDocsDir.exists()) {
- final boolean made = webApiDocsDir.mkdirs();
- if (!made) {
- throw new RuntimeException(webApiDocsDir.getAbsolutePath() + " could not be created");
- }
- }
-
- ServletHolder apiDocs = new ServletHolder("apiDocs", DefaultServlet.class);
- apiDocs.setInitParameter("resourceBase", webApiDocsDir.getPath());
- apiDocs.setInitParameter("dirAllowed", "false");
-
- docsContext.addServlet(apiDocs, "/rest-api/*");
-
- logger.info("Loading documents web app with context path set to " + docsContext.getContextPath());
-
- } catch (Exception ex) {
- logger.error("Unhandled Exception in createDocsWebApp: " + ex.getMessage());
- startUpFailure(ex);
- }
- }
-
public void start() {
try {
- // start the server
server.start();
- // ensure everything started successfully
- for (Handler handler : server.getChildHandlers()) {
- // see if the handler is a web app
- if (handler instanceof WebAppContext) {
- WebAppContext context = (WebAppContext) handler;
+ final Optional unavailableExceptionFound = Arrays.stream(server.getChildHandlers())
+ .filter(handler -> handler instanceof WebAppContext)
+ .map(handler -> (WebAppContext) handler)
+ .map(WebAppContext::getUnavailableException)
+ .filter(Objects::nonNull)
+ .findFirst();
- // see if this webapp had any exceptions that would
- // cause it to be unavailable
- if (context.getUnavailableException() != null) {
- startUpFailure(context.getUnavailableException());
- }
- }
+ if (unavailableExceptionFound.isPresent()) {
+ final Throwable unavailableException = unavailableExceptionFound.get();
+ shutdown(unavailableException);
}
- dumpUrls();
+ final List applicationUrls = getApplicationUrls();
+ if (applicationUrls.isEmpty()) {
+ logger.warn("Started Server without connectors");
+ } else {
+ for (final URI applicationUrl : applicationUrls) {
+ logger.info("Started Server on {}", applicationUrl);
+ }
+ }
} catch (final Throwable t) {
- startUpFailure(t);
- }
- }
-
- private void startUpFailure(Throwable t) {
- System.err.println("Failed to start web server: " + t.getMessage());
- System.err.println("Shutting down...");
- logger.warn("Failed to start web server... shutting down.", t);
- System.exit(1);
- }
-
- private void dumpUrls() throws SocketException {
- final List urls = new ArrayList<>();
-
- for (Connector connector : server.getConnectors()) {
- if (connector instanceof ServerConnector) {
- final ServerConnector serverConnector = (ServerConnector) connector;
-
- Set hosts = new HashSet<>();
-
- // determine the hosts
- if (StringUtils.isNotBlank(serverConnector.getHost())) {
- hosts.add(serverConnector.getHost());
- } else {
- Enumeration networkInterfaces = NetworkInterface.getNetworkInterfaces();
- if (networkInterfaces != null) {
- for (NetworkInterface networkInterface : Collections.list(networkInterfaces)) {
- for (InetAddress inetAddress : Collections.list(networkInterface.getInetAddresses())) {
- hosts.add(inetAddress.getHostAddress());
- }
- }
- }
- }
-
- // ensure some hosts were found
- if (!hosts.isEmpty()) {
- String scheme = "http";
- if (properties.getSslPort() != null && serverConnector.getPort() == properties.getSslPort()) {
- scheme = "https";
- }
-
- // dump each url
- for (String host : hosts) {
- urls.add(String.format("%s://%s:%s", scheme, host, serverConnector.getPort()));
- }
- }
- }
- }
-
- if (urls.isEmpty()) {
- logger.warn("NiFi Registry has started, but the UI is not available on any hosts. Please verify the host properties.");
- } else {
- // log the ui location
- logger.info("NiFi Registry has started. The UI is available at the following URLs:");
- for (final String url : urls) {
- logger.info(String.format("%s/nifi-registry", url));
- }
+ shutdown(t);
}
}
public void stop() {
try {
server.stop();
- } catch (Exception ex) {
- logger.warn("Failed to stop web server", ex);
+ } catch (final Exception e) {
+ logger.warn("Server stop failed", e);
}
}
+
+ protected List getApplicationUrls() {
+ return Arrays.stream(server.getConnectors())
+ .map(connector -> (ServerConnector) connector)
+ .map(serverConnector -> {
+ final SslConnectionFactory sslConnectionFactory = serverConnector.getConnectionFactory(SslConnectionFactory.class);
+ final String scheme = sslConnectionFactory == null ? HTTP_SCHEME : HTTPS_SCHEME;
+ final int port = serverConnector.getLocalPort();
+ final String connectorHost = serverConnector.getHost();
+ final String host = StringUtils.defaultIfEmpty(connectorHost, HOST_UNSPECIFIED);
+ return URI.create(String.format(APPLICATION_URL_FORMAT, scheme, host, port));
+ })
+ .collect(Collectors.toList());
+ }
+
+ private void shutdown(final Throwable throwable) {
+ System.err.printf("Server start failed: %s%n", throwable.getMessage());
+ logger.error("Server start failed", throwable);
+ System.exit(1);
+ }
}
diff --git a/nifi-registry/nifi-registry-core/nifi-registry-jetty/src/main/java/org/apache/nifi/registry/jetty/handler/HandlerProvider.java b/nifi-registry/nifi-registry-core/nifi-registry-jetty/src/main/java/org/apache/nifi/registry/jetty/handler/HandlerProvider.java
new file mode 100644
index 0000000000..b1334d44ee
--- /dev/null
+++ b/nifi-registry/nifi-registry-core/nifi-registry-jetty/src/main/java/org/apache/nifi/registry/jetty/handler/HandlerProvider.java
@@ -0,0 +1,33 @@
+/*
+ * 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.nifi.registry.jetty.handler;
+
+import org.apache.nifi.registry.properties.NiFiRegistryProperties;
+import org.eclipse.jetty.server.Handler;
+
+/**
+ * Jetty Handler Provider abstraction for loading and configuring web applications
+ */
+public interface HandlerProvider {
+ /**
+ * Get Jetty Handler using configuration properties
+ *
+ * @param properties Registry properties
+ * @return Jetty Handler
+ */
+ Handler getHandler(NiFiRegistryProperties properties);
+}
diff --git a/nifi-registry/nifi-registry-core/nifi-registry-jetty/src/main/java/org/apache/nifi/registry/jetty/handler/HeaderWriterHandler.java b/nifi-registry/nifi-registry-core/nifi-registry-jetty/src/main/java/org/apache/nifi/registry/jetty/handler/HeaderWriterHandler.java
new file mode 100644
index 0000000000..d31296078b
--- /dev/null
+++ b/nifi-registry/nifi-registry-core/nifi-registry-jetty/src/main/java/org/apache/nifi/registry/jetty/handler/HeaderWriterHandler.java
@@ -0,0 +1,59 @@
+/*
+ * 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.nifi.registry.jetty.handler;
+
+import org.eclipse.jetty.server.Request;
+import org.eclipse.jetty.server.handler.ScopedHandler;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+/**
+ * HTTP Response Header Writer Handler applies standard headers to HTTP responses
+ */
+public class HeaderWriterHandler extends ScopedHandler {
+ protected static final String CONTENT_SECURITY_POLICY_HEADER = "Content-Security-Policy";
+ protected static final String CONTENT_SECURITY_POLICY = "frame-ancestors 'self'";
+
+ protected static final String FRAME_OPTIONS_HEADER = "X-Frame-Options";
+ protected static final String FRAME_OPTIONS = "SAMEORIGIN";
+
+ protected static final String STRICT_TRANSPORT_SECURITY_HEADER = "Strict-Transport-Security";
+ protected static final String STRICT_TRANSPORT_SECURITY = "max-age=31540000";
+
+ protected static final String XSS_PROTECTION_HEADER = "X-XSS-Protection";
+ protected static final String XSS_PROTECTION = "1; mode=block";
+
+ /**
+ * Handle requests and set HTTP response headers
+ *
+ * @param target Target URI
+ * @param request Jetty Request
+ * @param httpServletRequest HTTP Servlet Request
+ * @param httpServletResponse HTTP Servlet Response
+ */
+ @Override
+ public void doHandle(final String target, final Request request, final HttpServletRequest httpServletRequest, final HttpServletResponse httpServletResponse) {
+ httpServletResponse.setHeader(CONTENT_SECURITY_POLICY_HEADER, CONTENT_SECURITY_POLICY);
+ httpServletResponse.setHeader(FRAME_OPTIONS_HEADER, FRAME_OPTIONS);
+ httpServletResponse.setHeader(XSS_PROTECTION_HEADER, XSS_PROTECTION);
+
+ if (httpServletRequest.isSecure()) {
+ httpServletResponse.setHeader(STRICT_TRANSPORT_SECURITY_HEADER, STRICT_TRANSPORT_SECURITY);
+ }
+ }
+}
diff --git a/nifi-registry/nifi-registry-core/nifi-registry-jetty/src/main/java/org/apache/nifi/registry/jetty/handler/StandardHandlerProvider.java b/nifi-registry/nifi-registry-core/nifi-registry-jetty/src/main/java/org/apache/nifi/registry/jetty/handler/StandardHandlerProvider.java
new file mode 100644
index 0000000000..a86c217872
--- /dev/null
+++ b/nifi-registry/nifi-registry-core/nifi-registry-jetty/src/main/java/org/apache/nifi/registry/jetty/handler/StandardHandlerProvider.java
@@ -0,0 +1,257 @@
+/*
+ * 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.nifi.registry.jetty.handler;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.registry.properties.NiFiRegistryProperties;
+import org.apache.nifi.registry.security.crypto.CryptoKeyProvider;
+import org.eclipse.jetty.server.Handler;
+import org.eclipse.jetty.server.handler.HandlerCollection;
+import org.eclipse.jetty.servlet.DefaultServlet;
+import org.eclipse.jetty.servlet.ServletHolder;
+import org.eclipse.jetty.webapp.WebAppClassLoader;
+import org.eclipse.jetty.webapp.WebAppContext;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Objects;
+import java.util.regex.Pattern;
+
+/**
+ * Standard Jetty Handler Provider responsible for loading web applications
+ */
+public class StandardHandlerProvider implements HandlerProvider {
+ private static final String DEFAULTS_DESCRIPTOR = "org/apache/nifi-registry/web/webdefault.xml";
+
+ private static final int MAX_FORM_CONTENT_SIZE = 600000;
+
+ private static final String UI_CONTEXT_PATH = "/nifi-registry";
+
+ private static final Pattern UI_FILE_PATTERN = Pattern.compile("^nifi-registry-web-ui-.+?\\.war$");
+
+ private static final String API_CONTEXT_PATH = "/nifi-registry-api";
+
+ private static final Pattern API_FILE_PATTERN = Pattern.compile("^nifi-registry-web-api-.+?\\.war$");
+
+ private static final String DOCS_CONTEXT_PATH = "/nifi-registry-docs";
+
+ private static final Pattern DOCS_FILE_PATTERN = Pattern.compile("^nifi-registry-web-docs-.+?\\.war$");
+
+ private static final String HTML_DOCS_PATH = "/html/*";
+
+ private static final String REST_API_DOCS_PATH = "/rest-api/*";
+
+ private static final String REST_API_DOCS_RELATIVE_PATH = "webapp/docs";
+
+ private static final String OIDC_SUPPORTED_PARAMETER = "oidc-supported";
+
+ private static final String PROPERTIES_PARAMETER = "nifi-registry.properties";
+
+ private static final String KEY_PROVIDER_PARAMETER = "nifi-registry.key";
+
+ private static final String RESOURCE_BASE_PARAMETER = "resourceBase";
+
+ private static final String DIR_ALLOWED_PARAMETER = "dirAllowed";
+
+ private static final String WEB_INF_JAR_PATTERN_ATTRIBUTE = "org.eclipse.jetty.server.webapp.WebInfIncludeJarPattern";
+
+ private static final String WEB_INF_JAR_PATTERN = ".*/spring-[^/]*\\.jar$";
+
+ private final CryptoKeyProvider cryptoKeyProvider;
+
+ private final String docsDirectory;
+
+ public StandardHandlerProvider(final CryptoKeyProvider cryptoKeyProvider, final String docsDirectory) {
+ this.cryptoKeyProvider = Objects.requireNonNull(cryptoKeyProvider, "Key Provider required");
+ this.docsDirectory = docsDirectory;
+ }
+
+ /**
+ * Get Jetty Handler for Registry Server containing mappings to web applications
+ *
+ * @param properties Registry properties
+ * @return Jetty Handler
+ */
+ @Override
+ public Handler getHandler(final NiFiRegistryProperties properties) {
+ Objects.requireNonNull(properties, "Properties required");
+
+ final File libDirectory = properties.getWarLibDirectory();
+ final File workDirectory = properties.getWebWorkingDirectory();
+
+ final HandlerCollection handlers = new HandlerCollection();
+ // Add Header Writer Handler before others
+ handlers.addHandler(new HeaderWriterHandler());
+
+ final WebAppContext userInterfaceContext = getWebAppContext(libDirectory, workDirectory, ClassLoader.getSystemClassLoader(), UI_FILE_PATTERN, UI_CONTEXT_PATH);
+ userInterfaceContext.setInitParameter(OIDC_SUPPORTED_PARAMETER, Boolean.toString(properties.isOidcEnabled()));
+ handlers.addHandler(userInterfaceContext);
+
+ final ClassLoader apiClassLoader = getApiClassLoader(properties.getDatabaseDriverDirectory());
+ final WebAppContext apiContext = getWebAppContext(libDirectory, workDirectory, apiClassLoader, API_FILE_PATTERN, API_CONTEXT_PATH);
+ apiContext.setAttribute(PROPERTIES_PARAMETER, properties);
+ apiContext.setAttribute(KEY_PROVIDER_PARAMETER, cryptoKeyProvider);
+ handlers.addHandler(apiContext);
+
+ final WebAppContext docsContext = getWebAppContext(libDirectory, workDirectory, ClassLoader.getSystemClassLoader(), DOCS_FILE_PATTERN, DOCS_CONTEXT_PATH);
+ final File docsDir = getDocsDir();
+ final ServletHolder docsServletHolder = getDocsServletHolder(docsDir);
+ docsContext.addServlet(docsServletHolder, HTML_DOCS_PATH);
+
+ final File apiDocsDir = getApiDocsDir(apiContext);
+ final ServletHolder apiDocsServletHolder = getDocsServletHolder(apiDocsDir);
+ docsContext.addServlet(apiDocsServletHolder, REST_API_DOCS_PATH);
+
+ handlers.addHandler(docsContext);
+
+ return handlers;
+ }
+
+ private ClassLoader getApiClassLoader(final String databaseDriverDirectory) {
+ final URL[] resourceLocations = getResourceLocations(databaseDriverDirectory);
+ final ClassLoader apiClassLoader;
+ if (resourceLocations.length == 0) {
+ apiClassLoader = ClassLoader.getSystemClassLoader();
+ } else {
+ apiClassLoader = new URLClassLoader(resourceLocations, ClassLoader.getSystemClassLoader());
+ }
+ return apiClassLoader;
+ }
+
+ private WebAppContext getWebAppContext(
+ final File libDirectory,
+ final File workDirectory,
+ final ClassLoader parentClassLoader,
+ final Pattern applicationFilePattern,
+ final String contextPath
+ ) {
+ final File applicationFile = getApplicationFile(libDirectory, applicationFilePattern);
+ final WebAppContext webAppContext = new WebAppContext(applicationFile.getPath(), contextPath);
+ webAppContext.setContextPath(contextPath);
+ webAppContext.setDefaultsDescriptor(DEFAULTS_DESCRIPTOR);
+ webAppContext.setMaxFormContentSize(MAX_FORM_CONTENT_SIZE);
+ webAppContext.setAttribute(WEB_INF_JAR_PATTERN_ATTRIBUTE, WEB_INF_JAR_PATTERN);
+
+ final File tempDirectory = getTempDirectory(workDirectory, applicationFile.getName());
+ webAppContext.setTempDirectory(tempDirectory);
+
+ try {
+ final WebAppClassLoader webAppClassLoader = new WebAppClassLoader(parentClassLoader, webAppContext);
+ webAppContext.setClassLoader(webAppClassLoader);
+ } catch (final IOException e) {
+ throw new IllegalStateException(String.format("Application Context Path [%s] ClassLoader creation failed", contextPath), e);
+ }
+
+ return webAppContext;
+ }
+
+ private File getApplicationFile(final File directory, final Pattern filenamePattern) {
+ final File[] applicationFiles = directory.listFiles((file, filename) -> filenamePattern.matcher(filename).matches());
+ if (applicationFiles == null || applicationFiles.length == 0) {
+ throw new IllegalStateException(String.format("Required Application matching [%s] not found in directory [%s]", filenamePattern, directory));
+ }
+ return applicationFiles[0];
+ }
+
+ private File getTempDirectory(final File webWorkingDirectory, final String filename) {
+ final File tempDirectory = new File(webWorkingDirectory, filename);
+ if (tempDirectory.isDirectory()) {
+ if (tempDirectory.canWrite()) {
+ return tempDirectory;
+ } else {
+ throw new IllegalStateException(String.format("Temporary Directory [%s] not writable", tempDirectory));
+ }
+ } else {
+ if (tempDirectory.mkdirs()) {
+ return tempDirectory;
+ } else {
+ throw new IllegalStateException(String.format("Temporary Directory [%s] creation failed", tempDirectory));
+ }
+ }
+ }
+
+ private URL[] getResourceLocations(final String databaseDriverDirectory) {
+ final URL[] resourceLocations;
+
+ if (StringUtils.isBlank(databaseDriverDirectory)) {
+ resourceLocations = new URL[0];
+ } else {
+ final File driverDirectory = new File(databaseDriverDirectory);
+ if (driverDirectory.canRead()) {
+ final List locations = new ArrayList<>();
+ final URL driverDirectoryUrl = getUrl(driverDirectory);
+ locations.add(driverDirectoryUrl);
+
+ final File[] files = driverDirectory.listFiles();
+ if (files != null) {
+ Arrays.stream(files)
+ .filter(File::isFile)
+ .map(this::getUrl)
+ .forEach(locations::add);
+ }
+ resourceLocations = locations.toArray(new URL[0]);
+ } else {
+ resourceLocations = new URL[0];
+ }
+ }
+ return resourceLocations;
+ }
+
+ private URL getUrl(final File file) {
+ try {
+ return file.toURI().toURL();
+ } catch (final MalformedURLException e) {
+ throw new IllegalStateException(String.format("File URL [%s] conversion failed", file), e);
+ }
+ }
+
+ private File getDocsDir() {
+ File docsDir;
+ try {
+ docsDir = Paths.get(docsDirectory).toRealPath().toFile();
+ } catch (IOException e) {
+ docsDir = new File(docsDirectory).getAbsoluteFile();
+ if (!docsDir.mkdirs()) {
+ final String message = String.format("Documentation Directory [%s] creation failed", docsDir.getAbsolutePath());
+ throw new IllegalStateException(message);
+ }
+ }
+ return docsDir;
+ }
+
+ private ServletHolder getDocsServletHolder(final File directory) {
+ final ServletHolder servletHolder = new ServletHolder(directory.getPath(), DefaultServlet.class);
+ servletHolder.setInitParameter(RESOURCE_BASE_PARAMETER, directory.getPath());
+ servletHolder.setInitParameter(DIR_ALLOWED_PARAMETER, Boolean.FALSE.toString());
+ return servletHolder;
+ }
+
+ private File getApiDocsDir(final WebAppContext apiContext) {
+ final File apiDocsDir = new File(apiContext.getTempDirectory(), REST_API_DOCS_RELATIVE_PATH);
+ if (apiDocsDir.canRead() || apiDocsDir.mkdirs()) {
+ return apiDocsDir;
+ }
+ throw new IllegalStateException(String.format("REST API Documentation Directory [%s] not readable", apiDocsDir.getAbsolutePath()));
+ }
+}
diff --git a/nifi-registry/nifi-registry-core/nifi-registry-jetty/src/main/java/org/apache/nifi/registry/jetty/headers/ContentSecurityPolicyFilter.java b/nifi-registry/nifi-registry-core/nifi-registry-jetty/src/main/java/org/apache/nifi/registry/jetty/headers/ContentSecurityPolicyFilter.java
deleted file mode 100644
index 758e939599..0000000000
--- a/nifi-registry/nifi-registry-core/nifi-registry-jetty/src/main/java/org/apache/nifi/registry/jetty/headers/ContentSecurityPolicyFilter.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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.nifi.registry.jetty.headers;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.servlet.Filter;
-import javax.servlet.FilterChain;
-import javax.servlet.FilterConfig;
-import javax.servlet.ServletException;
-import javax.servlet.ServletRequest;
-import javax.servlet.ServletResponse;
-import javax.servlet.http.HttpServletResponse;
-import java.io.IOException;
-
-/**
- * A filter to apply the Content Security Policy header.
- *
- */
-public class ContentSecurityPolicyFilter implements Filter {
- private static final String HEADER = "Content-Security-Policy";
- private static final String POLICY = "frame-ancestors 'self'";
-
- private static final Logger logger = LoggerFactory.getLogger(ContentSecurityPolicyFilter.class);
-
- @Override
- public void doFilter(final ServletRequest req, final ServletResponse resp, final FilterChain filterChain)
- throws IOException, ServletException {
-
- final HttpServletResponse response = (HttpServletResponse) resp;
- response.setHeader(HEADER, POLICY);
-
- filterChain.doFilter(req, resp);
- }
-
- @Override
- public void init(final FilterConfig config) {
- }
-
- @Override
- public void destroy() {
- }
-}
\ No newline at end of file
diff --git a/nifi-registry/nifi-registry-core/nifi-registry-jetty/src/main/java/org/apache/nifi/registry/jetty/headers/StrictTransportSecurityFilter.java b/nifi-registry/nifi-registry-core/nifi-registry-jetty/src/main/java/org/apache/nifi/registry/jetty/headers/StrictTransportSecurityFilter.java
deleted file mode 100644
index 7f0f913885..0000000000
--- a/nifi-registry/nifi-registry-core/nifi-registry-jetty/src/main/java/org/apache/nifi/registry/jetty/headers/StrictTransportSecurityFilter.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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.nifi.registry.jetty.headers;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.servlet.Filter;
-import javax.servlet.FilterChain;
-import javax.servlet.FilterConfig;
-import javax.servlet.ServletException;
-import javax.servlet.ServletRequest;
-import javax.servlet.ServletResponse;
-import javax.servlet.http.HttpServletResponse;
-import java.io.IOException;
-
-/**
- * A filter to apply the HTTP Strict Transport Security (HSTS) HTTP header. This forces the browser to use HTTPS for
- * all
- */
-public class StrictTransportSecurityFilter implements Filter {
- private static final String HEADER = "Strict-Transport-Security";
- private static final String POLICY = "max-age=31540000";
-
- private static final Logger logger = LoggerFactory.getLogger(StrictTransportSecurityFilter.class);
-
- @Override
- public void doFilter(final ServletRequest req, final ServletResponse resp, final FilterChain filterChain)
- throws IOException, ServletException {
-
- final HttpServletResponse response = (HttpServletResponse) resp;
- response.setHeader(HEADER, POLICY);
-
- filterChain.doFilter(req, resp);
- }
-
- @Override
- public void init(final FilterConfig config) {
- }
-
- @Override
- public void destroy() {
- }
-}
\ No newline at end of file
diff --git a/nifi-registry/nifi-registry-core/nifi-registry-jetty/src/main/java/org/apache/nifi/registry/jetty/headers/XFrameOptionsFilter.java b/nifi-registry/nifi-registry-core/nifi-registry-jetty/src/main/java/org/apache/nifi/registry/jetty/headers/XFrameOptionsFilter.java
deleted file mode 100644
index fad5bbcb05..0000000000
--- a/nifi-registry/nifi-registry-core/nifi-registry-jetty/src/main/java/org/apache/nifi/registry/jetty/headers/XFrameOptionsFilter.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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.nifi.registry.jetty.headers;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.servlet.Filter;
-import javax.servlet.FilterChain;
-import javax.servlet.FilterConfig;
-import javax.servlet.ServletException;
-import javax.servlet.ServletRequest;
-import javax.servlet.ServletResponse;
-import javax.servlet.http.HttpServletResponse;
-import java.io.IOException;
-
-/**
- * A filter to apply the X-Frame-Options header.
- *
- */
-public class XFrameOptionsFilter implements Filter {
- private static final String HEADER = "X-Frame-Options";
- private static final String POLICY = "SAMEORIGIN";
-
- private static final Logger logger = LoggerFactory.getLogger(XFrameOptionsFilter.class);
-
- @Override
- public void doFilter(final ServletRequest req, final ServletResponse resp, final FilterChain filterChain)
- throws IOException, ServletException {
-
- final HttpServletResponse response = (HttpServletResponse) resp;
- response.setHeader(HEADER, POLICY);
-
- filterChain.doFilter(req, resp);
- }
-
- @Override
- public void init(final FilterConfig config) {
- }
-
- @Override
- public void destroy() {
- }
-}
\ No newline at end of file
diff --git a/nifi-registry/nifi-registry-core/nifi-registry-jetty/src/main/java/org/apache/nifi/registry/jetty/headers/XSSProtectionFilter.java b/nifi-registry/nifi-registry-core/nifi-registry-jetty/src/main/java/org/apache/nifi/registry/jetty/headers/XSSProtectionFilter.java
deleted file mode 100644
index 62792f1b4f..0000000000
--- a/nifi-registry/nifi-registry-core/nifi-registry-jetty/src/main/java/org/apache/nifi/registry/jetty/headers/XSSProtectionFilter.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * 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.nifi.registry.jetty.headers;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.servlet.Filter;
-import javax.servlet.FilterChain;
-import javax.servlet.FilterConfig;
-import javax.servlet.ServletException;
-import javax.servlet.ServletRequest;
-import javax.servlet.ServletResponse;
-import javax.servlet.http.HttpServletResponse;
-import java.io.IOException;
-
-/**
- * A filter to apply the Cross Site Scripting (XSS) HTTP header. Protects against reflected cross-site scripting attacks.
- * The browser will prevent rendering of the page if an attack is detected.
- */
-
-public class XSSProtectionFilter implements Filter {
- private static final String HEADER = "X-XSS-Protection";
- private static final String POLICY = "1; mode=block";
-
- private static final Logger logger = LoggerFactory.getLogger(XSSProtectionFilter.class);
-
- @Override
- public void doFilter(final ServletRequest req, final ServletResponse resp, final FilterChain filterChain)
- throws IOException, ServletException {
-
- final HttpServletResponse response = (HttpServletResponse) resp;
- response.setHeader(HEADER, POLICY);
-
- filterChain.doFilter(req, resp);
- }
-
- @Override
- public void init(final FilterConfig config) {
- }
-
- @Override
- public void destroy() {
- }
-}
\ No newline at end of file
diff --git a/nifi-registry/nifi-registry-core/nifi-registry-jetty/src/test/java/org/apache/nifi/registry/jetty/JettyServerTest.java b/nifi-registry/nifi-registry-core/nifi-registry-jetty/src/test/java/org/apache/nifi/registry/jetty/JettyServerTest.java
new file mode 100644
index 0000000000..2554533e10
--- /dev/null
+++ b/nifi-registry/nifi-registry-core/nifi-registry-jetty/src/test/java/org/apache/nifi/registry/jetty/JettyServerTest.java
@@ -0,0 +1,77 @@
+/*
+ * 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.nifi.registry.jetty;
+
+import org.apache.nifi.registry.jetty.handler.HandlerProvider;
+import org.apache.nifi.registry.properties.NiFiRegistryProperties;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+import java.net.HttpURLConnection;
+import java.net.URI;
+import java.net.URL;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@ExtendWith(MockitoExtension.class)
+public class JettyServerTest {
+ private static final String RANDOM_PORT = "0";
+
+ private static final String LOCALHOST = "127.0.0.1";
+
+ private static final int TIMEOUT = 5000;
+
+ @Mock
+ private HandlerProvider handlerProvider;
+
+ @Test
+ void testStartStop() throws Exception {
+ final Map requiredProperties = new LinkedHashMap<>();
+ requiredProperties.put(NiFiRegistryProperties.WEB_HTTP_PORT, RANDOM_PORT);
+ requiredProperties.put(NiFiRegistryProperties.WEB_HTTP_HOST, LOCALHOST);
+ final NiFiRegistryProperties properties = new NiFiRegistryProperties(requiredProperties);
+
+ final JettyServer server = new JettyServer(properties, handlerProvider);
+
+ try {
+ server.start();
+
+ final Iterator applicationUrls = server.getApplicationUrls().iterator();
+ assertTrue(applicationUrls.hasNext());
+ assertServerRunning(applicationUrls.next());
+ } finally {
+ server.stop();
+ }
+ }
+
+ private void assertServerRunning(final URI applicationUrl) throws Exception {
+ final URL url = applicationUrl.toURL();
+ final HttpURLConnection connection = (HttpURLConnection) url.openConnection();
+ connection.setConnectTimeout(TIMEOUT);
+ connection.setReadTimeout(TIMEOUT);
+ connection.connect();
+
+ final int responseCode = connection.getResponseCode();
+ assertEquals(HttpURLConnection.HTTP_NOT_FOUND, responseCode);
+ }
+}
diff --git a/nifi-registry/nifi-registry-core/nifi-registry-jetty/src/test/java/org/apache/nifi/registry/jetty/handler/HeaderWriterHandlerTest.java b/nifi-registry/nifi-registry-core/nifi-registry-jetty/src/test/java/org/apache/nifi/registry/jetty/handler/HeaderWriterHandlerTest.java
new file mode 100644
index 0000000000..8ed3457c06
--- /dev/null
+++ b/nifi-registry/nifi-registry-core/nifi-registry-jetty/src/test/java/org/apache/nifi/registry/jetty/handler/HeaderWriterHandlerTest.java
@@ -0,0 +1,76 @@
+/*
+ * 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.nifi.registry.jetty.handler;
+
+import org.eclipse.jetty.server.Request;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+@ExtendWith(MockitoExtension.class)
+class HeaderWriterHandlerTest {
+ private static final String TARGET = HeaderWriterHandler.class.getSimpleName();
+
+ @Mock
+ private Request request;
+
+ @Mock
+ private HttpServletRequest httpServletRequest;
+
+ @Mock
+ private HttpServletResponse httpServletResponse;
+
+ private HeaderWriterHandler handler;
+
+ @BeforeEach
+ void setHandler() {
+ handler = new HeaderWriterHandler();
+ }
+
+ @Test
+ void testDoHandle() {
+ handler.doHandle(TARGET, request, httpServletRequest, httpServletResponse);
+
+ verifyStandardHeaders();
+ }
+
+ @Test
+ void testDoHandleSecure() {
+ when(httpServletRequest.isSecure()).thenReturn(true);
+
+ handler.doHandle(TARGET, request, httpServletRequest, httpServletResponse);
+
+ verifyStandardHeaders();
+ verify(httpServletResponse).setHeader(eq(HeaderWriterHandler.STRICT_TRANSPORT_SECURITY_HEADER), any());
+ }
+
+ private void verifyStandardHeaders() {
+ verify(httpServletResponse).setHeader(eq(HeaderWriterHandler.CONTENT_SECURITY_POLICY_HEADER), any());
+ verify(httpServletResponse).setHeader(eq(HeaderWriterHandler.FRAME_OPTIONS_HEADER), any());
+ verify(httpServletResponse).setHeader(eq(HeaderWriterHandler.XSS_PROTECTION_HEADER), any());
+ }
+}
diff --git a/nifi-registry/nifi-registry-core/nifi-registry-jetty/src/test/java/org/apache/nifi/registry/jetty/handler/StandardHandlerProviderTest.java b/nifi-registry/nifi-registry-core/nifi-registry-jetty/src/test/java/org/apache/nifi/registry/jetty/handler/StandardHandlerProviderTest.java
new file mode 100644
index 0000000000..c1ec0e32e0
--- /dev/null
+++ b/nifi-registry/nifi-registry-core/nifi-registry-jetty/src/test/java/org/apache/nifi/registry/jetty/handler/StandardHandlerProviderTest.java
@@ -0,0 +1,51 @@
+/*
+ * 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.nifi.registry.jetty.handler;
+
+import org.apache.nifi.registry.properties.NiFiRegistryProperties;
+import org.apache.nifi.registry.security.crypto.CryptoKeyProvider;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.io.TempDir;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+import java.io.File;
+
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+@ExtendWith(MockitoExtension.class)
+class StandardHandlerProviderTest {
+ @Mock
+ private CryptoKeyProvider cryptoKeyProvider;
+
+ private StandardHandlerProvider provider;
+
+ @BeforeEach
+ void setProvider(@TempDir final File tempDir) {
+ final String docsDirectory = tempDir.getAbsolutePath();
+ provider = new StandardHandlerProvider(cryptoKeyProvider, docsDirectory);
+ }
+
+ @Test
+ void testGetHandlerApplicationNotFound() {
+ final NiFiRegistryProperties properties = new NiFiRegistryProperties();
+
+ assertThrows(IllegalStateException.class, () -> provider.getHandler(properties));
+ }
+}
diff --git a/nifi-registry/nifi-registry-core/nifi-registry-runtime/src/main/java/org/apache/nifi/registry/NiFiRegistry.java b/nifi-registry/nifi-registry-core/nifi-registry-runtime/src/main/java/org/apache/nifi/registry/NiFiRegistry.java
index e3e6447fb4..601fc49572 100644
--- a/nifi-registry/nifi-registry-core/nifi-registry-runtime/src/main/java/org/apache/nifi/registry/NiFiRegistry.java
+++ b/nifi-registry/nifi-registry-core/nifi-registry-runtime/src/main/java/org/apache/nifi/registry/NiFiRegistry.java
@@ -17,6 +17,8 @@
package org.apache.nifi.registry;
import org.apache.nifi.registry.jetty.JettyServer;
+import org.apache.nifi.registry.jetty.handler.HandlerProvider;
+import org.apache.nifi.registry.jetty.handler.StandardHandlerProvider;
import org.apache.nifi.registry.properties.NiFiRegistryProperties;
import org.apache.nifi.registry.security.crypto.BootstrapFileCryptoKeyProvider;
import org.apache.nifi.registry.security.crypto.CryptoKeyProvider;
@@ -55,25 +57,13 @@ public class NiFiRegistry {
private final BootstrapListener bootstrapListener;
private volatile boolean shutdown = false;
- public NiFiRegistry(final NiFiRegistryProperties properties, CryptoKeyProvider masterKeyProvider)
- throws ClassNotFoundException, IOException, NoSuchMethodException, InstantiationException, IllegalAccessException, IllegalArgumentException, InvocationTargetException {
+ public NiFiRegistry(final NiFiRegistryProperties properties, CryptoKeyProvider masterKeyProvider) throws IOException, IllegalArgumentException {
- Thread.setDefaultUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() {
- @Override
- public void uncaughtException(final Thread t, final Throwable e) {
- LOGGER.error("An Unknown Error Occurred in Thread {}: {}", t, e.toString());
- LOGGER.error("", e);
- }
- });
+ Thread.setDefaultUncaughtExceptionHandler((t, e) -> LOGGER.error("An Unknown Error Occurred in Thread {}", t, e));
// register the shutdown hook
- Runtime.getRuntime().addShutdownHook(new Thread(new Runnable() {
- @Override
- public void run() {
- // shutdown the jetty server
- shutdownHook();
- }
- }));
+ // shutdown the jetty server
+ Runtime.getRuntime().addShutdownHook(new Thread(this::shutdownHook));
final String bootstrapPort = System.getProperty(BOOTSTRAP_PORT_PROPERTY);
if (bootstrapPort != null) {
@@ -107,11 +97,12 @@ public class NiFiRegistry {
SLF4JBridgeHandler.removeHandlersForRootLogger();
SLF4JBridgeHandler.install();
- final String docsDir = System.getProperty(NiFiRegistryProperties.NIFI_REGISTRY_BOOTSTRAP_DOCS_DIR_PROPERTY,
- NiFiRegistryProperties.RELATIVE_DOCS_LOCATION);
-
final long startTime = System.nanoTime();
- server = new JettyServer(properties, masterKeyProvider, docsDir);
+
+ final String docsDirectory = System.getProperty(NiFiRegistryProperties.NIFI_REGISTRY_BOOTSTRAP_DOCS_DIR_PROPERTY,
+ NiFiRegistryProperties.RELATIVE_DOCS_LOCATION);
+ final HandlerProvider handlerProvider = new StandardHandlerProvider(masterKeyProvider, docsDirectory);
+ server = new JettyServer(properties, handlerProvider);
if (shutdown) {
LOGGER.info("NiFi Registry has been shutdown via NiFi Registry Bootstrap. Will not start Controller");
@@ -123,8 +114,8 @@ public class NiFiRegistry {
}
final long duration = System.nanoTime() - startTime;
- LOGGER.info("Registry initialization took " + duration + " nanoseconds "
- + "(" + (int) TimeUnit.SECONDS.convert(duration, TimeUnit.NANOSECONDS) + " seconds).");
+ final double durationSeconds = TimeUnit.NANOSECONDS.toMillis(duration) / 1000.0;
+ LOGGER.info("Started Application in {} seconds ({} ns)", durationSeconds, duration);
}
}