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 2021/02/26 18:16:07 UTC

[GitHub] [nifi] markap14 opened a new pull request #4852: NIFI-8212: Refactored StandardExtensionDiscoveringManager to avoid us…

markap14 opened a new pull request #4852:
URL: https://github.com/apache/nifi/pull/4852


   …ing ServiceLoader. Instead, it will look at the ServiceLoader file and read the names of the classes but avoid instantiating all of the objects or loading the classes into memory.
   
   - Updated Doc Generation so that if the documentation for a given NAR already exists, it doesn't delete it and re-generate it. This was necessary because we are no longer instantiating an instance of each component and instead lazily creating the components as necessary.
   - Removed stateless version of extension registry because it's no longer necessary
   
   Thank you for submitting a contribution to Apache NiFi.
   
   Please provide a short description of the PR here:
   
   #### Description of PR
   
   _Enables X functionality; fixes bug NIFI-YYYY._
   
   In order to streamline the review of the contribution we ask you
   to ensure the following steps have been taken:
   
   ### For all changes:
   - [ ] Is there a JIRA ticket associated with this PR? Is it referenced 
        in the commit message?
   
   - [ ] 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.
   
   - [ ] Has your PR been rebased against the latest commit within the target branch (typically `main`)?
   
   - [ ] 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] exceptionfactory commented on a change in pull request #4852: NIFI-8212: Refactored StandardExtensionDiscoveringManager to avoid us…

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



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/StandardExtensionDiscoveringManager.java
##########
@@ -128,7 +131,11 @@ public void discoverExtensions(final Set<Bundle> narBundles) {
             // so that static initialization techniques that depend on the context class loader will work properly
             final ClassLoader ncl = bundle.getClassLoader();
             Thread.currentThread().setContextClassLoader(ncl);
+
+            final long loadStart = System.currentTimeMillis();
             loadExtensions(bundle);
+            final long loadMillis = System.currentTimeMillis() - loadStart;
+            logger.info("Loaded extensions for {} in {} millis", bundle.getBundleDetails(), loadMillis);

Review comment:
       Thanks for the explanation, I'm fine with leaving it at the INFO level since it is only at startup, just wanted to make sure it was intentional.




----------------------------------------------------------------
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 #4852: NIFI-8212: Refactored StandardExtensionDiscoveringManager to avoid us…

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



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/service/ServiceStateTransition.java
##########
@@ -58,6 +65,14 @@ public boolean enable() {
             state = ControllerServiceState.ENABLED;
 
             enabledFutures.forEach(future -> future.complete(null));
+

Review comment:
       Maybe some debug logging here?




----------------------------------------------------------------
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 #4852: NIFI-8212: Refactored StandardExtensionDiscoveringManager to avoid us…

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



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/ExtensionDefinition.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.nar;
+
+import org.apache.nifi.bundle.Bundle;
+
+import java.util.Objects;
+
+public class ExtensionDefinition {
+    private final String implementationClassName;
+    private final Bundle bundle;
+    private final Class<?> extensionType;
+

Review comment:
       maybe a java doc on purpose / usage 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 #4852: NIFI-8212: Refactored StandardExtensionDiscoveringManager to avoid us…

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



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/ExtensionBuilder.java
##########
@@ -402,6 +409,60 @@ private ControllerServiceNode createControllerServiceNode() throws ClassNotFound
         }
     }
 
+    private static void verifyControllerServiceReferences(final ConfigurableComponent component, final ClassLoader bundleClassLoader) throws InstantiationException {
+        // If a component lives in the same NAR as a Controller Service API, and the component references the Controller Service API (either
+        // by itself implementing the API or by having a Property Descriptor that identifies the Controller Service), then the component is not
+        // allowed to Require Instance Class Loading. This is done because when a component requires Instance Class Loading, the jars within the
+        // NAR and its parents must be copied to a new class loader all the way up to the point of the Controller Service APIs. If the Controller
+        // Service API lives in the same NAR as the implementation itself, then we cannot duplicate the NAR ClassLoader. Otherwise, we would have
+        // two different NAR ClassLoaders that each define the Service API. And the Service API class must live in the parent ClassLoader for both
+        // the referencing component AND the implementing component.
+
+        // if the extension does not require instance classloading, there is no concern.
+        final boolean requiresInstanceClassLoading = component.getClass().isAnnotationPresent(RequiresInstanceClassLoading.class);

Review comment:
       Fair enough. This method was transplanted here from another class. But I do agree that there's some stuff in here that debug logging may be helpful for. Will add.




----------------------------------------------------------------
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] asfgit closed pull request #4852: NIFI-8212: Refactored StandardExtensionDiscoveringManager to avoid us…

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


   


----------------------------------------------------------------
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 #4852: NIFI-8212: Refactored StandardExtensionDiscoveringManager to avoid us…

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


   Thanks for the reviews all! I have pushed a commit that I think addresses all outstanding concerns.


----------------------------------------------------------------
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 #4852: NIFI-8212: Refactored StandardExtensionDiscoveringManager to avoid us…

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



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/service/ServiceStateTransition.java
##########
@@ -58,6 +65,14 @@ public boolean enable() {
             state = ControllerServiceState.ENABLED;
 
             enabledFutures.forEach(future -> future.complete(null));
+

Review comment:
       i take it back.




----------------------------------------------------------------
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 #4852: NIFI-8212: Refactored StandardExtensionDiscoveringManager to avoid us…

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



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/StatelessNiFi.java
##########
@@ -69,7 +69,7 @@ public static void main(final String[] args) throws IOException, ClassNotFoundEx
             logger.info("Unpacking {} NARs", narFiles.length);
             final long startUnpack = System.nanoTime();
             for (final File narFile : narFiles) {

Review comment:
       I created a nar class loader to work without unpacking the files, I always wonder if in stateless deployments this wouldn't be better.  Although I did not do the docs, but in stateless, do you need the docs?




----------------------------------------------------------------
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] exceptionfactory commented on a change in pull request #4852: NIFI-8212: Refactored StandardExtensionDiscoveringManager to avoid us…

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



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/repository/scheduling/ConnectableProcessContext.java
##########
@@ -41,18 +33,31 @@
 import org.apache.nifi.scheduling.ExecutionNode;
 import org.apache.nifi.util.Connectables;
 
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
 /**
  * This class is essentially an empty shell for {@link Connectable}s that are not Processors
  */
 public class ConnectableProcessContext implements ProcessContext {
 
     private final Connectable connectable;
-    private final PropertyEncryptor encryptor;
+    private final Supplier<PropertyEncryptor> encryptorFactory;

Review comment:
       One other option to consider instead of changing references for `PropertyEncryptor` to `Supplier<PropertyEncryptor>` would be to create a lazily initialized PropertyEncryptor specific to the stateless engine.  This implementation would take the sensitive properties key as a constructor argument, but only call `PropertyEncryptorFactory` on initial calls to `encrypt()` or `decrypt()`.  Not a big difference either way, just fewer classes impacted.




----------------------------------------------------------------
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 #4852: NIFI-8212: Refactored StandardExtensionDiscoveringManager to avoid us…

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



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/StandardExtensionDiscoveringManager.java
##########
@@ -145,74 +152,103 @@ public void discoverExtensions(final Set<Bundle> narBundles) {
      *
      * @param bundle from which to load extensions
      */
-    @SuppressWarnings("unchecked")
     private void loadExtensions(final Bundle bundle) {
-        for (final Map.Entry<Class, Set<Class>> entry : definitionMap.entrySet()) {
-            final boolean isControllerService = ControllerService.class.equals(entry.getKey());
-            final boolean isProcessor = Processor.class.equals(entry.getKey());
-            final boolean isReportingTask = ReportingTask.class.equals(entry.getKey());
-
-            final ServiceLoader<?> serviceLoader = ServiceLoader.load(entry.getKey(), bundle.getClassLoader());
-            for (final Object o : serviceLoader) {
-                try {
-                    loadExtension(o, entry.getKey(), bundle);
-                } catch (Exception e) {
-                    logger.warn("Failed to register extension {} due to: {}" , new Object[]{o.getClass().getCanonicalName(), e.getMessage()});
-                    if (logger.isDebugEnabled()) {
-                        logger.debug("", e);
+        for (final Class extensionType : definitionMap.keySet()) {
+            final String serviceType = extensionType.getName();
+
+            try {
+                final Set<URL> serviceResourceUrls = getServiceFileURLs(bundle, extensionType);
+                logger.debug("Bundle {} has the following Services File URLs for {}: {}", bundle, serviceType, serviceResourceUrls);
+
+                for (final URL serviceResourceUrl : serviceResourceUrls) {
+                    final Set<String> implementationClassNames = getServiceFileImplementationClassNames(serviceResourceUrl);
+                    logger.debug("Bundle {} defines {} implementations of interface {}", bundle, implementationClassNames.size(), serviceType);
+
+                    for (final String implementationClassName : implementationClassNames) {
+                        try {
+                            loadExtension(implementationClassName, extensionType, bundle);
+                            logger.debug("Successfully loaded {} {} from {}", extensionType.getSimpleName(), implementationClassName, bundle);
+                        } catch (final Exception e) {
+                            logger.error("Failed to register {} of type {} in bundle {}" , extensionType.getSimpleName(), implementationClassName, bundle, e);
+                        }
                     }
                 }
+            } catch (final IOException e) {
+                throw new RuntimeException("Failed to get resources of type " + serviceType + " from bundle " + bundle);
             }
-
-            classLoaderBundleLookup.put(bundle.getClassLoader(), bundle);
         }
+
+        classLoaderBundleLookup.put(bundle.getClassLoader(), bundle);
     }
 
-    protected void loadExtension(final Object extension, final Class<?> extensionType, final Bundle bundle) {
-        final boolean isControllerService = ControllerService.class.equals(extensionType);
-        final boolean isProcessor = Processor.class.equals(extensionType);
-        final boolean isReportingTask = ReportingTask.class.equals(extensionType);
+    private Set<String> getServiceFileImplementationClassNames(final URL serviceFileUrl) throws IOException {
+        final Set<String> implementationClassNames = new HashSet<>();
 
-        // create a cache of temp ConfigurableComponent instances, the initialize here has to happen before the checks below
-        if ((isControllerService || isProcessor || isReportingTask) && extension instanceof ConfigurableComponent) {
-            final ConfigurableComponent configurableComponent = (ConfigurableComponent) extension;
-            initializeTempComponent(configurableComponent);
+        try (final InputStream in = serviceFileUrl.openStream();
+             final Reader inputStreamReader = new InputStreamReader(in);
+             final BufferedReader reader = new BufferedReader(inputStreamReader)) {
 
-            final String cacheKey = getClassBundleKey(extension.getClass().getCanonicalName(), bundle.getBundleDetails().getCoordinate());
-            tempComponentLookup.put(cacheKey, configurableComponent);
-        }
+            String line;
+            while ((line = reader.readLine()) != null) {
+                // Remove anything after the #
+                final int index = line.indexOf("#");
+                if (index >= 0) {
+                    line = line.substring(0, index);
+                }
 
-        // only consider extensions discovered directly in this bundle
-        boolean registerExtension = bundle.getClassLoader().equals(extension.getClass().getClassLoader());
+                // Ignore empty line
+                line = line.trim();
+                if (line.isEmpty()) {
+                    continue;
+                }
 
-        if (registerExtension) {
-            final Class<?> extensionClass = extension.getClass();
-            if (isControllerService && !checkControllerServiceEligibility(extensionClass)) {
-                registerExtension = false;
-                logger.error(String.format(
-                    "Skipping Controller Service %s because it is bundled with its supporting APIs and requires instance class loading.", extensionClass.getName()));
+                implementationClassNames.add(line);

Review comment:
       in https://github.com/palindromicity/bundles, which is my breaking out of the nar system, I switched from service loader to https://github.com/atteo/classindex which is faster and have other benefits ( like marking services at build time so you don't have to manually maintain the service file ).  You may want to think about it
   




----------------------------------------------------------------
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 #4852: NIFI-8212: Refactored StandardExtensionDiscoveringManager to avoid us…

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



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/ExtensionBuilder.java
##########
@@ -402,6 +409,60 @@ private ControllerServiceNode createControllerServiceNode() throws ClassNotFound
         }
     }
 
+    private static void verifyControllerServiceReferences(final ConfigurableComponent component, final ClassLoader bundleClassLoader) throws InstantiationException {
+        // If a component lives in the same NAR as a Controller Service API, and the component references the Controller Service API (either
+        // by itself implementing the API or by having a Property Descriptor that identifies the Controller Service), then the component is not
+        // allowed to Require Instance Class Loading. This is done because when a component requires Instance Class Loading, the jars within the
+        // NAR and its parents must be copied to a new class loader all the way up to the point of the Controller Service APIs. If the Controller
+        // Service API lives in the same NAR as the implementation itself, then we cannot duplicate the NAR ClassLoader. Otherwise, we would have
+        // two different NAR ClassLoaders that each define the Service API. And the Service API class must live in the parent ClassLoader for both
+        // the referencing component AND the implementing component.
+
+        // if the extension does not require instance classloading, there is no concern.
+        final boolean requiresInstanceClassLoading = component.getClass().isAnnotationPresent(RequiresInstanceClassLoading.class);

Review comment:
       There are dragons here that should be logged




----------------------------------------------------------------
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 #4852: NIFI-8212: Refactored StandardExtensionDiscoveringManager to avoid us…

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



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceNode.java
##########
@@ -181,6 +182,8 @@
      */
     boolean isActive();
 

Review comment:
       Good catch.




----------------------------------------------------------------
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] exceptionfactory commented on a change in pull request #4852: NIFI-8212: Refactored StandardExtensionDiscoveringManager to avoid us…

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



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/provenance/ComponentIdentifierLookup.java
##########
@@ -46,14 +48,15 @@ public ComponentIdentifierLookup(final FlowController flowController) {
 
     @Override
     public List<String> getComponentTypes() {
-        final Set<Class> procClasses = flowController.getExtensionManager().getExtensions(Processor.class);
+        final Set<ExtensionDefinition> procDefinitions = flowController.getExtensionManager().getExtensions(Processor.class);
 
-        final List<String> componentTypes = new ArrayList<>(procClasses.size() + 2);
+        final List<String> componentTypes = new ArrayList<>(procDefinitions.size() + 2);
         componentTypes.add(ProvenanceEventRecord.REMOTE_INPUT_PORT_TYPE);
         componentTypes.add(ProvenanceEventRecord.REMOTE_OUTPUT_PORT_TYPE);
 
-        procClasses.stream()
-            .map(Class::getSimpleName)
+        procDefinitions.stream()
+            .map(ExtensionDefinition::getImplementationClassName)
+            .map(className -> className.contains(".") ? StringUtils.substringAfterLast(className, ".") : className)

Review comment:
       The `StandardStatelessEngine` uses the same approach attempting to derive the simple class name, does it make sense to add a method on `ExtensionDefinition` to return the simple class name?

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarUnpacker.java
##########
@@ -288,17 +280,20 @@ private static void unpackBundleDocs(final File docsDirectory, final ExtensionMa
      *
      * @param nar the nar to unpack
      * @param baseWorkingDirectory the directory to unpack to
+     * @param verifyHash if the NAR has already been unpacked, indicates whether or not the hash should be verified. If this value is true,
+     * and the NAR's hash does not match the hash written to the unpacked directory, the working directory will be deleted and the NAR will be
+     * unpacked again. If false, the NAR will not be unpacked again and its hash will not be checked.
      * @return the directory to the unpacked NAR
      * @throws IOException if unable to explode nar
      */
-    public static File unpackNar(final File nar, final File baseWorkingDirectory) throws IOException {
+    public static File unpackNar(final File nar, final File baseWorkingDirectory, final boolean verifyHash) throws IOException {
         final File narWorkingDirectory = new File(baseWorkingDirectory, nar.getName() + "-unpacked");
 
         // if the working directory doesn't exist, unpack the nar
         if (!narWorkingDirectory.exists()) {
             unpack(nar, narWorkingDirectory, FileDigestUtils.getDigest(nar));
-        } else {
-            // the working directory does exist. Run digest against the nar
+        } else if (verifyHash) {
+            // the working directory does exist. Run MD5 sum against the nar

Review comment:
       NIFI-8132 changed the hash to SHA-256, so the previous comment used the general term `digest` as opposed to referencing the particular algorithm.  This could be changed to say `SHA-256`, or reverted.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/StandardExtensionDiscoveringManager.java
##########
@@ -145,74 +152,103 @@ public void discoverExtensions(final Set<Bundle> narBundles) {
      *
      * @param bundle from which to load extensions
      */
-    @SuppressWarnings("unchecked")
     private void loadExtensions(final Bundle bundle) {
-        for (final Map.Entry<Class, Set<Class>> entry : definitionMap.entrySet()) {
-            final boolean isControllerService = ControllerService.class.equals(entry.getKey());
-            final boolean isProcessor = Processor.class.equals(entry.getKey());
-            final boolean isReportingTask = ReportingTask.class.equals(entry.getKey());
-
-            final ServiceLoader<?> serviceLoader = ServiceLoader.load(entry.getKey(), bundle.getClassLoader());
-            for (final Object o : serviceLoader) {
-                try {
-                    loadExtension(o, entry.getKey(), bundle);
-                } catch (Exception e) {
-                    logger.warn("Failed to register extension {} due to: {}" , new Object[]{o.getClass().getCanonicalName(), e.getMessage()});
-                    if (logger.isDebugEnabled()) {
-                        logger.debug("", e);
+        for (final Class extensionType : definitionMap.keySet()) {
+            final String serviceType = extensionType.getName();
+
+            try {
+                final Set<URL> serviceResourceUrls = getServiceFileURLs(bundle, extensionType);
+                logger.debug("Bundle {} has the following Services File URLs for {}: {}", bundle, serviceType, serviceResourceUrls);
+
+                for (final URL serviceResourceUrl : serviceResourceUrls) {
+                    final Set<String> implementationClassNames = getServiceFileImplementationClassNames(serviceResourceUrl);
+                    logger.debug("Bundle {} defines {} implementations of interface {}", bundle, implementationClassNames.size(), serviceType);
+
+                    for (final String implementationClassName : implementationClassNames) {
+                        try {
+                            loadExtension(implementationClassName, extensionType, bundle);
+                            logger.debug("Successfully loaded {} {} from {}", extensionType.getSimpleName(), implementationClassName, bundle);
+                        } catch (final Exception e) {
+                            logger.error("Failed to register {} of type {} in bundle {}" , extensionType.getSimpleName(), implementationClassName, bundle, e);
+                        }
                     }
                 }
+            } catch (final IOException e) {
+                throw new RuntimeException("Failed to get resources of type " + serviceType + " from bundle " + bundle);
             }
-
-            classLoaderBundleLookup.put(bundle.getClassLoader(), bundle);
         }
+
+        classLoaderBundleLookup.put(bundle.getClassLoader(), bundle);
     }
 
-    protected void loadExtension(final Object extension, final Class<?> extensionType, final Bundle bundle) {
-        final boolean isControllerService = ControllerService.class.equals(extensionType);
-        final boolean isProcessor = Processor.class.equals(extensionType);
-        final boolean isReportingTask = ReportingTask.class.equals(extensionType);
+    private Set<String> getServiceFileImplementationClassNames(final URL serviceFileUrl) throws IOException {
+        final Set<String> implementationClassNames = new HashSet<>();
 
-        // create a cache of temp ConfigurableComponent instances, the initialize here has to happen before the checks below
-        if ((isControllerService || isProcessor || isReportingTask) && extension instanceof ConfigurableComponent) {
-            final ConfigurableComponent configurableComponent = (ConfigurableComponent) extension;
-            initializeTempComponent(configurableComponent);
+        try (final InputStream in = serviceFileUrl.openStream();
+             final Reader inputStreamReader = new InputStreamReader(in);
+             final BufferedReader reader = new BufferedReader(inputStreamReader)) {
 
-            final String cacheKey = getClassBundleKey(extension.getClass().getCanonicalName(), bundle.getBundleDetails().getCoordinate());
-            tempComponentLookup.put(cacheKey, configurableComponent);
-        }
+            String line;
+            while ((line = reader.readLine()) != null) {
+                // Remove anything after the #
+                final int index = line.indexOf("#");
+                if (index >= 0) {
+                    line = line.substring(0, index);
+                }
 
-        // only consider extensions discovered directly in this bundle
-        boolean registerExtension = bundle.getClassLoader().equals(extension.getClass().getClassLoader());
+                // Ignore empty line
+                line = line.trim();
+                if (line.isEmpty()) {
+                    continue;
+                }
 
-        if (registerExtension) {
-            final Class<?> extensionClass = extension.getClass();
-            if (isControllerService && !checkControllerServiceEligibility(extensionClass)) {
-                registerExtension = false;
-                logger.error(String.format(
-                    "Skipping Controller Service %s because it is bundled with its supporting APIs and requires instance class loading.", extensionClass.getName()));
+                implementationClassNames.add(line);

Review comment:
       The JDK `ServiceLoader` includes a check of the first character in a line that calls `Character.isJavaIdentifierStart()`, is it worth introducing such as check at this point?  The result of the check could throw an exception indicating an invalid line in the service file, as opposed to throwing an error somewhere higher in the chain when attempting to instantiate an class from an invalid string.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/DocGenerator.java
##########
@@ -69,25 +70,34 @@ public static void generate(final NiFiProperties properties, final ExtensionMana
     /**
      * Documents a type of configurable component.
      *
-     * @param extensionClasses types of a configurable component
+     * @param extensionDefinitions definitions of the extensions to document
      * @param explodedNiFiDocsDir base directory of component documentation
      */
-    public static void documentConfigurableComponent(final Set<Class> extensionClasses, final File explodedNiFiDocsDir, final ExtensionManager extensionManager) {
-        for (final Class<?> extensionClass : extensionClasses) {
-            if (ConfigurableComponent.class.isAssignableFrom(extensionClass)) {
-                final String extensionClassName = extensionClass.getCanonicalName();
-
-                final Bundle bundle = extensionManager.getBundle(extensionClass.getClassLoader());
-                if (bundle == null) {
-                    logger.warn("No coordinate found for {}, skipping...", new Object[] {extensionClassName});
-                    continue;
-                }
-                final BundleCoordinate coordinate = bundle.getBundleDetails().getCoordinate();
+    public static void documentConfigurableComponent(final Set<ExtensionDefinition> extensionDefinitions, final File explodedNiFiDocsDir, final ExtensionManager extensionManager) {
+        for (final ExtensionDefinition extensionDefinition : extensionDefinitions) {
+            final Bundle bundle = extensionDefinition.getBundle();
+            if (bundle == null) {
+                logger.warn("Cannot document extension {} because it has no bundle associated with it", extensionDefinition);
+                continue;
+            }
+
+            final BundleCoordinate coordinate = bundle.getBundleDetails().getCoordinate();
+
+            final String extensionClassName = extensionDefinition.getImplementationClassName();
+            final String path = coordinate.getGroup() + "/" + coordinate.getId() + "/" + coordinate.getVersion() + "/" + extensionClassName;
+            final File componentDirectory = new File(explodedNiFiDocsDir, path);
+            final File indexHtml = new File(componentDirectory, "index.html");

Review comment:
       It looks like the `index.html` filename string is referenced multiple times in this class.  Is it worth creating a static variable to define it once and reuse it here as well as in the `document()` method?

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/extensions/FileSystemExtensionRepository.java
##########
@@ -135,6 +135,8 @@ public BundleAvailability getBundleAvailability(final BundleCoordinate bundleCoo
         }
 
         final Set<Bundle> loadedBundles = narLoadResult.getLoadedBundles();
+
+//        extensionManager.registerBundles(loadedBundles);

Review comment:
       Should this line remain commented, or be removed?

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardStatelessDataflowFactory.java
##########
@@ -148,7 +149,19 @@ public StatelessDataflow createDataflow(final StatelessEngineConfiguration engin
                 narClassLoaders, extensionClients);
 
             final VariableRegistry variableRegistry = VariableRegistry.EMPTY_REGISTRY;
-            final PropertyEncryptor encryptor = getPropertyEncryptor(engineConfiguration.getSensitivePropsKey());
+            final Supplier<PropertyEncryptor> encryptorFactory = new Supplier<PropertyEncryptor>() {
+                private PropertyEncryptor created = null;
+
+                @Override
+                public synchronized PropertyEncryptor get() {

Review comment:
       Is it worth considering a more optimized locking strategy that method-level synchronization since this may get called multiple times?  Some options include checking the `PropertyEncryptor` instance for null and then synchronizing around that object, or leveraging Apache Commons `AtomicInitializer`, or implementing something along the lines of that class.




----------------------------------------------------------------
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] factoryexception commented on a change in pull request #4852: NIFI-8212: Refactored StandardExtensionDiscoveringManager to avoid us…

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



##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestListenTCP.java
##########
@@ -156,7 +156,7 @@ public void testTLSClientAuthRequiredAndClientCertNotProvided() throws Initializ
         messages.add("This is message 5\n");
 
         // Make an SSLContext that only has the trust store, this should not work since the processor has client auth REQUIRED
-        Assert.assertThrows(SSLException.class, () ->

Review comment:
       This is a subtle change that works differently based on the specific update revision of Java 8.  More recent versions, after Update 271 in general, should throw an `SSLException` whereas older versions will throw an IOException.  Did you get a test failure?  The recent GitHub CI checks failed with a Checkstyle violation due to the unused import of `SSLException`.  Since `SSLException` is a subclass of `IOException`, your change to the more general exception seems like the better way to go, just remove the unused import and that should work.




----------------------------------------------------------------
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 #4852: NIFI-8212: Refactored StandardExtensionDiscoveringManager to avoid us…

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



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/StatelessNiFi.java
##########
@@ -69,7 +69,7 @@ public static void main(final String[] args) throws IOException, ClassNotFoundEx
             logger.info("Unpacking {} NARs", narFiles.length);
             final long startUnpack = System.nanoTime();
             for (final File narFile : narFiles) {

Review comment:
       Interesting. I looked into this briefly a couple months ago and realized that we still needed to unpack nars for NiFi, so I didn't bother looking much further. Not sure if the need was only for docs or not, i don't remember the details. Just remember that I stopped looking into it for some reason or another (it's possible that I just forgot about it??) But yeah it could be worth pursuing. 




----------------------------------------------------------------
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 #4852: NIFI-8212: Refactored StandardExtensionDiscoveringManager to avoid us…

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



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/provenance/ComponentIdentifierLookup.java
##########
@@ -46,14 +48,15 @@ public ComponentIdentifierLookup(final FlowController flowController) {
 
     @Override
     public List<String> getComponentTypes() {
-        final Set<Class> procClasses = flowController.getExtensionManager().getExtensions(Processor.class);
+        final Set<ExtensionDefinition> procDefinitions = flowController.getExtensionManager().getExtensions(Processor.class);
 
-        final List<String> componentTypes = new ArrayList<>(procClasses.size() + 2);
+        final List<String> componentTypes = new ArrayList<>(procDefinitions.size() + 2);
         componentTypes.add(ProvenanceEventRecord.REMOTE_INPUT_PORT_TYPE);
         componentTypes.add(ProvenanceEventRecord.REMOTE_OUTPUT_PORT_TYPE);
 
-        procClasses.stream()
-            .map(Class::getSimpleName)
+        procDefinitions.stream()
+            .map(ExtensionDefinition::getImplementationClassName)
+            .map(className -> className.contains(".") ? StringUtils.substringAfterLast(className, ".") : className)

Review comment:
       To do that, we would have to either implement the logic of substringAfterLast in ExtensionDefinition (which isn't difficult but is probably uglier than repeating this trivial tertiary check twice) or bring in a dependency on commons-utils, and that dependency cannot exist there due to classloader hierarchy. So I think it makes more sense to just leave as-is for now.




----------------------------------------------------------------
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 #4852: NIFI-8212: Refactored StandardExtensionDiscoveringManager to avoid us…

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



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/StandardExtensionDiscoveringManager.java
##########
@@ -145,74 +152,103 @@ public void discoverExtensions(final Set<Bundle> narBundles) {
      *
      * @param bundle from which to load extensions
      */
-    @SuppressWarnings("unchecked")
     private void loadExtensions(final Bundle bundle) {
-        for (final Map.Entry<Class, Set<Class>> entry : definitionMap.entrySet()) {
-            final boolean isControllerService = ControllerService.class.equals(entry.getKey());
-            final boolean isProcessor = Processor.class.equals(entry.getKey());
-            final boolean isReportingTask = ReportingTask.class.equals(entry.getKey());
-
-            final ServiceLoader<?> serviceLoader = ServiceLoader.load(entry.getKey(), bundle.getClassLoader());
-            for (final Object o : serviceLoader) {
-                try {
-                    loadExtension(o, entry.getKey(), bundle);
-                } catch (Exception e) {
-                    logger.warn("Failed to register extension {} due to: {}" , new Object[]{o.getClass().getCanonicalName(), e.getMessage()});
-                    if (logger.isDebugEnabled()) {
-                        logger.debug("", e);
+        for (final Class extensionType : definitionMap.keySet()) {
+            final String serviceType = extensionType.getName();
+
+            try {
+                final Set<URL> serviceResourceUrls = getServiceFileURLs(bundle, extensionType);
+                logger.debug("Bundle {} has the following Services File URLs for {}: {}", bundle, serviceType, serviceResourceUrls);
+
+                for (final URL serviceResourceUrl : serviceResourceUrls) {
+                    final Set<String> implementationClassNames = getServiceFileImplementationClassNames(serviceResourceUrl);
+                    logger.debug("Bundle {} defines {} implementations of interface {}", bundle, implementationClassNames.size(), serviceType);
+
+                    for (final String implementationClassName : implementationClassNames) {
+                        try {
+                            loadExtension(implementationClassName, extensionType, bundle);
+                            logger.debug("Successfully loaded {} {} from {}", extensionType.getSimpleName(), implementationClassName, bundle);
+                        } catch (final Exception e) {
+                            logger.error("Failed to register {} of type {} in bundle {}" , extensionType.getSimpleName(), implementationClassName, bundle, e);
+                        }
                     }
                 }
+            } catch (final IOException e) {
+                throw new RuntimeException("Failed to get resources of type " + serviceType + " from bundle " + bundle);
             }
-
-            classLoaderBundleLookup.put(bundle.getClassLoader(), bundle);
         }
+
+        classLoaderBundleLookup.put(bundle.getClassLoader(), bundle);
     }
 
-    protected void loadExtension(final Object extension, final Class<?> extensionType, final Bundle bundle) {
-        final boolean isControllerService = ControllerService.class.equals(extensionType);
-        final boolean isProcessor = Processor.class.equals(extensionType);
-        final boolean isReportingTask = ReportingTask.class.equals(extensionType);
+    private Set<String> getServiceFileImplementationClassNames(final URL serviceFileUrl) throws IOException {
+        final Set<String> implementationClassNames = new HashSet<>();
 
-        // create a cache of temp ConfigurableComponent instances, the initialize here has to happen before the checks below
-        if ((isControllerService || isProcessor || isReportingTask) && extension instanceof ConfigurableComponent) {
-            final ConfigurableComponent configurableComponent = (ConfigurableComponent) extension;
-            initializeTempComponent(configurableComponent);
+        try (final InputStream in = serviceFileUrl.openStream();
+             final Reader inputStreamReader = new InputStreamReader(in);
+             final BufferedReader reader = new BufferedReader(inputStreamReader)) {
 
-            final String cacheKey = getClassBundleKey(extension.getClass().getCanonicalName(), bundle.getBundleDetails().getCoordinate());
-            tempComponentLookup.put(cacheKey, configurableComponent);
-        }
+            String line;
+            while ((line = reader.readLine()) != null) {
+                // Remove anything after the #
+                final int index = line.indexOf("#");
+                if (index >= 0) {
+                    line = line.substring(0, index);
+                }
 
-        // only consider extensions discovered directly in this bundle
-        boolean registerExtension = bundle.getClassLoader().equals(extension.getClass().getClassLoader());
+                // Ignore empty line
+                line = line.trim();
+                if (line.isEmpty()) {
+                    continue;
+                }
 
-        if (registerExtension) {
-            final Class<?> extensionClass = extension.getClass();
-            if (isControllerService && !checkControllerServiceEligibility(extensionClass)) {
-                registerExtension = false;
-                logger.error(String.format(
-                    "Skipping Controller Service %s because it is bundled with its supporting APIs and requires instance class loading.", extensionClass.getName()));
+                implementationClassNames.add(line);

Review comment:
       @ottobackwards I'm not familiar with that project. Will have to look into it. 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



[GitHub] [nifi] markap14 commented on a change in pull request #4852: NIFI-8212: Refactored StandardExtensionDiscoveringManager to avoid us…

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



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/DocGenerator.java
##########
@@ -69,25 +70,34 @@ public static void generate(final NiFiProperties properties, final ExtensionMana
     /**
      * Documents a type of configurable component.
      *
-     * @param extensionClasses types of a configurable component
+     * @param extensionDefinitions definitions of the extensions to document
      * @param explodedNiFiDocsDir base directory of component documentation
      */
-    public static void documentConfigurableComponent(final Set<Class> extensionClasses, final File explodedNiFiDocsDir, final ExtensionManager extensionManager) {
-        for (final Class<?> extensionClass : extensionClasses) {
-            if (ConfigurableComponent.class.isAssignableFrom(extensionClass)) {
-                final String extensionClassName = extensionClass.getCanonicalName();
-
-                final Bundle bundle = extensionManager.getBundle(extensionClass.getClassLoader());
-                if (bundle == null) {
-                    logger.warn("No coordinate found for {}, skipping...", new Object[] {extensionClassName});
-                    continue;
-                }
-                final BundleCoordinate coordinate = bundle.getBundleDetails().getCoordinate();
+    public static void documentConfigurableComponent(final Set<ExtensionDefinition> extensionDefinitions, final File explodedNiFiDocsDir, final ExtensionManager extensionManager) {
+        for (final ExtensionDefinition extensionDefinition : extensionDefinitions) {
+            final Bundle bundle = extensionDefinition.getBundle();
+            if (bundle == null) {
+                logger.warn("Cannot document extension {} because it has no bundle associated with it", extensionDefinition);
+                continue;
+            }
+
+            final BundleCoordinate coordinate = bundle.getBundleDetails().getCoordinate();
+
+            final String extensionClassName = extensionDefinition.getImplementationClassName();
+            final String path = coordinate.getGroup() + "/" + coordinate.getId() + "/" + coordinate.getVersion() + "/" + extensionClassName;
+            final File componentDirectory = new File(explodedNiFiDocsDir, path);
+            final File indexHtml = new File(componentDirectory, "index.html");

Review comment:
       Not IMO. I am very much not a fan of extracting strings into static member variables unless they are pretty heavily used or likely to change. It results in lots of statically defined strings that really muddy up the 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 #4852: NIFI-8212: Refactored StandardExtensionDiscoveringManager to avoid us…

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



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/service/ServiceStateTransition.java
##########
@@ -58,6 +65,14 @@ public boolean enable() {
             state = ControllerServiceState.ENABLED;
 
             enabledFutures.forEach(future -> future.complete(null));
+

Review comment:
       What are you thinking makes sense to log at a debug level?




----------------------------------------------------------------
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 #4852: NIFI-8212: Refactored StandardExtensionDiscoveringManager to avoid us…

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



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarUnpacker.java
##########
@@ -288,17 +280,20 @@ private static void unpackBundleDocs(final File docsDirectory, final ExtensionMa
      *
      * @param nar the nar to unpack
      * @param baseWorkingDirectory the directory to unpack to
+     * @param verifyHash if the NAR has already been unpacked, indicates whether or not the hash should be verified. If this value is true,
+     * and the NAR's hash does not match the hash written to the unpacked directory, the working directory will be deleted and the NAR will be
+     * unpacked again. If false, the NAR will not be unpacked again and its hash will not be checked.
      * @return the directory to the unpacked NAR
      * @throws IOException if unable to explode nar
      */
-    public static File unpackNar(final File nar, final File baseWorkingDirectory) throws IOException {
+    public static File unpackNar(final File nar, final File baseWorkingDirectory, final boolean verifyHash) throws IOException {
         final File narWorkingDirectory = new File(baseWorkingDirectory, nar.getName() + "-unpacked");
 
         // if the working directory doesn't exist, unpack the nar
         if (!narWorkingDirectory.exists()) {
             unpack(nar, narWorkingDirectory, FileDigestUtils.getDigest(nar));
-        } else {
-            // the working directory does exist. Run digest against the nar
+        } else if (verifyHash) {
+            // the working directory does exist. Run MD5 sum against the nar

Review comment:
       Good catch. I missed that when rebasing.




----------------------------------------------------------------
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 #4852: NIFI-8212: Refactored StandardExtensionDiscoveringManager to avoid us…

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



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/StandardExtensionDiscoveringManager.java
##########
@@ -145,74 +152,103 @@ public void discoverExtensions(final Set<Bundle> narBundles) {
      *
      * @param bundle from which to load extensions
      */
-    @SuppressWarnings("unchecked")
     private void loadExtensions(final Bundle bundle) {
-        for (final Map.Entry<Class, Set<Class>> entry : definitionMap.entrySet()) {
-            final boolean isControllerService = ControllerService.class.equals(entry.getKey());
-            final boolean isProcessor = Processor.class.equals(entry.getKey());
-            final boolean isReportingTask = ReportingTask.class.equals(entry.getKey());
-
-            final ServiceLoader<?> serviceLoader = ServiceLoader.load(entry.getKey(), bundle.getClassLoader());
-            for (final Object o : serviceLoader) {
-                try {
-                    loadExtension(o, entry.getKey(), bundle);
-                } catch (Exception e) {
-                    logger.warn("Failed to register extension {} due to: {}" , new Object[]{o.getClass().getCanonicalName(), e.getMessage()});
-                    if (logger.isDebugEnabled()) {
-                        logger.debug("", e);
+        for (final Class extensionType : definitionMap.keySet()) {
+            final String serviceType = extensionType.getName();
+
+            try {
+                final Set<URL> serviceResourceUrls = getServiceFileURLs(bundle, extensionType);
+                logger.debug("Bundle {} has the following Services File URLs for {}: {}", bundle, serviceType, serviceResourceUrls);
+
+                for (final URL serviceResourceUrl : serviceResourceUrls) {
+                    final Set<String> implementationClassNames = getServiceFileImplementationClassNames(serviceResourceUrl);
+                    logger.debug("Bundle {} defines {} implementations of interface {}", bundle, implementationClassNames.size(), serviceType);
+
+                    for (final String implementationClassName : implementationClassNames) {
+                        try {
+                            loadExtension(implementationClassName, extensionType, bundle);
+                            logger.debug("Successfully loaded {} {} from {}", extensionType.getSimpleName(), implementationClassName, bundle);
+                        } catch (final Exception e) {
+                            logger.error("Failed to register {} of type {} in bundle {}" , extensionType.getSimpleName(), implementationClassName, bundle, e);
+                        }
                     }
                 }
+            } catch (final IOException e) {
+                throw new RuntimeException("Failed to get resources of type " + serviceType + " from bundle " + bundle);
             }
-
-            classLoaderBundleLookup.put(bundle.getClassLoader(), bundle);
         }
+
+        classLoaderBundleLookup.put(bundle.getClassLoader(), bundle);
     }
 
-    protected void loadExtension(final Object extension, final Class<?> extensionType, final Bundle bundle) {
-        final boolean isControllerService = ControllerService.class.equals(extensionType);
-        final boolean isProcessor = Processor.class.equals(extensionType);
-        final boolean isReportingTask = ReportingTask.class.equals(extensionType);
+    private Set<String> getServiceFileImplementationClassNames(final URL serviceFileUrl) throws IOException {
+        final Set<String> implementationClassNames = new HashSet<>();
 
-        // create a cache of temp ConfigurableComponent instances, the initialize here has to happen before the checks below
-        if ((isControllerService || isProcessor || isReportingTask) && extension instanceof ConfigurableComponent) {
-            final ConfigurableComponent configurableComponent = (ConfigurableComponent) extension;
-            initializeTempComponent(configurableComponent);
+        try (final InputStream in = serviceFileUrl.openStream();
+             final Reader inputStreamReader = new InputStreamReader(in);
+             final BufferedReader reader = new BufferedReader(inputStreamReader)) {
 
-            final String cacheKey = getClassBundleKey(extension.getClass().getCanonicalName(), bundle.getBundleDetails().getCoordinate());
-            tempComponentLookup.put(cacheKey, configurableComponent);
-        }
+            String line;
+            while ((line = reader.readLine()) != null) {
+                // Remove anything after the #
+                final int index = line.indexOf("#");
+                if (index >= 0) {
+                    line = line.substring(0, index);
+                }
 
-        // only consider extensions discovered directly in this bundle
-        boolean registerExtension = bundle.getClassLoader().equals(extension.getClass().getClassLoader());
+                // Ignore empty line
+                line = line.trim();
+                if (line.isEmpty()) {
+                    continue;
+                }
 
-        if (registerExtension) {
-            final Class<?> extensionClass = extension.getClass();
-            if (isControllerService && !checkControllerServiceEligibility(extensionClass)) {
-                registerExtension = false;
-                logger.error(String.format(
-                    "Skipping Controller Service %s because it is bundled with its supporting APIs and requires instance class loading.", extensionClass.getName()));
+                implementationClassNames.add(line);

Review comment:
       @exceptionfactory we could add a check like that, but I don't know that it really buys us anything. It seems very arbitrary to me, to check one character on each line. Unless you have some deeper understanding that I am lacking, as to why that is an important check, I don't think it's worth muddying the codebase to try to discover an anomaly with the first character of each line, in particular, when any other character could be an issue as well




----------------------------------------------------------------
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] exceptionfactory commented on a change in pull request #4852: NIFI-8212: Refactored StandardExtensionDiscoveringManager to avoid us…

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



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardStatelessDataflowFactory.java
##########
@@ -148,7 +149,19 @@ public StatelessDataflow createDataflow(final StatelessEngineConfiguration engin
                 narClassLoaders, extensionClients);
 
             final VariableRegistry variableRegistry = VariableRegistry.EMPTY_REGISTRY;
-            final PropertyEncryptor encryptor = getPropertyEncryptor(engineConfiguration.getSensitivePropsKey());
+            final Supplier<PropertyEncryptor> encryptorFactory = new Supplier<PropertyEncryptor>() {
+                private PropertyEncryptor created = null;
+
+                @Override
+                public synchronized PropertyEncryptor get() {

Review comment:
       Is it worth considering a more optimized locking strategy than method-level synchronization since this may get called multiple times?  Some options include checking the `PropertyEncryptor` instance for null and then synchronizing around that object, or leveraging Apache Commons `AtomicInitializer`, or implementing something along the lines of that class.




----------------------------------------------------------------
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 #4852: NIFI-8212: Refactored StandardExtensionDiscoveringManager to avoid us…

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



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardStatelessDataflowFactory.java
##########
@@ -148,7 +149,19 @@ public StatelessDataflow createDataflow(final StatelessEngineConfiguration engin
                 narClassLoaders, extensionClients);
 
             final VariableRegistry variableRegistry = VariableRegistry.EMPTY_REGISTRY;
-            final PropertyEncryptor encryptor = getPropertyEncryptor(engineConfiguration.getSensitivePropsKey());
+            final Supplier<PropertyEncryptor> encryptorFactory = new Supplier<PropertyEncryptor>() {
+                private PropertyEncryptor created = null;
+
+                @Override
+                public synchronized PropertyEncryptor get() {

Review comment:
       I don't think lock contention is a concern here. It'll be called rarely. The synchronization won't add any significant overhead within the context that it's evaluated.




----------------------------------------------------------------
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] bbende commented on a change in pull request #4852: NIFI-8212: Refactored StandardExtensionDiscoveringManager to avoid us…

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



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/ExtensionBuilder.java
##########
@@ -463,10 +524,12 @@ private ControllerServiceNode createGhostControllerServiceNode() {
             Thread.currentThread().setContextClassLoader(detectedClassLoader);
 
             final Object extensionInstance = rawClass.newInstance();
+            final T cast = nodeType.cast(extensionInstance);
+            verifyControllerServiceReferences(cast, bundle.getClassLoader());

Review comment:
       When testing creating some processors I decided to try adding PutHDFS since it has `@RequiresInstanceClassLoading`, and it produced a ghost component with this exception in the logs...
   ```
   Caused by: java.lang.NullPointerException: null
   	at java.util.ArrayList.<init>(ArrayList.java:178)
   	at org.apache.nifi.processors.hadoop.PutHDFS.getSupportedPropertyDescriptors(PutHDFS.java:202)
   	at org.apache.nifi.components.AbstractConfigurableComponent.getPropertyDescriptors(AbstractConfigurableComponent.java:247)
   	at org.apache.nifi.controller.ExtensionBuilder.verifyControllerServiceReferences(ExtensionBuilder.java:433)
   	at org.apache.nifi.controller.ExtensionBuilder.createLoggableComponent(ExtensionBuilder.java:528)
   	at org.apache.nifi.controller.ExtensionBuilder.createLoggableProcessor(ExtensionBuilder.java:485)
   	... 170 common frames omitted
   ```
   I think the issue is that verify is calls `getSupportedPropertyDescritptors`, but we haven't called `initialize` on the component yet and we don't know if initialize is what sets up the component's property descriptors.




----------------------------------------------------------------
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] exceptionfactory commented on a change in pull request #4852: NIFI-8212: Refactored StandardExtensionDiscoveringManager to avoid us…

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



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/StandardExtensionDiscoveringManager.java
##########
@@ -128,7 +131,11 @@ public void discoverExtensions(final Set<Bundle> narBundles) {
             // so that static initialization techniques that depend on the context class loader will work properly
             final ClassLoader ncl = bundle.getClassLoader();
             Thread.currentThread().setContextClassLoader(ncl);
+
+            final long loadStart = System.currentTimeMillis();
             loadExtensions(bundle);
+            final long loadMillis = System.currentTimeMillis() - loadStart;
+            logger.info("Loaded extensions for {} in {} millis", bundle.getBundleDetails(), loadMillis);

Review comment:
       Should this timing information be logged at the info level, that can produce a number of logs given that it runs for each bundle.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/DocGenerator.java
##########
@@ -69,25 +70,34 @@ public static void generate(final NiFiProperties properties, final ExtensionMana
     /**
      * Documents a type of configurable component.
      *
-     * @param extensionClasses types of a configurable component
+     * @param extensionDefinitions definitions of the extensions to document
      * @param explodedNiFiDocsDir base directory of component documentation
      */
-    public static void documentConfigurableComponent(final Set<Class> extensionClasses, final File explodedNiFiDocsDir, final ExtensionManager extensionManager) {
-        for (final Class<?> extensionClass : extensionClasses) {
-            if (ConfigurableComponent.class.isAssignableFrom(extensionClass)) {
-                final String extensionClassName = extensionClass.getCanonicalName();
-
-                final Bundle bundle = extensionManager.getBundle(extensionClass.getClassLoader());
-                if (bundle == null) {
-                    logger.warn("No coordinate found for {}, skipping...", new Object[] {extensionClassName});
-                    continue;
-                }
-                final BundleCoordinate coordinate = bundle.getBundleDetails().getCoordinate();
+    public static void documentConfigurableComponent(final Set<ExtensionDefinition> extensionDefinitions, final File explodedNiFiDocsDir, final ExtensionManager extensionManager) {
+        for (final ExtensionDefinition extensionDefinition : extensionDefinitions) {
+            final Bundle bundle = extensionDefinition.getBundle();
+            if (bundle == null) {
+                logger.warn("Cannot document extension {} because it has no bundle associated with it", extensionDefinition);
+                continue;
+            }
+
+            final BundleCoordinate coordinate = bundle.getBundleDetails().getCoordinate();
+
+            final String extensionClassName = extensionDefinition.getImplementationClassName();
+            final String path = coordinate.getGroup() + "/" + coordinate.getId() + "/" + coordinate.getVersion() + "/" + extensionClassName;
+            final File componentDirectory = new File(explodedNiFiDocsDir, path);
+            final File indexHtml = new File(componentDirectory, "index.html");

Review comment:
       Thanks for the reply, that sounds reasonable.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/provenance/ComponentIdentifierLookup.java
##########
@@ -46,14 +48,15 @@ public ComponentIdentifierLookup(final FlowController flowController) {
 
     @Override
     public List<String> getComponentTypes() {
-        final Set<Class> procClasses = flowController.getExtensionManager().getExtensions(Processor.class);
+        final Set<ExtensionDefinition> procDefinitions = flowController.getExtensionManager().getExtensions(Processor.class);
 
-        final List<String> componentTypes = new ArrayList<>(procClasses.size() + 2);
+        final List<String> componentTypes = new ArrayList<>(procDefinitions.size() + 2);
         componentTypes.add(ProvenanceEventRecord.REMOTE_INPUT_PORT_TYPE);
         componentTypes.add(ProvenanceEventRecord.REMOTE_OUTPUT_PORT_TYPE);
 
-        procClasses.stream()
-            .map(Class::getSimpleName)
+        procDefinitions.stream()
+            .map(ExtensionDefinition::getImplementationClassName)
+            .map(className -> className.contains(".") ? StringUtils.substringAfterLast(className, ".") : className)

Review comment:
       That makes sense, given it is only in two places, not a real concern right now.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/StandardExtensionDiscoveringManager.java
##########
@@ -145,74 +152,103 @@ public void discoverExtensions(final Set<Bundle> narBundles) {
      *
      * @param bundle from which to load extensions
      */
-    @SuppressWarnings("unchecked")
     private void loadExtensions(final Bundle bundle) {
-        for (final Map.Entry<Class, Set<Class>> entry : definitionMap.entrySet()) {
-            final boolean isControllerService = ControllerService.class.equals(entry.getKey());
-            final boolean isProcessor = Processor.class.equals(entry.getKey());
-            final boolean isReportingTask = ReportingTask.class.equals(entry.getKey());
-
-            final ServiceLoader<?> serviceLoader = ServiceLoader.load(entry.getKey(), bundle.getClassLoader());
-            for (final Object o : serviceLoader) {
-                try {
-                    loadExtension(o, entry.getKey(), bundle);
-                } catch (Exception e) {
-                    logger.warn("Failed to register extension {} due to: {}" , new Object[]{o.getClass().getCanonicalName(), e.getMessage()});
-                    if (logger.isDebugEnabled()) {
-                        logger.debug("", e);
+        for (final Class extensionType : definitionMap.keySet()) {
+            final String serviceType = extensionType.getName();
+
+            try {
+                final Set<URL> serviceResourceUrls = getServiceFileURLs(bundle, extensionType);
+                logger.debug("Bundle {} has the following Services File URLs for {}: {}", bundle, serviceType, serviceResourceUrls);
+
+                for (final URL serviceResourceUrl : serviceResourceUrls) {
+                    final Set<String> implementationClassNames = getServiceFileImplementationClassNames(serviceResourceUrl);
+                    logger.debug("Bundle {} defines {} implementations of interface {}", bundle, implementationClassNames.size(), serviceType);
+
+                    for (final String implementationClassName : implementationClassNames) {
+                        try {
+                            loadExtension(implementationClassName, extensionType, bundle);
+                            logger.debug("Successfully loaded {} {} from {}", extensionType.getSimpleName(), implementationClassName, bundle);
+                        } catch (final Exception e) {
+                            logger.error("Failed to register {} of type {} in bundle {}" , extensionType.getSimpleName(), implementationClassName, bundle, e);
+                        }
                     }
                 }
+            } catch (final IOException e) {
+                throw new RuntimeException("Failed to get resources of type " + serviceType + " from bundle " + bundle);
             }
-
-            classLoaderBundleLookup.put(bundle.getClassLoader(), bundle);
         }
+
+        classLoaderBundleLookup.put(bundle.getClassLoader(), bundle);
     }
 
-    protected void loadExtension(final Object extension, final Class<?> extensionType, final Bundle bundle) {
-        final boolean isControllerService = ControllerService.class.equals(extensionType);
-        final boolean isProcessor = Processor.class.equals(extensionType);
-        final boolean isReportingTask = ReportingTask.class.equals(extensionType);
+    private Set<String> getServiceFileImplementationClassNames(final URL serviceFileUrl) throws IOException {
+        final Set<String> implementationClassNames = new HashSet<>();
 
-        // create a cache of temp ConfigurableComponent instances, the initialize here has to happen before the checks below
-        if ((isControllerService || isProcessor || isReportingTask) && extension instanceof ConfigurableComponent) {
-            final ConfigurableComponent configurableComponent = (ConfigurableComponent) extension;
-            initializeTempComponent(configurableComponent);
+        try (final InputStream in = serviceFileUrl.openStream();
+             final Reader inputStreamReader = new InputStreamReader(in);
+             final BufferedReader reader = new BufferedReader(inputStreamReader)) {
 
-            final String cacheKey = getClassBundleKey(extension.getClass().getCanonicalName(), bundle.getBundleDetails().getCoordinate());
-            tempComponentLookup.put(cacheKey, configurableComponent);
-        }
+            String line;
+            while ((line = reader.readLine()) != null) {
+                // Remove anything after the #
+                final int index = line.indexOf("#");
+                if (index >= 0) {
+                    line = line.substring(0, index);
+                }
 
-        // only consider extensions discovered directly in this bundle
-        boolean registerExtension = bundle.getClassLoader().equals(extension.getClass().getClassLoader());
+                // Ignore empty line
+                line = line.trim();
+                if (line.isEmpty()) {
+                    continue;
+                }
 
-        if (registerExtension) {
-            final Class<?> extensionClass = extension.getClass();
-            if (isControllerService && !checkControllerServiceEligibility(extensionClass)) {
-                registerExtension = false;
-                logger.error(String.format(
-                    "Skipping Controller Service %s because it is bundled with its supporting APIs and requires instance class loading.", extensionClass.getName()));
+                implementationClassNames.add(line);

Review comment:
       Thanks for the reply.  The Character is check appears to be just a sanity check in ServiceLoader.  Invalid class names will throw other errors, so it is really just a matter of where NiFi catches the problem.  Leaving out the check is fine if it is reasonable to wait until the class gets loaded to indicate what is most likely a typo.




----------------------------------------------------------------
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 #4852: NIFI-8212: Refactored StandardExtensionDiscoveringManager to avoid us…

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



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/ExtensionBuilder.java
##########
@@ -463,10 +524,12 @@ private ControllerServiceNode createGhostControllerServiceNode() {
             Thread.currentThread().setContextClassLoader(detectedClassLoader);
 
             final Object extensionInstance = rawClass.newInstance();
+            final T cast = nodeType.cast(extensionInstance);
+            verifyControllerServiceReferences(cast, bundle.getClassLoader());

Review comment:
       Great catch




----------------------------------------------------------------
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 #4852: NIFI-8212: Refactored StandardExtensionDiscoveringManager to avoid us…

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



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/repository/scheduling/ConnectableProcessContext.java
##########
@@ -41,18 +33,31 @@
 import org.apache.nifi.scheduling.ExecutionNode;
 import org.apache.nifi.util.Connectables;
 
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
 /**
  * This class is essentially an empty shell for {@link Connectable}s that are not Processors
  */
 public class ConnectableProcessContext implements ProcessContext {
 
     private final Connectable connectable;
-    private final PropertyEncryptor encryptor;
+    private final Supplier<PropertyEncryptor> encryptorFactory;

Review comment:
       Yeah I think the PropertyEncryptor interface was introduced just recently, after I have written the code for this PR. It's worth considering. Will look into it.




----------------------------------------------------------------
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 #4852: NIFI-8212: Refactored StandardExtensionDiscoveringManager to avoid us…

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



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/StandardExtensionDiscoveringManager.java
##########
@@ -128,7 +131,11 @@ public void discoverExtensions(final Set<Bundle> narBundles) {
             // so that static initialization techniques that depend on the context class loader will work properly
             final ClassLoader ncl = bundle.getClassLoader();
             Thread.currentThread().setContextClassLoader(ncl);
+
+            final long loadStart = System.currentTimeMillis();
             loadExtensions(bundle);
+            final long loadMillis = System.currentTimeMillis() - loadStart;
+            logger.info("Loaded extensions for {} in {} millis", bundle.getBundleDetails(), loadMillis);

Review comment:
       I did consider moving that down to DEBUG level. But I think it makes more sense at INFO level. It is verbose, yes. But it's only on startup, and can be good info to have, as a long load time would indicate that something may be problematic. Plus, it was certainly convenient for measuring timing before & after the change. But I am ok decreasing to DEBUG if you think it would be annoying in the logs.




----------------------------------------------------------------
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 #4852: NIFI-8212: Refactored StandardExtensionDiscoveringManager to avoid us…

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



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceNode.java
##########
@@ -181,6 +182,8 @@
      */
     boolean isActive();
 

Review comment:
       javadoc?




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