You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by "mkalavala (via GitHub)" <gi...@apache.org> on 2023/04/17 20:54:35 UTC

[GitHub] [nifi] mkalavala opened a new pull request, #7182: NIFI-11360 Support Client-Side Encryption for Azure Blob v12 Processors

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

   <!-- Licensed to the Apache Software Foundation (ASF) under one or more -->
   <!-- contributor license agreements.  See the NOTICE file distributed with -->
   <!-- this work for additional information regarding copyright ownership. -->
   <!-- The ASF licenses this file to You under the Apache License, Version 2.0 -->
   <!-- (the "License"); you may not use this file except in compliance with -->
   <!-- the License.  You may obtain a copy of the License at -->
   <!--     http://www.apache.org/licenses/LICENSE-2.0 -->
   <!-- Unless required by applicable law or agreed to in writing, software -->
   <!-- distributed under the License is distributed on an "AS IS" BASIS, -->
   <!-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -->
   <!-- See the License for the specific language governing permissions and -->
   <!-- limitations under the License. -->
   
   # Summary
   
   [NIFI-11360](https://issues.apache.org/jira/browse/NIFI-00000)
   
   # Tracking
   
   Please complete the following tracking steps prior to pull request creation.
   
   ### Issue Tracking
   
   - [x] [Apache NiFi Jira](https://issues.apache.org/jira/browse/NIFI) issue created
   
   ### Pull Request Tracking
   
   - [x] Pull Request title starts with Apache NiFi Jira issue number, such as `NIFI-00000`
   - [x] Pull Request commit message starts with Apache NiFi Jira issue number, as such `NIFI-00000`
   
   ### Pull Request Formatting
   
   - [x] Pull Request based on current revision of the `main` branch
   - [ ] Pull Request refers to a feature branch with one commit containing changes
   
   # Verification
   
   Please indicate the verification steps performed prior to pull request creation.
   
   ### Build
   
   - [x] Build completed using `mvn clean install -P contrib-check`
     - [x] JDK 11
     - [ ] JDK 17
   
   ### Licensing
   
   - [x] New dependencies are compatible with the [Apache License 2.0](https://apache.org/licenses/LICENSE-2.0) according to the [License Policy](https://www.apache.org/legal/resolved.html)
   - [ ] New dependencies are documented in applicable `LICENSE` and `NOTICE` files
   
   ### Documentation
   
   - [ ] Documentation formatting appears as expected in rendered files
   


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

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

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


[GitHub] [nifi] exceptionfactory closed pull request #7182: NIFI-11360 Support Client-Side Encryption for Azure Blob v12 Processors

Posted by "exceptionfactory (via GitHub)" <gi...@apache.org>.
exceptionfactory closed pull request #7182: NIFI-11360 Support Client-Side Encryption for Azure Blob v12 Processors
URL: https://github.com/apache/nifi/pull/7182


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

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

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


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #7182: NIFI-11360 Support Client-Side Encryption for Azure Blob v12 Processors

Posted by "exceptionfactory (via GitHub)" <gi...@apache.org>.
exceptionfactory commented on code in PR #7182:
URL: https://github.com/apache/nifi/pull/7182#discussion_r1181571272


##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/ClientSideEncryptionSupport.java:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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;
+
+import com.azure.core.cryptography.AsyncKeyEncryptionKey;
+import com.azure.security.keyvault.keys.cryptography.KeyEncryptionKeyClientBuilder;
+import com.azure.security.keyvault.keys.cryptography.models.KeyWrapAlgorithm;
+import com.azure.security.keyvault.keys.models.JsonWebKey;
+import com.azure.security.keyvault.keys.models.KeyOperation;
+import com.azure.storage.blob.BlobClient;
+import com.azure.storage.blob.BlobContainerClient;
+import com.azure.storage.blob.specialized.cryptography.EncryptedBlobClientBuilder;
+import com.azure.storage.blob.specialized.cryptography.EncryptionVersion;
+import org.apache.commons.codec.DecoderException;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.azure.storage.utils.ClientSideEncryptionMethod;
+import org.apache.nifi.util.StringUtils;
+
+import javax.crypto.spec.SecretKeySpec;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+public interface ClientSideEncryptionSupport {
+    List<KeyOperation> KEY_OPERATIONS = Arrays.asList(KeyOperation.WRAP_KEY, KeyOperation.UNWRAP_KEY);
+
+    PropertyDescriptor CSE_KEY_TYPE = new PropertyDescriptor.Builder()
+            .name("cse-key-type")
+            .displayName("Client-Side Encryption Key Type")
+            .required(true)
+            .allowableValues(ClientSideEncryptionMethod.class)
+            .defaultValue(ClientSideEncryptionMethod.NONE.name())
+            .description("Specifies the key type to use for client-side encryption.")
+            .build();
+
+    PropertyDescriptor CSE_KEY_ID = new PropertyDescriptor.Builder()
+            .name("cse-key-id")
+            .displayName("Client-Side Encryption Key ID")
+            .description("Specifies the ID of the key to use for client-side encryption.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(CSE_KEY_TYPE, ClientSideEncryptionMethod.LOCAL.name())
+            .build();
+
+    PropertyDescriptor CSE_LOCAL_KEY_HEX = new PropertyDescriptor.Builder()
+            .name("cse-local-key-hex")
+            .displayName("Client-Side Encryption Local Key")
+            .description("When using local client-side encryption, this is the raw key, encoded in hexadecimal")
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(CSE_KEY_TYPE, ClientSideEncryptionMethod.LOCAL.name())
+            .sensitive(true)
+            .build();
+
+    default Collection<ValidationResult> validateClientSideEncryptionProperties(ValidationContext validationContext) {
+        final List<ValidationResult> validationResults = new ArrayList<>();
+        final String cseKeyTypeValue = validationContext.getProperty(CSE_KEY_TYPE).getValue();
+        final ClientSideEncryptionMethod cseKeyType = ClientSideEncryptionMethod.valueOf(cseKeyTypeValue);
+        final String cseKeyId = validationContext.getProperty(CSE_KEY_ID).getValue();
+        final String cseLocalKeyHex = validationContext.getProperty(CSE_LOCAL_KEY_HEX).getValue();
+        if (cseKeyType != ClientSideEncryptionMethod.NONE && StringUtils.isBlank(cseKeyId)) {
+            validationResults.add(new ValidationResult.Builder().subject(CSE_KEY_ID.getDisplayName())
+                    .explanation("a key ID must be set when client-side encryption is enabled.").build());
+        }
+        if (cseKeyType == ClientSideEncryptionMethod.LOCAL) {
+            validationResults.addAll(validateLocalKey(cseLocalKeyHex));
+        }
+        return validationResults;
+    }
+
+    default List<ValidationResult> validateLocalKey(String keyHex) {
+        final List<ValidationResult> validationResults = new ArrayList<>();
+        if (StringUtils.isBlank(keyHex)) {
+            validationResults.add(new ValidationResult.Builder().subject(CSE_LOCAL_KEY_HEX.getDisplayName())
+                    .explanation("a local key must be set when client-side encryption is enabled with local encryption.").build());
+        } else {
+            byte[] keyBytes;
+            try {
+                keyBytes = Hex.decodeHex(keyHex);
+                if (getKeyWrapAlgorithm(keyBytes) == null) {
+                    validationResults.add(new ValidationResult.Builder().subject(CSE_LOCAL_KEY_HEX.getDisplayName())
+                            .explanation("the local key must be 128, 192, 256, 384 or 512 bits of data.").build());

Review Comment:
   Recommend adjusting the wording and including the actual length.
   ```suggestion
                               .explanation(String.format("Key size in bits must be one of [128, 192, 256, 384, 512] instead of [%d]", keyBytes.length * 8)).build());
   ```



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/ClientSideEncryptionSupport.java:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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;
+
+import com.azure.core.cryptography.AsyncKeyEncryptionKey;
+import com.azure.security.keyvault.keys.cryptography.KeyEncryptionKeyClientBuilder;
+import com.azure.security.keyvault.keys.cryptography.models.KeyWrapAlgorithm;
+import com.azure.security.keyvault.keys.models.JsonWebKey;
+import com.azure.security.keyvault.keys.models.KeyOperation;
+import com.azure.storage.blob.BlobClient;
+import com.azure.storage.blob.BlobContainerClient;
+import com.azure.storage.blob.specialized.cryptography.EncryptedBlobClientBuilder;
+import com.azure.storage.blob.specialized.cryptography.EncryptionVersion;
+import org.apache.commons.codec.DecoderException;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.azure.storage.utils.ClientSideEncryptionMethod;
+import org.apache.nifi.util.StringUtils;
+
+import javax.crypto.spec.SecretKeySpec;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+public interface ClientSideEncryptionSupport {
+    List<KeyOperation> KEY_OPERATIONS = Arrays.asList(KeyOperation.WRAP_KEY, KeyOperation.UNWRAP_KEY);
+
+    PropertyDescriptor CSE_KEY_TYPE = new PropertyDescriptor.Builder()
+            .name("cse-key-type")
+            .displayName("Client-Side Encryption Key Type")
+            .required(true)
+            .allowableValues(ClientSideEncryptionMethod.class)
+            .defaultValue(ClientSideEncryptionMethod.NONE.name())
+            .description("Specifies the key type to use for client-side encryption.")
+            .build();
+
+    PropertyDescriptor CSE_KEY_ID = new PropertyDescriptor.Builder()
+            .name("cse-key-id")
+            .displayName("Client-Side Encryption Key ID")
+            .description("Specifies the ID of the key to use for client-side encryption.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(CSE_KEY_TYPE, ClientSideEncryptionMethod.LOCAL.name())
+            .build();
+
+    PropertyDescriptor CSE_LOCAL_KEY_HEX = new PropertyDescriptor.Builder()
+            .name("cse-local-key-hex")
+            .displayName("Client-Side Encryption Local Key")
+            .description("When using local client-side encryption, this is the raw key, encoded in hexadecimal")
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(CSE_KEY_TYPE, ClientSideEncryptionMethod.LOCAL.name())
+            .sensitive(true)
+            .build();
+
+    default Collection<ValidationResult> validateClientSideEncryptionProperties(ValidationContext validationContext) {
+        final List<ValidationResult> validationResults = new ArrayList<>();
+        final String cseKeyTypeValue = validationContext.getProperty(CSE_KEY_TYPE).getValue();
+        final ClientSideEncryptionMethod cseKeyType = ClientSideEncryptionMethod.valueOf(cseKeyTypeValue);
+        final String cseKeyId = validationContext.getProperty(CSE_KEY_ID).getValue();
+        final String cseLocalKeyHex = validationContext.getProperty(CSE_LOCAL_KEY_HEX).getValue();
+        if (cseKeyType != ClientSideEncryptionMethod.NONE && StringUtils.isBlank(cseKeyId)) {
+            validationResults.add(new ValidationResult.Builder().subject(CSE_KEY_ID.getDisplayName())
+                    .explanation("a key ID must be set when client-side encryption is enabled.").build());

Review Comment:
   ```suggestion
                       .explanation("a key ID must be set when client-side encryption is enabled").build());
   ```



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/ClientSideEncryptionSupport.java:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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;
+
+import com.azure.core.cryptography.AsyncKeyEncryptionKey;
+import com.azure.security.keyvault.keys.cryptography.KeyEncryptionKeyClientBuilder;
+import com.azure.security.keyvault.keys.cryptography.models.KeyWrapAlgorithm;
+import com.azure.security.keyvault.keys.models.JsonWebKey;
+import com.azure.security.keyvault.keys.models.KeyOperation;
+import com.azure.storage.blob.BlobClient;
+import com.azure.storage.blob.BlobContainerClient;
+import com.azure.storage.blob.specialized.cryptography.EncryptedBlobClientBuilder;
+import com.azure.storage.blob.specialized.cryptography.EncryptionVersion;
+import org.apache.commons.codec.DecoderException;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.azure.storage.utils.ClientSideEncryptionMethod;
+import org.apache.nifi.util.StringUtils;
+
+import javax.crypto.spec.SecretKeySpec;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+public interface ClientSideEncryptionSupport {
+    List<KeyOperation> KEY_OPERATIONS = Arrays.asList(KeyOperation.WRAP_KEY, KeyOperation.UNWRAP_KEY);
+
+    PropertyDescriptor CSE_KEY_TYPE = new PropertyDescriptor.Builder()
+            .name("cse-key-type")
+            .displayName("Client-Side Encryption Key Type")
+            .required(true)
+            .allowableValues(ClientSideEncryptionMethod.class)
+            .defaultValue(ClientSideEncryptionMethod.NONE.name())
+            .description("Specifies the key type to use for client-side encryption.")
+            .build();
+
+    PropertyDescriptor CSE_KEY_ID = new PropertyDescriptor.Builder()
+            .name("cse-key-id")

Review Comment:
   ```suggestion
               .name("Client-Side Encryption Key ID")
   ```



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/PutAzureBlobStorage_v12.java:
##########
@@ -124,9 +129,19 @@ public class PutAzureBlobStorage_v12 extends AbstractAzureBlobProcessor_v12 {
             CREATE_CONTAINER,
             CONFLICT_RESOLUTION,
             BLOB_NAME,
-            AzureStorageUtils.PROXY_CONFIGURATION_SERVICE
+            AzureStorageUtils.PROXY_CONFIGURATION_SERVICE,
+            CSE_KEY_ID,
+            CSE_KEY_TYPE,

Review Comment:
   The Key Type should be listed before the Key ID:
   ```suggestion
               CSE_KEY_TYPE,
               CSE_KEY_ID,
   ```



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/ClientSideEncryptionSupport.java:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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;
+
+import com.azure.core.cryptography.AsyncKeyEncryptionKey;
+import com.azure.security.keyvault.keys.cryptography.KeyEncryptionKeyClientBuilder;
+import com.azure.security.keyvault.keys.cryptography.models.KeyWrapAlgorithm;
+import com.azure.security.keyvault.keys.models.JsonWebKey;
+import com.azure.security.keyvault.keys.models.KeyOperation;
+import com.azure.storage.blob.BlobClient;
+import com.azure.storage.blob.BlobContainerClient;
+import com.azure.storage.blob.specialized.cryptography.EncryptedBlobClientBuilder;
+import com.azure.storage.blob.specialized.cryptography.EncryptionVersion;
+import org.apache.commons.codec.DecoderException;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.azure.storage.utils.ClientSideEncryptionMethod;
+import org.apache.nifi.util.StringUtils;
+
+import javax.crypto.spec.SecretKeySpec;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+public interface ClientSideEncryptionSupport {
+    List<KeyOperation> KEY_OPERATIONS = Arrays.asList(KeyOperation.WRAP_KEY, KeyOperation.UNWRAP_KEY);
+
+    PropertyDescriptor CSE_KEY_TYPE = new PropertyDescriptor.Builder()
+            .name("cse-key-type")
+            .displayName("Client-Side Encryption Key Type")
+            .required(true)
+            .allowableValues(ClientSideEncryptionMethod.class)
+            .defaultValue(ClientSideEncryptionMethod.NONE.name())
+            .description("Specifies the key type to use for client-side encryption.")
+            .build();
+
+    PropertyDescriptor CSE_KEY_ID = new PropertyDescriptor.Builder()
+            .name("cse-key-id")
+            .displayName("Client-Side Encryption Key ID")
+            .description("Specifies the ID of the key to use for client-side encryption.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(CSE_KEY_TYPE, ClientSideEncryptionMethod.LOCAL.name())
+            .build();
+
+    PropertyDescriptor CSE_LOCAL_KEY_HEX = new PropertyDescriptor.Builder()
+            .name("cse-local-key-hex")
+            .displayName("Client-Side Encryption Local Key")
+            .description("When using local client-side encryption, this is the raw key, encoded in hexadecimal")
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(CSE_KEY_TYPE, ClientSideEncryptionMethod.LOCAL.name())
+            .sensitive(true)
+            .build();
+
+    default Collection<ValidationResult> validateClientSideEncryptionProperties(ValidationContext validationContext) {
+        final List<ValidationResult> validationResults = new ArrayList<>();
+        final String cseKeyTypeValue = validationContext.getProperty(CSE_KEY_TYPE).getValue();
+        final ClientSideEncryptionMethod cseKeyType = ClientSideEncryptionMethod.valueOf(cseKeyTypeValue);
+        final String cseKeyId = validationContext.getProperty(CSE_KEY_ID).getValue();
+        final String cseLocalKeyHex = validationContext.getProperty(CSE_LOCAL_KEY_HEX).getValue();
+        if (cseKeyType != ClientSideEncryptionMethod.NONE && StringUtils.isBlank(cseKeyId)) {
+            validationResults.add(new ValidationResult.Builder().subject(CSE_KEY_ID.getDisplayName())
+                    .explanation("a key ID must be set when client-side encryption is enabled.").build());
+        }
+        if (cseKeyType == ClientSideEncryptionMethod.LOCAL) {
+            validationResults.addAll(validateLocalKey(cseLocalKeyHex));
+        }
+        return validationResults;
+    }
+
+    default List<ValidationResult> validateLocalKey(String keyHex) {
+        final List<ValidationResult> validationResults = new ArrayList<>();
+        if (StringUtils.isBlank(keyHex)) {
+            validationResults.add(new ValidationResult.Builder().subject(CSE_LOCAL_KEY_HEX.getDisplayName())
+                    .explanation("a local key must be set when client-side encryption is enabled with local encryption.").build());

Review Comment:
   ```suggestion
                       .explanation("a local key must be set when client-side encryption is enabled").build());
   ```



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/storage/TestClientSideEncryptionSupport.java:
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.storage;
+
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.util.MockProcessContext;
+import org.apache.nifi.util.MockValidationContext;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.Collection;
+
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class TestClientSideEncryptionSupport {
+    private static final String KEY_ID_VALUE = "key:id";
+    private static final String KEY_64B_VALUE = "1234567890ABCDEF";
+    private static final String KEY_128B_VALUE = KEY_64B_VALUE + KEY_64B_VALUE;
+    private static final String KEY_192B_VALUE = KEY_128B_VALUE + KEY_64B_VALUE;
+    private static final String KEY_256B_VALUE = KEY_128B_VALUE + KEY_128B_VALUE;
+    private static final String KEY_384B_VALUE = KEY_256B_VALUE + KEY_128B_VALUE;
+    private static final String KEY_512B_VALUE = KEY_256B_VALUE + KEY_256B_VALUE;
+
+    private MockProcessContext processContext;
+    private MockValidationContext validationContext;
+    private PutAzureBlobStorage_v12 putAzureBlobStorage_v12;
+
+    @BeforeEach
+    public void setUp() {
+        putAzureBlobStorage_v12 = new PutAzureBlobStorage_v12();
+        processContext = new MockProcessContext(putAzureBlobStorage_v12);
+        validationContext = new MockValidationContext(processContext);
+    }
+
+    @Test
+    public void testNoCesConfiguredOnProcessor() {
+        configureProcessorProperties("NONE", null, null);
+
+        Collection<ValidationResult> result = putAzureBlobStorage_v12.validateClientSideEncryptionProperties(validationContext);
+
+        assertValid(result);
+    }
+
+    @Test
+    public void testLocalCesNoKeyIdOnProcessor() {
+        configureProcessorProperties("LOCAL", null, KEY_128B_VALUE);
+
+        Collection<ValidationResult> result = putAzureBlobStorage_v12.validateClientSideEncryptionProperties(validationContext);
+
+        assertNotValid(result);
+    }
+
+    @Test
+    public void testLocalCesNoKeyOnProcessor() {
+        configureProcessorProperties("LOCAL", KEY_ID_VALUE, null);
+
+        Collection<ValidationResult> result = putAzureBlobStorage_v12.validateClientSideEncryptionProperties(validationContext);
+
+        assertNotValid(result);
+    }
+
+    @Test
+    public void testLocalCesInvalidHexKeyOnProcessor() {
+        configureProcessorProperties("LOCAL", KEY_ID_VALUE, "ZZ");
+
+        Collection<ValidationResult> result = putAzureBlobStorage_v12.validateClientSideEncryptionProperties(validationContext);
+
+        assertNotValid(result);
+    }
+
+    @Test
+    public void testLocalCesInvalidKeyLengthOnProcessor() {
+        configureProcessorProperties("LOCAL", KEY_ID_VALUE, KEY_64B_VALUE);
+
+        Collection<ValidationResult> result = putAzureBlobStorage_v12.validateClientSideEncryptionProperties(validationContext);
+
+        assertNotValid(result);
+        assertContains(result, "the local key must be 128, 192, 256, 384 or 512 bits of data.");
+    }
+
+    @Test
+    public void testLocalCes128BitKeyOnProcessor() {
+        configureProcessorProperties("LOCAL", KEY_ID_VALUE, KEY_128B_VALUE);
+
+        Collection<ValidationResult> result = putAzureBlobStorage_v12.validateClientSideEncryptionProperties(validationContext);
+
+        assertValid(result);
+    }
+
+    @Test
+    public void testLocalCes192BitKeyOnProcessor() {
+        configureProcessorProperties("LOCAL", KEY_ID_VALUE, KEY_192B_VALUE);
+
+        Collection<ValidationResult> result = putAzureBlobStorage_v12.validateClientSideEncryptionProperties(validationContext);
+
+        assertValid(result);
+    }
+
+    @Test
+    public void testLocalCes256BitKeyOnProcessor() {
+        configureProcessorProperties("LOCAL", KEY_ID_VALUE, KEY_256B_VALUE);
+
+        Collection<ValidationResult> result = putAzureBlobStorage_v12.validateClientSideEncryptionProperties(validationContext);
+
+        assertValid(result);
+    }
+
+    @Test
+    public void testLocalCes384BitKeyOnProcessor() {
+        configureProcessorProperties("LOCAL", KEY_ID_VALUE, KEY_384B_VALUE);
+
+        Collection<ValidationResult> result = putAzureBlobStorage_v12.validateClientSideEncryptionProperties(validationContext);
+
+        assertValid(result);
+    }
+
+    @Test
+    public void testLocalCes512BitKeyOnProcessor() {
+        configureProcessorProperties("LOCAL", KEY_ID_VALUE, KEY_512B_VALUE);
+
+        Collection<ValidationResult> result = putAzureBlobStorage_v12.validateClientSideEncryptionProperties(validationContext);
+
+        assertValid(result);
+    }
+
+    private void configureProcessorProperties(String keyType, String keyId, String localKeyHex) {

Review Comment:
   The `keyType` should be changed to use the `ClientSideEncryptionMethod` enum instead of a string.
   ```suggestion
       private void configureProcessorProperties(ClientSideEncryptionMethod encryptionMethod, String keyId, String localKeyHex) {
   ```



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/ClientSideEncryptionSupport.java:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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;
+
+import com.azure.core.cryptography.AsyncKeyEncryptionKey;
+import com.azure.security.keyvault.keys.cryptography.KeyEncryptionKeyClientBuilder;
+import com.azure.security.keyvault.keys.cryptography.models.KeyWrapAlgorithm;
+import com.azure.security.keyvault.keys.models.JsonWebKey;
+import com.azure.security.keyvault.keys.models.KeyOperation;
+import com.azure.storage.blob.BlobClient;
+import com.azure.storage.blob.BlobContainerClient;
+import com.azure.storage.blob.specialized.cryptography.EncryptedBlobClientBuilder;
+import com.azure.storage.blob.specialized.cryptography.EncryptionVersion;
+import org.apache.commons.codec.DecoderException;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.azure.storage.utils.ClientSideEncryptionMethod;
+import org.apache.nifi.util.StringUtils;
+
+import javax.crypto.spec.SecretKeySpec;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+public interface ClientSideEncryptionSupport {
+    List<KeyOperation> KEY_OPERATIONS = Arrays.asList(KeyOperation.WRAP_KEY, KeyOperation.UNWRAP_KEY);
+
+    PropertyDescriptor CSE_KEY_TYPE = new PropertyDescriptor.Builder()
+            .name("cse-key-type")
+            .displayName("Client-Side Encryption Key Type")
+            .required(true)
+            .allowableValues(ClientSideEncryptionMethod.class)
+            .defaultValue(ClientSideEncryptionMethod.NONE.name())
+            .description("Specifies the key type to use for client-side encryption.")
+            .build();
+
+    PropertyDescriptor CSE_KEY_ID = new PropertyDescriptor.Builder()
+            .name("cse-key-id")
+            .displayName("Client-Side Encryption Key ID")
+            .description("Specifies the ID of the key to use for client-side encryption.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(CSE_KEY_TYPE, ClientSideEncryptionMethod.LOCAL.name())
+            .build();
+
+    PropertyDescriptor CSE_LOCAL_KEY_HEX = new PropertyDescriptor.Builder()
+            .name("cse-local-key-hex")
+            .displayName("Client-Side Encryption Local Key")
+            .description("When using local client-side encryption, this is the raw key, encoded in hexadecimal")
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(CSE_KEY_TYPE, ClientSideEncryptionMethod.LOCAL.name())
+            .sensitive(true)
+            .build();
+
+    default Collection<ValidationResult> validateClientSideEncryptionProperties(ValidationContext validationContext) {
+        final List<ValidationResult> validationResults = new ArrayList<>();
+        final String cseKeyTypeValue = validationContext.getProperty(CSE_KEY_TYPE).getValue();
+        final ClientSideEncryptionMethod cseKeyType = ClientSideEncryptionMethod.valueOf(cseKeyTypeValue);
+        final String cseKeyId = validationContext.getProperty(CSE_KEY_ID).getValue();
+        final String cseLocalKeyHex = validationContext.getProperty(CSE_LOCAL_KEY_HEX).getValue();
+        if (cseKeyType != ClientSideEncryptionMethod.NONE && StringUtils.isBlank(cseKeyId)) {
+            validationResults.add(new ValidationResult.Builder().subject(CSE_KEY_ID.getDisplayName())
+                    .explanation("a key ID must be set when client-side encryption is enabled.").build());
+        }
+        if (cseKeyType == ClientSideEncryptionMethod.LOCAL) {
+            validationResults.addAll(validateLocalKey(cseLocalKeyHex));
+        }
+        return validationResults;
+    }
+
+    default List<ValidationResult> validateLocalKey(String keyHex) {
+        final List<ValidationResult> validationResults = new ArrayList<>();
+        if (StringUtils.isBlank(keyHex)) {
+            validationResults.add(new ValidationResult.Builder().subject(CSE_LOCAL_KEY_HEX.getDisplayName())
+                    .explanation("a local key must be set when client-side encryption is enabled with local encryption.").build());
+        } else {
+            byte[] keyBytes;
+            try {
+                keyBytes = Hex.decodeHex(keyHex);
+                if (getKeyWrapAlgorithm(keyBytes) == null) {
+                    validationResults.add(new ValidationResult.Builder().subject(CSE_LOCAL_KEY_HEX.getDisplayName())
+                            .explanation("the local key must be 128, 192, 256, 384 or 512 bits of data.").build());
+                }
+            } catch (DecoderException e) {
+                validationResults.add(new ValidationResult.Builder().subject(CSE_LOCAL_KEY_HEX.getDisplayName())
+                        .explanation("the local key must be a valid hexadecimal string.").build());

Review Comment:
   ```suggestion
                           .explanation("Key must be a valid hexadecimal string").build());
   ```



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/ClientSideEncryptionSupport.java:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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;
+
+import com.azure.core.cryptography.AsyncKeyEncryptionKey;
+import com.azure.security.keyvault.keys.cryptography.KeyEncryptionKeyClientBuilder;
+import com.azure.security.keyvault.keys.cryptography.models.KeyWrapAlgorithm;
+import com.azure.security.keyvault.keys.models.JsonWebKey;
+import com.azure.security.keyvault.keys.models.KeyOperation;
+import com.azure.storage.blob.BlobClient;
+import com.azure.storage.blob.BlobContainerClient;
+import com.azure.storage.blob.specialized.cryptography.EncryptedBlobClientBuilder;
+import com.azure.storage.blob.specialized.cryptography.EncryptionVersion;
+import org.apache.commons.codec.DecoderException;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.azure.storage.utils.ClientSideEncryptionMethod;
+import org.apache.nifi.util.StringUtils;
+
+import javax.crypto.spec.SecretKeySpec;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+public interface ClientSideEncryptionSupport {
+    List<KeyOperation> KEY_OPERATIONS = Arrays.asList(KeyOperation.WRAP_KEY, KeyOperation.UNWRAP_KEY);
+
+    PropertyDescriptor CSE_KEY_TYPE = new PropertyDescriptor.Builder()
+            .name("cse-key-type")
+            .displayName("Client-Side Encryption Key Type")
+            .required(true)
+            .allowableValues(ClientSideEncryptionMethod.class)
+            .defaultValue(ClientSideEncryptionMethod.NONE.name())
+            .description("Specifies the key type to use for client-side encryption.")
+            .build();
+
+    PropertyDescriptor CSE_KEY_ID = new PropertyDescriptor.Builder()
+            .name("cse-key-id")
+            .displayName("Client-Side Encryption Key ID")
+            .description("Specifies the ID of the key to use for client-side encryption.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(CSE_KEY_TYPE, ClientSideEncryptionMethod.LOCAL.name())
+            .build();
+
+    PropertyDescriptor CSE_LOCAL_KEY_HEX = new PropertyDescriptor.Builder()
+            .name("cse-local-key-hex")
+            .displayName("Client-Side Encryption Local Key")
+            .description("When using local client-side encryption, this is the raw key, encoded in hexadecimal")
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(CSE_KEY_TYPE, ClientSideEncryptionMethod.LOCAL.name())
+            .sensitive(true)
+            .build();
+
+    default Collection<ValidationResult> validateClientSideEncryptionProperties(ValidationContext validationContext) {
+        final List<ValidationResult> validationResults = new ArrayList<>();
+        final String cseKeyTypeValue = validationContext.getProperty(CSE_KEY_TYPE).getValue();
+        final ClientSideEncryptionMethod cseKeyType = ClientSideEncryptionMethod.valueOf(cseKeyTypeValue);
+        final String cseKeyId = validationContext.getProperty(CSE_KEY_ID).getValue();
+        final String cseLocalKeyHex = validationContext.getProperty(CSE_LOCAL_KEY_HEX).getValue();
+        if (cseKeyType != ClientSideEncryptionMethod.NONE && StringUtils.isBlank(cseKeyId)) {
+            validationResults.add(new ValidationResult.Builder().subject(CSE_KEY_ID.getDisplayName())
+                    .explanation("a key ID must be set when client-side encryption is enabled.").build());
+        }
+        if (cseKeyType == ClientSideEncryptionMethod.LOCAL) {
+            validationResults.addAll(validateLocalKey(cseLocalKeyHex));
+        }
+        return validationResults;
+    }
+
+    default List<ValidationResult> validateLocalKey(String keyHex) {
+        final List<ValidationResult> validationResults = new ArrayList<>();
+        if (StringUtils.isBlank(keyHex)) {
+            validationResults.add(new ValidationResult.Builder().subject(CSE_LOCAL_KEY_HEX.getDisplayName())
+                    .explanation("a local key must be set when client-side encryption is enabled with local encryption.").build());
+        } else {
+            byte[] keyBytes;
+            try {
+                keyBytes = Hex.decodeHex(keyHex);
+                if (getKeyWrapAlgorithm(keyBytes) == null) {
+                    validationResults.add(new ValidationResult.Builder().subject(CSE_LOCAL_KEY_HEX.getDisplayName())
+                            .explanation("the local key must be 128, 192, 256, 384 or 512 bits of data.").build());
+                }
+            } catch (DecoderException e) {
+                validationResults.add(new ValidationResult.Builder().subject(CSE_LOCAL_KEY_HEX.getDisplayName())
+                        .explanation("the local key must be a valid hexadecimal string.").build());
+            } catch (IllegalArgumentException e) {
+                validationResults.add(new ValidationResult.Builder().subject(CSE_LOCAL_KEY_HEX.getDisplayName())
+                        .explanation(e.getMessage()).build());
+            }
+        }
+
+        return validationResults;
+    }
+
+    default boolean isClientSideEncryptionEnabled(PropertyContext context) {
+        final String cseKeyTypeValue = context.getProperty(CSE_KEY_TYPE).getValue();
+        final ClientSideEncryptionMethod cseKeyType = ClientSideEncryptionMethod.valueOf(cseKeyTypeValue);
+        return cseKeyType != ClientSideEncryptionMethod.NONE;
+    }
+
+    default BlobClient getEncryptedBlobClient(PropertyContext context, BlobContainerClient containerClient, String blobName) throws DecoderException {
+        final String cseKeyId = context.getProperty(CSE_KEY_ID).getValue();
+        final String cseLocalKeyHex = context.getProperty(CSE_LOCAL_KEY_HEX).getValue();
+        BlobClient blobClient = containerClient.getBlobClient(blobName);
+        byte[] keyBytes = Hex.decodeHex(cseLocalKeyHex);
+        JsonWebKey localKey = JsonWebKey.fromAes(new SecretKeySpec(keyBytes, "AES"), KEY_OPERATIONS)
+                .setId(cseKeyId);
+        AsyncKeyEncryptionKey akek = new KeyEncryptionKeyClientBuilder()
+                .buildAsyncKeyEncryptionKey(localKey).block();
+        return new EncryptedBlobClientBuilder(EncryptionVersion.V2)
+                .key(akek, getKeyWrapAlgorithm(keyBytes))
+                .blobClient(blobClient)
+                .buildEncryptedBlobClient();
+    }
+
+    default String getKeyWrapAlgorithm(byte[] keyBytes) {
+        final int KeySize128 = 128 >> 3;
+        final int KeySize192 = 192 >> 3;
+        final int KeySize256 = 256 >> 3;
+        final int KeySize384 = 384 >> 3;
+        final int KeySize512 = 512 >> 3;
+        switch (keyBytes.length) {
+            case KeySize128:
+                return KeyWrapAlgorithm.A128KW.toString();
+            case KeySize192:
+                return KeyWrapAlgorithm.A192KW.toString();
+            case KeySize256:
+            case KeySize512:
+            case KeySize384:
+                // Default to longest allowed key length for wrap
+                return KeyWrapAlgorithm.A256KW.toString();
+            default:
+                return null;

Review Comment:
   This condition should throw an IllegalArgumentException, or the `getKeyWrapAlgorithm` should return an `Optional<String>` to avoid potential `null` return values.



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/ClientSideEncryptionSupport.java:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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;
+
+import com.azure.core.cryptography.AsyncKeyEncryptionKey;
+import com.azure.security.keyvault.keys.cryptography.KeyEncryptionKeyClientBuilder;
+import com.azure.security.keyvault.keys.cryptography.models.KeyWrapAlgorithm;
+import com.azure.security.keyvault.keys.models.JsonWebKey;
+import com.azure.security.keyvault.keys.models.KeyOperation;
+import com.azure.storage.blob.BlobClient;
+import com.azure.storage.blob.BlobContainerClient;
+import com.azure.storage.blob.specialized.cryptography.EncryptedBlobClientBuilder;
+import com.azure.storage.blob.specialized.cryptography.EncryptionVersion;
+import org.apache.commons.codec.DecoderException;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.azure.storage.utils.ClientSideEncryptionMethod;
+import org.apache.nifi.util.StringUtils;
+
+import javax.crypto.spec.SecretKeySpec;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+public interface ClientSideEncryptionSupport {
+    List<KeyOperation> KEY_OPERATIONS = Arrays.asList(KeyOperation.WRAP_KEY, KeyOperation.UNWRAP_KEY);
+
+    PropertyDescriptor CSE_KEY_TYPE = new PropertyDescriptor.Builder()
+            .name("cse-key-type")
+            .displayName("Client-Side Encryption Key Type")
+            .required(true)
+            .allowableValues(ClientSideEncryptionMethod.class)
+            .defaultValue(ClientSideEncryptionMethod.NONE.name())
+            .description("Specifies the key type to use for client-side encryption.")
+            .build();
+
+    PropertyDescriptor CSE_KEY_ID = new PropertyDescriptor.Builder()
+            .name("cse-key-id")
+            .displayName("Client-Side Encryption Key ID")
+            .description("Specifies the ID of the key to use for client-side encryption.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(CSE_KEY_TYPE, ClientSideEncryptionMethod.LOCAL.name())
+            .build();
+
+    PropertyDescriptor CSE_LOCAL_KEY_HEX = new PropertyDescriptor.Builder()
+            .name("cse-local-key-hex")
+            .displayName("Client-Side Encryption Local Key")
+            .description("When using local client-side encryption, this is the raw key, encoded in hexadecimal")
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(CSE_KEY_TYPE, ClientSideEncryptionMethod.LOCAL.name())
+            .sensitive(true)
+            .build();
+
+    default Collection<ValidationResult> validateClientSideEncryptionProperties(ValidationContext validationContext) {
+        final List<ValidationResult> validationResults = new ArrayList<>();
+        final String cseKeyTypeValue = validationContext.getProperty(CSE_KEY_TYPE).getValue();
+        final ClientSideEncryptionMethod cseKeyType = ClientSideEncryptionMethod.valueOf(cseKeyTypeValue);
+        final String cseKeyId = validationContext.getProperty(CSE_KEY_ID).getValue();
+        final String cseLocalKeyHex = validationContext.getProperty(CSE_LOCAL_KEY_HEX).getValue();
+        if (cseKeyType != ClientSideEncryptionMethod.NONE && StringUtils.isBlank(cseKeyId)) {
+            validationResults.add(new ValidationResult.Builder().subject(CSE_KEY_ID.getDisplayName())
+                    .explanation("a key ID must be set when client-side encryption is enabled.").build());
+        }
+        if (cseKeyType == ClientSideEncryptionMethod.LOCAL) {
+            validationResults.addAll(validateLocalKey(cseLocalKeyHex));
+        }
+        return validationResults;
+    }
+
+    default List<ValidationResult> validateLocalKey(String keyHex) {
+        final List<ValidationResult> validationResults = new ArrayList<>();
+        if (StringUtils.isBlank(keyHex)) {
+            validationResults.add(new ValidationResult.Builder().subject(CSE_LOCAL_KEY_HEX.getDisplayName())
+                    .explanation("a local key must be set when client-side encryption is enabled with local encryption.").build());
+        } else {
+            byte[] keyBytes;
+            try {
+                keyBytes = Hex.decodeHex(keyHex);
+                if (getKeyWrapAlgorithm(keyBytes) == null) {
+                    validationResults.add(new ValidationResult.Builder().subject(CSE_LOCAL_KEY_HEX.getDisplayName())
+                            .explanation("the local key must be 128, 192, 256, 384 or 512 bits of data.").build());
+                }
+            } catch (DecoderException e) {
+                validationResults.add(new ValidationResult.Builder().subject(CSE_LOCAL_KEY_HEX.getDisplayName())
+                        .explanation("the local key must be a valid hexadecimal string.").build());
+            } catch (IllegalArgumentException e) {
+                validationResults.add(new ValidationResult.Builder().subject(CSE_LOCAL_KEY_HEX.getDisplayName())
+                        .explanation(e.getMessage()).build());
+            }
+        }
+
+        return validationResults;
+    }
+
+    default boolean isClientSideEncryptionEnabled(PropertyContext context) {
+        final String cseKeyTypeValue = context.getProperty(CSE_KEY_TYPE).getValue();
+        final ClientSideEncryptionMethod cseKeyType = ClientSideEncryptionMethod.valueOf(cseKeyTypeValue);
+        return cseKeyType != ClientSideEncryptionMethod.NONE;
+    }
+
+    default BlobClient getEncryptedBlobClient(PropertyContext context, BlobContainerClient containerClient, String blobName) throws DecoderException {
+        final String cseKeyId = context.getProperty(CSE_KEY_ID).getValue();
+        final String cseLocalKeyHex = context.getProperty(CSE_LOCAL_KEY_HEX).getValue();
+        BlobClient blobClient = containerClient.getBlobClient(blobName);
+        byte[] keyBytes = Hex.decodeHex(cseLocalKeyHex);
+        JsonWebKey localKey = JsonWebKey.fromAes(new SecretKeySpec(keyBytes, "AES"), KEY_OPERATIONS)
+                .setId(cseKeyId);
+        AsyncKeyEncryptionKey akek = new KeyEncryptionKeyClientBuilder()
+                .buildAsyncKeyEncryptionKey(localKey).block();
+        return new EncryptedBlobClientBuilder(EncryptionVersion.V2)
+                .key(akek, getKeyWrapAlgorithm(keyBytes))
+                .blobClient(blobClient)
+                .buildEncryptedBlobClient();
+    }
+
+    default String getKeyWrapAlgorithm(byte[] keyBytes) {
+        final int KeySize128 = 128 >> 3;
+        final int KeySize192 = 192 >> 3;
+        final int KeySize256 = 256 >> 3;
+        final int KeySize384 = 384 >> 3;
+        final int KeySize512 = 512 >> 3;

Review Comment:
   Variable names should follow standard coding conventions and the value should be declared in bytes instead of calculated.
   ```suggestion
           final int keySize128 = 16;
           final int keySize192 = 24;
           final int keySize256 = 32;
           final int keySize384 = 48;
           final int keySize512 = 64;
   ```
   



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/ClientSideEncryptionSupport.java:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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;
+
+import com.azure.core.cryptography.AsyncKeyEncryptionKey;
+import com.azure.security.keyvault.keys.cryptography.KeyEncryptionKeyClientBuilder;
+import com.azure.security.keyvault.keys.cryptography.models.KeyWrapAlgorithm;
+import com.azure.security.keyvault.keys.models.JsonWebKey;
+import com.azure.security.keyvault.keys.models.KeyOperation;
+import com.azure.storage.blob.BlobClient;
+import com.azure.storage.blob.BlobContainerClient;
+import com.azure.storage.blob.specialized.cryptography.EncryptedBlobClientBuilder;
+import com.azure.storage.blob.specialized.cryptography.EncryptionVersion;
+import org.apache.commons.codec.DecoderException;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.azure.storage.utils.ClientSideEncryptionMethod;
+import org.apache.nifi.util.StringUtils;
+
+import javax.crypto.spec.SecretKeySpec;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+public interface ClientSideEncryptionSupport {
+    List<KeyOperation> KEY_OPERATIONS = Arrays.asList(KeyOperation.WRAP_KEY, KeyOperation.UNWRAP_KEY);
+
+    PropertyDescriptor CSE_KEY_TYPE = new PropertyDescriptor.Builder()
+            .name("cse-key-type")
+            .displayName("Client-Side Encryption Key Type")
+            .required(true)
+            .allowableValues(ClientSideEncryptionMethod.class)
+            .defaultValue(ClientSideEncryptionMethod.NONE.name())
+            .description("Specifies the key type to use for client-side encryption.")
+            .build();
+
+    PropertyDescriptor CSE_KEY_ID = new PropertyDescriptor.Builder()
+            .name("cse-key-id")
+            .displayName("Client-Side Encryption Key ID")
+            .description("Specifies the ID of the key to use for client-side encryption.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(CSE_KEY_TYPE, ClientSideEncryptionMethod.LOCAL.name())
+            .build();
+
+    PropertyDescriptor CSE_LOCAL_KEY_HEX = new PropertyDescriptor.Builder()
+            .name("cse-local-key-hex")
+            .displayName("Client-Side Encryption Local Key")
+            .description("When using local client-side encryption, this is the raw key, encoded in hexadecimal")
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(CSE_KEY_TYPE, ClientSideEncryptionMethod.LOCAL.name())
+            .sensitive(true)
+            .build();
+
+    default Collection<ValidationResult> validateClientSideEncryptionProperties(ValidationContext validationContext) {
+        final List<ValidationResult> validationResults = new ArrayList<>();
+        final String cseKeyTypeValue = validationContext.getProperty(CSE_KEY_TYPE).getValue();
+        final ClientSideEncryptionMethod cseKeyType = ClientSideEncryptionMethod.valueOf(cseKeyTypeValue);
+        final String cseKeyId = validationContext.getProperty(CSE_KEY_ID).getValue();
+        final String cseLocalKeyHex = validationContext.getProperty(CSE_LOCAL_KEY_HEX).getValue();
+        if (cseKeyType != ClientSideEncryptionMethod.NONE && StringUtils.isBlank(cseKeyId)) {
+            validationResults.add(new ValidationResult.Builder().subject(CSE_KEY_ID.getDisplayName())
+                    .explanation("a key ID must be set when client-side encryption is enabled.").build());
+        }
+        if (cseKeyType == ClientSideEncryptionMethod.LOCAL) {
+            validationResults.addAll(validateLocalKey(cseLocalKeyHex));
+        }
+        return validationResults;
+    }
+
+    default List<ValidationResult> validateLocalKey(String keyHex) {
+        final List<ValidationResult> validationResults = new ArrayList<>();
+        if (StringUtils.isBlank(keyHex)) {
+            validationResults.add(new ValidationResult.Builder().subject(CSE_LOCAL_KEY_HEX.getDisplayName())
+                    .explanation("a local key must be set when client-side encryption is enabled with local encryption.").build());
+        } else {
+            byte[] keyBytes;
+            try {
+                keyBytes = Hex.decodeHex(keyHex);

Review Comment:
   The declaration and assignment can be merged.
   ```suggestion
               try {
                   final byte[] keyBytes = Hex.decodeHex(keyHex);
   ```



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/ClientSideEncryptionSupport.java:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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;
+
+import com.azure.core.cryptography.AsyncKeyEncryptionKey;
+import com.azure.security.keyvault.keys.cryptography.KeyEncryptionKeyClientBuilder;
+import com.azure.security.keyvault.keys.cryptography.models.KeyWrapAlgorithm;
+import com.azure.security.keyvault.keys.models.JsonWebKey;
+import com.azure.security.keyvault.keys.models.KeyOperation;
+import com.azure.storage.blob.BlobClient;
+import com.azure.storage.blob.BlobContainerClient;
+import com.azure.storage.blob.specialized.cryptography.EncryptedBlobClientBuilder;
+import com.azure.storage.blob.specialized.cryptography.EncryptionVersion;
+import org.apache.commons.codec.DecoderException;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.azure.storage.utils.ClientSideEncryptionMethod;
+import org.apache.nifi.util.StringUtils;
+
+import javax.crypto.spec.SecretKeySpec;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+public interface ClientSideEncryptionSupport {
+    List<KeyOperation> KEY_OPERATIONS = Arrays.asList(KeyOperation.WRAP_KEY, KeyOperation.UNWRAP_KEY);
+
+    PropertyDescriptor CSE_KEY_TYPE = new PropertyDescriptor.Builder()
+            .name("cse-key-type")
+            .displayName("Client-Side Encryption Key Type")
+            .required(true)
+            .allowableValues(ClientSideEncryptionMethod.class)
+            .defaultValue(ClientSideEncryptionMethod.NONE.name())
+            .description("Specifies the key type to use for client-side encryption.")
+            .build();
+
+    PropertyDescriptor CSE_KEY_ID = new PropertyDescriptor.Builder()
+            .name("cse-key-id")
+            .displayName("Client-Side Encryption Key ID")
+            .description("Specifies the ID of the key to use for client-side encryption.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(CSE_KEY_TYPE, ClientSideEncryptionMethod.LOCAL.name())
+            .build();
+
+    PropertyDescriptor CSE_LOCAL_KEY_HEX = new PropertyDescriptor.Builder()
+            .name("cse-local-key-hex")
+            .displayName("Client-Side Encryption Local Key")
+            .description("When using local client-side encryption, this is the raw key, encoded in hexadecimal")
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(CSE_KEY_TYPE, ClientSideEncryptionMethod.LOCAL.name())

Review Comment:
   ```suggestion
               .dependsOn(CSE_KEY_TYPE, ClientSideEncryptionMethod.LOCAL.getValue())
   ```



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/ClientSideEncryptionSupport.java:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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;
+
+import com.azure.core.cryptography.AsyncKeyEncryptionKey;
+import com.azure.security.keyvault.keys.cryptography.KeyEncryptionKeyClientBuilder;
+import com.azure.security.keyvault.keys.cryptography.models.KeyWrapAlgorithm;
+import com.azure.security.keyvault.keys.models.JsonWebKey;
+import com.azure.security.keyvault.keys.models.KeyOperation;
+import com.azure.storage.blob.BlobClient;
+import com.azure.storage.blob.BlobContainerClient;
+import com.azure.storage.blob.specialized.cryptography.EncryptedBlobClientBuilder;
+import com.azure.storage.blob.specialized.cryptography.EncryptionVersion;
+import org.apache.commons.codec.DecoderException;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.azure.storage.utils.ClientSideEncryptionMethod;
+import org.apache.nifi.util.StringUtils;
+
+import javax.crypto.spec.SecretKeySpec;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+public interface ClientSideEncryptionSupport {
+    List<KeyOperation> KEY_OPERATIONS = Arrays.asList(KeyOperation.WRAP_KEY, KeyOperation.UNWRAP_KEY);
+
+    PropertyDescriptor CSE_KEY_TYPE = new PropertyDescriptor.Builder()
+            .name("cse-key-type")
+            .displayName("Client-Side Encryption Key Type")
+            .required(true)
+            .allowableValues(ClientSideEncryptionMethod.class)
+            .defaultValue(ClientSideEncryptionMethod.NONE.name())

Review Comment:
   ```suggestion
               .defaultValue(ClientSideEncryptionMethod.NONE.getValue())
   ```



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/ClientSideEncryptionSupport.java:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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;
+
+import com.azure.core.cryptography.AsyncKeyEncryptionKey;
+import com.azure.security.keyvault.keys.cryptography.KeyEncryptionKeyClientBuilder;
+import com.azure.security.keyvault.keys.cryptography.models.KeyWrapAlgorithm;
+import com.azure.security.keyvault.keys.models.JsonWebKey;
+import com.azure.security.keyvault.keys.models.KeyOperation;
+import com.azure.storage.blob.BlobClient;
+import com.azure.storage.blob.BlobContainerClient;
+import com.azure.storage.blob.specialized.cryptography.EncryptedBlobClientBuilder;
+import com.azure.storage.blob.specialized.cryptography.EncryptionVersion;
+import org.apache.commons.codec.DecoderException;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.azure.storage.utils.ClientSideEncryptionMethod;
+import org.apache.nifi.util.StringUtils;
+
+import javax.crypto.spec.SecretKeySpec;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+public interface ClientSideEncryptionSupport {
+    List<KeyOperation> KEY_OPERATIONS = Arrays.asList(KeyOperation.WRAP_KEY, KeyOperation.UNWRAP_KEY);
+
+    PropertyDescriptor CSE_KEY_TYPE = new PropertyDescriptor.Builder()
+            .name("cse-key-type")

Review Comment:
   Recommend using the same value for the property name since these are new properties.
   ```suggestion
               .name("Client-Side Encryption Key Type")
   ```



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/ClientSideEncryptionSupport.java:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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;
+
+import com.azure.core.cryptography.AsyncKeyEncryptionKey;
+import com.azure.security.keyvault.keys.cryptography.KeyEncryptionKeyClientBuilder;
+import com.azure.security.keyvault.keys.cryptography.models.KeyWrapAlgorithm;
+import com.azure.security.keyvault.keys.models.JsonWebKey;
+import com.azure.security.keyvault.keys.models.KeyOperation;
+import com.azure.storage.blob.BlobClient;
+import com.azure.storage.blob.BlobContainerClient;
+import com.azure.storage.blob.specialized.cryptography.EncryptedBlobClientBuilder;
+import com.azure.storage.blob.specialized.cryptography.EncryptionVersion;
+import org.apache.commons.codec.DecoderException;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.azure.storage.utils.ClientSideEncryptionMethod;
+import org.apache.nifi.util.StringUtils;
+
+import javax.crypto.spec.SecretKeySpec;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+public interface ClientSideEncryptionSupport {
+    List<KeyOperation> KEY_OPERATIONS = Arrays.asList(KeyOperation.WRAP_KEY, KeyOperation.UNWRAP_KEY);
+
+    PropertyDescriptor CSE_KEY_TYPE = new PropertyDescriptor.Builder()
+            .name("cse-key-type")
+            .displayName("Client-Side Encryption Key Type")
+            .required(true)
+            .allowableValues(ClientSideEncryptionMethod.class)
+            .defaultValue(ClientSideEncryptionMethod.NONE.name())
+            .description("Specifies the key type to use for client-side encryption.")
+            .build();
+
+    PropertyDescriptor CSE_KEY_ID = new PropertyDescriptor.Builder()
+            .name("cse-key-id")
+            .displayName("Client-Side Encryption Key ID")
+            .description("Specifies the ID of the key to use for client-side encryption.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(CSE_KEY_TYPE, ClientSideEncryptionMethod.LOCAL.name())

Review Comment:
   To avoid potential future confusion, `name()` should be changed to `getValue()`.
   ```suggestion
               .dependsOn(CSE_KEY_TYPE, ClientSideEncryptionMethod.LOCAL.getValue())
   ```



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/ClientSideEncryptionSupport.java:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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;
+
+import com.azure.core.cryptography.AsyncKeyEncryptionKey;
+import com.azure.security.keyvault.keys.cryptography.KeyEncryptionKeyClientBuilder;
+import com.azure.security.keyvault.keys.cryptography.models.KeyWrapAlgorithm;
+import com.azure.security.keyvault.keys.models.JsonWebKey;
+import com.azure.security.keyvault.keys.models.KeyOperation;
+import com.azure.storage.blob.BlobClient;
+import com.azure.storage.blob.BlobContainerClient;
+import com.azure.storage.blob.specialized.cryptography.EncryptedBlobClientBuilder;
+import com.azure.storage.blob.specialized.cryptography.EncryptionVersion;
+import org.apache.commons.codec.DecoderException;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.azure.storage.utils.ClientSideEncryptionMethod;
+import org.apache.nifi.util.StringUtils;
+
+import javax.crypto.spec.SecretKeySpec;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+public interface ClientSideEncryptionSupport {
+    List<KeyOperation> KEY_OPERATIONS = Arrays.asList(KeyOperation.WRAP_KEY, KeyOperation.UNWRAP_KEY);
+
+    PropertyDescriptor CSE_KEY_TYPE = new PropertyDescriptor.Builder()
+            .name("cse-key-type")
+            .displayName("Client-Side Encryption Key Type")
+            .required(true)
+            .allowableValues(ClientSideEncryptionMethod.class)
+            .defaultValue(ClientSideEncryptionMethod.NONE.name())
+            .description("Specifies the key type to use for client-side encryption.")
+            .build();
+
+    PropertyDescriptor CSE_KEY_ID = new PropertyDescriptor.Builder()
+            .name("cse-key-id")
+            .displayName("Client-Side Encryption Key ID")
+            .description("Specifies the ID of the key to use for client-side encryption.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(CSE_KEY_TYPE, ClientSideEncryptionMethod.LOCAL.name())
+            .build();
+
+    PropertyDescriptor CSE_LOCAL_KEY_HEX = new PropertyDescriptor.Builder()

Review Comment:
   ```suggestion
       PropertyDescriptor CSE_LOCAL_KEY = new PropertyDescriptor.Builder()
   ```



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/ClientSideEncryptionSupport.java:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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;
+
+import com.azure.core.cryptography.AsyncKeyEncryptionKey;
+import com.azure.security.keyvault.keys.cryptography.KeyEncryptionKeyClientBuilder;
+import com.azure.security.keyvault.keys.cryptography.models.KeyWrapAlgorithm;
+import com.azure.security.keyvault.keys.models.JsonWebKey;
+import com.azure.security.keyvault.keys.models.KeyOperation;
+import com.azure.storage.blob.BlobClient;
+import com.azure.storage.blob.BlobContainerClient;
+import com.azure.storage.blob.specialized.cryptography.EncryptedBlobClientBuilder;
+import com.azure.storage.blob.specialized.cryptography.EncryptionVersion;
+import org.apache.commons.codec.DecoderException;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.azure.storage.utils.ClientSideEncryptionMethod;
+import org.apache.nifi.util.StringUtils;
+
+import javax.crypto.spec.SecretKeySpec;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+public interface ClientSideEncryptionSupport {
+    List<KeyOperation> KEY_OPERATIONS = Arrays.asList(KeyOperation.WRAP_KEY, KeyOperation.UNWRAP_KEY);
+
+    PropertyDescriptor CSE_KEY_TYPE = new PropertyDescriptor.Builder()
+            .name("cse-key-type")
+            .displayName("Client-Side Encryption Key Type")
+            .required(true)
+            .allowableValues(ClientSideEncryptionMethod.class)
+            .defaultValue(ClientSideEncryptionMethod.NONE.name())
+            .description("Specifies the key type to use for client-side encryption.")
+            .build();
+
+    PropertyDescriptor CSE_KEY_ID = new PropertyDescriptor.Builder()
+            .name("cse-key-id")
+            .displayName("Client-Side Encryption Key ID")
+            .description("Specifies the ID of the key to use for client-side encryption.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(CSE_KEY_TYPE, ClientSideEncryptionMethod.LOCAL.name())
+            .build();
+
+    PropertyDescriptor CSE_LOCAL_KEY_HEX = new PropertyDescriptor.Builder()
+            .name("cse-local-key-hex")

Review Comment:
   ```suggestion
               .name("Client-Side Encryption Local Key")
   ```



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/ClientSideEncryptionSupport.java:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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;
+
+import com.azure.core.cryptography.AsyncKeyEncryptionKey;
+import com.azure.security.keyvault.keys.cryptography.KeyEncryptionKeyClientBuilder;
+import com.azure.security.keyvault.keys.cryptography.models.KeyWrapAlgorithm;
+import com.azure.security.keyvault.keys.models.JsonWebKey;
+import com.azure.security.keyvault.keys.models.KeyOperation;
+import com.azure.storage.blob.BlobClient;
+import com.azure.storage.blob.BlobContainerClient;
+import com.azure.storage.blob.specialized.cryptography.EncryptedBlobClientBuilder;
+import com.azure.storage.blob.specialized.cryptography.EncryptionVersion;
+import org.apache.commons.codec.DecoderException;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.azure.storage.utils.ClientSideEncryptionMethod;
+import org.apache.nifi.util.StringUtils;
+
+import javax.crypto.spec.SecretKeySpec;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+public interface ClientSideEncryptionSupport {
+    List<KeyOperation> KEY_OPERATIONS = Arrays.asList(KeyOperation.WRAP_KEY, KeyOperation.UNWRAP_KEY);
+
+    PropertyDescriptor CSE_KEY_TYPE = new PropertyDescriptor.Builder()
+            .name("cse-key-type")
+            .displayName("Client-Side Encryption Key Type")
+            .required(true)
+            .allowableValues(ClientSideEncryptionMethod.class)
+            .defaultValue(ClientSideEncryptionMethod.NONE.name())
+            .description("Specifies the key type to use for client-side encryption.")
+            .build();
+
+    PropertyDescriptor CSE_KEY_ID = new PropertyDescriptor.Builder()
+            .name("cse-key-id")
+            .displayName("Client-Side Encryption Key ID")
+            .description("Specifies the ID of the key to use for client-side encryption.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(CSE_KEY_TYPE, ClientSideEncryptionMethod.LOCAL.name())
+            .build();
+
+    PropertyDescriptor CSE_LOCAL_KEY_HEX = new PropertyDescriptor.Builder()
+            .name("cse-local-key-hex")
+            .displayName("Client-Side Encryption Local Key")
+            .description("When using local client-side encryption, this is the raw key, encoded in hexadecimal")
+            .required(false)

Review Comment:
   ```suggestion
               .required(true)
   ```



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/storage/AbstractAzureBlobStorage_v12IT.java:
##########
@@ -128,6 +144,26 @@ protected BlobClient uploadBlob(String blobName, byte[] blobData) throws Excepti
         return blobClient;
     }
 
+    protected BlobClient uploadBlobWithCSE(String blobName, byte[] blobData, String hexKey, String keyId, String keyWrapAlgorithm) throws Exception {

Review Comment:
   Thanks for the reply, keeping them sounds reasonable under the circumstances.



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/ClientSideEncryptionSupport.java:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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;
+
+import com.azure.core.cryptography.AsyncKeyEncryptionKey;
+import com.azure.security.keyvault.keys.cryptography.KeyEncryptionKeyClientBuilder;
+import com.azure.security.keyvault.keys.cryptography.models.KeyWrapAlgorithm;
+import com.azure.security.keyvault.keys.models.JsonWebKey;
+import com.azure.security.keyvault.keys.models.KeyOperation;
+import com.azure.storage.blob.BlobClient;
+import com.azure.storage.blob.BlobContainerClient;
+import com.azure.storage.blob.specialized.cryptography.EncryptedBlobClientBuilder;
+import com.azure.storage.blob.specialized.cryptography.EncryptionVersion;
+import org.apache.commons.codec.DecoderException;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.azure.storage.utils.ClientSideEncryptionMethod;
+import org.apache.nifi.util.StringUtils;
+
+import javax.crypto.spec.SecretKeySpec;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+public interface ClientSideEncryptionSupport {
+    List<KeyOperation> KEY_OPERATIONS = Arrays.asList(KeyOperation.WRAP_KEY, KeyOperation.UNWRAP_KEY);
+
+    PropertyDescriptor CSE_KEY_TYPE = new PropertyDescriptor.Builder()
+            .name("cse-key-type")
+            .displayName("Client-Side Encryption Key Type")
+            .required(true)
+            .allowableValues(ClientSideEncryptionMethod.class)
+            .defaultValue(ClientSideEncryptionMethod.NONE.name())
+            .description("Specifies the key type to use for client-side encryption.")
+            .build();
+
+    PropertyDescriptor CSE_KEY_ID = new PropertyDescriptor.Builder()
+            .name("cse-key-id")
+            .displayName("Client-Side Encryption Key ID")
+            .description("Specifies the ID of the key to use for client-side encryption.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)

Review Comment:
   This can be changed to `required(true)` since it depends on the Local Type, which should remove the need for custom validation.



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/FetchAzureBlobStorage_v12.java:
##########
@@ -114,9 +119,19 @@ public class FetchAzureBlobStorage_v12 extends AbstractAzureBlobProcessor_v12 {
             BLOB_NAME,
             RANGE_START,
             RANGE_LENGTH,
-            AzureStorageUtils.PROXY_CONFIGURATION_SERVICE
+            AzureStorageUtils.PROXY_CONFIGURATION_SERVICE,
+            CSE_KEY_ID,
+            CSE_KEY_TYPE,

Review Comment:
   ```suggestion
               CSE_KEY_TYPE,
               CSE_KEY_ID,
   ```



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

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

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


[GitHub] [nifi] exceptionfactory commented on pull request #7182: NIFI-11360 Support Client-Side Encryption for Azure Blob v12 Processors

Posted by "exceptionfactory (via GitHub)" <gi...@apache.org>.
exceptionfactory commented on PR #7182:
URL: https://github.com/apache/nifi/pull/7182#issuecomment-1530060143

   @mkalavala I also cherry-picked the fix from NIFI-11475 on to the pull request branch to correct a missing runtime dependency on jackson-dataformat-xml, which is already corrected in the main branch.


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

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

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


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #7182: NIFI-11360 Support Client-Side Encryption for Azure Blob v12 Processors

Posted by "exceptionfactory (via GitHub)" <gi...@apache.org>.
exceptionfactory commented on code in PR #7182:
URL: https://github.com/apache/nifi/pull/7182#discussion_r1177018700


##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/AbstractAzureBlobProcessor_v12.java:
##########
@@ -113,6 +131,59 @@ protected BlobServiceClient getStorageClient() {
         return storageClient;
     }
 
+    protected BlobClient getBlobClient(PropertyContext context, BlobContainerClient containerClient, String blobName) throws DecoderException {
+        final String cseKeyTypeValue = context.getProperty(AzureBlobClientSideEncryptionUtils_v12.CSE_KEY_TYPE).getValue();
+        final AzureBlobClientSideEncryptionMethod_v12 cseKeyType = AzureBlobClientSideEncryptionMethod_v12.valueOf(cseKeyTypeValue);
+
+        final String cseKeyId = context.getProperty(AzureBlobClientSideEncryptionUtils_v12.CSE_KEY_ID).getValue();
+
+        final String cseLocalKeyHex = context.getProperty(AzureBlobClientSideEncryptionUtils_v12.CSE_LOCAL_KEY_HEX).getValue();
+
+        BlobClient blobClient = containerClient.getBlobClient(blobName);
+
+        if (cseKeyType == AzureBlobClientSideEncryptionMethod_v12.LOCAL) {
+            byte[] keyBytes = Hex.decodeHex(cseLocalKeyHex.toCharArray());
+            JsonWebKey localKey = JsonWebKey.fromAes(new SecretKeySpec(keyBytes, "AES"),
+                    Arrays.asList(KeyOperation.WRAP_KEY, KeyOperation.UNWRAP_KEY))

Review Comment:
   The array of KeyOperations can be declared statically and reused.



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/utils/AzureBlobClientSideEncryptionMethod_v12.java:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.storage.utils;
+
+/**
+ * Enumeration capturing essential information about the various client-side
+ * encryption methods supported by Azure
+ */
+public enum AzureBlobClientSideEncryptionMethod_v12 {
+
+    NONE("None", "The blobs sent to Azure are not encrypted."),
+    LOCAL("Local", "The blobs sent to Azure are encrypted using a local key.");

Review Comment:
   ```suggestion
       LOCAL("Client-Side Encryption enabled using local key");
   ```



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/utils/AzureBlobClientSideEncryptionUtils_v12.java:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.storage.utils;
+
+import org.apache.commons.codec.DecoderException;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.nifi.components.AllowableValue;
+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;
+import org.apache.nifi.util.StringUtils;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+public class AzureBlobClientSideEncryptionUtils_v12 {

Review Comment:
   Similar to the enum naming, recommend a simplified name for this class.
   ```suggestion
   public class ClientSideEncryptionUtils {
   ```



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/utils/AzureBlobClientSideEncryptionUtils_v12.java:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.storage.utils;
+
+import org.apache.commons.codec.DecoderException;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.nifi.components.AllowableValue;
+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;
+import org.apache.nifi.util.StringUtils;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+public class AzureBlobClientSideEncryptionUtils_v12 {
+
+    private static final String DEFAULT_KEY_ID = "nifi";
+    public static final int KeySize128 = 128 >> 3;
+    public static final int KeySize192 = 192 >> 3;
+    public static final int KeySize256 = 256 >> 3;
+    public static final int KeySize384 = 384 >> 3;
+    public static final int KeySize512 = 512 >> 3;
+
+    public static final PropertyDescriptor CSE_KEY_TYPE = new PropertyDescriptor.Builder()
+            .name("cse-key-type")
+            .displayName("Client-Side Encryption Key Type")

Review Comment:
   Although the conventions are not consistent, in this case, the new properties can have the same `name` and `displayName`.
   ```suggestion
               .name("Client-Side Encryption Key Type")
               .displayName("Client-Side Encryption Key Type")
   ```



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/utils/AzureBlobClientSideEncryptionUtils_v12.java:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.storage.utils;
+
+import org.apache.commons.codec.DecoderException;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.nifi.components.AllowableValue;
+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;
+import org.apache.nifi.util.StringUtils;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+public class AzureBlobClientSideEncryptionUtils_v12 {
+
+    private static final String DEFAULT_KEY_ID = "nifi";
+    public static final int KeySize128 = 128 >> 3;
+    public static final int KeySize192 = 192 >> 3;
+    public static final int KeySize256 = 256 >> 3;
+    public static final int KeySize384 = 384 >> 3;
+    public static final int KeySize512 = 512 >> 3;
+
+    public static final PropertyDescriptor CSE_KEY_TYPE = new PropertyDescriptor.Builder()
+            .name("cse-key-type")
+            .displayName("Client-Side Encryption Key Type")
+            .required(true)
+            .allowableValues(buildCseEncryptionMethodAllowableValues())
+            .defaultValue(AzureBlobClientSideEncryptionMethod_v12.NONE.name())
+            .description("Specifies the key type to use for client-side encryption.")
+            .build();
+
+    public static final PropertyDescriptor CSE_KEY_ID = new PropertyDescriptor.Builder()
+            .name("cse-key-id")
+            .displayName("Client-Side Encryption Key ID")
+            .description("Specifies the ID of the key to use for client-side encryption.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(CSE_KEY_TYPE, AzureBlobClientSideEncryptionMethod_v12.LOCAL.name())
+            .build();
+
+    public static final PropertyDescriptor CSE_LOCAL_KEY_HEX = new PropertyDescriptor.Builder()
+            .name("cse-local-key-hex")
+            .displayName("Local Key")
+            .description("When using local client-side encryption, this is the raw key, encoded in hexadecimal")
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(CSE_KEY_TYPE, AzureBlobClientSideEncryptionMethod_v12.LOCAL.name())
+            .sensitive(true)
+            .build();
+
+    private static AllowableValue[] buildCseEncryptionMethodAllowableValues() {
+        return Arrays.stream(AzureBlobClientSideEncryptionMethod_v12.values())
+            .map(v -> new AllowableValue(v.name(), v.name(), v.getDescription()))
+            .toArray(AllowableValue[]::new);
+    }

Review Comment:
   As mentioned above, this method can be removed and replaced with an approached based on implemented `DescribedValue` in the `ClientSideEncryptionMethod` enum.



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/storage/AbstractAzureBlobStorage_v12IT.java:
##########
@@ -128,6 +144,26 @@ protected BlobClient uploadBlob(String blobName, byte[] blobData) throws Excepti
         return blobClient;
     }
 
+    protected BlobClient uploadBlobWithCSE(String blobName, byte[] blobData, String hexKey, String keyId, String keyWrapAlgorithm) throws Exception {

Review Comment:
   The Integration Tests are not run automatically, so I recommend avoiding adding new integration tests for these changes.



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/utils/AzureBlobClientSideEncryptionMethod_v12.java:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.storage.utils;
+
+/**
+ * Enumeration capturing essential information about the various client-side
+ * encryption methods supported by Azure
+ */
+public enum AzureBlobClientSideEncryptionMethod_v12 {
+
+    NONE("None", "The blobs sent to Azure are not encrypted."),

Review Comment:
   I recommend removing the `None` label and instead using the enum value itself. I also recommend adjusting the description.
   ```suggestion
       NONE("Client-Side Encryption disabled"),
   ```



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/utils/AzureBlobClientSideEncryptionUtils_v12.java:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.storage.utils;
+
+import org.apache.commons.codec.DecoderException;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.nifi.components.AllowableValue;
+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;
+import org.apache.nifi.util.StringUtils;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+public class AzureBlobClientSideEncryptionUtils_v12 {
+
+    private static final String DEFAULT_KEY_ID = "nifi";
+    public static final int KeySize128 = 128 >> 3;
+    public static final int KeySize192 = 192 >> 3;
+    public static final int KeySize256 = 256 >> 3;
+    public static final int KeySize384 = 384 >> 3;
+    public static final int KeySize512 = 512 >> 3;
+
+    public static final PropertyDescriptor CSE_KEY_TYPE = new PropertyDescriptor.Builder()
+            .name("cse-key-type")
+            .displayName("Client-Side Encryption Key Type")
+            .required(true)
+            .allowableValues(buildCseEncryptionMethodAllowableValues())
+            .defaultValue(AzureBlobClientSideEncryptionMethod_v12.NONE.name())
+            .description("Specifies the key type to use for client-side encryption.")
+            .build();
+
+    public static final PropertyDescriptor CSE_KEY_ID = new PropertyDescriptor.Builder()
+            .name("cse-key-id")
+            .displayName("Client-Side Encryption Key ID")
+            .description("Specifies the ID of the key to use for client-side encryption.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(CSE_KEY_TYPE, AzureBlobClientSideEncryptionMethod_v12.LOCAL.name())
+            .build();
+
+    public static final PropertyDescriptor CSE_LOCAL_KEY_HEX = new PropertyDescriptor.Builder()
+            .name("cse-local-key-hex")
+            .displayName("Local Key")

Review Comment:
   ```suggestion
               .displayName("Client-Side Encryption Local Key")
   ```



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/utils/AzureBlobClientSideEncryptionUtils_v12.java:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.storage.utils;
+
+import org.apache.commons.codec.DecoderException;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.nifi.components.AllowableValue;
+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;
+import org.apache.nifi.util.StringUtils;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+public class AzureBlobClientSideEncryptionUtils_v12 {
+
+    private static final String DEFAULT_KEY_ID = "nifi";
+    public static final int KeySize128 = 128 >> 3;
+    public static final int KeySize192 = 192 >> 3;
+    public static final int KeySize256 = 256 >> 3;
+    public static final int KeySize384 = 384 >> 3;
+    public static final int KeySize512 = 512 >> 3;
+
+    public static final PropertyDescriptor CSE_KEY_TYPE = new PropertyDescriptor.Builder()
+            .name("cse-key-type")
+            .displayName("Client-Side Encryption Key Type")
+            .required(true)
+            .allowableValues(buildCseEncryptionMethodAllowableValues())
+            .defaultValue(AzureBlobClientSideEncryptionMethod_v12.NONE.name())
+            .description("Specifies the key type to use for client-side encryption.")
+            .build();
+
+    public static final PropertyDescriptor CSE_KEY_ID = new PropertyDescriptor.Builder()
+            .name("cse-key-id")
+            .displayName("Client-Side Encryption Key ID")
+            .description("Specifies the ID of the key to use for client-side encryption.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(CSE_KEY_TYPE, AzureBlobClientSideEncryptionMethod_v12.LOCAL.name())
+            .build();
+
+    public static final PropertyDescriptor CSE_LOCAL_KEY_HEX = new PropertyDescriptor.Builder()
+            .name("cse-local-key-hex")
+            .displayName("Local Key")
+            .description("When using local client-side encryption, this is the raw key, encoded in hexadecimal")
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(CSE_KEY_TYPE, AzureBlobClientSideEncryptionMethod_v12.LOCAL.name())
+            .sensitive(true)
+            .build();

Review Comment:
   Instead of defining these property descriptors on the `Utils` class, recommend creating a new interface named `ClientSideEncryptionSupport` and adding the Property Descriptors on that interface. Then the implementing Processors and implement the `ClientSideEncryptionSupport` interface to access the properties.



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/utils/AzureBlobClientSideEncryptionMethod_v12.java:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.storage.utils;
+
+/**
+ * Enumeration capturing essential information about the various client-side
+ * encryption methods supported by Azure
+ */
+public enum AzureBlobClientSideEncryptionMethod_v12 {

Review Comment:
   Although this follows the pattern of the Processor names, this can be named more simply. The `AzureBlob` prefix seems unnecessary, so I recommend `ClientSideEncryptionMethod`.
   ```suggestion
   public enum ClientSideEncryptionMethod {
   ```



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/utils/AzureBlobClientSideEncryptionUtils_v12.java:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.storage.utils;
+
+import org.apache.commons.codec.DecoderException;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.nifi.components.AllowableValue;
+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;
+import org.apache.nifi.util.StringUtils;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+public class AzureBlobClientSideEncryptionUtils_v12 {
+
+    private static final String DEFAULT_KEY_ID = "nifi";
+    public static final int KeySize128 = 128 >> 3;
+    public static final int KeySize192 = 192 >> 3;
+    public static final int KeySize256 = 256 >> 3;
+    public static final int KeySize384 = 384 >> 3;
+    public static final int KeySize512 = 512 >> 3;

Review Comment:
   Although this follows a pattern of earlier versions, these static fields do not follow standard naming conventions. These key sizes should not be exposed as public variables, and instead, the method for determining the algorithm should be encapsulated in this class.



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/AbstractAzureBlobProcessor_v12.java:
##########
@@ -113,6 +131,59 @@ protected BlobServiceClient getStorageClient() {
         return storageClient;
     }
 
+    protected BlobClient getBlobClient(PropertyContext context, BlobContainerClient containerClient, String blobName) throws DecoderException {
+        final String cseKeyTypeValue = context.getProperty(AzureBlobClientSideEncryptionUtils_v12.CSE_KEY_TYPE).getValue();
+        final AzureBlobClientSideEncryptionMethod_v12 cseKeyType = AzureBlobClientSideEncryptionMethod_v12.valueOf(cseKeyTypeValue);
+
+        final String cseKeyId = context.getProperty(AzureBlobClientSideEncryptionUtils_v12.CSE_KEY_ID).getValue();
+
+        final String cseLocalKeyHex = context.getProperty(AzureBlobClientSideEncryptionUtils_v12.CSE_LOCAL_KEY_HEX).getValue();
+
+        BlobClient blobClient = containerClient.getBlobClient(blobName);
+
+        if (cseKeyType == AzureBlobClientSideEncryptionMethod_v12.LOCAL) {
+            byte[] keyBytes = Hex.decodeHex(cseLocalKeyHex.toCharArray());

Review Comment:
   The `toCharArray()` is unnecessary.
   ```suggestion
               byte[] keyBytes = Hex.decodeHex(cseLocalKeyHex);
   ```



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/utils/AzureBlobClientSideEncryptionUtils_v12.java:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.storage.utils;
+
+import org.apache.commons.codec.DecoderException;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.nifi.components.AllowableValue;
+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;
+import org.apache.nifi.util.StringUtils;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+public class AzureBlobClientSideEncryptionUtils_v12 {
+
+    private static final String DEFAULT_KEY_ID = "nifi";
+    public static final int KeySize128 = 128 >> 3;
+    public static final int KeySize192 = 192 >> 3;
+    public static final int KeySize256 = 256 >> 3;
+    public static final int KeySize384 = 384 >> 3;
+    public static final int KeySize512 = 512 >> 3;
+
+    public static final PropertyDescriptor CSE_KEY_TYPE = new PropertyDescriptor.Builder()
+            .name("cse-key-type")
+            .displayName("Client-Side Encryption Key Type")
+            .required(true)
+            .allowableValues(buildCseEncryptionMethodAllowableValues())

Review Comment:
   Instead of this method-based approach, the `enum` can implemented the `DescribedValue` interface, and the enum class reference can be provided to the `allowableValues()` builder method.



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

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

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


[GitHub] [nifi] mkalavala commented on a diff in pull request #7182: NIFI-11360 Support Client-Side Encryption for Azure Blob v12 Processors

Posted by "mkalavala (via GitHub)" <gi...@apache.org>.
mkalavala commented on code in PR #7182:
URL: https://github.com/apache/nifi/pull/7182#discussion_r1179429970


##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/utils/AzureBlobClientSideEncryptionUtils_v12.java:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.storage.utils;
+
+import org.apache.commons.codec.DecoderException;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.nifi.components.AllowableValue;
+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;
+import org.apache.nifi.util.StringUtils;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+public class AzureBlobClientSideEncryptionUtils_v12 {

Review Comment:
   I replaced utils class with interface ClientSideEncryptionSupport



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

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

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


[GitHub] [nifi] mkalavala commented on a diff in pull request #7182: NIFI-11360 Support Client-Side Encryption for Azure Blob v12 Processors

Posted by "mkalavala (via GitHub)" <gi...@apache.org>.
mkalavala commented on code in PR #7182:
URL: https://github.com/apache/nifi/pull/7182#discussion_r1179435232


##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/storage/AbstractAzureBlobStorage_v12IT.java:
##########
@@ -128,6 +144,26 @@ protected BlobClient uploadBlob(String blobName, byte[] blobData) throws Excepti
         return blobClient;
     }
 
+    protected BlobClient uploadBlobWithCSE(String blobName, byte[] blobData, String hexKey, String keyId, String keyWrapAlgorithm) throws Exception {

Review Comment:
   I would suggest to keep them as they helped me in identifying that KeyWrapAlgorithm cannot be constant and needs to derived from key size. Also helped me in verifying different key algorithm sizes



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

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

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