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

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

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