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/20 19:16:15 UTC

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

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