You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2020/06/30 19:44:33 UTC

[GitHub] [nifi] mattyb149 opened a new pull request #4376: NIFI-7592: Allow NiFi to be started without a GUI/REST interface

mattyb149 opened a new pull request #4376:
URL: https://github.com/apache/nifi/pull/4376


   Thank you for submitting a contribution to Apache NiFi.
   
   Please provide a short description of the PR here:
   
   #### Description of PR
   
   Refactors the framework code to move Web/UI-related stuff into its own NAR, and to use ServiceLoader to find a NiFiServer implementation rather than hardcoding the JettyServer class as the required implementation (and thus a required NAR).
   
   In order to streamline the review of the contribution we ask you
   to ensure the following steps have been taken:
   
   ### For all changes:
   - [x] Is there a JIRA ticket associated with this PR? Is it referenced 
        in the commit message?
   
   - [x] Does your PR title start with **NIFI-XXXX** where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
   
   - [x] Has your PR been rebased against the latest commit within the target branch (typically `master`)?
   
   - [x] Is your initial contribution a single, squashed commit? _Additional commits in response to PR reviewer feedback should be made on this branch and pushed to allow change tracking. Do not `squash` or use `--force` when pushing to allow for clean monitoring of changes._
   
   ### For code changes:
   - [ ] Have you ensured that the full suite of tests is executed via `mvn -Pcontrib-check clean install` at the root `nifi` folder?
   - [ ] Have you written or updated unit tests to verify your changes?
   - [ ] Have you verified that the full build is successful on JDK 8?
   - [ ] Have you verified that the full build is successful on JDK 11?
   - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)? 
   - [ ] If applicable, have you updated the `LICENSE` file, including the main `LICENSE` file under `nifi-assembly`?
   - [ ] If applicable, have you updated the `NOTICE` file, including the main `NOTICE` file found under `nifi-assembly`?
   - [ ] If adding new Properties, have you added `.displayName` in addition to .name (programmatic access) for each of the new properties?
   
   ### For documentation related changes:
   - [ ] Have you ensured that format looks appropriate for the output in which it is rendered?
   
   ### Note:
   Please ensure that once the PR is submitted, you check GitHub Actions CI for build issues and submit an update to your PR as soon as possible.
   


----------------------------------------------------------------
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.

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



[GitHub] [nifi] markap14 commented on a change in pull request #4376: NIFI-7592: Allow NiFi to be started without a GUI/REST interface

