You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2022/10/19 14:01:40 UTC

[GitHub] [nifi] exceptionfactory opened a new pull request, #6553: NIFI-10667 Add Private Key Service interface and implementation

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

   # Summary
   
   [NIFI-10667](https://issues.apache.org/jira/browse/NIFI-10667) Adds a new `PrivateKeyService` Controller Service interface and `StandardPrivateKeyService` implementation capable of reading encrypted or unencrypted Private Keys.
   
   The service interface is defined in a new `nifi-key-service-api` module under `nifi-standard-services`. The standard implementation uses the Bouncy Castle library and `PEMParser` class to handle decoding and decrypting Private Keys. The initial implementation supports [PEM](https://en.wikipedia.org/wiki/Privacy-Enhanced_Mail) encoding and [PKCS 8](https://en.wikipedia.org/wiki/PKCS_8), which can be used for several standard key algorithms including RSA and DSA.
   
   # 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
   - [X] 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 8
     - [ ] JDK 11
     - [ ] JDK 17
   
   ### Licensing
   
   - [ ] New dependencies are compatible with the [Apache License 2.0](https://apache.org/licenses/LICENSE-2.0) according to the [License Policy](https://www.apache.org/legal/resolved.html)
   - [ ] New dependencies are documented in applicable `LICENSE` and `NOTICE` files
   
   ### Documentation
   
   - [ ] Documentation formatting appears as expected in rendered files
   


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

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

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


[GitHub] [nifi] turcsanyip commented on a diff in pull request #6553: NIFI-10667 Add Private Key Service interface and implementation

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


##########
nifi-nar-bundles/nifi-standard-services/nifi-key-service-bundle/nifi-key-service/src/main/java/org/apache/nifi/key/service/StandardPrivateKeyService.java:
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.key.service;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.resource.ResourceCardinality;
+import org.apache.nifi.components.resource.ResourceType;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.key.service.api.PrivateKeyService;
+import org.apache.nifi.key.service.reader.BouncyCastlePrivateKeyReader;
+import org.apache.nifi.key.service.reader.PrivateKeyReader;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.UncheckedIOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.PrivateKey;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * Standard implementation of Private Key Service supporting encrypted or unencrypted sources
+ */
+@Tags({"PEM", "PKCS8"})
+@CapabilityDescription("Private Key Service provides access to a Private Key loaded from configured sources")
+public class StandardPrivateKeyService extends AbstractControllerService implements PrivateKeyService {
+    public static final PropertyDescriptor KEY_FILE = new PropertyDescriptor.Builder()
+            .name("key-file")
+            .displayName("Key File")
+            .description("File path to Private Key structured using PKCS8 and encoded as PEM")
+            .required(false)
+            .identifiesExternalResource(ResourceCardinality.SINGLE, ResourceType.FILE)
+            .build();
+
+    public static final PropertyDescriptor KEY = new PropertyDescriptor.Builder()
+            .name("key")
+            .displayName("Key")
+            .description("Private Key structured using PKCS8 and encoded as PEM")
+            .required(false)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor KEY_PASSWORD = new PropertyDescriptor.Builder()
+            .name("key-password")
+            .displayName("Key Password")
+            .description("Password used for decrypting Private Keys")
+            .required(false)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    private static final List<PropertyDescriptor> DESCRIPTORS = Arrays.asList(
+            KEY_FILE,
+            KEY,
+            KEY_PASSWORD
+    );
+
+    private static final Charset KEY_CHARACTER_SET = StandardCharsets.US_ASCII;
+
+    private static final PrivateKeyReader PRIVATE_KEY_READER = new BouncyCastlePrivateKeyReader();
+
+    private final AtomicReference<PrivateKey> keyReference = new AtomicReference<>();
+
+    @Override
+    public PrivateKey getPrivateKey() {
+        return keyReference.get();
+    }
+
+    /**
+     * On Enabled reads Private Keys using configured properties
+     *
+     * @param context Configuration Context with properties
+     * @throws InitializationException Thrown when unable to load
+     */
+    @OnEnabled
+    public void onEnabled(final ConfigurationContext context) throws InitializationException {
+        try {
+            final PrivateKey readKey = readKey(context);
+            keyReference.set(readKey);
+        } catch (final RuntimeException e) {
+            throw new InitializationException("Reading Private Key Failed", e);
+        }
+    }
+
+    /**
+     * On Disabled clears Private Keys
+     */
+    @OnDisabled
+    public void onDisabled() {
+        keyReference.set(null);
+    }
+
+    /**
+     * Get Supported Property Descriptors
+     *
+     * @return Supported Property Descriptors
+     */
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    /**
+     * Custom Validate reads key using configured password for encrypted keys
+     *
+     * @param context Validation Context
+     * @return Validation Results
+     */
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext context) {
+        final Collection<ValidationResult> results = new ArrayList<>();
+
+
+        final PropertyValue keyFileProperty = context.getProperty(KEY_FILE);
+        final PropertyValue keyProperty = context.getProperty(KEY);
+        if (keyFileProperty.isSet() && keyProperty.isSet()) {
+            final String explanation = String.format("Both [%s] and [%s] properties configured", KEY_FILE.getDisplayName(), KEY.getDisplayName());
+            final ValidationResult result = new ValidationResult.Builder()
+                    .valid(false)
+                    .subject(KEY.getDisplayName())
+                    .explanation(explanation)
+                    .build();
+            results.add(result);
+        } else {
+            try {
+                readKey(context);

Review Comment:
   `customValidate()` reads the private key file at every execution so a disabled controller service will load the file periodically (when the validation thread runs). It may be worth implementing `VerifiableControllerService` and moving this check there or using a "validation cache" as in [StandardSSLContextService](https://github.com/apache/nifi/blob/a5d93d96a30b3af73146e4b3c2a36a86332bb774/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-bundle/nifi-ssl-context-service/src/main/java/org/apache/nifi/ssl/StandardSSLContextService.java#L187-L197).



##########
nifi-nar-bundles/nifi-standard-services/nifi-key-service-bundle/nifi-key-service/src/main/java/org/apache/nifi/key/service/StandardPrivateKeyService.java:
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.key.service;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.resource.ResourceCardinality;
+import org.apache.nifi.components.resource.ResourceType;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.key.service.api.PrivateKeyService;
+import org.apache.nifi.key.service.reader.BouncyCastlePrivateKeyReader;
+import org.apache.nifi.key.service.reader.PrivateKeyReader;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.UncheckedIOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.PrivateKey;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * Standard implementation of Private Key Service supporting encrypted or unencrypted sources
+ */
+@Tags({"PEM", "PKCS8"})
+@CapabilityDescription("Private Key Service provides access to a Private Key loaded from configured sources")
+public class StandardPrivateKeyService extends AbstractControllerService implements PrivateKeyService {
+    public static final PropertyDescriptor KEY_FILE = new PropertyDescriptor.Builder()
+            .name("key-file")
+            .displayName("Key File")
+            .description("File path to Private Key structured using PKCS8 and encoded as PEM")
+            .required(false)
+            .identifiesExternalResource(ResourceCardinality.SINGLE, ResourceType.FILE)
+            .build();
+
+    public static final PropertyDescriptor KEY = new PropertyDescriptor.Builder()
+            .name("key")
+            .displayName("Key")
+            .description("Private Key structured using PKCS8 and encoded as PEM")
+            .required(false)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor KEY_PASSWORD = new PropertyDescriptor.Builder()
+            .name("key-password")
+            .displayName("Key Password")
+            .description("Password used for decrypting Private Keys")
+            .required(false)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    private static final List<PropertyDescriptor> DESCRIPTORS = Arrays.asList(
+            KEY_FILE,
+            KEY,
+            KEY_PASSWORD
+    );
+
+    private static final Charset KEY_CHARACTER_SET = StandardCharsets.US_ASCII;
+
+    private static final PrivateKeyReader PRIVATE_KEY_READER = new BouncyCastlePrivateKeyReader();
+
+    private final AtomicReference<PrivateKey> keyReference = new AtomicReference<>();
+
+    @Override
+    public PrivateKey getPrivateKey() {
+        return keyReference.get();
+    }
+
+    /**
+     * On Enabled reads Private Keys using configured properties
+     *
+     * @param context Configuration Context with properties
+     * @throws InitializationException Thrown when unable to load
+     */
+    @OnEnabled
+    public void onEnabled(final ConfigurationContext context) throws InitializationException {
+        try {
+            final PrivateKey readKey = readKey(context);
+            keyReference.set(readKey);
+        } catch (final RuntimeException e) {
+            throw new InitializationException("Reading Private Key Failed", e);
+        }
+    }
+
+    /**
+     * On Disabled clears Private Keys
+     */
+    @OnDisabled
+    public void onDisabled() {
+        keyReference.set(null);
+    }
+
+    /**
+     * Get Supported Property Descriptors
+     *
+     * @return Supported Property Descriptors
+     */
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    /**
+     * Custom Validate reads key using configured password for encrypted keys
+     *
+     * @param context Validation Context
+     * @return Validation Results
+     */
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext context) {
+        final Collection<ValidationResult> results = new ArrayList<>();
+
+
+        final PropertyValue keyFileProperty = context.getProperty(KEY_FILE);
+        final PropertyValue keyProperty = context.getProperty(KEY);
+        if (keyFileProperty.isSet() && keyProperty.isSet()) {
+            final String explanation = String.format("Both [%s] and [%s] properties configured", KEY_FILE.getDisplayName(), KEY.getDisplayName());

Review Comment:
   I prefer lower case start of the explanations because it will be concatenated in the middle of a sentence: _"XXX invalid because [explanation]"_.



##########
nifi-nar-bundles/nifi-standard-services/nifi-key-service-bundle/nifi-key-service/src/main/java/org/apache/nifi/key/service/reader/BouncyCastlePrivateKeyReader.java:
##########
@@ -0,0 +1,109 @@
+/*
+ * 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.key.service.reader;
+
+import org.bouncycastle.asn1.pkcs.PrivateKeyInfo;
+import org.bouncycastle.openssl.PEMDecryptorProvider;
+import org.bouncycastle.openssl.PEMEncryptedKeyPair;
+import org.bouncycastle.openssl.PEMException;
+import org.bouncycastle.openssl.PEMKeyPair;
+import org.bouncycastle.openssl.PEMParser;
+import org.bouncycastle.openssl.jcajce.JcaPEMKeyConverter;
+import org.bouncycastle.openssl.jcajce.JceOpenSSLPKCS8DecryptorProviderBuilder;
+import org.bouncycastle.openssl.jcajce.JcePEMDecryptorProviderBuilder;
+import org.bouncycastle.operator.InputDecryptorProvider;
+import org.bouncycastle.operator.OperatorCreationException;
+import org.bouncycastle.pkcs.PKCS8EncryptedPrivateKeyInfo;
+import org.bouncycastle.pkcs.PKCSException;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.UncheckedIOException;
+import java.security.PrivateKey;
+
+/**
+ * Bouncy Castle implementation of Private Key Reader supporting PEM files
+ */
+public class BouncyCastlePrivateKeyReader implements PrivateKeyReader {
+    /**
+     * Read Private Key using Bouncy Castle PEM Parser
+     *
+     * @param inputStream Key stream
+     * @param keyPassword Password
+     * @return Private Key
+     */
+    @Override
+    public PrivateKey readPrivateKey(final InputStream inputStream, final char[] keyPassword) {
+        try (final PEMParser parser = new PEMParser(new InputStreamReader(inputStream))) {
+            final Object object = parser.readObject();
+            final Class<?> objectClass = object.getClass();

Review Comment:
   If an invalid key is set (e.g. "dummy" string in the `Key` property), then the parser returns `null` and `getClass()` throws NPE. The result is _"'Key' is invalid because"_ validation error on the UI (that is no explanation in the message).



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

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

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


[GitHub] [nifi] thenatog closed pull request #6553: NIFI-10667 Add Private Key Service interface and implementation

Posted by GitBox <gi...@apache.org>.
thenatog closed pull request #6553: NIFI-10667 Add Private Key Service interface and implementation
URL: https://github.com/apache/nifi/pull/6553


-- 
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 #6553: NIFI-10667 Add Private Key Service interface and implementation

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


##########
nifi-nar-bundles/nifi-standard-services/nifi-key-service-bundle/nifi-key-service/src/main/java/org/apache/nifi/key/service/StandardPrivateKeyService.java:
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.key.service;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.resource.ResourceCardinality;
+import org.apache.nifi.components.resource.ResourceType;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.key.service.api.PrivateKeyService;
+import org.apache.nifi.key.service.reader.BouncyCastlePrivateKeyReader;
+import org.apache.nifi.key.service.reader.PrivateKeyReader;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.UncheckedIOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.PrivateKey;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * Standard implementation of Private Key Service supporting encrypted or unencrypted sources
+ */
+@Tags({"PEM", "PKCS8"})
+@CapabilityDescription("Private Key Service provides access to a Private Key loaded from configured sources")
+public class StandardPrivateKeyService extends AbstractControllerService implements PrivateKeyService {
+    public static final PropertyDescriptor KEY_FILE = new PropertyDescriptor.Builder()
+            .name("key-file")
+            .displayName("Key File")
+            .description("File path to Private Key structured using PKCS8 and encoded as PEM")
+            .required(false)
+            .identifiesExternalResource(ResourceCardinality.SINGLE, ResourceType.FILE)
+            .build();
+
+    public static final PropertyDescriptor KEY = new PropertyDescriptor.Builder()
+            .name("key")
+            .displayName("Key")
+            .description("Private Key structured using PKCS8 and encoded as PEM")
+            .required(false)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor KEY_PASSWORD = new PropertyDescriptor.Builder()
+            .name("key-password")
+            .displayName("Key Password")
+            .description("Password used for decrypting Private Keys")
+            .required(false)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    private static final List<PropertyDescriptor> DESCRIPTORS = Arrays.asList(
+            KEY_FILE,
+            KEY,
+            KEY_PASSWORD
+    );
+
+    private static final Charset KEY_CHARACTER_SET = StandardCharsets.US_ASCII;
+
+    private static final PrivateKeyReader PRIVATE_KEY_READER = new BouncyCastlePrivateKeyReader();
+
+    private final AtomicReference<PrivateKey> keyReference = new AtomicReference<>();
+
+    @Override
+    public PrivateKey getPrivateKey() {
+        return keyReference.get();
+    }
+
+    /**
+     * On Enabled reads Private Keys using configured properties
+     *
+     * @param context Configuration Context with properties
+     * @throws InitializationException Thrown when unable to load
+     */
+    @OnEnabled
+    public void onEnabled(final ConfigurationContext context) throws InitializationException {
+        try {
+            final PrivateKey readKey = readKey(context);
+            keyReference.set(readKey);
+        } catch (final RuntimeException e) {
+            throw new InitializationException("Reading Private Key Failed", e);
+        }
+    }
+
+    /**
+     * On Disabled clears Private Keys
+     */
+    @OnDisabled
+    public void onDisabled() {
+        keyReference.set(null);
+    }
+
+    /**
+     * Get Supported Property Descriptors
+     *
+     * @return Supported Property Descriptors
+     */
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    /**
+     * Custom Validate reads key using configured password for encrypted keys
+     *
+     * @param context Validation Context
+     * @return Validation Results
+     */
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext context) {
+        final Collection<ValidationResult> results = new ArrayList<>();
+
+
+        final PropertyValue keyFileProperty = context.getProperty(KEY_FILE);
+        final PropertyValue keyProperty = context.getProperty(KEY);
+        if (keyFileProperty.isSet() && keyProperty.isSet()) {
+            final String explanation = String.format("Both [%s] and [%s] properties configured", KEY_FILE.getDisplayName(), KEY.getDisplayName());
+            final ValidationResult result = new ValidationResult.Builder()
+                    .valid(false)
+                    .subject(KEY.getDisplayName())
+                    .explanation(explanation)
+                    .build();
+            results.add(result);
+        } else {
+            try {
+                readKey(context);

Review Comment:
   That's a good point. I made an adjustment to set the key reference on a successful read to avoid repetitive reads.



-- 
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] thenatog commented on pull request #6553: NIFI-10667 Add Private Key Service interface and implementation

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

   Will review


-- 
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 #6553: NIFI-10667 Add Private Key Service interface and implementation

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


##########
nifi-nar-bundles/nifi-standard-services/nifi-key-service-bundle/nifi-key-service/src/main/java/org/apache/nifi/key/service/reader/BouncyCastlePrivateKeyReader.java:
##########
@@ -0,0 +1,109 @@
+/*
+ * 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.key.service.reader;
+
+import org.bouncycastle.asn1.pkcs.PrivateKeyInfo;
+import org.bouncycastle.openssl.PEMDecryptorProvider;
+import org.bouncycastle.openssl.PEMEncryptedKeyPair;
+import org.bouncycastle.openssl.PEMException;
+import org.bouncycastle.openssl.PEMKeyPair;
+import org.bouncycastle.openssl.PEMParser;
+import org.bouncycastle.openssl.jcajce.JcaPEMKeyConverter;
+import org.bouncycastle.openssl.jcajce.JceOpenSSLPKCS8DecryptorProviderBuilder;
+import org.bouncycastle.openssl.jcajce.JcePEMDecryptorProviderBuilder;
+import org.bouncycastle.operator.InputDecryptorProvider;
+import org.bouncycastle.operator.OperatorCreationException;
+import org.bouncycastle.pkcs.PKCS8EncryptedPrivateKeyInfo;
+import org.bouncycastle.pkcs.PKCSException;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.UncheckedIOException;
+import java.security.PrivateKey;
+
+/**
+ * Bouncy Castle implementation of Private Key Reader supporting PEM files
+ */
+public class BouncyCastlePrivateKeyReader implements PrivateKeyReader {
+    /**
+     * Read Private Key using Bouncy Castle PEM Parser
+     *
+     * @param inputStream Key stream
+     * @param keyPassword Password
+     * @return Private Key
+     */
+    @Override
+    public PrivateKey readPrivateKey(final InputStream inputStream, final char[] keyPassword) {
+        try (final PEMParser parser = new PEMParser(new InputStreamReader(inputStream))) {
+            final Object object = parser.readObject();
+            final Class<?> objectClass = object.getClass();

Review Comment:
   Thanks for catching that issue, I will add a null check to avoid the NullPointerException.



-- 
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] thenatog commented on pull request #6553: NIFI-10667 Add Private Key Service interface and implementation

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

   +1 looks good to me


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

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 #6553: NIFI-10667 Add Private Key Service interface and implementation

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


##########
nifi-nar-bundles/nifi-standard-services/nifi-key-service-bundle/nifi-key-service/src/main/java/org/apache/nifi/key/service/StandardPrivateKeyService.java:
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.key.service;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.resource.ResourceCardinality;
+import org.apache.nifi.components.resource.ResourceType;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.key.service.api.PrivateKeyService;
+import org.apache.nifi.key.service.reader.BouncyCastlePrivateKeyReader;
+import org.apache.nifi.key.service.reader.PrivateKeyReader;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.UncheckedIOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.PrivateKey;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * Standard implementation of Private Key Service supporting encrypted or unencrypted sources
+ */
+@Tags({"PEM", "PKCS8"})
+@CapabilityDescription("Private Key Service provides access to a Private Key loaded from configured sources")
+public class StandardPrivateKeyService extends AbstractControllerService implements PrivateKeyService {
+    public static final PropertyDescriptor KEY_FILE = new PropertyDescriptor.Builder()
+            .name("key-file")
+            .displayName("Key File")
+            .description("File path to Private Key structured using PKCS8 and encoded as PEM")
+            .required(false)
+            .identifiesExternalResource(ResourceCardinality.SINGLE, ResourceType.FILE)
+            .build();
+
+    public static final PropertyDescriptor KEY = new PropertyDescriptor.Builder()
+            .name("key")
+            .displayName("Key")
+            .description("Private Key structured using PKCS8 and encoded as PEM")
+            .required(false)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor KEY_PASSWORD = new PropertyDescriptor.Builder()
+            .name("key-password")
+            .displayName("Key Password")
+            .description("Password used for decrypting Private Keys")
+            .required(false)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    private static final List<PropertyDescriptor> DESCRIPTORS = Arrays.asList(
+            KEY_FILE,
+            KEY,
+            KEY_PASSWORD
+    );
+
+    private static final Charset KEY_CHARACTER_SET = StandardCharsets.US_ASCII;
+
+    private static final PrivateKeyReader PRIVATE_KEY_READER = new BouncyCastlePrivateKeyReader();
+
+    private final AtomicReference<PrivateKey> keyReference = new AtomicReference<>();
+
+    @Override
+    public PrivateKey getPrivateKey() {
+        return keyReference.get();
+    }
+
+    /**
+     * On Enabled reads Private Keys using configured properties
+     *
+     * @param context Configuration Context with properties
+     * @throws InitializationException Thrown when unable to load
+     */
+    @OnEnabled
+    public void onEnabled(final ConfigurationContext context) throws InitializationException {
+        try {
+            final PrivateKey readKey = readKey(context);
+            keyReference.set(readKey);
+        } catch (final RuntimeException e) {
+            throw new InitializationException("Reading Private Key Failed", e);
+        }
+    }
+
+    /**
+     * On Disabled clears Private Keys
+     */
+    @OnDisabled
+    public void onDisabled() {
+        keyReference.set(null);
+    }
+
+    /**
+     * Get Supported Property Descriptors
+     *
+     * @return Supported Property Descriptors
+     */
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    /**
+     * Custom Validate reads key using configured password for encrypted keys
+     *
+     * @param context Validation Context
+     * @return Validation Results
+     */
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext context) {
+        final Collection<ValidationResult> results = new ArrayList<>();
+
+
+        final PropertyValue keyFileProperty = context.getProperty(KEY_FILE);
+        final PropertyValue keyProperty = context.getProperty(KEY);
+        if (keyFileProperty.isSet() && keyProperty.isSet()) {
+            final String explanation = String.format("Both [%s] and [%s] properties configured", KEY_FILE.getDisplayName(), KEY.getDisplayName());

Review Comment:
   That's a good point. Given that other components follow a similar pattern, it seems like it would be better to change the general message prefix to improve the overall reading, as opposed to removing the capitalization in this explanation. Perhaps that could be address in a separate issue.



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