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/23 03:38:21 UTC

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

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