Posted by GitBox <gi...@apache.org>.
markap14 commented on a change in pull request #4376:
URL: https://github.com/apache/nifi/pull/4376#discussion_r459535677



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/minifi-server/src/main/java/org/apache/nifi/minifi/FlowParser.java
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.minifi;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.nifi.util.LoggingXmlParserErrorHandler;
+import org.apache.nifi.util.file.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+import org.xml.sax.SAXException;
+
+import javax.xml.XMLConstants;
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.transform.Result;
+import javax.xml.transform.Source;
+import javax.xml.transform.TransformerException;
+import javax.xml.transform.TransformerFactory;
+import javax.xml.transform.dom.DOMSource;
+import javax.xml.transform.stream.StreamResult;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.zip.GZIPInputStream;
+import java.util.zip.GZIPOutputStream;
+
+/**
+ * Parses a flow from its xml.gz format into an XML {@link Document}.  This class is primarily toward utilities for assisting
+ * with the handling of component bundles.
+ * <p>
+ * Provides auxiliary methods to aid in evaluating and manipulating the flow.
+ */
+public class FlowParser {
+
+    private static final Logger logger = LoggerFactory.getLogger(FlowParser.class);
+
+    /**
+     * Generates a {@link Document} from the flow configuration file provided
+     */
+    public Document parse(final File flowConfigurationFile) {
+        if (flowConfigurationFile == null) {
+            logger.debug("Flow Configuration file was null");
+            return null;
+        }
+
+        // if the flow doesn't exist or is 0 bytes, then return null
+        final Path flowPath = flowConfigurationFile.toPath();
+        try {
+            if (!Files.exists(flowPath) || Files.size(flowPath) == 0) {
+                logger.warn("Flow Configuration does not exist or was empty");
+                return null;
+            }
+        } catch (IOException e) {
+            logger.error("An error occurred determining the size of the Flow Configuration file");
+            return null;
+        }
+
+        // otherwise create the appropriate input streams to read the file
+        try (final InputStream in = Files.newInputStream(flowPath, StandardOpenOption.READ);
+             final InputStream gzipIn = new GZIPInputStream(in)) {
+
+            final byte[] flowBytes = IOUtils.toByteArray(gzipIn);
+            if (flowBytes == null || flowBytes.length == 0) {
+                logger.warn("Could not extract root group id because Flow Configuration File was empty");
+                return null;
+            }
+
+            final DocumentBuilderFactory docFactory = DocumentBuilderFactory.newInstance();

Review comment:
       Should use the `SafeXMLConfiguration` class defined in `nifi-security-utils`, if adding that dependency is okay here. Otherwise, should at least ensure that we are setting all of the appropriate flags in the way that is handled there.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarUnpacker.java
##########
@@ -243,7 +243,9 @@ private static void mapExtensions(final Map<File, BundleCoordinate> unpackedNars
 
     public static void mapExtension(final File unpackedNar, final BundleCoordinate bundleCoordinate, final File docsDirectory, final ExtensionMapping mapping) throws IOException {
         final File bundledDependencies = new File(unpackedNar, BUNDLED_DEPENDENCIES_DIRECTORY);
-        unpackBundleDocs(docsDirectory, mapping, bundleCoordinate, bundledDependencies);
+        if(docsDirectory != null) {

Review comment:
       `if (docsDirectory != null) {...` I'm guessing the implication here is that docsDirectory == null implies that it's running headless? If so, can you add a comment to this effect to make it clear why we'd have such a conditional?

##########
File path: nifi-server-api/src/main/java/org/apache/nifi/NiFiServer.java
##########
@@ -33,6 +34,10 @@
 
     void setBundles(Bundle systemBundle, Set<Bundle> bundles);
 
+    void setNiFiProperties(NiFiProperties properties);
+
+    default void init() {}

Review comment:
       Rather than having two setters that must be called, and then having an init() method (which I'm guessing is expected to be called only after the setters?) perhaps it makes more sense to instead of an `initialize(NiFiProperties properties, Bundle systemBundle, Set<Bundle> additionalBundles);` method?




----------------------------------------------------------------
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.

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



[GitHub] [nifi] ottobackwards commented on a change in pull request #4376: NIFI-7592: Allow NiFi to be started without a GUI/REST interface

Posted by GitBox <gi...@apache.org>.
ottobackwards commented on a change in pull request #4376:
URL: https://github.com/apache/nifi/pull/4376#discussion_r449782692



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java
##########
@@ -192,7 +192,7 @@ public static NarThreadContextClassLoader getInstance() {
     public static <T> T createInstance(final ExtensionManager extensionManager, final String implementationClassName, final Class<T> typeDefinition, final NiFiProperties nifiProperties)
             throws InstantiationException, IllegalAccessException, ClassNotFoundException {
         final ClassLoader originalClassLoader = Thread.currentThread().getContextClassLoader();
-        Thread.currentThread().setContextClassLoader(NarThreadContextClassLoader.getInstance());
+        //Thread.currentThread().setContextClassLoader(NarThreadContextClassLoader.getInstance());

Review comment:
       Is this on purpose?  Or a left over from debugging?




----------------------------------------------------------------
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.

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



[GitHub] [nifi] mattyb149 commented on a change in pull request #4376: NIFI-7592: Allow NiFi to be started without a GUI/REST interface

Posted by GitBox <gi...@apache.org>.
mattyb149 commented on a change in pull request #4376:
URL: https://github.com/apache/nifi/pull/4376#discussion_r450919766



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java
##########
@@ -192,7 +192,7 @@ public static NarThreadContextClassLoader getInstance() {
     public static <T> T createInstance(final ExtensionManager extensionManager, final String implementationClassName, final Class<T> typeDefinition, final NiFiProperties nifiProperties)
             throws InstantiationException, IllegalAccessException, ClassNotFoundException {
         final ClassLoader originalClassLoader = Thread.currentThread().getContextClassLoader();
-        Thread.currentThread().setContextClassLoader(NarThreadContextClassLoader.getInstance());
+        //Thread.currentThread().setContextClassLoader(NarThreadContextClassLoader.getInstance());

Review comment:
       It's on purpose, but I'll remove the dead code. We want the NAR's classloader to be used to load things, not the singleton




----------------------------------------------------------------
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.

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



[GitHub] [nifi] mattyb149 commented on a change in pull request #4376: NIFI-7592: Allow NiFi to be started without a GUI/REST interface

Posted by GitBox <gi...@apache.org>.
mattyb149 commented on a change in pull request #4376:
URL: https://github.com/apache/nifi/pull/4376#discussion_r460155636



##########
File path: nifi-server-api/src/main/java/org/apache/nifi/NiFiServer.java
##########
@@ -33,6 +34,10 @@
 
     void setBundles(Bundle systemBundle, Set<Bundle> bundles);
 
+    void setNiFiProperties(NiFiProperties properties);
+
+    default void init() {}

Review comment:
       Makes sense, the `setBundles` method was already there so I just kept the approach consistent, but since they're all called in the same place (including `setExtensionMapping`), I can just move everything to a single `initialize` method.




----------------------------------------------------------------
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.

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



[GitHub] [nifi] ottobackwards commented on pull request #4376: NIFI-7592: Allow NiFi to be started without a GUI/REST interface

Posted by GitBox <gi...@apache.org>.
ottobackwards commented on pull request #4376:
URL: https://github.com/apache/nifi/pull/4376#issuecomment-655007142


   OK, if there are no classes that are removed from what a 3rd party would see as the default set of classes, then there is 0 risk.
   Otherwise, I'm not sure you can assume that no 3rd party would break, since you can't assume the nar dependency.
   
   There might be other places in the documentation that talk about nar dependencies and mention jetty, they should all be updated.
   


----------------------------------------------------------------
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.

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



[GitHub] [nifi] ottobackwards commented on a change in pull request #4376: NIFI-7592: Allow NiFi to be started without a GUI/REST interface

Posted by GitBox <gi...@apache.org>.
ottobackwards commented on a change in pull request #4376:
URL: https://github.com/apache/nifi/pull/4376#discussion_r450959476



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java
##########
@@ -192,7 +192,7 @@ public static NarThreadContextClassLoader getInstance() {
     public static <T> T createInstance(final ExtensionManager extensionManager, final String implementationClassName, final Class<T> typeDefinition, final NiFiProperties nifiProperties)
             throws InstantiationException, IllegalAccessException, ClassNotFoundException {
         final ClassLoader originalClassLoader = Thread.currentThread().getContextClassLoader();
-        Thread.currentThread().setContextClassLoader(NarThreadContextClassLoader.getInstance());
+        //Thread.currentThread().setContextClassLoader(NarThreadContextClassLoader.getInstance());

Review comment:
       OK, that makes sense, just wasn't sure with the commented out code ;)
   




----------------------------------------------------------------
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.

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



[GitHub] [nifi] markap14 commented on a change in pull request #4376: NIFI-7592: Allow NiFi to be started without a GUI/REST interface

Posted by GitBox <gi...@apache.org>.
markap14 commented on a change in pull request #4376:
URL: https://github.com/apache/nifi/pull/4376#discussion_r459715604



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/minifi-server/src/main/java/org/apache/nifi/minifi/MiNiFiServer.java
##########
@@ -0,0 +1,211 @@
+/*
+ * 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.minifi;
+
+import org.apache.nifi.NiFiServer;
+import org.apache.nifi.admin.service.AuditService;
+import org.apache.nifi.admin.service.impl.StandardAuditService;
+import org.apache.nifi.authorization.AuthorizationRequest;
+import org.apache.nifi.authorization.AuthorizationResult;
+import org.apache.nifi.authorization.Authorizer;
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.AuthorizerInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.bundle.Bundle;
+import org.apache.nifi.controller.FlowController;
+import org.apache.nifi.controller.StandardFlowService;
+import org.apache.nifi.controller.flow.FlowManager;
+import org.apache.nifi.controller.repository.FlowFileEventRepository;
+import org.apache.nifi.controller.repository.metrics.RingBufferEventRepository;
+import org.apache.nifi.diagnostics.DiagnosticsFactory;
+import org.apache.nifi.encrypt.StringEncryptor;
+import org.apache.nifi.events.VolatileBulletinRepository;
+import org.apache.nifi.nar.ExtensionDiscoveringManager;
+import org.apache.nifi.nar.ExtensionManagerHolder;
+import org.apache.nifi.nar.ExtensionMapping;
+import org.apache.nifi.nar.StandardExtensionDiscoveringManager;
+import org.apache.nifi.registry.VariableRegistry;
+import org.apache.nifi.registry.flow.StandardFlowRegistryClient;
+import org.apache.nifi.registry.variable.FileBasedVariableRegistry;
+import org.apache.nifi.reporting.BulletinRepository;
+import org.apache.nifi.services.FlowService;
+import org.apache.nifi.util.NiFiProperties;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.Set;
+
+/**
+ */
+public class MiNiFiServer implements NiFiServer {
+
+    private static final Logger logger = LoggerFactory.getLogger(MiNiFiServer.class);
+    private NiFiProperties props;
+    private Bundle systemBundle;
+    private Set<Bundle> bundles;
+    private ExtensionMapping extensionMapping;
+    private FlowService flowService;
+
+    private static final String DEFAULT_SENSITIVE_PROPS_KEY = "nififtw!";
+
+    /**
+     * Default constructor
+     */
+    public MiNiFiServer() {
+    }
+
+    public void start() {
+        try {
+
+            // Create a standard extension manager and discover extensions
+            final ExtensionDiscoveringManager extensionManager = new StandardExtensionDiscoveringManager();
+            extensionManager.discoverExtensions(systemBundle, bundles);
+            extensionManager.logClassLoaderMapping();
+
+            // Set the extension manager into the holder which makes it available to the Spring context via a factory bean
+            ExtensionManagerHolder.init(extensionManager);
+
+            // Enrich the flow xml using the Extension Manager mapping
+            final FlowParser flowParser = new FlowParser();
+            final FlowEnricher flowEnricher = new FlowEnricher(this, flowParser, props);
+            flowEnricher.enrichFlowWithBundleInformation();
+            logger.info("Loading Flow...");
+
+            FlowFileEventRepository flowFileEventRepository = new RingBufferEventRepository(5);
+            AuditService auditService = new StandardAuditService();
+            Authorizer authorizer = new Authorizer() {
+                @Override
+                public AuthorizationResult authorize(AuthorizationRequest request) throws AuthorizationAccessException {
+                    return AuthorizationResult.approved();
+                }
+
+                @Override
+                public void initialize(AuthorizerInitializationContext initializationContext) throws AuthorizerCreationException {
+                    // do nothing
+                }
+
+                @Override
+                public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+                    // do nothing
+                }
+
+                @Override
+                public void preDestruction() throws AuthorizerDestructionException {
+                    // do nothing
+                }
+            };
+
+            final String sensitivePropAlgorithmVal = props.getProperty(StringEncryptor.NF_SENSITIVE_PROPS_ALGORITHM);
+            final String sensitivePropProviderVal = props.getProperty(StringEncryptor.NF_SENSITIVE_PROPS_PROVIDER);
+            final String sensitivePropValueNifiPropVar = props.getProperty(StringEncryptor.NF_SENSITIVE_PROPS_KEY, DEFAULT_SENSITIVE_PROPS_KEY);
+
+            StringEncryptor encryptor = StringEncryptor.createEncryptor(sensitivePropAlgorithmVal, sensitivePropProviderVal, sensitivePropValueNifiPropVar);
+            VariableRegistry variableRegistry = new FileBasedVariableRegistry(props.getVariableRegistryPropertiesPaths());
+            BulletinRepository bulletinRepository = new VolatileBulletinRepository();
+
+            FlowController flowController = FlowController.createStandaloneInstance(
+                    flowFileEventRepository,
+                    props,
+                    authorizer,
+                    auditService,
+                    encryptor,
+                    bulletinRepository,
+                    variableRegistry,
+                    new StandardFlowRegistryClient(),

Review comment:
       The StandardFlowRegistryClient needs the nifi properties to be provided. So I think this needs to be pulled into a local variable and have `setNifiProperties` called before passing to the FlowController.




----------------------------------------------------------------
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.

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



[GitHub] [nifi] ottobackwards commented on pull request #4376: NIFI-7592: Allow NiFi to be started without a GUI/REST interface

Posted by GitBox <gi...@apache.org>.
ottobackwards commented on pull request #4376:
URL: https://github.com/apache/nifi/pull/4376#issuecomment-654958033


   So this changes the 'root' classloader from 'jetty' to the current classloader?


----------------------------------------------------------------
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.

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



[GitHub] [nifi] ottobackwards commented on pull request #4376: NIFI-7592: Allow NiFi to be started without a GUI/REST interface

Posted by GitBox <gi...@apache.org>.
ottobackwards commented on pull request #4376:
URL: https://github.com/apache/nifi/pull/4376#issuecomment-654989630


   "The @RequiresInstanceClassLoading annotation also provides an optional flag `cloneAncestorResources'. If set to true, the instance ClassLoader will include ancestor resources up to the first ClassLoader containing a controller service API referenced by the component, or up to the Jetty NAR. If set to false, or not specified, only the resources from the component’s NAR will be included."
   
   I think you need to update the documentation then.
   I understand about taking jetty out, in my fork of the bundle "system" I did the same ( since it isn't in nifi) .
   
   Saying that, I'm not super familiar with what is actually _in_ that nar.  So there is no chance that this will break existing 3rd party nars which may have picked something up from the jetty NAR?
   


----------------------------------------------------------------
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.

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



[GitHub] [nifi] mattyb149 commented on pull request #4376: NIFI-7592: Allow NiFi to be started without a GUI/REST interface

Posted by GitBox <gi...@apache.org>.
mattyb149 commented on pull request #4376:
URL: https://github.com/apache/nifi/pull/4376#issuecomment-654986964


   The PR removes nifi-jetty (which has the JettyServer class, an implementation of the NiFiServer interface) as the parent classloader for all NARs that don't have one. Some NARs still need Jetty itself (some HTTP processors, e.g.) so our JettyServer class and its dependencies are now in a nifi-server NAR, which has the "nifi-jetty-bundle" as a parent. 
   The naming gets confusing here because NiFi code has a JettyServer class, but the nifi-jetty-bundle just has pure Jetty dependencies for those NARs that need them. For example all NARs with custom UIs (the Advanced button on the component dialog), components that need to start a server (ListenHttp, e.g.) have nifi-jetty-bundle as the top-level parent.
   
   This PR also includes refactoring the way the NiFiServer implementation is instantiated. It used to look for a hardcoded class name (for JettyServer) and use reflection to instantiate it. This PR removes that code and replaces it with a ServiceLoader that looks in the NARs for an implementation of NiFiServer. For the normal NiFi assembly, the nifi-server NAR (with JettyServer) will be there so it will get discovered as expected. But for other assemblies (such as MiNiFi Java), the nifi-server NAR would be replaced with a different implementation, and this PR offers one called MiNiFiServer in a minifi-server NAR. If you replace the nifi-server NAR with minifi-server, the UI/REST components are no longer available, and the flow controller and all components are created immediately and the flow is loaded/initialized.


----------------------------------------------------------------
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.

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



[GitHub] [nifi] markap14 commented on pull request #4376: NIFI-7592: Allow NiFi to be started without a GUI/REST interface

Posted by GitBox <gi...@apache.org>.
markap14 commented on pull request #4376:
URL: https://github.com/apache/nifi/pull/4376#issuecomment-667273585


   Awesome, thanks @mattyb149! Looks great! I was able to verify running headless and not. Same flow worked great in both. Code looks good. I changed the "headless" profile to be activeByDefault = false instead of true and fixed a typo in the comment in the assembly's pom.xml because I'm 99.9% sure that's what was intended. If not then please reopen and we can address. Otherwise, all is good, +1 merged to main!


----------------------------------------------------------------
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.

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



[GitHub] [nifi] mattyb149 commented on pull request #4376: NIFI-7592: Allow NiFi to be started without a GUI/REST interface

Posted by GitBox <gi...@apache.org>.
mattyb149 commented on pull request #4376:
URL: https://github.com/apache/nifi/pull/4376#issuecomment-667176270


   Makes sense to me, will rename and update PR


----------------------------------------------------------------
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.

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



[GitHub] [nifi] markap14 closed pull request #4376: NIFI-7592: Allow NiFi to be started without a GUI/REST interface

Posted by GitBox <gi...@apache.org>.
markap14 closed pull request #4376:
URL: https://github.com/apache/nifi/pull/4376


   


----------------------------------------------------------------
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.

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



[GitHub] [nifi] markap14 commented on pull request #4376: NIFI-7592: Allow NiFi to be started without a GUI/REST interface

Posted by GitBox <gi...@apache.org>.
markap14 commented on pull request #4376:
URL: https://github.com/apache/nifi/pull/4376#issuecomment-667166106


   @mattyb149 thanks for the updates! I reviewed and code looks good. Did some quick tests and everything still seems to be working well. The only thing that gives me a bit of pause here is the naming convention. Currently, the naming for the new classes/packages/nars are very specific to MiNiFi. But this change itself does not introduce MiNiFi, but rather a headless version of nifi, which is a prereq, I would say, to a fully merging of MiNiFi. What do you think of naming these `nifi-headless-server-nar`, `HeadlessNiFiServer`, etc.?


----------------------------------------------------------------
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.

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



[GitHub] [nifi] markap14 commented on a change in pull request #4376: NIFI-7592: Allow NiFi to be started without a GUI/REST interface

Posted by GitBox <gi...@apache.org>.
markap14 commented on a change in pull request #4376:
URL: https://github.com/apache/nifi/pull/4376#discussion_r459702685



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarClassLoaders.java
##########
@@ -262,6 +263,15 @@ private InitContext load(final ClassLoader rootClassloader,
                         narDirectoryBundleLookup.put(narDetail.getWorkingDirectory().getCanonicalPath(), new Bundle(narDetail, bundleClassLoader));
                         narCoordinateClassLoaderLookup.put(narDetail.getCoordinate().getCoordinate(), narClassLoader);
                         narDetailsIter.remove();
+                        // Search for a NiFiServer implementation
+                        ServiceLoader<NiFiServer> niFiServerServiceLoader = ServiceLoader.load(NiFiServer.class, narClassLoader);
+                        for(NiFiServer server : niFiServerServiceLoader) {
+                            if(serverInstance == null) {
+                                serverInstance = server;
+                            } else {
+                                throw new IOException("Multiple implementations of NiFiServer found, there must be exactly one implementation.");

Review comment:
       I tried adding both nifi-server-nar and minifi-server-nar into the lib/ directory. Indeed, I ran into this. But this error can probably be made a bit clearer, by listing the classnames of the NiFiServer implementations and the associated NAR's. For example:
   
   `Expected exactly one implementation of NiFiServer but found 2: JettyServer from nifi-server-nar-1.12.0-SNAPSHOT.nar, MiNiFiServer from minifi-server-nar-1.12.0-SNAPSHOT.nar`
   
   Or something to that effect.




----------------------------------------------------------------
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.

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



[GitHub] [nifi] mattyb149 commented on pull request #4376: NIFI-7592: Allow NiFi to be started without a GUI/REST interface

Posted by GitBox <gi...@apache.org>.
mattyb149 commented on pull request #4376:
URL: https://github.com/apache/nifi/pull/4376#issuecomment-654995453


   I'm still looking at various NARs to see how/if they're affected. Many NARs end up using nifi-standard-services-api-nar as a parent, which itself has the nifi-jetty-bundle as a parent, so there should be no change there. We're not removing Jetty, just decoupling NiFi framework stuff that also uses Jetty, so we can exclude things like the NiFi UI/REST interface but still support processors that use Jetty.
   
   Good catch on the doc, I didn't know that was in there :) Will update the doc as appropriate, thanks!


----------------------------------------------------------------
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.

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