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/28 21:28:45 UTC

[GitHub] [nifi] turcsanyip commented on a change in pull request #4226: NIFI-6149: Azure EventHub Managed identity support patch

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