You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by "exceptionfactory (via GitHub)" <gi...@apache.org> on 2023/05/17 15:41:50 UTC

[GitHub] [nifi] exceptionfactory opened a new pull request, #7258: NIFI-11558 Apply Security Headers to All Registry Responses

exceptionfactory opened a new pull request, #7258:
URL: https://github.com/apache/nifi/pull/7258

   # Summary
   
   [NIFI-11558](https://issues.apache.org/jira/browse/NIFI-11558) Updates NiFi Registry Jetty Server configuration to apply standard HTTP response headers to all requests regardless of application.
   
   The existing implementation uses individual Servlet Filters applied to the UI and REST API applications, which covers normal usage, but does not include requests to the root path of the server.
   
   The new implementation uses a Jetty Handler implementation that applies standard headers to all responses. Adjustments include refactoring the Registry Jetty Server to abstract application preparation from the server startup for improved maintenance and testing.
   
   # Tracking
   
   Please complete the following tracking steps prior to pull request creation.
   
   ### Issue Tracking
   
   - [X] [Apache NiFi Jira](https://issues.apache.org/jira/browse/NIFI) issue created
   
   ### Pull Request Tracking
   
   - [X] Pull Request title starts with Apache NiFi Jira issue number, such as `NIFI-00000`
   - [X] Pull Request commit message starts with Apache NiFi Jira issue number, as such `NIFI-00000`
   
   ### Pull Request Formatting
   
   - [X] Pull Request based on current revision of the `main` branch
   - [X] Pull Request refers to a feature branch with one commit containing changes
   
   # Verification
   
   Please indicate the verification steps performed prior to pull request creation.
   
   ### Build
   
   - [X] Build completed using `mvn clean install -P contrib-check`
     - [X] JDK 11
     - [ ] JDK 17
   
   ### Licensing
   
   - [ ] New dependencies are compatible with the [Apache License 2.0](https://apache.org/licenses/LICENSE-2.0) according to the [License Policy](https://www.apache.org/legal/resolved.html)
   - [ ] New dependencies are documented in applicable `LICENSE` and `NOTICE` files
   
   ### Documentation
   
   - [ ] Documentation formatting appears as expected in rendered files
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] gresockj commented on a diff in pull request #7258: NIFI-11558 Apply Security Headers to All Registry Responses

Posted by "gresockj (via GitHub)" <gi...@apache.org>.
gresockj commented on code in PR #7258:
URL: https://github.com/apache/nifi/pull/7258#discussion_r1200318430


##########
nifi-registry/nifi-registry-core/nifi-registry-jetty/src/main/java/org/apache/nifi/registry/jetty/JettyServer.java:
##########
@@ -178,303 +118,60 @@ private void configureConnectors() {
         }
     }
 
-    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<String> 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<Class<? extends Filter>> 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<URL> 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<Throwable> 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();
-        } 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<String> urls = new ArrayList<>();
-
-        for (Connector connector : server.getConnectors()) {
-            if (connector instanceof ServerConnector) {
-                final ServerConnector serverConnector = (ServerConnector) connector;
-
-                Set<String> hosts = new HashSet<>();
-
-                // determine the hosts
-                if (StringUtils.isNotBlank(serverConnector.getHost())) {
-                    hosts.add(serverConnector.getHost());
-                } else {
-                    Enumeration<NetworkInterface> networkInterfaces = NetworkInterface.getNetworkInterfaces();
-                    if (networkInterfaces != null) {
-                        for (NetworkInterface networkInterface : Collections.list(networkInterfaces)) {
-                            for (InetAddress inetAddress : Collections.list(networkInterface.getInetAddresses())) {
-                                hosts.add(inetAddress.getHostAddress());
-                            }
-                        }
-                    }
+            final List<URI> applicationUrls = getApplicationUrls();
+            if (applicationUrls.isEmpty()) {
+                logger.warn("Started Server without connectors");

Review Comment:
   If this PR is applied to `support/nifi-1.x`, we should be aware that the server startup text is changing.  Prior to this PR, the startup text looked like this:
   ```
   2023-05-22 06:14:36,044 INFO [main] org.eclipse.jetty.server.Server Started @10783ms
   2023-05-22 06:14:36,048 INFO [main] o.apache.nifi.registry.jetty.JettyServer NiFi Registry has started. The UI is available at the following URLs:
   2023-05-22 06:14:36,049 INFO [main] o.apache.nifi.registry.jetty.JettyServer http://192.168.1.146:18080/nifi-registry
   2023-05-22 06:14:36,049 INFO [main] o.apache.nifi.registry.jetty.JettyServer http://10.19.8.251:18080/nifi-registry
   2023-05-22 06:14:36,049 INFO [main] o.apache.nifi.registry.jetty.JettyServer http://127.0.0.1:18080/nifi-registry
   2023-05-22 06:14:36,050 INFO [main] o.apache.nifi.registry.BootstrapListener Successfully initiated communication with Bootstrap
   2023-05-22 06:14:36,050 INFO [main] org.apache.nifi.registry.NiFiRegistry Registry initialization took 9841683488 nanoseconds (9 seconds).
   ```
   
   Now it looks like this:
   ```
   2023-05-22 06:23:40,314 INFO [main] org.eclipse.jetty.server.Server Started @10053ms
   2023-05-22 06:23:40,315 INFO [main] o.apache.nifi.registry.jetty.JettyServer Started Server on http://0.0.0.0:18080/nifi-registry
   2023-05-22 06:23:40,316 INFO [main] o.apache.nifi.registry.BootstrapListener Successfully initiated communication with Bootstrap
   2023-05-22 06:23:40,317 INFO [main] org.apache.nifi.registry.NiFiRegistry Started Application in 9.26 seconds (9260276707 ns)
   ```
   
   This may not be an issue, but I wanted to at least point it out in case there are any users who detect server startup using any of these log statements.  The line `org.eclipse.jetty.server.Server Started` is at least consistent.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #7258: NIFI-11558 Apply Security Headers to All Registry Responses

Posted by "exceptionfactory (via GitHub)" <gi...@apache.org>.
exceptionfactory commented on code in PR #7258:
URL: https://github.com/apache/nifi/pull/7258#discussion_r1201471521


##########
nifi-registry/nifi-registry-core/nifi-registry-jetty/src/main/java/org/apache/nifi/registry/jetty/JettyServer.java:
##########
@@ -178,303 +118,60 @@ private void configureConnectors() {
         }
     }
 
-    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<String> 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<Class<? extends Filter>> 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<URL> 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<Throwable> 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();
-        } 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<String> urls = new ArrayList<>();
-
-        for (Connector connector : server.getConnectors()) {
-            if (connector instanceof ServerConnector) {
-                final ServerConnector serverConnector = (ServerConnector) connector;
-
-                Set<String> hosts = new HashSet<>();
-
-                // determine the hosts
-                if (StringUtils.isNotBlank(serverConnector.getHost())) {
-                    hosts.add(serverConnector.getHost());
-                } else {
-                    Enumeration<NetworkInterface> networkInterfaces = NetworkInterface.getNetworkInterfaces();
-                    if (networkInterfaces != null) {
-                        for (NetworkInterface networkInterface : Collections.list(networkInterfaces)) {
-                            for (InetAddress inetAddress : Collections.list(networkInterface.getInetAddresses())) {
-                                hosts.add(inetAddress.getHostAddress());
-                            }
-                        }
-                    }
+            final List<URI> applicationUrls = getApplicationUrls();
+            if (applicationUrls.isEmpty()) {
+                logger.warn("Started Server without connectors");

Review Comment:
   Thanks for highlighting this change @gresockj.
   
   The adjustment to the listening addresses is intentional as it reflects the Jetty Server Connector address of `0.0.0.0` when not explicitly configured. The current approach enumerates all available interfaces and addresses, which is correct in the sense that `0.0.0.0` means all interfaces, but the change brings the NiFi Registry logs in line with Jetty configuration.
   
   The startup time logging change reflects more recent adjustments in NiFi where the rounded number of seconds is listed, followed by the nanonseconds.
   
   I think the change should be backported, but thanks for highlighting the differences.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] asfgit closed pull request #7258: NIFI-11558 Apply Security Headers to All Registry Responses

Posted by "asfgit (via GitHub)" <gi...@apache.org>.
asfgit closed pull request #7258: NIFI-11558 Apply Security Headers to All Registry Responses
URL: https://github.com/apache/nifi/pull/7258


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #7258: NIFI-11558 Apply Security Headers to All Registry Responses

Posted by "exceptionfactory (via GitHub)" <gi...@apache.org>.
exceptionfactory commented on code in PR #7258:
URL: https://github.com/apache/nifi/pull/7258#discussion_r1201472151


##########
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);

Review Comment:
   Thanks for catching this problem @gresockj! The problem was due to using the same name for the regular documentation and REST API documentation Servlet Holders. As a result, the REST API Holder overwrote the regular documentation. I corrected the naming to use the directory path, which is unique, now all documentation pages render as expected.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on pull request #7258: NIFI-11558 Apply Security Headers to All Registry Responses

Posted by "exceptionfactory (via GitHub)" <gi...@apache.org>.
exceptionfactory commented on PR #7258:
URL: https://github.com/apache/nifi/pull/7258#issuecomment-1558464977

   Thanks for the review @gresockj, I addressed the documentation issue as described.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] gresockj commented on a diff in pull request #7258: NIFI-11558 Apply Security Headers to All Registry Responses

Posted by "gresockj (via GitHub)" <gi...@apache.org>.
gresockj commented on code in PR #7258:
URL: https://github.com/apache/nifi/pull/7258#discussion_r1200342923


##########
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);

Review Comment:
   Something appears to be wrong with the docs:
   
   <img width="1156" alt="image" src="https://github.com/apache/nifi/assets/423565/f894ac1f-0d97-4b1b-89f5-30d84736f7c0">
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] gresockj commented on a diff in pull request #7258: NIFI-11558 Apply Security Headers to All Registry Responses

Posted by "gresockj (via GitHub)" <gi...@apache.org>.
gresockj commented on code in PR #7258:
URL: https://github.com/apache/nifi/pull/7258#discussion_r1200342923


##########
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);

Review Comment:
   Something appears to be wrong with the docs:
   
   ![image](https://github.com/apache/nifi/assets/423565/9264a194-d7ac-4b04-9e77-142280319240)
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org