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

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

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