You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2020/04/22 21:12:45 UTC

[GitHub] [nifi] sjyang18 opened a new pull request #4226: NIFI-6149: Azure EventHub Managed identity support patch

sjyang18 opened a new pull request #4226:
URL: https://github.com/apache/nifi/pull/4226


   Thank you for submitting a contribution to Apache NiFi.
   
   Please provide a short description of the PR here:
   
   #### Description of PR
   
   Azure EventHub Managed identity support patch
   
   In order to streamline the review of the contribution we ask you
   to ensure the following steps have been taken:
   
   ### For all changes:
   - [X] Is there a JIRA ticket associated with this PR? Is it referenced 
        in the commit message?
   
   - [X] Does your PR title start with **NIFI-XXXX** where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
   
   - [X] Has your PR been rebased against the latest commit within the target branch (typically `master`)?
   
   - [X] Is your initial contribution a single, squashed commit? _Additional commits in response to PR reviewer feedback should be made on this branch and pushed to allow change tracking. Do not `squash` or use `--force` when pushing to allow for clean monitoring of changes._
   
   ### For code changes:
   - [X] Have you ensured that the full suite of tests is executed via `mvn -Pcontrib-check clean install` at the root `nifi` folder?
   - [X] Have you written or updated unit tests to verify your changes?
   - [X] Have you verified that the full build is successful on both JDK 8 and JDK 11?
   - [X] 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 travis-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] sjyang18 commented on a change in pull request #4226: NIFI-6149: Azure EventHub Managed identity support patch

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



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/utils/AzureEventHubUtils.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.processors.azure.eventhub.utils;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import com.microsoft.azure.eventhubs.ConnectionStringBuilder;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.util.StandardValidators;
+
+public final class AzureEventHubUtils {
+
+    public static final String MANANGED_IDENDITY_POLICY = ConnectionStringBuilder.MANAGED_IDENTITY_AUTHENTICATION;
+
+    public static final PropertyDescriptor POLICY_PRIMARY_KEY = new PropertyDescriptor.Builder()
+        .name("Shared Access Policy Primary Key")
+        .description("The primary key of the shared access policy")
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+        .sensitive(true)
+        .required(false)
+        .build();
+
+    public static final PropertyDescriptor USE_MANANGED_IDENTITY = new PropertyDescriptor.Builder()
+        .name("use-managed-identity")
+        .displayName("Use Azure Managed Identity")
+        .description("Choose whether or not to use the managed identity of azure vm/vmss. ")
+        .required(false).defaultValue("false").allowableValues("true", "false")
+        .addValidator(StandardValidators.BOOLEAN_VALIDATOR).build();
+
+    public static List<ValidationResult> customValidate(PropertyDescriptor accessPolicyDescriptor, ValidationContext context) {
+        List<ValidationResult> retVal = new ArrayList<>();
+
+        boolean accessPolicyIsSet  = context.getProperty(accessPolicyDescriptor).isSet();
+        boolean policyKeyIsSet     = context.getProperty(POLICY_PRIMARY_KEY).isSet();
+        boolean useManagedIdentity = context.getProperty(USE_MANANGED_IDENTITY).asBoolean();
+
+        if (useManagedIdentity && (accessPolicyIsSet || policyKeyIsSet) ) {
+            final String msg = String.format(
+                "%s and %s with %s fields cannot be set at the same time.",

Review comment:
       fixed.




----------------------------------------------------------------
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] turcsanyip commented on a change in pull request #4226: NIFI-6149: Azure EventHub Managed identity support patch

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



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/GetAzureEventHub.java
##########
@@ -103,16 +107,10 @@
             .description("The name of the shared access policy. This policy must have Listen claims.")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .expressionLanguageSupported(ExpressionLanguageScope.NONE)
-            .required(true)
-            .build();
-    static final PropertyDescriptor POLICY_PRIMARY_KEY = new PropertyDescriptor.Builder()
-            .name("Shared Access Policy Primary Key")
-            .description("The primary key of the shared access policy")
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
-            .sensitive(true)
-            .required(true)
+            .required(false)
             .build();
+    static final PropertyDescriptor POLICY_PRIMARY_KEY =  AzureEventHubUtils.POLICY_PRIMARY_KEY;
+    static final PropertyDescriptor USE_MANANGED_IDENTITY = AzureEventHubUtils.USE_MANAGED_IDENTITY;

Review comment:
       I did not notice the BogusCoonectionStringMockPutAzureEventHub test class. Your workaround is fine with me.




----------------------------------------------------------------
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] pvillard31 commented on pull request #4226: NIFI-6149: Azure EventHub Managed identity support patch

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


   > @pvillard31 Yes on ADLS side. I am looking into it and I will create another PR later.
   > As for BlobStorage, the existing processors are using legacy azure-storage SDK (com.microsoft.azure groupID), and the legacy SDK does not have the higher version that support managed identities. Thus, migration of existing BlobStorage processors with the new azure SDK (com.azure groupID) is the first step, if we want to add this feature to Blob storage processors. I expects there will be impact on the existing NIFI flow with existing blob storage, in this case. Do you have any migration guidance example that I reference?
   
   Thanks @sjyang18 - it makes sense. I don't have any strong feeling about doing this for BlobStorage processors. Having it for ADLS would be a great start IMHO. This was more out of curiosity as I'm not as familiar with Azure as I can be with another cloud provider.


----------------------------------------------------------------
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] sjyang18 commented on a change in pull request #4226: NIFI-6149: Azure EventHub Managed identity support patch

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



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/ConsumeAzureEventHub.java
##########
@@ -627,9 +616,26 @@ private void registerEventProcessor(final ProcessContext context) throws Excepti
 
         final String storageConnectionString = String.format(FORMAT_STORAGE_CONNECTION_STRING, storageAccountName, storageAccountKey);
 
-        final ConnectionStringBuilder eventHubConnectionString = new ConnectionStringBuilder().setNamespaceName(namespaceName).setEventHubName( eventHubName).setSasKeyName(sasName).setSasKey(sasKey);
-
-        eventProcessorHost = new EventProcessorHost(consumerHostname, eventHubName, consumerGroupName, eventHubConnectionString.toString(), storageConnectionString, containerName);
+        final String connectionString;
+        final boolean useManagedIdentity = context.getProperty(USE_MANANGED_IDENTITY).asBoolean();
+        if(useManagedIdentity) {
+            connectionString = AzureEventHubUtils.getManagedIdentityConnectionString(namespaceName, eventHubName);
+        } else {
+            final String sasName = context.getProperty(ACCESS_POLICY_NAME).evaluateAttributeExpressions().getValue();
+            validateRequiredProperty(ACCESS_POLICY_NAME, sasName);
+            final String sasKey = context.getProperty(POLICY_PRIMARY_KEY).evaluateAttributeExpressions().getValue();
+            validateRequiredProperty(POLICY_PRIMARY_KEY, sasKey);
+            connectionString = new ConnectionStringBuilder()

Review comment:
       Done.

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/utils/AzureEventHubUtils.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.processors.azure.eventhub.utils;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import com.microsoft.azure.eventhubs.ConnectionStringBuilder;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.util.StandardValidators;
+
+public final class AzureEventHubUtils {
+
+    public static final String MANANGED_IDENDITY_POLICY = ConnectionStringBuilder.MANAGED_IDENTITY_AUTHENTICATION;
+
+    public static final PropertyDescriptor POLICY_PRIMARY_KEY = new PropertyDescriptor.Builder()
+        .name("Shared Access Policy Primary Key")
+        .description("The primary key of the shared access policy")
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+        .sensitive(true)
+        .required(false)
+        .build();
+
+    public static final PropertyDescriptor USE_MANANGED_IDENTITY = new PropertyDescriptor.Builder()
+        .name("use-managed-identity")
+        .displayName("Use Azure Managed Identity")
+        .description("Choose whether or not to use the managed identity of azure vm/vmss. ")
+        .required(false).defaultValue("false").allowableValues("true", "false")
+        .addValidator(StandardValidators.BOOLEAN_VALIDATOR).build();
+
+    public static List<ValidationResult> customValidate(PropertyDescriptor accessPolicyDescriptor, ValidationContext context) {
+        List<ValidationResult> retVal = new ArrayList<>();
+
+        boolean accessPolicyIsSet  = context.getProperty(accessPolicyDescriptor).isSet();
+        boolean policyKeyIsSet     = context.getProperty(POLICY_PRIMARY_KEY).isSet();
+        boolean useManagedIdentity = context.getProperty(USE_MANANGED_IDENTITY).asBoolean();
+
+        if (useManagedIdentity && (accessPolicyIsSet || policyKeyIsSet) ) {
+            final String msg = String.format(
+                "%s and %s with %s fields cannot be set at the same time.",
+                USE_MANANGED_IDENTITY.getDisplayName(),
+                accessPolicyDescriptor.getDisplayName(),
+                POLICY_PRIMARY_KEY.getDisplayName()
+            );
+            retVal.add(new ValidationResult.Builder().valid(false).explanation(msg).build());
+        } else if (!useManagedIdentity && (!accessPolicyIsSet || !policyKeyIsSet)) {
+            final String msg = String.format(
+                "Either %s or %s with %s must be set",

Review comment:
       done

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/GetAzureEventHub.java
##########
@@ -301,11 +308,24 @@ public void onScheduled(final ProcessContext context) throws ProcessException, U
         }
         this.partitionNames = partitionNames;
 
-        final String policyName = context.getProperty(ACCESS_POLICY).getValue();
-        final String policyKey = context.getProperty(POLICY_PRIMARY_KEY).getValue();
         final String namespace = context.getProperty(NAMESPACE).getValue();
         final String eventHubName = context.getProperty(EVENT_HUB_NAME).getValue();
         final String serviceBusEndpoint = context.getProperty(SERVICE_BUS_ENDPOINT).getValue();
+        final boolean useManagedIdentity = context.getProperty(USE_MANANGED_IDENTITY).asBoolean();
+        final String policyName, policyKey, connectionString;
+        if(useManagedIdentity){
+            policyName =  MANANGED_IDENDITY_POLICY;
+            policyKey = null;

Review comment:
       done




----------------------------------------------------------------
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] sjyang18 commented on a change in pull request #4226: NIFI-6149: Azure EventHub Managed identity support patch

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



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/GetAzureEventHub.java
##########
@@ -103,16 +107,10 @@
             .description("The name of the shared access policy. This policy must have Listen claims.")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .expressionLanguageSupported(ExpressionLanguageScope.NONE)
-            .required(true)
-            .build();
-    static final PropertyDescriptor POLICY_PRIMARY_KEY = new PropertyDescriptor.Builder()
-            .name("Shared Access Policy Primary Key")
-            .description("The primary key of the shared access policy")
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
-            .sensitive(true)
-            .required(true)
+            .required(false)
             .build();
+    static final PropertyDescriptor POLICY_PRIMARY_KEY =  AzureEventHubUtils.POLICY_PRIMARY_KEY;
+    static final PropertyDescriptor USE_MANANGED_IDENTITY = AzureEventHubUtils.USE_MANAGED_IDENTITY;

Review comment:
       BogusCoonectionStringMockPutAzureEventHub class is defined in PutAzureEventHubTest:351 and its overriding method getConnectionString.




----------------------------------------------------------------
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] sjyang18 commented on a change in pull request #4226: NIFI-6149: Azure EventHub Managed identity support patch

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



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/ConsumeAzureEventHub.java
##########
@@ -115,17 +116,10 @@
             .description("The name of the shared access policy. This policy must have Listen claims.")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
-            .required(true)
-            .build();
-    static final PropertyDescriptor POLICY_PRIMARY_KEY = new PropertyDescriptor.Builder()
-            .name("event-hub-shared-access-policy-primary-key")

Review comment:
       modified as u suggested.




----------------------------------------------------------------
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] sjyang18 commented on a change in pull request #4226: NIFI-6149: Azure EventHub Managed identity support patch

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



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/utils/AzureEventHubUtils.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.processors.azure.eventhub.utils;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import com.microsoft.azure.eventhubs.ConnectionStringBuilder;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.util.StandardValidators;
+
+public final class AzureEventHubUtils {
+
+    public static final String MANANGED_IDENDITY_POLICY = ConnectionStringBuilder.MANAGED_IDENTITY_AUTHENTICATION;
+
+    public static final PropertyDescriptor POLICY_PRIMARY_KEY = new PropertyDescriptor.Builder()
+        .name("Shared Access Policy Primary Key")
+        .description("The primary key of the shared access policy")
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+        .sensitive(true)
+        .required(false)
+        .build();
+
+    public static final PropertyDescriptor USE_MANANGED_IDENTITY = new PropertyDescriptor.Builder()
+        .name("use-managed-identity")
+        .displayName("Use Azure Managed Identity")
+        .description("Choose whether or not to use the managed identity of azure vm/vmss. ")

Review comment:
       fixed.




----------------------------------------------------------------
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] turcsanyip commented on a change in pull request #4226: NIFI-6149: Azure EventHub Managed identity support patch

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



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/GetAzureEventHub.java
##########
@@ -103,16 +107,10 @@
             .description("The name of the shared access policy. This policy must have Listen claims.")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .expressionLanguageSupported(ExpressionLanguageScope.NONE)
-            .required(true)
-            .build();
-    static final PropertyDescriptor POLICY_PRIMARY_KEY = new PropertyDescriptor.Builder()
-            .name("Shared Access Policy Primary Key")
-            .description("The primary key of the shared access policy")
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
-            .sensitive(true)
-            .required(true)
+            .required(false)
             .build();
+    static final PropertyDescriptor POLICY_PRIMARY_KEY =  AzureEventHubUtils.POLICY_PRIMARY_KEY;
+    static final PropertyDescriptor USE_MANANGED_IDENTITY = AzureEventHubUtils.USE_MANAGED_IDENTITY;

Review comment:
       @sjyang18 I cannot see this fix in your last commit. It is MANANGED in all the 3 classes.
   
   Please also remove the unused getConnectionString() from PutAzureEventHub (with the import).




----------------------------------------------------------------
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] sjyang18 commented on a change in pull request #4226: NIFI-6149: Azure EventHub Managed identity support patch

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



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/ConsumeAzureEventHub.java
##########
@@ -76,6 +76,7 @@
 import java.util.concurrent.TimeUnit;

Review comment:
       removed.




----------------------------------------------------------------
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] sjyang18 commented on pull request #4226: NIFI-6149: Azure EventHub Managed identity support patch

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


   @pvillard31  Yes on ADLS side. I am looking into it and I will create another PR later. 
   As for BlobStorage, the existing processors are using legacy azure-storage SDK (com.microsoft.azure groupID), and the legacy SDK does not have the higher version that support managed identities. Thus, migration of existing BlobStorage processors with the new azure SDK (com.azure groupID) is the first step, if we want to add this feature to Blob storage processors.  I expects there will be impact on the existing NIFI flow with existing blob storage, in this case. Do you have any migration guidance example that I reference?


----------------------------------------------------------------
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] turcsanyip commented on a change in pull request #4226: NIFI-6149: Azure EventHub Managed identity support patch

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



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/utils/AzureEventHubUtils.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.processors.azure.eventhub.utils;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import com.microsoft.azure.eventhubs.ConnectionStringBuilder;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.util.StandardValidators;
+
+public final class AzureEventHubUtils {
+
+    public static final String MANAGED_IDENDITY_POLICY = ConnectionStringBuilder.MANAGED_IDENTITY_AUTHENTICATION;

Review comment:
       I did not catch this typo earlier: IDENTITY

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/utils/AzureEventHubUtils.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.processors.azure.eventhub.utils;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import com.microsoft.azure.eventhubs.ConnectionStringBuilder;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.util.StandardValidators;
+
+public final class AzureEventHubUtils {
+
+    public static final String MANAGED_IDENDITY_POLICY = ConnectionStringBuilder.MANAGED_IDENTITY_AUTHENTICATION;
+
+    public static final PropertyDescriptor POLICY_PRIMARY_KEY = new PropertyDescriptor.Builder()
+        .name("Shared Access Policy Primary Key")
+        .description("The primary key of the shared access policy")
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+        .sensitive(true)
+        .required(false)
+        .build();
+
+    public static final PropertyDescriptor USE_MANAGED_IDENTITY = new PropertyDescriptor.Builder()
+        .name("use-managed-identity")
+        .displayName("Use Azure Managed Identity")
+        .description("Choose whether or not to use the managed identity of Azure VM/VMSS")
+        .required(false).defaultValue("false").allowableValues("true", "false")
+        .addValidator(StandardValidators.BOOLEAN_VALIDATOR).build();
+
+    public static List<ValidationResult> customValidate(PropertyDescriptor accessPolicyDescriptor,
+        PropertyDescriptor policyKeyDescriptor,
+        ValidationContext context) {
+        List<ValidationResult> retVal = new ArrayList<>();
+
+        boolean accessPolicyIsSet  = context.getProperty(accessPolicyDescriptor).isSet();
+        boolean policyKeyIsSet     = context.getProperty(policyKeyDescriptor).isSet();
+        boolean useManagedIdentity = context.getProperty(USE_MANAGED_IDENTITY).asBoolean();
+
+        if (useManagedIdentity && (accessPolicyIsSet || policyKeyIsSet) ) {
+            final String msg = String.format(
+                "('%s') and ('%s' with '%s') fields cannot be set at the same time.",
+                USE_MANAGED_IDENTITY.getDisplayName(),
+                accessPolicyDescriptor.getDisplayName(),
+                POLICY_PRIMARY_KEY.getDisplayName()
+            );
+            retVal.add(new ValidationResult.Builder().valid(false).explanation(msg).build());
+        } else if (!useManagedIdentity && (!accessPolicyIsSet || !policyKeyIsSet)) {
+            final String msg = String.format(
+                "either('%s') or (%s with '%s') must be set",
+                USE_MANAGED_IDENTITY.getDisplayName(),
+                accessPolicyDescriptor.getDisplayName(),
+                POLICY_PRIMARY_KEY.getDisplayName()
+            );
+            retVal.add(new ValidationResult.Builder().valid(false).explanation(msg).build());
+        }
+        return retVal;
+    }
+
+    public static String getManagedIdentityConnectionString(final String namespace, final String eventHubName){
+        return new ConnectionStringBuilder().setNamespaceName(namespace).setEventHubName(eventHubName)
+                    .setAuthentication(MANAGED_IDENDITY_POLICY).toString();
+    }
+    public static String getSharedAccessSignatureConnectionString(final String namespace, final String eventHubName, final String sasName, final String sasKey) {

Review comment:
       This could also be used from `PutAzureEventHub` now (be careful with the unused import there).

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/ConsumeAzureEventHub.java
##########
@@ -76,6 +76,7 @@
 import java.util.concurrent.TimeUnit;

Review comment:
       com.microsoft.azure.eventhubs.ConnectionStringBuilder is an unused import above.

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/GetAzureEventHub.java
##########
@@ -103,16 +107,10 @@
             .description("The name of the shared access policy. This policy must have Listen claims.")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .expressionLanguageSupported(ExpressionLanguageScope.NONE)
-            .required(true)
-            .build();
-    static final PropertyDescriptor POLICY_PRIMARY_KEY = new PropertyDescriptor.Builder()
-            .name("Shared Access Policy Primary Key")
-            .description("The primary key of the shared access policy")
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
-            .sensitive(true)
-            .required(true)
+            .required(false)
             .build();
+    static final PropertyDescriptor POLICY_PRIMARY_KEY =  AzureEventHubUtils.POLICY_PRIMARY_KEY;
+    static final PropertyDescriptor USE_MANANGED_IDENTITY = AzureEventHubUtils.USE_MANAGED_IDENTITY;

Review comment:
       I did not catch this typo earlier: MANANGED here too, and also in the other 2 processor classes.

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/utils/AzureEventHubUtils.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.processors.azure.eventhub.utils;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import com.microsoft.azure.eventhubs.ConnectionStringBuilder;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.util.StandardValidators;
+
+public final class AzureEventHubUtils {
+
+    public static final String MANANGED_IDENDITY_POLICY = ConnectionStringBuilder.MANAGED_IDENTITY_AUTHENTICATION;
+
+    public static final PropertyDescriptor POLICY_PRIMARY_KEY = new PropertyDescriptor.Builder()
+        .name("Shared Access Policy Primary Key")
+        .description("The primary key of the shared access policy")
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+        .sensitive(true)
+        .required(false)
+        .build();
+
+    public static final PropertyDescriptor USE_MANANGED_IDENTITY = new PropertyDescriptor.Builder()
+        .name("use-managed-identity")
+        .displayName("Use Azure Managed Identity")
+        .description("Choose whether or not to use the managed identity of azure vm/vmss. ")
+        .required(false).defaultValue("false").allowableValues("true", "false")
+        .addValidator(StandardValidators.BOOLEAN_VALIDATOR).build();
+
+    public static List<ValidationResult> customValidate(PropertyDescriptor accessPolicyDescriptor, ValidationContext context) {
+        List<ValidationResult> retVal = new ArrayList<>();
+
+        boolean accessPolicyIsSet  = context.getProperty(accessPolicyDescriptor).isSet();
+        boolean policyKeyIsSet     = context.getProperty(POLICY_PRIMARY_KEY).isSet();
+        boolean useManagedIdentity = context.getProperty(USE_MANANGED_IDENTITY).asBoolean();
+
+        if (useManagedIdentity && (accessPolicyIsSet || policyKeyIsSet) ) {
+            final String msg = String.format(
+                "%s and %s with %s fields cannot be set at the same time.",
+                USE_MANANGED_IDENTITY.getDisplayName(),
+                accessPolicyDescriptor.getDisplayName(),
+                POLICY_PRIMARY_KEY.getDisplayName()
+            );
+            retVal.add(new ValidationResult.Builder().valid(false).explanation(msg).build());
+        } else if (!useManagedIdentity && (!accessPolicyIsSet || !policyKeyIsSet)) {
+            final String msg = String.format(
+                "Either %s or %s with %s must be set",
+                USE_MANANGED_IDENTITY.getDisplayName(),
+                accessPolicyDescriptor.getDisplayName(),
+                POLICY_PRIMARY_KEY.getDisplayName()
+            );
+            retVal.add(new ValidationResult.Builder().valid(false).explanation(msg).build());

Review comment:
       You might have missed this comment. I meant `new ValidationResult.Builder().subject("Credentials config ").valid(false).explanation(msg).build())` (or similar string in the subject) which would be more informative than the empty string in the validation error message.




----------------------------------------------------------------
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] sjyang18 commented on a change in pull request #4226: NIFI-6149: Azure EventHub Managed identity support patch

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



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/GetAzureEventHub.java
##########
@@ -103,16 +107,10 @@
             .description("The name of the shared access policy. This policy must have Listen claims.")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .expressionLanguageSupported(ExpressionLanguageScope.NONE)
-            .required(true)
-            .build();
-    static final PropertyDescriptor POLICY_PRIMARY_KEY = new PropertyDescriptor.Builder()
-            .name("Shared Access Policy Primary Key")
-            .description("The primary key of the shared access policy")
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
-            .sensitive(true)
-            .required(true)
+            .required(false)
             .build();
+    static final PropertyDescriptor POLICY_PRIMARY_KEY =  AzureEventHubUtils.POLICY_PRIMARY_KEY;
+    static final PropertyDescriptor USE_MANANGED_IDENTITY = AzureEventHubUtils.USE_MANAGED_IDENTITY;

Review comment:
       I fixed typos, not sure what went wrong with my IDE not to change in bulk update last time This time, I used grep to double check. 
   And, I cannot delete getConnectionString() without impacting existing test case and its BogusConnectionStringMockPutAzureEventHub. So, I decided to move AzureEventHubUtils.getSharedAccessSignatureConnectionString inside getConnectionString.  




----------------------------------------------------------------
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] sjyang18 commented on a change in pull request #4226: NIFI-6149: Azure EventHub Managed identity support patch

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



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/utils/AzureEventHubUtils.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.processors.azure.eventhub.utils;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import com.microsoft.azure.eventhubs.ConnectionStringBuilder;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.util.StandardValidators;
+
+public final class AzureEventHubUtils {
+
+    public static final String MANANGED_IDENDITY_POLICY = ConnectionStringBuilder.MANAGED_IDENTITY_AUTHENTICATION;
+
+    public static final PropertyDescriptor POLICY_PRIMARY_KEY = new PropertyDescriptor.Builder()
+        .name("Shared Access Policy Primary Key")

Review comment:
       Sounds good.




----------------------------------------------------------------
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] pvillard31 commented on pull request #4226: NIFI-6149: Azure EventHub Managed identity support patch

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


   Thanks for the pull request and recent changes @sjyang18. I think this is close to the finish line based on @turcsanyip's review. Just a quick question (unrelated to this specific pull request): would managed identities also make sense for storage processors (Blob storage / ADLS)? if yes, is it something you'd be willing to look at?


----------------------------------------------------------------
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] sjyang18 commented on a change in pull request #4226: NIFI-6149: Azure EventHub Managed identity support patch

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



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/utils/AzureEventHubUtils.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.processors.azure.eventhub.utils;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import com.microsoft.azure.eventhubs.ConnectionStringBuilder;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.util.StandardValidators;
+
+public final class AzureEventHubUtils {
+
+    public static final String MANAGED_IDENDITY_POLICY = ConnectionStringBuilder.MANAGED_IDENTITY_AUTHENTICATION;
+
+    public static final PropertyDescriptor POLICY_PRIMARY_KEY = new PropertyDescriptor.Builder()
+        .name("Shared Access Policy Primary Key")
+        .description("The primary key of the shared access policy")
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+        .sensitive(true)
+        .required(false)
+        .build();
+
+    public static final PropertyDescriptor USE_MANAGED_IDENTITY = new PropertyDescriptor.Builder()
+        .name("use-managed-identity")
+        .displayName("Use Azure Managed Identity")
+        .description("Choose whether or not to use the managed identity of Azure VM/VMSS")
+        .required(false).defaultValue("false").allowableValues("true", "false")
+        .addValidator(StandardValidators.BOOLEAN_VALIDATOR).build();
+
+    public static List<ValidationResult> customValidate(PropertyDescriptor accessPolicyDescriptor,
+        PropertyDescriptor policyKeyDescriptor,
+        ValidationContext context) {
+        List<ValidationResult> retVal = new ArrayList<>();
+
+        boolean accessPolicyIsSet  = context.getProperty(accessPolicyDescriptor).isSet();
+        boolean policyKeyIsSet     = context.getProperty(policyKeyDescriptor).isSet();
+        boolean useManagedIdentity = context.getProperty(USE_MANAGED_IDENTITY).asBoolean();
+
+        if (useManagedIdentity && (accessPolicyIsSet || policyKeyIsSet) ) {
+            final String msg = String.format(
+                "('%s') and ('%s' with '%s') fields cannot be set at the same time.",
+                USE_MANAGED_IDENTITY.getDisplayName(),
+                accessPolicyDescriptor.getDisplayName(),
+                POLICY_PRIMARY_KEY.getDisplayName()
+            );
+            retVal.add(new ValidationResult.Builder().valid(false).explanation(msg).build());
+        } else if (!useManagedIdentity && (!accessPolicyIsSet || !policyKeyIsSet)) {
+            final String msg = String.format(
+                "either('%s') or (%s with '%s') must be set",
+                USE_MANAGED_IDENTITY.getDisplayName(),
+                accessPolicyDescriptor.getDisplayName(),
+                POLICY_PRIMARY_KEY.getDisplayName()
+            );
+            retVal.add(new ValidationResult.Builder().valid(false).explanation(msg).build());
+        }
+        return retVal;
+    }
+
+    public static String getManagedIdentityConnectionString(final String namespace, final String eventHubName){
+        return new ConnectionStringBuilder().setNamespaceName(namespace).setEventHubName(eventHubName)
+                    .setAuthentication(MANAGED_IDENDITY_POLICY).toString();
+    }
+    public static String getSharedAccessSignatureConnectionString(final String namespace, final String eventHubName, final String sasName, final String sasKey) {

Review comment:
       done as you suggested




----------------------------------------------------------------
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] sjyang18 commented on a change in pull request #4226: NIFI-6149: Azure EventHub Managed identity support patch

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



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/utils/AzureEventHubUtils.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.processors.azure.eventhub.utils;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import com.microsoft.azure.eventhubs.ConnectionStringBuilder;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.util.StandardValidators;
+
+public final class AzureEventHubUtils {
+
+    public static final String MANANGED_IDENDITY_POLICY = ConnectionStringBuilder.MANAGED_IDENTITY_AUTHENTICATION;
+
+    public static final PropertyDescriptor POLICY_PRIMARY_KEY = new PropertyDescriptor.Builder()
+        .name("Shared Access Policy Primary Key")
+        .description("The primary key of the shared access policy")
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+        .sensitive(true)
+        .required(false)
+        .build();
+
+    public static final PropertyDescriptor USE_MANANGED_IDENTITY = new PropertyDescriptor.Builder()
+        .name("use-managed-identity")
+        .displayName("Use Azure Managed Identity")
+        .description("Choose whether or not to use the managed identity of azure vm/vmss. ")

Review comment:
       fixed.




----------------------------------------------------------------
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] sjyang18 commented on a change in pull request #4226: NIFI-6149: Azure EventHub Managed identity support patch

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



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/GetAzureEventHub.java
##########
@@ -163,6 +161,8 @@
             .description("Any FlowFile that is successfully received from the event hub will be transferred to this Relationship.")
             .build();
 
+    static final String MANANGED_IDENDITY_POLICY = AzureEventHubUtils.MANANGED_IDENDITY_POLICY;

Review comment:
       removed




----------------------------------------------------------------
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] sjyang18 commented on a change in pull request #4226: NIFI-6149: Azure EventHub Managed identity support patch

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



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/utils/AzureEventHubUtils.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.processors.azure.eventhub.utils;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import com.microsoft.azure.eventhubs.ConnectionStringBuilder;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.util.StandardValidators;
+
+public final class AzureEventHubUtils {
+
+    public static final String MANAGED_IDENDITY_POLICY = ConnectionStringBuilder.MANAGED_IDENTITY_AUTHENTICATION;

Review comment:
       Thank you for catching my mistakes. 

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/GetAzureEventHub.java
##########
@@ -103,16 +107,10 @@
             .description("The name of the shared access policy. This policy must have Listen claims.")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .expressionLanguageSupported(ExpressionLanguageScope.NONE)
-            .required(true)
-            .build();
-    static final PropertyDescriptor POLICY_PRIMARY_KEY = new PropertyDescriptor.Builder()
-            .name("Shared Access Policy Primary Key")
-            .description("The primary key of the shared access policy")
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
-            .sensitive(true)
-            .required(true)
+            .required(false)
             .build();
+    static final PropertyDescriptor POLICY_PRIMARY_KEY =  AzureEventHubUtils.POLICY_PRIMARY_KEY;
+    static final PropertyDescriptor USE_MANANGED_IDENTITY = AzureEventHubUtils.USE_MANAGED_IDENTITY;

Review comment:
       fixed.




----------------------------------------------------------------
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] turcsanyip commented on a change in pull request #4226: NIFI-6149: Azure EventHub Managed identity support patch

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



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/utils/AzureEventHubUtils.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.processors.azure.eventhub.utils;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import com.microsoft.azure.eventhubs.ConnectionStringBuilder;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.util.StandardValidators;
+
+public final class AzureEventHubUtils {
+
+    public static final String MANANGED_IDENDITY_POLICY = ConnectionStringBuilder.MANAGED_IDENTITY_AUTHENTICATION;

Review comment:
       Typo: MANAGED

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/utils/AzureEventHubUtils.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.processors.azure.eventhub.utils;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import com.microsoft.azure.eventhubs.ConnectionStringBuilder;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.util.StandardValidators;
+
+public final class AzureEventHubUtils {
+
+    public static final String MANANGED_IDENDITY_POLICY = ConnectionStringBuilder.MANAGED_IDENTITY_AUTHENTICATION;
+
+    public static final PropertyDescriptor POLICY_PRIMARY_KEY = new PropertyDescriptor.Builder()
+        .name("Shared Access Policy Primary Key")
+        .description("The primary key of the shared access policy")
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+        .sensitive(true)
+        .required(false)
+        .build();
+
+    public static final PropertyDescriptor USE_MANANGED_IDENTITY = new PropertyDescriptor.Builder()

Review comment:
       Typo: MANAGED

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/ConsumeAzureEventHub.java
##########
@@ -115,17 +116,10 @@
             .description("The name of the shared access policy. This policy must have Listen claims.")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
-            .required(true)
-            .build();
-    static final PropertyDescriptor POLICY_PRIMARY_KEY = new PropertyDescriptor.Builder()
-            .name("event-hub-shared-access-policy-primary-key")

Review comment:
       `POLICY_PRIMARY_KEY` in `ConsumeAzureEventHub` had a different name which must be kept for backward compatibility (my old ConsumeAzureEventHub processor became invalid after updated the Azure NAR in NiFi).
   
   For this reason, it is not possible to use the exact same `POLICY_PRIMARY_KEY` from `AzureEventHubUtils`.
   You can do something like this:
   ```
       static final PropertyDescriptor POLICY_PRIMARY_KEY = new PropertyDescriptor.Builder()
               .fromPropertyDescriptor(AzureEventHubUtils.POLICY_PRIMARY_KEY)
               .name("event-hub-shared-access-policy-primary-key")
               .build();
   ```
   Unfortunately, it will also break the common validation logic in `AzureEventHubUtils`. You will need to pass in the actual PropertyDescriptor of the `POLICY_PRIMARY_KEY` (similar to `accessPolicyDescriptor`).

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/utils/AzureEventHubUtils.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.processors.azure.eventhub.utils;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import com.microsoft.azure.eventhubs.ConnectionStringBuilder;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.util.StandardValidators;
+
+public final class AzureEventHubUtils {
+
+    public static final String MANANGED_IDENDITY_POLICY = ConnectionStringBuilder.MANAGED_IDENTITY_AUTHENTICATION;
+
+    public static final PropertyDescriptor POLICY_PRIMARY_KEY = new PropertyDescriptor.Builder()
+        .name("Shared Access Policy Primary Key")
+        .description("The primary key of the shared access policy")
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+        .sensitive(true)
+        .required(false)
+        .build();
+
+    public static final PropertyDescriptor USE_MANANGED_IDENTITY = new PropertyDescriptor.Builder()
+        .name("use-managed-identity")
+        .displayName("Use Azure Managed Identity")
+        .description("Choose whether or not to use the managed identity of azure vm/vmss. ")

Review comment:
       I would correct it: "... Azure VM/VMSS."

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/ConsumeAzureEventHub.java
##########
@@ -627,9 +616,26 @@ private void registerEventProcessor(final ProcessContext context) throws Excepti
 
         final String storageConnectionString = String.format(FORMAT_STORAGE_CONNECTION_STRING, storageAccountName, storageAccountKey);
 
-        final ConnectionStringBuilder eventHubConnectionString = new ConnectionStringBuilder().setNamespaceName(namespaceName).setEventHubName( eventHubName).setSasKeyName(sasName).setSasKey(sasKey);
-
-        eventProcessorHost = new EventProcessorHost(consumerHostname, eventHubName, consumerGroupName, eventHubConnectionString.toString(), storageConnectionString, containerName);
+        final String connectionString;
+        final boolean useManagedIdentity = context.getProperty(USE_MANANGED_IDENTITY).asBoolean();
+        if(useManagedIdentity) {
+            connectionString = AzureEventHubUtils.getManagedIdentityConnectionString(namespaceName, eventHubName);
+        } else {
+            final String sasName = context.getProperty(ACCESS_POLICY_NAME).evaluateAttributeExpressions().getValue();
+            validateRequiredProperty(ACCESS_POLICY_NAME, sasName);
+            final String sasKey = context.getProperty(POLICY_PRIMARY_KEY).evaluateAttributeExpressions().getValue();
+            validateRequiredProperty(POLICY_PRIMARY_KEY, sasKey);
+            connectionString = new ConnectionStringBuilder()

Review comment:
       You could create a common `getSharedAccessSignatureConnectionString()` method in `AzureEventHubUtils`, similar to `getManagedIdentityConnectionString()`.

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/utils/AzureEventHubUtils.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.processors.azure.eventhub.utils;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import com.microsoft.azure.eventhubs.ConnectionStringBuilder;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.util.StandardValidators;
+
+public final class AzureEventHubUtils {
+
+    public static final String MANANGED_IDENDITY_POLICY = ConnectionStringBuilder.MANAGED_IDENTITY_AUTHENTICATION;
+
+    public static final PropertyDescriptor POLICY_PRIMARY_KEY = new PropertyDescriptor.Builder()
+        .name("Shared Access Policy Primary Key")
+        .description("The primary key of the shared access policy")
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+        .sensitive(true)
+        .required(false)
+        .build();
+
+    public static final PropertyDescriptor USE_MANANGED_IDENTITY = new PropertyDescriptor.Builder()
+        .name("use-managed-identity")
+        .displayName("Use Azure Managed Identity")
+        .description("Choose whether or not to use the managed identity of azure vm/vmss. ")
+        .required(false).defaultValue("false").allowableValues("true", "false")
+        .addValidator(StandardValidators.BOOLEAN_VALIDATOR).build();
+
+    public static List<ValidationResult> customValidate(PropertyDescriptor accessPolicyDescriptor, ValidationContext context) {
+        List<ValidationResult> retVal = new ArrayList<>();
+
+        boolean accessPolicyIsSet  = context.getProperty(accessPolicyDescriptor).isSet();
+        boolean policyKeyIsSet     = context.getProperty(POLICY_PRIMARY_KEY).isSet();
+        boolean useManagedIdentity = context.getProperty(USE_MANANGED_IDENTITY).asBoolean();
+
+        if (useManagedIdentity && (accessPolicyIsSet || policyKeyIsSet) ) {
+            final String msg = String.format(
+                "%s and %s with %s fields cannot be set at the same time.",
+                USE_MANANGED_IDENTITY.getDisplayName(),
+                accessPolicyDescriptor.getDisplayName(),
+                POLICY_PRIMARY_KEY.getDisplayName()
+            );
+            retVal.add(new ValidationResult.Builder().valid(false).explanation(msg).build());
+        } else if (!useManagedIdentity && (!accessPolicyIsSet || !policyKeyIsSet)) {
+            final String msg = String.format(
+                "Either %s or %s with %s must be set",
+                USE_MANANGED_IDENTITY.getDisplayName(),
+                accessPolicyDescriptor.getDisplayName(),
+                POLICY_PRIMARY_KEY.getDisplayName()
+            );
+            retVal.add(new ValidationResult.Builder().valid(false).explanation(msg).build());

Review comment:
       `subject()` should be set, otherwise the error message will look like: '' is invalid...
   For example, you can specify "Credentials config" as subject. This way, the error message will be: 'Credentials config' is invalid...
   
   The same can be applied on line 62 above.

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/utils/AzureEventHubUtils.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.processors.azure.eventhub.utils;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import com.microsoft.azure.eventhubs.ConnectionStringBuilder;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.util.StandardValidators;
+
+public final class AzureEventHubUtils {
+
+    public static final String MANANGED_IDENDITY_POLICY = ConnectionStringBuilder.MANAGED_IDENTITY_AUTHENTICATION;
+
+    public static final PropertyDescriptor POLICY_PRIMARY_KEY = new PropertyDescriptor.Builder()
+        .name("Shared Access Policy Primary Key")
+        .description("The primary key of the shared access policy")
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+        .sensitive(true)
+        .required(false)
+        .build();
+
+    public static final PropertyDescriptor USE_MANANGED_IDENTITY = new PropertyDescriptor.Builder()
+        .name("use-managed-identity")
+        .displayName("Use Azure Managed Identity")
+        .description("Choose whether or not to use the managed identity of azure vm/vmss. ")
+        .required(false).defaultValue("false").allowableValues("true", "false")
+        .addValidator(StandardValidators.BOOLEAN_VALIDATOR).build();
+
+    public static List<ValidationResult> customValidate(PropertyDescriptor accessPolicyDescriptor, ValidationContext context) {
+        List<ValidationResult> retVal = new ArrayList<>();
+
+        boolean accessPolicyIsSet  = context.getProperty(accessPolicyDescriptor).isSet();
+        boolean policyKeyIsSet     = context.getProperty(POLICY_PRIMARY_KEY).isSet();
+        boolean useManagedIdentity = context.getProperty(USE_MANANGED_IDENTITY).asBoolean();
+
+        if (useManagedIdentity && (accessPolicyIsSet || policyKeyIsSet) ) {
+            final String msg = String.format(
+                "%s and %s with %s fields cannot be set at the same time.",
+                USE_MANANGED_IDENTITY.getDisplayName(),
+                accessPolicyDescriptor.getDisplayName(),
+                POLICY_PRIMARY_KEY.getDisplayName()
+            );
+            retVal.add(new ValidationResult.Builder().valid(false).explanation(msg).build());
+        } else if (!useManagedIdentity && (!accessPolicyIsSet || !policyKeyIsSet)) {
+            final String msg = String.format(
+                "Either %s or %s with %s must be set",

Review comment:
       It will land in the middle of the error message, so "either ..." would be more appropriate.

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/utils/AzureEventHubUtils.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.processors.azure.eventhub.utils;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import com.microsoft.azure.eventhubs.ConnectionStringBuilder;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.util.StandardValidators;
+
+public final class AzureEventHubUtils {
+
+    public static final String MANANGED_IDENDITY_POLICY = ConnectionStringBuilder.MANAGED_IDENTITY_AUTHENTICATION;
+
+    public static final PropertyDescriptor POLICY_PRIMARY_KEY = new PropertyDescriptor.Builder()
+        .name("Shared Access Policy Primary Key")
+        .description("The primary key of the shared access policy")
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+        .sensitive(true)
+        .required(false)
+        .build();
+
+    public static final PropertyDescriptor USE_MANANGED_IDENTITY = new PropertyDescriptor.Builder()
+        .name("use-managed-identity")
+        .displayName("Use Azure Managed Identity")
+        .description("Choose whether or not to use the managed identity of azure vm/vmss. ")
+        .required(false).defaultValue("false").allowableValues("true", "false")
+        .addValidator(StandardValidators.BOOLEAN_VALIDATOR).build();
+
+    public static List<ValidationResult> customValidate(PropertyDescriptor accessPolicyDescriptor, ValidationContext context) {
+        List<ValidationResult> retVal = new ArrayList<>();
+
+        boolean accessPolicyIsSet  = context.getProperty(accessPolicyDescriptor).isSet();
+        boolean policyKeyIsSet     = context.getProperty(POLICY_PRIMARY_KEY).isSet();
+        boolean useManagedIdentity = context.getProperty(USE_MANANGED_IDENTITY).asBoolean();
+
+        if (useManagedIdentity && (accessPolicyIsSet || policyKeyIsSet) ) {
+            final String msg = String.format(
+                "%s and %s with %s fields cannot be set at the same time.",

Review comment:
       If you enclose the property names in apostrophes ('%s'), the error message would be more readable on the UI.
   
   The same can be applied on line 67 below.

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/GetAzureEventHub.java
##########
@@ -301,11 +308,24 @@ public void onScheduled(final ProcessContext context) throws ProcessException, U
         }
         this.partitionNames = partitionNames;
 
-        final String policyName = context.getProperty(ACCESS_POLICY).getValue();
-        final String policyKey = context.getProperty(POLICY_PRIMARY_KEY).getValue();
         final String namespace = context.getProperty(NAMESPACE).getValue();
         final String eventHubName = context.getProperty(EVENT_HUB_NAME).getValue();
         final String serviceBusEndpoint = context.getProperty(SERVICE_BUS_ENDPOINT).getValue();
+        final boolean useManagedIdentity = context.getProperty(USE_MANANGED_IDENTITY).asBoolean();
+        final String policyName, policyKey, connectionString;
+        if(useManagedIdentity){
+            policyName =  MANANGED_IDENDITY_POLICY;
+            policyKey = null;

Review comment:
       These are unused assignments that could be omitted.
   `policyName` and `policyKey` could be declared in the `else` branch only.

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/utils/AzureEventHubUtils.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.processors.azure.eventhub.utils;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import com.microsoft.azure.eventhubs.ConnectionStringBuilder;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.util.StandardValidators;
+
+public final class AzureEventHubUtils {
+
+    public static final String MANANGED_IDENDITY_POLICY = ConnectionStringBuilder.MANAGED_IDENTITY_AUTHENTICATION;
+
+    public static final PropertyDescriptor POLICY_PRIMARY_KEY = new PropertyDescriptor.Builder()
+        .name("Shared Access Policy Primary Key")

Review comment:
       Not related to your current change but the `primary` sounds weird to me because one can actually configure the _secondary_ policy key for the property too (and it will work).
   'Shared Access Policy Key' would be more adequate. 
   I'll create a separate jira ticket to change it (or more precisely add a modified display name because the existing property name should not be changed).

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/GetAzureEventHub.java
##########
@@ -163,6 +161,8 @@
             .description("Any FlowFile that is successfully received from the event hub will be transferred to this Relationship.")
             .build();
 
+    static final String MANANGED_IDENDITY_POLICY = AzureEventHubUtils.MANANGED_IDENDITY_POLICY;

Review comment:
       Does this indirection needed here?
   The `AzureEventHubUtils.MANANGED_IDENDITY_POLICY` constant could be used directly as in the Put processor.




----------------------------------------------------------------
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] sjyang18 commented on a change in pull request #4226: NIFI-6149: Azure EventHub Managed identity support patch

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



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/utils/AzureEventHubUtils.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.processors.azure.eventhub.utils;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import com.microsoft.azure.eventhubs.ConnectionStringBuilder;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.util.StandardValidators;
+
+public final class AzureEventHubUtils {
+
+    public static final String MANANGED_IDENDITY_POLICY = ConnectionStringBuilder.MANAGED_IDENTITY_AUTHENTICATION;
+
+    public static final PropertyDescriptor POLICY_PRIMARY_KEY = new PropertyDescriptor.Builder()
+        .name("Shared Access Policy Primary Key")
+        .description("The primary key of the shared access policy")
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+        .sensitive(true)
+        .required(false)
+        .build();
+
+    public static final PropertyDescriptor USE_MANANGED_IDENTITY = new PropertyDescriptor.Builder()

Review comment:
       fixed




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