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 2022/09/20 09:05:36 UTC

[GitHub] [nifi] simonbence opened a new pull request, #6433: NIFI-10497 Making RegistryClient an extension point

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

   [NIFI-10497](https://issues.apache.org/jira/browse/NIFI-10497)
   
   Currently NiFi is capable to connect only to the NiFi Registry as a "registry repository". It would be beneficial to give NiFi the capability to depend on other services.
   
   In order to this, I suggest to decouple the registry behaviour from the NiFi Registry as much as it is possible (inlcuding but not only the API and the Resources) and move the actual implementation behind this new API.
   
   To be able to add new implementations, this new API must be an extension point applying the usual NiFi instruments for this. Also it is paramunt to keep the continuity with the current usages and make the implementation capable to process the current REST call format.
   
   This is a draft PR for early review, but please expect a PR soonish.
   
   <!-- 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. -->
   
   # Summary
   
   [NIFI-00000](https://issues.apache.org/jira/browse/NIFI-00000)
   
   # Tracking
   
   Please complete the following tracking steps prior to pull request creation.
   
   ### Issue Tracking
   
   - [ ] [Apache NiFi Jira](https://issues.apache.org/jira/browse/NIFI) issue created
   
   ### Pull Request Tracking
   
   - [ ] Pull Request title starts with Apache NiFi Jira issue number, such as `NIFI-00000`
   - [ ] Pull Request commit message starts with Apache NiFi Jira issue number, as such `NIFI-00000`
   
   ### Pull Request Formatting
   
   - [ ] Pull Request based on current revision of the `main` branch
   - [ ] Pull Request refers to a feature branch with one commit containing changes
   
   # Verification
   
   Please indicate the verification steps performed prior to pull request creation.
   
   ### Build
   
   - [ ] Build completed using `mvn clean install -P contrib-check`
     - [ ] JDK 8
     - [ ] JDK 11
     - [ ] JDK 17
   
   ### Licensing
   
   - [ ] New dependencies are compatible with the [Apache License 2.0](https://apache.org/licenses/LICENSE-2.0) according to the [License Policy](https://www.apache.org/legal/resolved.html)
   - [ ] New dependencies are documented in applicable `LICENSE` and `NOTICE` files
   
   ### Documentation
   
   - [ ] Documentation formatting appears as expected in rendered files
   


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

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

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


[GitHub] [nifi] simonbence commented on a diff in pull request #6433: NIFI-10497 Making RegistryClient an extension point

Posted by GitBox <gi...@apache.org>.
simonbence commented on code in PR #6433:
URL: https://github.com/apache/nifi/pull/6433#discussion_r976209016


##########
nifi-nar-bundles/nifi-flow-registry-client-bundle/nifi-flow-registry-client-services/src/main/java/org/apache/nifi/registry/flow/NifiRegistryFlowRegistryClient.java:
##########
@@ -0,0 +1,376 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.registry.flow;
+
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.registry.bucket.Bucket;
+import org.apache.nifi.registry.client.BucketClient;
+import org.apache.nifi.registry.client.FlowClient;
+import org.apache.nifi.registry.client.FlowSnapshotClient;
+import org.apache.nifi.registry.client.NiFiRegistryClient;
+import org.apache.nifi.registry.client.NiFiRegistryClientConfig;
+import org.apache.nifi.registry.client.NiFiRegistryException;
+import org.apache.nifi.registry.client.impl.JerseyNiFiRegistryClient;
+import org.apache.nifi.registry.client.impl.request.ProxiedEntityRequestConfig;
+import org.apache.nifi.security.util.KeystoreType;
+import org.apache.nifi.security.util.SslContextFactory;
+import org.apache.nifi.security.util.StandardTlsConfiguration;
+import org.apache.nifi.security.util.TlsConfiguration;
+import org.apache.nifi.security.util.TlsException;
+
+import javax.net.ssl.SSLContext;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public class NifiRegistryFlowRegistryClient extends AbstractFlowRegistryClient {
+
+    public final static PropertyDescriptor PROPERTY_URL = new PropertyDescriptor.Builder()
+            .name("url")
+            .displayName("URL")
+            .description("URL of the NiFi Registry")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .required(true)
+            .build();
+    public final static PropertyDescriptor PROPERTY_KEYSTORE_PATH = new PropertyDescriptor.Builder()
+            .name("keystorePath")
+            .displayName("Keystore Path")
+            .description("The fully-qualified filename of the Keystore")
+            .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
+            .required(false)
+            .build();
+    public final static PropertyDescriptor PROPERTY_KEYSTORE_PASSWORD = new PropertyDescriptor.Builder()
+            .name("keystorePassword")
+            .displayName("Keystore Password")
+            .description("The password for the Keystore")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .sensitive(true)
+            .required(false)
+            .build();
+    public final static PropertyDescriptor PROPERTY_KEY_PASSWORD = new PropertyDescriptor.Builder()
+            .name("keyPassword")
+            .displayName("Key Password")
+            .description("The password for the key. If this is not specified, but the Keystore Filename, Password, and Type are specified, "
+                    + "then the Keystore Password will be assumed to be the same as the Key Password.")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .sensitive(true)
+            .required(false)
+            .build();
+    public final static PropertyDescriptor PROPERTY_KEYSTORE_TYPE = new PropertyDescriptor.Builder()
+            .name("keystoreType")
+            .displayName("Keystore Type")
+            .description("The Type of the Keystore")
+            .allowableValues(KeystoreType.values())
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .required(false)
+            .build();
+    public final static PropertyDescriptor PROPERTY_TRUSTSTORE_PATH = new PropertyDescriptor.Builder()
+            .name("truststorePath")
+            .displayName("Truststore Path")
+            .description("The fully-qualified filename of the Truststore")
+            .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
+            .required(false)
+            .build();
+    public final static PropertyDescriptor PROPERTY_TRUSTSTORE_PASSWORD = new PropertyDescriptor.Builder()
+            .name("truststorePassword")
+            .displayName("Truststore Password")
+            .description("The password for the Truststore")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .sensitive(true)
+            .required(false)
+            .build();
+    public final static PropertyDescriptor PROPERTY_TRUSTSTORE_TYPE = new PropertyDescriptor.Builder()
+            .name("truststoreType")
+            .displayName("Truststore Type")
+            .description("The Type of the Truststore")
+            .allowableValues(KeystoreType.values())
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .required(false)
+            .build();
+
+    private volatile String registryUrl;
+    private volatile NiFiRegistryClient registryClient;
+
+    private synchronized NiFiRegistryClient getRegistryClient(final FlowRegistryClientConfigurationContext context) {
+        final String configuredUrl = context.getProperty(PROPERTY_URL).evaluateAttributeExpressions().getValue();
+
+        final URI uri;
+
+        try {
+            // Handles case where the URI entered has a trailing slash, or includes the trailing /nifi-registry-api
+            uri = new URIBuilder(configuredUrl).setPath("").removeQuery().build();
+        } catch (URISyntaxException e) {
+            throw new IllegalArgumentException("The given Registry URL is not valid: " + configuredUrl);
+        }
+
+        final String uriScheme = uri.getScheme();
+        if (uriScheme == null) {
+            throw new IllegalArgumentException("The given Registry URL is not valid: " + configuredUrl);
+        }
+
+        final String proposedUrl = uri.toString();;
+
+        if (!proposedUrl.equals(registryUrl)) {
+            registryUrl = proposedUrl;
+            invalidateClient();
+        }
+
+        if (registryClient != null) {
+            return registryClient;
+        }
+
+        final NiFiRegistryClientConfig config = new NiFiRegistryClientConfig.Builder()
+                .connectTimeout(30000)
+                .readTimeout(30000)
+                .sslContext(extractSSLContext(context))
+                .baseUrl(registryUrl)
+                .build();
+        registryClient = new JerseyNiFiRegistryClient.Builder()
+                .config(config)
+                .build();
+
+        return registryClient;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return Arrays.asList(
+                PROPERTY_URL,
+                PROPERTY_KEYSTORE_PATH,
+                PROPERTY_KEYSTORE_TYPE,
+                PROPERTY_KEYSTORE_PASSWORD,
+                PROPERTY_KEY_PASSWORD,
+                PROPERTY_TRUSTSTORE_PATH,
+                PROPERTY_TRUSTSTORE_TYPE,
+                PROPERTY_TRUSTSTORE_PASSWORD

Review Comment:
   Yeah, I was considering that but as registries are not part of a process group (not even the root) contrary to the controller services, this I believe would bring in some issues



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

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

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


[GitHub] [nifi] simonbence commented on pull request #6433: NIFI-10497 Making RegistryClient an extension point

Posted by GitBox <gi...@apache.org>.
simonbence commented on PR #6433:
URL: https://github.com/apache/nifi/pull/6433#issuecomment-1253403132

   > 
   
   Thank you very much, and thank you for the quick feedback! As of the GitHub tasks: I am aware of that, an update will be pushed soonish which should fix those. I wanted to share the production changes in general as soon as they became mature enought in order to give some visibility on this. Hence the draft flag.
   
   Yes, this is defenitely something, retrospectively I think could be split into multiple chunks, but most of the changes came as necessity due to other changes, rippling through the code. They could be separated after the fact, but with it would be a paramount effort and I am afraid, the reason for some of the changes would be less clear that way


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

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

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


[GitHub] [nifi] simonbence commented on a diff in pull request #6433: NIFI-10497 Making RegistryClient an extension point

Posted by GitBox <gi...@apache.org>.
simonbence commented on code in PR #6433:
URL: https://github.com/apache/nifi/pull/6433#discussion_r976222555


##########
nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/AffectedComponentSet.java:
##########
@@ -80,6 +81,7 @@ public class AffectedComponentSet {
     private final Set<ControllerServiceNode> controllerServices = new HashSet<>();
     private final Set<ReportingTaskNode> reportingTasks = new HashSet<>();
     private final Set<ParameterProviderNode> parameterProviders = new HashSet<>();
+    private final Set<FlowRegistryClientNode> flowRegistryClients = new HashSet<>();

Review Comment:
   `VersionedFlowSynchronizer#inheritRegistries` uses this information, when syncing flows. Do we consider it unnecessary?



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

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

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


[GitHub] [nifi] markap14 commented on a diff in pull request #6433: NIFI-10497 Making RegistryClient an extension point

Posted by GitBox <gi...@apache.org>.
markap14 commented on code in PR #6433:
URL: https://github.com/apache/nifi/pull/6433#discussion_r975558630


##########
nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/NiFiRegistryFlowMapper.java:
##########
@@ -185,6 +186,11 @@ public InstantiatedVersionedProcessGroup mapProcessGroup(final ProcessGroup grou
         });
     }
 
+    // This is specific for the {@code NifiRegistryFlowRegistryClient}
+    private String getRegistryUrl(final FlowRegistryClientNode registry) {
+        return registry.getComponentType().equals("org.apache.nifi.registry.flow.NifiRegistryFlowRegistryClient") ? registry.getRawPropertyValue(registry.getPropertyDescriptor("URL")) : "";

Review Comment:
   Might be worth leaving a comment in the code here that we do this purely for backward compatibility purposes. Otherwise, we'll be scratching our heads at this later :)



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

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

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


[GitHub] [nifi] markap14 commented on a diff in pull request #6433: NIFI-10497 Making RegistryClient an extension point

Posted by GitBox <gi...@apache.org>.
markap14 commented on code in PR #6433:
URL: https://github.com/apache/nifi/pull/6433#discussion_r975551031


##########
nifi-nar-bundles/nifi-flow-registry-client-bundle/nifi-flow-registry-client-services/src/main/java/org/apache/nifi/registry/flow/NifiRegistryFlowRegistryClient.java:
##########
@@ -0,0 +1,376 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.registry.flow;
+
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.registry.bucket.Bucket;
+import org.apache.nifi.registry.client.BucketClient;
+import org.apache.nifi.registry.client.FlowClient;
+import org.apache.nifi.registry.client.FlowSnapshotClient;
+import org.apache.nifi.registry.client.NiFiRegistryClient;
+import org.apache.nifi.registry.client.NiFiRegistryClientConfig;
+import org.apache.nifi.registry.client.NiFiRegistryException;
+import org.apache.nifi.registry.client.impl.JerseyNiFiRegistryClient;
+import org.apache.nifi.registry.client.impl.request.ProxiedEntityRequestConfig;
+import org.apache.nifi.security.util.KeystoreType;
+import org.apache.nifi.security.util.SslContextFactory;
+import org.apache.nifi.security.util.StandardTlsConfiguration;
+import org.apache.nifi.security.util.TlsConfiguration;
+import org.apache.nifi.security.util.TlsException;
+
+import javax.net.ssl.SSLContext;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public class NifiRegistryFlowRegistryClient extends AbstractFlowRegistryClient {
+
+    public final static PropertyDescriptor PROPERTY_URL = new PropertyDescriptor.Builder()
+            .name("url")
+            .displayName("URL")
+            .description("URL of the NiFi Registry")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .required(true)
+            .build();
+    public final static PropertyDescriptor PROPERTY_KEYSTORE_PATH = new PropertyDescriptor.Builder()
+            .name("keystorePath")
+            .displayName("Keystore Path")
+            .description("The fully-qualified filename of the Keystore")
+            .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
+            .required(false)
+            .build();
+    public final static PropertyDescriptor PROPERTY_KEYSTORE_PASSWORD = new PropertyDescriptor.Builder()
+            .name("keystorePassword")
+            .displayName("Keystore Password")
+            .description("The password for the Keystore")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .sensitive(true)
+            .required(false)
+            .build();
+    public final static PropertyDescriptor PROPERTY_KEY_PASSWORD = new PropertyDescriptor.Builder()
+            .name("keyPassword")
+            .displayName("Key Password")
+            .description("The password for the key. If this is not specified, but the Keystore Filename, Password, and Type are specified, "
+                    + "then the Keystore Password will be assumed to be the same as the Key Password.")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .sensitive(true)
+            .required(false)
+            .build();
+    public final static PropertyDescriptor PROPERTY_KEYSTORE_TYPE = new PropertyDescriptor.Builder()
+            .name("keystoreType")
+            .displayName("Keystore Type")
+            .description("The Type of the Keystore")
+            .allowableValues(KeystoreType.values())
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .required(false)
+            .build();
+    public final static PropertyDescriptor PROPERTY_TRUSTSTORE_PATH = new PropertyDescriptor.Builder()
+            .name("truststorePath")
+            .displayName("Truststore Path")
+            .description("The fully-qualified filename of the Truststore")
+            .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
+            .required(false)
+            .build();
+    public final static PropertyDescriptor PROPERTY_TRUSTSTORE_PASSWORD = new PropertyDescriptor.Builder()
+            .name("truststorePassword")
+            .displayName("Truststore Password")
+            .description("The password for the Truststore")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .sensitive(true)
+            .required(false)
+            .build();
+    public final static PropertyDescriptor PROPERTY_TRUSTSTORE_TYPE = new PropertyDescriptor.Builder()
+            .name("truststoreType")
+            .displayName("Truststore Type")
+            .description("The Type of the Truststore")
+            .allowableValues(KeystoreType.values())
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .required(false)
+            .build();
+
+    private volatile String registryUrl;
+    private volatile NiFiRegistryClient registryClient;
+
+    private synchronized NiFiRegistryClient getRegistryClient(final FlowRegistryClientConfigurationContext context) {
+        final String configuredUrl = context.getProperty(PROPERTY_URL).evaluateAttributeExpressions().getValue();
+
+        final URI uri;
+
+        try {
+            // Handles case where the URI entered has a trailing slash, or includes the trailing /nifi-registry-api
+            uri = new URIBuilder(configuredUrl).setPath("").removeQuery().build();
+        } catch (URISyntaxException e) {
+            throw new IllegalArgumentException("The given Registry URL is not valid: " + configuredUrl);
+        }
+
+        final String uriScheme = uri.getScheme();
+        if (uriScheme == null) {
+            throw new IllegalArgumentException("The given Registry URL is not valid: " + configuredUrl);
+        }
+
+        final String proposedUrl = uri.toString();;
+
+        if (!proposedUrl.equals(registryUrl)) {
+            registryUrl = proposedUrl;
+            invalidateClient();
+        }
+
+        if (registryClient != null) {
+            return registryClient;
+        }
+
+        final NiFiRegistryClientConfig config = new NiFiRegistryClientConfig.Builder()
+                .connectTimeout(30000)
+                .readTimeout(30000)
+                .sslContext(extractSSLContext(context))
+                .baseUrl(registryUrl)
+                .build();
+        registryClient = new JerseyNiFiRegistryClient.Builder()
+                .config(config)
+                .build();
+
+        return registryClient;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return Arrays.asList(
+                PROPERTY_URL,
+                PROPERTY_KEYSTORE_PATH,
+                PROPERTY_KEYSTORE_TYPE,
+                PROPERTY_KEYSTORE_PASSWORD,
+                PROPERTY_KEY_PASSWORD,
+                PROPERTY_TRUSTSTORE_PATH,
+                PROPERTY_TRUSTSTORE_TYPE,
+                PROPERTY_TRUSTSTORE_PASSWORD

Review Comment:
   We shouldn't need keystore/truststore properties. Instead we should just take an SSLContext Service.



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

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

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


[GitHub] [nifi] markap14 commented on pull request #6433: NIFI-10497 Making RegistryClient an extension point

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

   So doing some testing I think the current approach will not work for embedded flows. In this PR, you make the VersionedFlowCoordinates now deprecate the Registry URL. This makes sense, since the URL is not a first-class citizen of the extension point. But it now uses the Registry ID in order to identify a flow. So if there is an inner Process Group that is versioned, and an outer Process Group that is versioned, the inner group is identified by this Registry ID.
   
   The problem, though, is that the Registry ID is specific to that nifi instance/cluster.
   If I then go to a different nifi cluster and create a Registry Client that points to the same registry, the Registry ID will not match, so that will create an issue.
   
   My recommendation would be: Instead of VersionedFlowCoordinates having a Registry ID, it would contain a 'Storage Location'. And that Storage Location would be a String. For the existing implementation that would be just the URL. And then, we'd update the FlowRegistryClient to have a new method: `boolean IsStorageLocationApplicable(String location)` or something of that nature. In this way, we can loop over all registry clients and find the one that allows for the storage location and use it to fetch the flow. In the case that there are multiple clients that indicate that the location is applicable, we can go through all clients. If it's able to find the flow, great. If not, go to the next one.
   
   Does that make sense & sound reasonable?


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

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

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


[GitHub] [nifi] markap14 commented on a diff in pull request #6433: NIFI-10497 Making RegistryClient an extension point

Posted by GitBox <gi...@apache.org>.
markap14 commented on code in PR #6433:
URL: https://github.com/apache/nifi/pull/6433#discussion_r975583020


##########
nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/RunningComponentSetFilter.java:
##########
@@ -39,6 +41,7 @@ public class RunningComponentSetFilter implements ComponentSetFilter {
     private final Map<String, VersionedControllerService> controllerServices = new HashMap<>();
     private final Map<String, VersionedProcessor> processors = new HashMap<>();
     private final Map<String, VersionedReportingTask> reportingTasks = new HashMap<>();
+    private final Map<String, VersionedFlowRegistryClient> flowRegistryClients = new HashMap<>();

Review Comment:
   I don't see any reason to keep track of the Flow Registry Clients here either. They are never used. Not sure that they need to be added to ComponentSetFilter at all, but I might be wrong. Don't see how they'd get used in that context yet.



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

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

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


[GitHub] [nifi] markap14 commented on a diff in pull request #6433: NIFI-10497 Making RegistryClient an extension point

Posted by GitBox <gi...@apache.org>.
markap14 commented on code in PR #6433:
URL: https://github.com/apache/nifi/pull/6433#discussion_r976694535


##########
nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/AffectedComponentSet.java:
##########
@@ -80,6 +81,7 @@ public class AffectedComponentSet {
     private final Set<ControllerServiceNode> controllerServices = new HashSet<>();
     private final Set<ReportingTaskNode> reportingTasks = new HashSet<>();
     private final Set<ParameterProviderNode> parameterProviders = new HashSet<>();
+    private final Set<FlowRegistryClientNode> flowRegistryClients = new HashSet<>();

Review Comment:
   Ah OK I missed that.



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

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

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


[GitHub] [nifi] markap14 commented on pull request #6433: NIFI-10497 Making RegistryClient an extension point

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

   Thanks for putting up the Draft PR @simonbence
   This is definitely a HUGE PR. I think a lot of it stems from the fact that there were some class name changes (VersionedFlowSnapshot -> RegisteredFlowSnapshot, for example). This clarifies things because it avoids naming conflicts. Perhaps the rename itself that should have been pushed up as a separate PR, but not a big deal.
   Putting up the Draft PR, though, is really helpful, I think, because it will allow us to move much more quickly when all is finished up because we've had a good chance to review the approach
   
   Overall, I think you're very much on the right track, too! I did make a few comments inline. Overall, though, I think this is definitely heading in the right direction.
   
   All of the GitHub actions failed and there are a few TODO's. So I know it's still a work in progress, but this is really good. 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.

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

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


[GitHub] [nifi] simonbence commented on a diff in pull request #6433: NIFI-10497 Making RegistryClient an extension point

Posted by GitBox <gi...@apache.org>.
simonbence commented on code in PR #6433:
URL: https://github.com/apache/nifi/pull/6433#discussion_r976217866


##########
nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/registry/StandardFlowRegistryClientNode.java:
##########
@@ -0,0 +1,353 @@
+/*
+ * 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.controller.registry;
+
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.documentation.DeprecationNotice;
+import org.apache.nifi.authorization.Resource;
+import org.apache.nifi.authorization.resource.Authorizable;
+import org.apache.nifi.authorization.resource.ResourceFactory;
+import org.apache.nifi.authorization.resource.ResourceType;
+import org.apache.nifi.bundle.BundleCoordinate;
+import org.apache.nifi.components.ConfigurableComponent;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.validation.ValidationStatus;
+import org.apache.nifi.components.validation.ValidationTrigger;
+import org.apache.nifi.controller.AbstractComponentNode;
+import org.apache.nifi.controller.FlowController;
+import org.apache.nifi.controller.LoggableComponent;
+import org.apache.nifi.controller.ReloadComponent;
+import org.apache.nifi.controller.TerminationAwareLogger;
+import org.apache.nifi.controller.ValidationContextFactory;
+import org.apache.nifi.controller.flow.FlowManager;
+import org.apache.nifi.controller.service.ControllerServiceProvider;
+import org.apache.nifi.flow.ExternalControllerServiceReference;
+import org.apache.nifi.flow.ParameterProviderReference;
+import org.apache.nifi.flow.VersionedFlowCoordinates;
+import org.apache.nifi.flow.VersionedParameterContext;
+import org.apache.nifi.flow.VersionedProcessGroup;
+import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.nar.NarCloseable;
+import org.apache.nifi.parameter.ParameterContext;
+import org.apache.nifi.parameter.ParameterLookup;
+import org.apache.nifi.registry.ComponentVariableRegistry;
+import org.apache.nifi.registry.flow.FlowRegistryBucket;
+import org.apache.nifi.registry.flow.FlowRegistryClient;
+import org.apache.nifi.registry.flow.FlowRegistryClientConfigurationContext;
+import org.apache.nifi.registry.flow.FlowRegistryClientNode;
+import org.apache.nifi.registry.flow.FlowRegistryClientUserContext;
+import org.apache.nifi.registry.flow.FlowRegistryException;
+import org.apache.nifi.registry.flow.FlowRegistryInvalidException;
+import org.apache.nifi.registry.flow.FlowRegistryUtil;
+import org.apache.nifi.registry.flow.RegisteredFlow;
+import org.apache.nifi.registry.flow.RegisteredFlowSnapshot;
+import org.apache.nifi.registry.flow.RegisteredFlowSnapshotMetadata;
+import org.apache.nifi.registry.flow.SimpleRegisteredFlowSnapshot;
+import org.apache.nifi.registry.flow.SimpleRegisteredFlowSnapshotMetadata;
+import org.apache.nifi.util.CharacterFilterUtils;
+import org.apache.nifi.util.file.classloader.ClassLoaderUtils;
+
+import java.io.IOException;
+import java.net.URL;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public final class StandardFlowRegistryClientNode extends AbstractComponentNode implements FlowRegistryClientNode {
+
+    private final FlowController flowController;
+    private final LoggableComponent<FlowRegistryClient> client;
+    private volatile String description;
+
+    public StandardFlowRegistryClientNode(
+            final FlowController flowController,
+            final LoggableComponent<FlowRegistryClient> client,
+            final String id,
+            final ValidationContextFactory validationContextFactory,
+            final ControllerServiceProvider serviceProvider,
+            final String componentType,
+            final String componentCanonicalClass,
+            final ComponentVariableRegistry variableRegistry,
+            final ReloadComponent reloadComponent,
+            final ExtensionManager extensionManager,
+            final ValidationTrigger validationTrigger,
+            final boolean isExtensionMissing) {
+        super(id, validationContextFactory, serviceProvider, componentType, componentCanonicalClass, variableRegistry, reloadComponent, extensionManager, validationTrigger, isExtensionMissing);
+        this.flowController = flowController;
+        this.client = client;
+    }
+
+    @Override
+    public Authorizable getParentAuthorizable() {
+        return flowController;
+    }
+
+    @Override
+    public Resource getResource() {
+        return ResourceFactory.getComponentResource(ResourceType.Controller, getIdentifier(), getName());
+    }
+
+    @Override
+    public String getProcessGroupIdentifier() {
+        return null;
+    }
+
+    @Override
+    protected List<ValidationResult> validateConfig() {
+        return Collections.emptyList();
+    }
+
+    @Override
+    public void verifyModifiable() throws IllegalStateException {
+        // FlowRegistryClients has no running state
+    }
+
+    @Override
+    public void setProperties(final Map<String, String> properties, final boolean allowRemovalOfRequiredProperties, final Set<String> updatedSensitiveDynamicPropertyNames) {
+        super.setProperties(properties, allowRemovalOfRequiredProperties, updatedSensitiveDynamicPropertyNames);
+    }

Review Comment:
   I think there was some -now obsolete- logic there. I will clean it up



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

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

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


[GitHub] [nifi] markap14 commented on a diff in pull request #6433: NIFI-10497 Making RegistryClient an extension point

Posted by GitBox <gi...@apache.org>.
markap14 commented on code in PR #6433:
URL: https://github.com/apache/nifi/pull/6433#discussion_r975581442


##########
nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/AffectedComponentSet.java:
##########
@@ -80,6 +81,7 @@ public class AffectedComponentSet {
     private final Set<ControllerServiceNode> controllerServices = new HashSet<>();
     private final Set<ReportingTaskNode> reportingTasks = new HashSet<>();
     private final Set<ParameterProviderNode> parameterProviders = new HashSet<>();
+    private final Set<FlowRegistryClientNode> flowRegistryClients = new HashSet<>();

Review Comment:
   The code here keeps track if Flow Registry Clients if ever they are added (but I don't think they ever should be). But we never do anything with them if they are added. I don't think we need to modify this class at all.



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

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

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


[GitHub] [nifi] markap14 commented on pull request #6433: NIFI-10497 Making RegistryClient an extension point

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

   Thanks for all the work you've done here @simonbence! This is a huge changeset and really great work put together here. I did quite a lot of testing:
   - Basics of adding a registry, starting version control, stopping version control, changing version, view local modifications, revert changes.
   - Built flow that use registry using 1.16, then moved that flow into a nifi built on this branch, in order to simulate an upgrade form an older version of NiFi
   - Built flow that uses embedded versioned flows
   - Built flow that uses embedded versioned flow in 1.16 and copied over to instance from this branch
   - Tried to remove a registry client that was in use, which correctly failed
   - Verified dirty / up-to-date / conflict / new version available states
   - Tested standalone and clustered
   - Tested flows with embedded PG's that are not themselves versioned
   
   All seems to check out with the current build.
   +1 will merge 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.

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

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


[GitHub] [nifi] markap14 commented on a diff in pull request #6433: NIFI-10497 Making RegistryClient an extension point

Posted by GitBox <gi...@apache.org>.
markap14 commented on code in PR #6433:
URL: https://github.com/apache/nifi/pull/6433#discussion_r975578025


##########
nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/registry/StandardFlowRegistryClientNode.java:
##########
@@ -0,0 +1,353 @@
+/*
+ * 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.controller.registry;
+
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.documentation.DeprecationNotice;
+import org.apache.nifi.authorization.Resource;
+import org.apache.nifi.authorization.resource.Authorizable;
+import org.apache.nifi.authorization.resource.ResourceFactory;
+import org.apache.nifi.authorization.resource.ResourceType;
+import org.apache.nifi.bundle.BundleCoordinate;
+import org.apache.nifi.components.ConfigurableComponent;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.validation.ValidationStatus;
+import org.apache.nifi.components.validation.ValidationTrigger;
+import org.apache.nifi.controller.AbstractComponentNode;
+import org.apache.nifi.controller.FlowController;
+import org.apache.nifi.controller.LoggableComponent;
+import org.apache.nifi.controller.ReloadComponent;
+import org.apache.nifi.controller.TerminationAwareLogger;
+import org.apache.nifi.controller.ValidationContextFactory;
+import org.apache.nifi.controller.flow.FlowManager;
+import org.apache.nifi.controller.service.ControllerServiceProvider;
+import org.apache.nifi.flow.ExternalControllerServiceReference;
+import org.apache.nifi.flow.ParameterProviderReference;
+import org.apache.nifi.flow.VersionedFlowCoordinates;
+import org.apache.nifi.flow.VersionedParameterContext;
+import org.apache.nifi.flow.VersionedProcessGroup;
+import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.nar.NarCloseable;
+import org.apache.nifi.parameter.ParameterContext;
+import org.apache.nifi.parameter.ParameterLookup;
+import org.apache.nifi.registry.ComponentVariableRegistry;
+import org.apache.nifi.registry.flow.FlowRegistryBucket;
+import org.apache.nifi.registry.flow.FlowRegistryClient;
+import org.apache.nifi.registry.flow.FlowRegistryClientConfigurationContext;
+import org.apache.nifi.registry.flow.FlowRegistryClientNode;
+import org.apache.nifi.registry.flow.FlowRegistryClientUserContext;
+import org.apache.nifi.registry.flow.FlowRegistryException;
+import org.apache.nifi.registry.flow.FlowRegistryInvalidException;
+import org.apache.nifi.registry.flow.FlowRegistryUtil;
+import org.apache.nifi.registry.flow.RegisteredFlow;
+import org.apache.nifi.registry.flow.RegisteredFlowSnapshot;
+import org.apache.nifi.registry.flow.RegisteredFlowSnapshotMetadata;
+import org.apache.nifi.registry.flow.SimpleRegisteredFlowSnapshot;
+import org.apache.nifi.registry.flow.SimpleRegisteredFlowSnapshotMetadata;
+import org.apache.nifi.util.CharacterFilterUtils;
+import org.apache.nifi.util.file.classloader.ClassLoaderUtils;
+
+import java.io.IOException;
+import java.net.URL;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public final class StandardFlowRegistryClientNode extends AbstractComponentNode implements FlowRegistryClientNode {
+
+    private final FlowController flowController;
+    private final LoggableComponent<FlowRegistryClient> client;
+    private volatile String description;
+
+    public StandardFlowRegistryClientNode(
+            final FlowController flowController,
+            final LoggableComponent<FlowRegistryClient> client,
+            final String id,
+            final ValidationContextFactory validationContextFactory,
+            final ControllerServiceProvider serviceProvider,
+            final String componentType,
+            final String componentCanonicalClass,
+            final ComponentVariableRegistry variableRegistry,
+            final ReloadComponent reloadComponent,
+            final ExtensionManager extensionManager,
+            final ValidationTrigger validationTrigger,
+            final boolean isExtensionMissing) {
+        super(id, validationContextFactory, serviceProvider, componentType, componentCanonicalClass, variableRegistry, reloadComponent, extensionManager, validationTrigger, isExtensionMissing);
+        this.flowController = flowController;
+        this.client = client;
+    }
+
+    @Override
+    public Authorizable getParentAuthorizable() {
+        return flowController;
+    }
+
+    @Override
+    public Resource getResource() {
+        return ResourceFactory.getComponentResource(ResourceType.Controller, getIdentifier(), getName());
+    }
+
+    @Override
+    public String getProcessGroupIdentifier() {
+        return null;
+    }
+
+    @Override
+    protected List<ValidationResult> validateConfig() {
+        return Collections.emptyList();
+    }
+
+    @Override
+    public void verifyModifiable() throws IllegalStateException {
+        // FlowRegistryClients has no running state
+    }
+
+    @Override
+    public void setProperties(final Map<String, String> properties, final boolean allowRemovalOfRequiredProperties, final Set<String> updatedSensitiveDynamicPropertyNames) {
+        super.setProperties(properties, allowRemovalOfRequiredProperties, updatedSensitiveDynamicPropertyNames);
+    }

Review Comment:
   Any reason we're overriding this method just to call super?



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

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

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


[GitHub] [nifi] simonbence commented on a diff in pull request #6433: NIFI-10497 Making RegistryClient an extension point

Posted by GitBox <gi...@apache.org>.
simonbence commented on code in PR #6433:
URL: https://github.com/apache/nifi/pull/6433#discussion_r976209016


##########
nifi-nar-bundles/nifi-flow-registry-client-bundle/nifi-flow-registry-client-services/src/main/java/org/apache/nifi/registry/flow/NifiRegistryFlowRegistryClient.java:
##########
@@ -0,0 +1,376 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.registry.flow;
+
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.registry.bucket.Bucket;
+import org.apache.nifi.registry.client.BucketClient;
+import org.apache.nifi.registry.client.FlowClient;
+import org.apache.nifi.registry.client.FlowSnapshotClient;
+import org.apache.nifi.registry.client.NiFiRegistryClient;
+import org.apache.nifi.registry.client.NiFiRegistryClientConfig;
+import org.apache.nifi.registry.client.NiFiRegistryException;
+import org.apache.nifi.registry.client.impl.JerseyNiFiRegistryClient;
+import org.apache.nifi.registry.client.impl.request.ProxiedEntityRequestConfig;
+import org.apache.nifi.security.util.KeystoreType;
+import org.apache.nifi.security.util.SslContextFactory;
+import org.apache.nifi.security.util.StandardTlsConfiguration;
+import org.apache.nifi.security.util.TlsConfiguration;
+import org.apache.nifi.security.util.TlsException;
+
+import javax.net.ssl.SSLContext;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public class NifiRegistryFlowRegistryClient extends AbstractFlowRegistryClient {
+
+    public final static PropertyDescriptor PROPERTY_URL = new PropertyDescriptor.Builder()
+            .name("url")
+            .displayName("URL")
+            .description("URL of the NiFi Registry")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .required(true)
+            .build();
+    public final static PropertyDescriptor PROPERTY_KEYSTORE_PATH = new PropertyDescriptor.Builder()
+            .name("keystorePath")
+            .displayName("Keystore Path")
+            .description("The fully-qualified filename of the Keystore")
+            .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
+            .required(false)
+            .build();
+    public final static PropertyDescriptor PROPERTY_KEYSTORE_PASSWORD = new PropertyDescriptor.Builder()
+            .name("keystorePassword")
+            .displayName("Keystore Password")
+            .description("The password for the Keystore")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .sensitive(true)
+            .required(false)
+            .build();
+    public final static PropertyDescriptor PROPERTY_KEY_PASSWORD = new PropertyDescriptor.Builder()
+            .name("keyPassword")
+            .displayName("Key Password")
+            .description("The password for the key. If this is not specified, but the Keystore Filename, Password, and Type are specified, "
+                    + "then the Keystore Password will be assumed to be the same as the Key Password.")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .sensitive(true)
+            .required(false)
+            .build();
+    public final static PropertyDescriptor PROPERTY_KEYSTORE_TYPE = new PropertyDescriptor.Builder()
+            .name("keystoreType")
+            .displayName("Keystore Type")
+            .description("The Type of the Keystore")
+            .allowableValues(KeystoreType.values())
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .required(false)
+            .build();
+    public final static PropertyDescriptor PROPERTY_TRUSTSTORE_PATH = new PropertyDescriptor.Builder()
+            .name("truststorePath")
+            .displayName("Truststore Path")
+            .description("The fully-qualified filename of the Truststore")
+            .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
+            .required(false)
+            .build();
+    public final static PropertyDescriptor PROPERTY_TRUSTSTORE_PASSWORD = new PropertyDescriptor.Builder()
+            .name("truststorePassword")
+            .displayName("Truststore Password")
+            .description("The password for the Truststore")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .sensitive(true)
+            .required(false)
+            .build();
+    public final static PropertyDescriptor PROPERTY_TRUSTSTORE_TYPE = new PropertyDescriptor.Builder()
+            .name("truststoreType")
+            .displayName("Truststore Type")
+            .description("The Type of the Truststore")
+            .allowableValues(KeystoreType.values())
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .required(false)
+            .build();
+
+    private volatile String registryUrl;
+    private volatile NiFiRegistryClient registryClient;
+
+    private synchronized NiFiRegistryClient getRegistryClient(final FlowRegistryClientConfigurationContext context) {
+        final String configuredUrl = context.getProperty(PROPERTY_URL).evaluateAttributeExpressions().getValue();
+
+        final URI uri;
+
+        try {
+            // Handles case where the URI entered has a trailing slash, or includes the trailing /nifi-registry-api
+            uri = new URIBuilder(configuredUrl).setPath("").removeQuery().build();
+        } catch (URISyntaxException e) {
+            throw new IllegalArgumentException("The given Registry URL is not valid: " + configuredUrl);
+        }
+
+        final String uriScheme = uri.getScheme();
+        if (uriScheme == null) {
+            throw new IllegalArgumentException("The given Registry URL is not valid: " + configuredUrl);
+        }
+
+        final String proposedUrl = uri.toString();;
+
+        if (!proposedUrl.equals(registryUrl)) {
+            registryUrl = proposedUrl;
+            invalidateClient();
+        }
+
+        if (registryClient != null) {
+            return registryClient;
+        }
+
+        final NiFiRegistryClientConfig config = new NiFiRegistryClientConfig.Builder()
+                .connectTimeout(30000)
+                .readTimeout(30000)
+                .sslContext(extractSSLContext(context))
+                .baseUrl(registryUrl)
+                .build();
+        registryClient = new JerseyNiFiRegistryClient.Builder()
+                .config(config)
+                .build();
+
+        return registryClient;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return Arrays.asList(
+                PROPERTY_URL,
+                PROPERTY_KEYSTORE_PATH,
+                PROPERTY_KEYSTORE_TYPE,
+                PROPERTY_KEYSTORE_PASSWORD,
+                PROPERTY_KEY_PASSWORD,
+                PROPERTY_TRUSTSTORE_PATH,
+                PROPERTY_TRUSTSTORE_TYPE,
+                PROPERTY_TRUSTSTORE_PASSWORD

Review Comment:
   Yeah, I was considering that but as registries are not part of a process group (not even the root) contrary to the controller services, this I believe would bring in some issues



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

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

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


[GitHub] [nifi] markap14 merged pull request #6433: NIFI-10497 Making RegistryClient an extension point

Posted by GitBox <gi...@apache.org>.
markap14 merged PR #6433:
URL: https://github.com/apache/nifi/pull/6433


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

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

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