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 2021/02/24 18:27:41 UTC

[GitHub] [nifi] exceptionfactory opened a new pull request #4842: NIFI-8251 Added EncryptContentPGP and DecryptContentPGP Processors

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


   #### Description of PR
   
   NIFI-8251 Adds the following Processors and Controller Services in a new `nifi-pgp-bundle` and related modules:
   
   - EncryptContentPGP
   - DecryptContentPGP
   - StandardPGPPrivateKeyService
   - StandardPGPPublicKeyService
   
   The new components address a number of latent issues related to PGP handling in the `EncryptContent` Processor:
   
   - NIFI-7396 EncryptContentPGP writes encryption metadata attributes
   - NIFI-6708 Controller Services support ElGamal Public and Private Keys
   - NIFI-5346 Controller Services support Keyring Files and ASCII Key properties
   - NIFI-5335 Controller Services support multiple public or private keys from keyrings
   - NIFI-2983 DecryptContentPGP finds and decrypts Encrypted Data Packets regardless of signing
   - NIFI-1694 Controller Services support individual key files or keyrings
   
   The EncryptContentPGP Processor supports several configuration properties that were previously hard-coded in `EncryptContent`:
   
   - File Encoding either Binary or ASCII Armor
   - Compression Algorithm
   
   The EncryptContentPGP supports AES or Camellia Symmetric-Key Algorithms with key sizes of 128, 192, or 256.  Support for other algorithms is not included in this initial version in order to discourage the use of less secure algorithms.  When configured with a PGPPublicKeyService, the EncryptContentPGP Processor requires the presence of a Public Key ID formatted using the long format as a 16 character hexadecimal string.  This approach provides direct control over which key to use for encryption and also supports both RSA and ElGamal Public Keys.
   
   The DecryptContentPGP Processor is capable of reading messages encrypted using any of the Symmetric-Key Algorithms supported in the BouncyCastle library.  The DecryptContentPGP Processor supports reading either binary or ASCII Armor encoded messages through content detection and also handles standard compression algorithms.  In addition, the DecryptContentPGP Processor is capable of handling signed messages and decrypting the contents.  Additional efforts to support signature verification can be implemented as separate processors under NIFI-7322.  The DecryptContentPGP Processor can be configured to read messages encrypted with either Password-Based Encryption or Public Key Encryption when configured with the necessary properties.  The Processor leverages the PGPPrivateKeyService to find associated Private Keys when processing messages encrypted using Public Key Encryption.
   
   The PGP Controller Services abstract access to Public and Private Key files.  Both services support binary or ASCII Armor keyrings, as well as the option to provide an ASCII Armored keyring as a property.  Public Keys are not considered sensitive, whereas Private Keys are considered sensitive and encrypted using standard NiFi Sensitive Properties encryption.
   
   In order to streamline the review of the contribution we ask you
   to ensure the following steps have been taken:
   
   ### For all changes:
   - [X] Is there a JIRA ticket associated with this PR? Is it referenced 
        in the commit message?
   
   - [X] Does your PR title start with **NIFI-XXXX** where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
   
   - [X] Has your PR been rebased against the latest commit within the target branch (typically `main`)?
   
   - [X] Is your initial contribution a single, squashed commit? _Additional commits in response to PR reviewer feedback should be made on this branch and pushed to allow change tracking. Do not `squash` or use `--force` when pushing to allow for clean monitoring of changes._
   
   ### For code changes:
   - [X] Have you ensured that the full suite of tests is executed via `mvn -Pcontrib-check clean install` at the root `nifi` folder?
   - [X] Have you written or updated unit tests to verify your changes?
   - [X] Have you verified that the full build is successful on JDK 8?
   - [X] Have you verified that the full build is successful on JDK 11?
   - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)? 
   - [ ] If applicable, have you updated the `LICENSE` file, including the main `LICENSE` file under `nifi-assembly`?
   - [ ] If applicable, have you updated the `NOTICE` file, including the main `NOTICE` file found under `nifi-assembly`?
   - [X] If adding new Properties, have you added `.displayName` in addition to .name (programmatic access) for each of the new properties?
   
   ### For documentation related changes:
   - [X] Have you ensured that format looks appropriate for the output in which it is rendered?
   
   ### Note:
   Please ensure that once the PR is submitted, you check GitHub Actions CI for build issues and submit an update to your PR as soon as possible.
   


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

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



[GitHub] [nifi] exceptionfactory commented on a change in pull request #4842: NIFI-8251 Added EncryptContentPGP and DecryptContentPGP Processors

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #4842:
URL: https://github.com/apache/nifi/pull/4842#discussion_r607374814



##########
File path: nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/test/java/org/apache/nifi/processors/pgp/DecryptContentPGPTest.java
##########
@@ -0,0 +1,391 @@
+/*
+ * 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.pgp;
+
+import org.apache.nifi.pgp.service.api.PGPPrivateKeyService;
+import org.apache.nifi.pgp.util.PGPSecretKeyGenerator;
+import org.apache.nifi.processors.pgp.exception.PGPDecryptionException;
+import org.apache.nifi.processors.pgp.exception.PGPProcessException;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.LogMessage;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.bouncycastle.bcpg.SymmetricKeyAlgorithmTags;
+import org.bouncycastle.openpgp.PGPCompressedData;
+import org.bouncycastle.openpgp.PGPCompressedDataGenerator;
+import org.bouncycastle.openpgp.PGPEncryptedDataGenerator;
+import org.bouncycastle.openpgp.PGPException;
+import org.bouncycastle.openpgp.PGPLiteralDataGenerator;
+import org.bouncycastle.openpgp.PGPPrivateKey;
+import org.bouncycastle.openpgp.PGPPublicKey;
+import org.bouncycastle.openpgp.PGPSecretKey;
+import org.bouncycastle.openpgp.PGPSecretKeyRing;
+import org.bouncycastle.openpgp.PGPSignature;
+import org.bouncycastle.openpgp.PGPSignatureGenerator;
+import org.bouncycastle.openpgp.PGPUtil;
+import org.bouncycastle.openpgp.operator.PBESecretKeyDecryptor;
+import org.bouncycastle.openpgp.operator.PGPContentSignerBuilder;
+import org.bouncycastle.openpgp.operator.PGPDataEncryptorBuilder;
+import org.bouncycastle.openpgp.operator.bc.BcPGPDataEncryptorBuilder;
+import org.bouncycastle.openpgp.operator.bc.BcPublicKeyKeyEncryptionMethodGenerator;
+import org.bouncycastle.openpgp.operator.jcajce.JcaPGPContentSignerBuilder;
+import org.bouncycastle.openpgp.operator.jcajce.JcePBEKeyEncryptionMethodGenerator;
+import org.bouncycastle.openpgp.operator.jcajce.JcePBESecretKeyDecryptorBuilder;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Date;
+import java.util.List;
+import java.util.Optional;
+import java.util.UUID;
+
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.CoreMatchers.isA;
+import static org.junit.Assert.assertTrue;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.when;
+
+@RunWith(MockitoJUnitRunner.class)
+public class DecryptContentPGPTest {
+    private static final int ENCRYPTION_ALGORITHM = SymmetricKeyAlgorithmTags.AES_256;
+
+    private static final boolean INTEGRITY_ENABLED = true;
+
+    private static final boolean INTEGRITY_DISABLED = false;
+
+    private static final String PASSPHRASE = UUID.randomUUID().toString();
+
+    private static final String FILE_NAME = String.class.getSimpleName();
+
+    private static final char FILE_TYPE = PGPLiteralDataGenerator.TEXT;
+
+    private static final long MODIFIED_MILLISECONDS = 86400000;
+
+    private static final Date MODIFIED = new Date(MODIFIED_MILLISECONDS);
+
+    private static final String DATA = String.class.getName();
+
+    private static final Charset DATA_CHARSET = StandardCharsets.UTF_8;
+
+    private static final int BUFFER_SIZE = 128;
+
+    private static final boolean NESTED_SIGNATURE_DISABLED = false;
+
+    private static final String SERVICE_ID = PGPPrivateKeyService.class.getSimpleName();
+
+    private static PGPSecretKey rsaSecretKey;
+
+    private static PGPPrivateKey rsaPrivateKey;
+
+    private static PGPPublicKey elGamalPublicKey;
+
+    private static PGPPrivateKey elGamalPrivateKey;
+
+    private TestRunner runner;
+
+    @Mock
+    private PGPPrivateKeyService privateKeyService;
+
+    @BeforeClass
+    public static void setKeys() throws Exception {
+        rsaSecretKey = PGPSecretKeyGenerator.generateRsaSecretKey(PASSPHRASE.toCharArray());
+
+        final PBESecretKeyDecryptor decryptor = new JcePBESecretKeyDecryptorBuilder().build(PASSPHRASE.toCharArray());
+        rsaPrivateKey = rsaSecretKey.extractPrivateKey(decryptor);
+        final PGPSecretKeyRing dsaElGamalSecretKeyRing = PGPSecretKeyGenerator.generateDsaElGamalSecretKeyRing(PASSPHRASE.toCharArray());
+        for (final PGPSecretKey secretKey : dsaElGamalSecretKeyRing) {
+            final PGPPublicKey publicKey = secretKey.getPublicKey();
+            if (PGPPublicKey.ELGAMAL_ENCRYPT == publicKey.getAlgorithm()) {
+                elGamalPrivateKey = secretKey.extractPrivateKey(decryptor);
+                elGamalPublicKey = publicKey;
+            }
+        }
+    }
+
+    @Before
+    public void setRunner() {
+        runner = TestRunners.newTestRunner(new DecryptContentPGP());
+    }
+
+    @Test
+    public void testMissingProperties() {
+        runner.assertNotValid();
+    }
+
+    @Test
+    public void testFailureEncryptedDataNotFound() {
+        runner.setProperty(DecryptContentPGP.PASSPHRASE, PASSPHRASE);
+        runner.enqueue(new byte[]{});
+        runner.run();
+
+        assertFailureExceptionLogged(PGPProcessException.class);
+    }
+

Review comment:
       Great question! RFC 4880 Section 5.3 describes this as a valid scenario, so I will include additional unit tests.




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

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



[GitHub] [nifi] exceptionfactory commented on a change in pull request #4842: NIFI-8251 Added EncryptContentPGP and DecryptContentPGP Processors

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #4842:
URL: https://github.com/apache/nifi/pull/4842#discussion_r607272225



##########
File path: nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/EncryptContentPGP.java
##########
@@ -0,0 +1,379 @@
+/*
+ * 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.pgp;
+
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+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.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.pgp.service.api.PGPPublicKeyService;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.io.StreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.pgp.attributes.CompressionAlgorithm;
+import org.apache.nifi.processors.pgp.attributes.FileEncoding;
+import org.apache.nifi.processors.pgp.attributes.SymmetricKeyAlgorithm;
+import org.apache.nifi.processors.pgp.exception.PGPEncryptionException;
+import org.apache.nifi.stream.io.StreamUtils;
+import org.apache.nifi.util.StringUtils;
+
+import org.bouncycastle.bcpg.ArmoredOutputStream;
+import org.bouncycastle.openpgp.PGPCompressedDataGenerator;
+import org.bouncycastle.openpgp.PGPEncryptedDataGenerator;
+import org.bouncycastle.openpgp.PGPException;
+import org.bouncycastle.openpgp.PGPLiteralData;
+import org.bouncycastle.openpgp.PGPLiteralDataGenerator;
+import org.bouncycastle.openpgp.PGPPublicKey;
+import org.bouncycastle.openpgp.operator.PGPDataEncryptorBuilder;
+import org.bouncycastle.openpgp.operator.PGPKeyEncryptionMethodGenerator;
+import org.bouncycastle.openpgp.operator.bc.BcPGPDataEncryptorBuilder;
+import org.bouncycastle.openpgp.operator.bc.BcPublicKeyKeyEncryptionMethodGenerator;
+import org.bouncycastle.openpgp.operator.jcajce.JcePBEKeyEncryptionMethodGenerator;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.security.SecureRandom;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * Encrypt Content using Open Pretty Good Privacy encryption methods
+ */
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Tags({"PGP", "GPG", "OpenPGP", "Encryption", "RFC 4880"})
+@CapabilityDescription("Encrypt Contents using OpenPGP")
+@WritesAttributes({
+        @WritesAttribute(attribute = PGPAttributeKey.SYMMETRIC_KEY_ALGORITHM, description = "Symmetric-Key Algorithm"),
+        @WritesAttribute(attribute = PGPAttributeKey.SYMMETRIC_KEY_ALGORITHM_BLOCK_CIPHER, description = "Symmetric-Key Algorithm Block Cipher"),
+        @WritesAttribute(attribute = PGPAttributeKey.SYMMETRIC_KEY_ALGORITHM_KEY_SIZE, description = "Symmetric-Key Algorithm Key Size"),
+        @WritesAttribute(attribute = PGPAttributeKey.SYMMETRIC_KEY_ALGORITHM_ID, description = "Symmetric-Key Algorithm Identifier"),
+        @WritesAttribute(attribute = PGPAttributeKey.FILE_ENCODING, description = "File Encoding"),
+        @WritesAttribute(attribute = PGPAttributeKey.COMPRESS_ALGORITHM, description = "Compression Algorithm"),
+        @WritesAttribute(attribute = PGPAttributeKey.COMPRESS_ALGORITHM_ID, description = "Compression Algorithm Identifier"),
+})
+public class EncryptContentPGP extends AbstractProcessor {
+
+    public static final Relationship SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Encryption Succeeded")
+            .build();
+
+    public static final Relationship FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Encryption Failed")
+            .build();
+
+    public static final PropertyDescriptor SYMMETRIC_KEY_ALGORITHM = new PropertyDescriptor.Builder()
+            .name("symmetric-key-algorithm")
+            .displayName("Symmetric-Key Algorithm")
+            .description("Symmetric-Key Algorithm for encryption")
+            .required(true)
+            .defaultValue(SymmetricKeyAlgorithm.AES_256.toString())
+            .allowableValues(SymmetricKeyAlgorithm.values())
+            .build();
+
+    public static final PropertyDescriptor COMPRESSION_ALGORITHM = new PropertyDescriptor.Builder()
+            .name("compression-algorithm")
+            .displayName("Compression Algorithm")
+            .description("Compression Algorithm for encryption")
+            .required(true)
+            .defaultValue(CompressionAlgorithm.ZIP.toString())
+            .allowableValues(CompressionAlgorithm.values())
+            .build();
+
+    public static final PropertyDescriptor FILE_ENCODING = new PropertyDescriptor.Builder()
+            .name("file-encoding")
+            .displayName("File Encoding")
+            .description("File Encoding for encryption")
+            .required(true)
+            .defaultValue(FileEncoding.BINARY.toString())
+            .allowableValues(FileEncoding.values())
+            .build();
+
+    public static final PropertyDescriptor PASSPHRASE = new PropertyDescriptor.Builder()
+            .name("passphrase")
+            .displayName("Passphrase")
+            .description("Passphrase used for encrypting data with Password-Based Encryption")
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor PUBLIC_KEY_SERVICE = new PropertyDescriptor.Builder()
+            .name("public-key-service")
+            .displayName("Public Key Service")
+            .description("PGP Public Key Service for encrypting data with Public Key Encryption")
+            .identifiesControllerService(PGPPublicKeyService.class)
+            .build();
+
+    public static final PropertyDescriptor PUBLIC_KEY_SEARCH = new PropertyDescriptor.Builder()
+            .name("public-key-search")
+            .displayName("Public Key Search")
+            .description("PGP Public Key Search will be used to match against the User ID or Key ID when formatted as uppercase hexadecimal string of 16 characters")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .addValidator(StandardValidators.ATTRIBUTE_EXPRESSION_LANGUAGE_VALIDATOR)
+            .dependsOn(PUBLIC_KEY_SERVICE)
+            .build();
+
+    /** Enable Integrity Protection as described in RFC 4880 Section 5.13 */
+    private static final boolean ENCRYPTION_INTEGRITY_PACKET_ENABLED = true;
+
+    private static final int OUTPUT_BUFFER_SIZE = 8192;
+
+    private static final Set<Relationship> RELATIONSHIPS = new HashSet<>(Arrays.asList(SUCCESS, FAILURE));
+
+    private static final List<PropertyDescriptor> DESCRIPTORS = Arrays.asList(
+            SYMMETRIC_KEY_ALGORITHM,
+            COMPRESSION_ALGORITHM,
+            FILE_ENCODING,
+            PASSPHRASE,
+            PUBLIC_KEY_SERVICE,
+            PUBLIC_KEY_SEARCH
+    );
+
+    /**
+     * Get Relationships
+     *
+     * @return Processor Relationships
+     */
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    /**
+     * Get Supported Property Descriptors
+     *
+     * @return Processor Supported Property Descriptors
+     */
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    /**
+     * On Trigger encrypts Flow File contents using configured properties
+     *
+     * @param context Process Context
+     * @param session Process Session
+     */
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        try {
+            final SymmetricKeyAlgorithm symmetricKeyAlgorithm = getSymmetricKeyAlgorithm(context);
+            final FileEncoding fileEncoding = getFileEncoding(context);
+            final CompressionAlgorithm compressionAlgorithm = getCompressionAlgorithm(context);
+            final StreamCallback callback = getEncryptStreamCallback(context, flowFile, symmetricKeyAlgorithm, compressionAlgorithm, fileEncoding);
+            flowFile = session.write(flowFile, callback);
+
+            final Map<String, String> attributes = getAttributes(symmetricKeyAlgorithm, fileEncoding, compressionAlgorithm);
+            flowFile = session.putAllAttributes(flowFile, attributes);
+
+            session.transfer(flowFile, SUCCESS);
+        } catch (final RuntimeException e) {
+            getLogger().error("Encryption Failed {}", flowFile, e);
+            session.transfer(flowFile, FAILURE);
+        }
+    }
+
+    /**
+     * Custom Validate requires at least one encryption property to be configured
+     *
+     * @param context Validation Context
+     * @return Collection of Validation Results
+     */
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext context) {
+        final Collection<ValidationResult> results = new ArrayList<>();
+
+        final String passphrase = context.getProperty(PASSPHRASE).getValue();
+        if (StringUtils.isBlank(passphrase)) {
+            final PGPPublicKeyService publicKeyService = context.getProperty(PUBLIC_KEY_SERVICE).asControllerService(PGPPublicKeyService.class);
+            if (publicKeyService == null) {
+                final String explanation = String.format("Neither [%s] nor [%s] configured", PASSPHRASE.getDisplayName(), PUBLIC_KEY_SERVICE.getDisplayName());
+                final ValidationResult result = new ValidationResult.Builder()
+                        .valid(false)
+                        .subject(getClass().getSimpleName())
+                        .explanation(explanation)
+                        .build();
+                results.add(result);
+            }
+        }
+
+        return results;

Review comment:
       That's a good point, although the `Public Key Search` property supports expression language, initial validation should at least determine that the property is not blank when the `Public Key Service` is configured.




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

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



[GitHub] [nifi] exceptionfactory commented on pull request #4842: NIFI-8251 Added EncryptContentPGP and DecryptContentPGP Processors

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on pull request #4842:
URL: https://github.com/apache/nifi/pull/4842#issuecomment-813697838


   > Great work, David. Aside from needing to add the pgp nars to the nifi-assembly pom.xml, I believe all of my suggested changes are captured in the review comments. I'm looking forward to using these processors!
   
   Thanks for the detailed and helpful feedback @gresockj!  I pushed an update to address your comments, which includes improving `DecryptContentPGP` to support handling multiple Encrypted Data packets, as is the case when content is encrypted with both a password and public key.


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

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



[GitHub] [nifi] thenatog commented on pull request #4842: NIFI-8251 Added EncryptContentPGP and DecryptContentPGP Processors

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


   I'll check this one out and see if we can get it merged in.


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

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



[GitHub] [nifi] thenatog closed pull request #4842: NIFI-8251 Added EncryptContentPGP and DecryptContentPGP Processors

Posted by GitBox <gi...@apache.org>.
thenatog closed pull request #4842:
URL: https://github.com/apache/nifi/pull/4842


   


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

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



[GitHub] [nifi] gresockj commented on a change in pull request #4842: NIFI-8251 Added EncryptContentPGP and DecryptContentPGP Processors

Posted by GitBox <gi...@apache.org>.
gresockj commented on a change in pull request #4842:
URL: https://github.com/apache/nifi/pull/4842#discussion_r606351601



##########
File path: nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-service/src/main/java/org/apache/nifi/pgp/service/standard/StandardPGPPrivateKeyService.java
##########
@@ -0,0 +1,282 @@
+/*
+ * 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.pgp.service.standard;
+
+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.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.pgp.service.api.PGPPrivateKeyService;
+import org.apache.nifi.pgp.service.standard.exception.PGPConfigurationException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.StringUtils;
+
+import org.bouncycastle.openpgp.PGPException;
+import org.bouncycastle.openpgp.PGPPrivateKey;
+import org.bouncycastle.openpgp.PGPSecretKey;
+import org.bouncycastle.openpgp.PGPSecretKeyRing;
+import org.bouncycastle.openpgp.PGPSecretKeyRingCollection;
+import org.bouncycastle.openpgp.PGPUtil;
+import org.bouncycastle.openpgp.operator.KeyFingerPrintCalculator;
+import org.bouncycastle.openpgp.operator.PBESecretKeyDecryptor;
+import org.bouncycastle.openpgp.operator.jcajce.JcaKeyFingerprintCalculator;
+import org.bouncycastle.openpgp.operator.jcajce.JcePBESecretKeyDecryptorBuilder;
+
+import java.io.ByteArrayInputStream;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+/**
+ * Standard Pretty Good Privacy Private Key Service reads Private Keys from configured Keyring files or properties
+ */
+@Tags({"PGP", "GPG", "OpenPGP", "Encryption", "Private", "Key", "RFC 4880"})
+@CapabilityDescription("PGP Private Key Service provides Private Keys loaded from files or properties")
+public class StandardPGPPrivateKeyService extends AbstractControllerService implements PGPPrivateKeyService {
+    public static final PropertyDescriptor KEYRING_FILE = new PropertyDescriptor.Builder()
+            .name("keyring-file")
+            .displayName("Keyring File")
+            .description("File path for PGP Keyring or Secret Key encoded in binary or ASCII Armor")

Review comment:
       I misread this the first time, thinking it meant the file path could be encoded in binary or ASCII Armor.  Perhaps "File path to either PGP Keyring or to Secret Key encoded in either binary or ASCII Armor"

##########
File path: nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-service/src/main/java/org/apache/nifi/pgp/service/standard/StandardPGPPublicKeyService.java
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.pgp.service.standard;
+
+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.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.pgp.service.api.PGPPublicKeyService;
+import org.apache.nifi.pgp.service.standard.exception.PGPConfigurationException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.StringUtils;
+
+import org.bouncycastle.openpgp.PGPException;
+import org.bouncycastle.openpgp.PGPPublicKey;
+import org.bouncycastle.openpgp.PGPPublicKeyRingCollection;
+import org.bouncycastle.openpgp.PGPUtil;
+import org.bouncycastle.openpgp.operator.KeyFingerPrintCalculator;
+import org.bouncycastle.openpgp.operator.jcajce.JcaKeyFingerprintCalculator;
+
+import java.io.ByteArrayInputStream;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+
+/**
+ * Standard Pretty Good Privacy Public Key Service reads Public Keys from configured Keyring files
+ */
+@Tags({"PGP", "GPG", "OpenPGP", "Encryption", "Private", "Key", "RFC 4880"})
+@CapabilityDescription("PGP Public Key Service providing Public Keys loaded from files")
+public class StandardPGPPublicKeyService extends AbstractControllerService implements PGPPublicKeyService {
+    public static final PropertyDescriptor KEYRING_FILE = new PropertyDescriptor.Builder()
+            .name("keyring-file")
+            .displayName("Keyring File")
+            .description("File path for PGP Keyring or Public Key encoded in binary or ASCII Armor")

Review comment:
       Same comment as in StandardPGPPrivateKeyService.   Perhaps "File path to either PGP Keyring or to Public Key encoded in either binary or ASCII Armor"

##########
File path: nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/EncryptContentPGP.java
##########
@@ -0,0 +1,379 @@
+/*
+ * 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.pgp;
+
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+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.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.pgp.service.api.PGPPublicKeyService;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.io.StreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.pgp.attributes.CompressionAlgorithm;
+import org.apache.nifi.processors.pgp.attributes.FileEncoding;
+import org.apache.nifi.processors.pgp.attributes.SymmetricKeyAlgorithm;
+import org.apache.nifi.processors.pgp.exception.PGPEncryptionException;
+import org.apache.nifi.stream.io.StreamUtils;
+import org.apache.nifi.util.StringUtils;
+
+import org.bouncycastle.bcpg.ArmoredOutputStream;
+import org.bouncycastle.openpgp.PGPCompressedDataGenerator;
+import org.bouncycastle.openpgp.PGPEncryptedDataGenerator;
+import org.bouncycastle.openpgp.PGPException;
+import org.bouncycastle.openpgp.PGPLiteralData;
+import org.bouncycastle.openpgp.PGPLiteralDataGenerator;
+import org.bouncycastle.openpgp.PGPPublicKey;
+import org.bouncycastle.openpgp.operator.PGPDataEncryptorBuilder;
+import org.bouncycastle.openpgp.operator.PGPKeyEncryptionMethodGenerator;
+import org.bouncycastle.openpgp.operator.bc.BcPGPDataEncryptorBuilder;
+import org.bouncycastle.openpgp.operator.bc.BcPublicKeyKeyEncryptionMethodGenerator;
+import org.bouncycastle.openpgp.operator.jcajce.JcePBEKeyEncryptionMethodGenerator;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.security.SecureRandom;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * Encrypt Content using Open Pretty Good Privacy encryption methods
+ */
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Tags({"PGP", "GPG", "OpenPGP", "Encryption", "RFC 4880"})
+@CapabilityDescription("Encrypt Contents using OpenPGP")
+@WritesAttributes({
+        @WritesAttribute(attribute = PGPAttributeKey.SYMMETRIC_KEY_ALGORITHM, description = "Symmetric-Key Algorithm"),
+        @WritesAttribute(attribute = PGPAttributeKey.SYMMETRIC_KEY_ALGORITHM_BLOCK_CIPHER, description = "Symmetric-Key Algorithm Block Cipher"),
+        @WritesAttribute(attribute = PGPAttributeKey.SYMMETRIC_KEY_ALGORITHM_KEY_SIZE, description = "Symmetric-Key Algorithm Key Size"),
+        @WritesAttribute(attribute = PGPAttributeKey.SYMMETRIC_KEY_ALGORITHM_ID, description = "Symmetric-Key Algorithm Identifier"),
+        @WritesAttribute(attribute = PGPAttributeKey.FILE_ENCODING, description = "File Encoding"),
+        @WritesAttribute(attribute = PGPAttributeKey.COMPRESS_ALGORITHM, description = "Compression Algorithm"),
+        @WritesAttribute(attribute = PGPAttributeKey.COMPRESS_ALGORITHM_ID, description = "Compression Algorithm Identifier"),
+})
+public class EncryptContentPGP extends AbstractProcessor {
+
+    public static final Relationship SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Encryption Succeeded")
+            .build();
+
+    public static final Relationship FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Encryption Failed")
+            .build();
+
+    public static final PropertyDescriptor SYMMETRIC_KEY_ALGORITHM = new PropertyDescriptor.Builder()
+            .name("symmetric-key-algorithm")
+            .displayName("Symmetric-Key Algorithm")
+            .description("Symmetric-Key Algorithm for encryption")
+            .required(true)
+            .defaultValue(SymmetricKeyAlgorithm.AES_256.toString())
+            .allowableValues(SymmetricKeyAlgorithm.values())
+            .build();
+
+    public static final PropertyDescriptor COMPRESSION_ALGORITHM = new PropertyDescriptor.Builder()
+            .name("compression-algorithm")
+            .displayName("Compression Algorithm")
+            .description("Compression Algorithm for encryption")
+            .required(true)
+            .defaultValue(CompressionAlgorithm.ZIP.toString())
+            .allowableValues(CompressionAlgorithm.values())
+            .build();
+
+    public static final PropertyDescriptor FILE_ENCODING = new PropertyDescriptor.Builder()
+            .name("file-encoding")
+            .displayName("File Encoding")
+            .description("File Encoding for encryption")
+            .required(true)
+            .defaultValue(FileEncoding.BINARY.toString())
+            .allowableValues(FileEncoding.values())
+            .build();
+
+    public static final PropertyDescriptor PASSPHRASE = new PropertyDescriptor.Builder()
+            .name("passphrase")
+            .displayName("Passphrase")
+            .description("Passphrase used for encrypting data with Password-Based Encryption")
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor PUBLIC_KEY_SERVICE = new PropertyDescriptor.Builder()
+            .name("public-key-service")
+            .displayName("Public Key Service")
+            .description("PGP Public Key Service for encrypting data with Public Key Encryption")
+            .identifiesControllerService(PGPPublicKeyService.class)
+            .build();
+
+    public static final PropertyDescriptor PUBLIC_KEY_SEARCH = new PropertyDescriptor.Builder()
+            .name("public-key-search")
+            .displayName("Public Key Search")
+            .description("PGP Public Key Search will be used to match against the User ID or Key ID when formatted as uppercase hexadecimal string of 16 characters")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .addValidator(StandardValidators.ATTRIBUTE_EXPRESSION_LANGUAGE_VALIDATOR)
+            .dependsOn(PUBLIC_KEY_SERVICE)
+            .build();
+
+    /** Enable Integrity Protection as described in RFC 4880 Section 5.13 */
+    private static final boolean ENCRYPTION_INTEGRITY_PACKET_ENABLED = true;
+
+    private static final int OUTPUT_BUFFER_SIZE = 8192;
+
+    private static final Set<Relationship> RELATIONSHIPS = new HashSet<>(Arrays.asList(SUCCESS, FAILURE));
+
+    private static final List<PropertyDescriptor> DESCRIPTORS = Arrays.asList(
+            SYMMETRIC_KEY_ALGORITHM,
+            COMPRESSION_ALGORITHM,
+            FILE_ENCODING,
+            PASSPHRASE,
+            PUBLIC_KEY_SERVICE,
+            PUBLIC_KEY_SEARCH
+    );
+
+    /**
+     * Get Relationships
+     *
+     * @return Processor Relationships
+     */
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    /**
+     * Get Supported Property Descriptors
+     *
+     * @return Processor Supported Property Descriptors
+     */
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    /**
+     * On Trigger encrypts Flow File contents using configured properties
+     *
+     * @param context Process Context
+     * @param session Process Session
+     */
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        try {
+            final SymmetricKeyAlgorithm symmetricKeyAlgorithm = getSymmetricKeyAlgorithm(context);
+            final FileEncoding fileEncoding = getFileEncoding(context);
+            final CompressionAlgorithm compressionAlgorithm = getCompressionAlgorithm(context);
+            final StreamCallback callback = getEncryptStreamCallback(context, flowFile, symmetricKeyAlgorithm, compressionAlgorithm, fileEncoding);
+            flowFile = session.write(flowFile, callback);
+
+            final Map<String, String> attributes = getAttributes(symmetricKeyAlgorithm, fileEncoding, compressionAlgorithm);
+            flowFile = session.putAllAttributes(flowFile, attributes);
+
+            session.transfer(flowFile, SUCCESS);
+        } catch (final RuntimeException e) {
+            getLogger().error("Encryption Failed {}", flowFile, e);
+            session.transfer(flowFile, FAILURE);
+        }
+    }
+
+    /**
+     * Custom Validate requires at least one encryption property to be configured
+     *
+     * @param context Validation Context
+     * @return Collection of Validation Results
+     */
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext context) {
+        final Collection<ValidationResult> results = new ArrayList<>();
+
+        final String passphrase = context.getProperty(PASSPHRASE).getValue();
+        if (StringUtils.isBlank(passphrase)) {
+            final PGPPublicKeyService publicKeyService = context.getProperty(PUBLIC_KEY_SERVICE).asControllerService(PGPPublicKeyService.class);
+            if (publicKeyService == null) {
+                final String explanation = String.format("Neither [%s] nor [%s] configured", PASSPHRASE.getDisplayName(), PUBLIC_KEY_SERVICE.getDisplayName());
+                final ValidationResult result = new ValidationResult.Builder()
+                        .valid(false)
+                        .subject(getClass().getSimpleName())
+                        .explanation(explanation)
+                        .build();
+                results.add(result);
+            }
+        }
+
+        return results;

Review comment:
       One thing I notice is that if a Public Key Service is specified, but not a Public Key Search value, custom validation doesn't report a problem, but when a flow file goes through the processor, you get an error: "no encryption methods specified".  Could we add some custom validation logic here to catch this ahead of time?

##########
File path: nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/test/java/org/apache/nifi/processors/pgp/DecryptContentPGPTest.java
##########
@@ -0,0 +1,391 @@
+/*
+ * 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.pgp;
+
+import org.apache.nifi.pgp.service.api.PGPPrivateKeyService;
+import org.apache.nifi.pgp.util.PGPSecretKeyGenerator;
+import org.apache.nifi.processors.pgp.exception.PGPDecryptionException;
+import org.apache.nifi.processors.pgp.exception.PGPProcessException;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.LogMessage;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.bouncycastle.bcpg.SymmetricKeyAlgorithmTags;
+import org.bouncycastle.openpgp.PGPCompressedData;
+import org.bouncycastle.openpgp.PGPCompressedDataGenerator;
+import org.bouncycastle.openpgp.PGPEncryptedDataGenerator;
+import org.bouncycastle.openpgp.PGPException;
+import org.bouncycastle.openpgp.PGPLiteralDataGenerator;
+import org.bouncycastle.openpgp.PGPPrivateKey;
+import org.bouncycastle.openpgp.PGPPublicKey;
+import org.bouncycastle.openpgp.PGPSecretKey;
+import org.bouncycastle.openpgp.PGPSecretKeyRing;
+import org.bouncycastle.openpgp.PGPSignature;
+import org.bouncycastle.openpgp.PGPSignatureGenerator;
+import org.bouncycastle.openpgp.PGPUtil;
+import org.bouncycastle.openpgp.operator.PBESecretKeyDecryptor;
+import org.bouncycastle.openpgp.operator.PGPContentSignerBuilder;
+import org.bouncycastle.openpgp.operator.PGPDataEncryptorBuilder;
+import org.bouncycastle.openpgp.operator.bc.BcPGPDataEncryptorBuilder;
+import org.bouncycastle.openpgp.operator.bc.BcPublicKeyKeyEncryptionMethodGenerator;
+import org.bouncycastle.openpgp.operator.jcajce.JcaPGPContentSignerBuilder;
+import org.bouncycastle.openpgp.operator.jcajce.JcePBEKeyEncryptionMethodGenerator;
+import org.bouncycastle.openpgp.operator.jcajce.JcePBESecretKeyDecryptorBuilder;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Date;
+import java.util.List;
+import java.util.Optional;
+import java.util.UUID;
+
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.CoreMatchers.isA;
+import static org.junit.Assert.assertTrue;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.when;
+
+@RunWith(MockitoJUnitRunner.class)
+public class DecryptContentPGPTest {
+    private static final int ENCRYPTION_ALGORITHM = SymmetricKeyAlgorithmTags.AES_256;
+
+    private static final boolean INTEGRITY_ENABLED = true;
+
+    private static final boolean INTEGRITY_DISABLED = false;
+
+    private static final String PASSPHRASE = UUID.randomUUID().toString();
+
+    private static final String FILE_NAME = String.class.getSimpleName();
+
+    private static final char FILE_TYPE = PGPLiteralDataGenerator.TEXT;
+
+    private static final long MODIFIED_MILLISECONDS = 86400000;
+
+    private static final Date MODIFIED = new Date(MODIFIED_MILLISECONDS);
+
+    private static final String DATA = String.class.getName();
+
+    private static final Charset DATA_CHARSET = StandardCharsets.UTF_8;
+
+    private static final int BUFFER_SIZE = 128;
+
+    private static final boolean NESTED_SIGNATURE_DISABLED = false;
+
+    private static final String SERVICE_ID = PGPPrivateKeyService.class.getSimpleName();
+
+    private static PGPSecretKey rsaSecretKey;
+
+    private static PGPPrivateKey rsaPrivateKey;
+
+    private static PGPPublicKey elGamalPublicKey;
+
+    private static PGPPrivateKey elGamalPrivateKey;
+
+    private TestRunner runner;
+
+    @Mock
+    private PGPPrivateKeyService privateKeyService;
+
+    @BeforeClass
+    public static void setKeys() throws Exception {
+        rsaSecretKey = PGPSecretKeyGenerator.generateRsaSecretKey(PASSPHRASE.toCharArray());
+
+        final PBESecretKeyDecryptor decryptor = new JcePBESecretKeyDecryptorBuilder().build(PASSPHRASE.toCharArray());
+        rsaPrivateKey = rsaSecretKey.extractPrivateKey(decryptor);
+        final PGPSecretKeyRing dsaElGamalSecretKeyRing = PGPSecretKeyGenerator.generateDsaElGamalSecretKeyRing(PASSPHRASE.toCharArray());
+        for (final PGPSecretKey secretKey : dsaElGamalSecretKeyRing) {
+            final PGPPublicKey publicKey = secretKey.getPublicKey();
+            if (PGPPublicKey.ELGAMAL_ENCRYPT == publicKey.getAlgorithm()) {
+                elGamalPrivateKey = secretKey.extractPrivateKey(decryptor);
+                elGamalPublicKey = publicKey;
+            }
+        }
+    }
+
+    @Before
+    public void setRunner() {
+        runner = TestRunners.newTestRunner(new DecryptContentPGP());
+    }
+
+    @Test
+    public void testMissingProperties() {
+        runner.assertNotValid();
+    }
+
+    @Test
+    public void testFailureEncryptedDataNotFound() {
+        runner.setProperty(DecryptContentPGP.PASSPHRASE, PASSPHRASE);
+        runner.enqueue(new byte[]{});
+        runner.run();
+
+        assertFailureExceptionLogged(PGPProcessException.class);
+    }
+

Review comment:
       Should it be possible to configure both Password-based encryption and Private Key Encryption at the same time in this processor?  Either way, it would be great to see a unit test showing what happens when you configure the processor both ways.

##########
File path: nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/test/java/org/apache/nifi/processors/pgp/EncryptContentPGPTest.java
##########
@@ -0,0 +1,293 @@
+/*
+ * 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.pgp;
+
+import org.apache.nifi.pgp.service.api.PGPPublicKeyService;
+import org.apache.nifi.processors.pgp.attributes.CompressionAlgorithm;
+import org.apache.nifi.processors.pgp.attributes.FileEncoding;
+import org.apache.nifi.processors.pgp.attributes.SymmetricKeyAlgorithm;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.stream.io.StreamUtils;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.apache.nifi.pgp.util.PGPSecretKeyGenerator;
+
+import org.bouncycastle.bcpg.ArmoredInputStream;
+import org.bouncycastle.openpgp.PGPCompressedData;
+import org.bouncycastle.openpgp.PGPEncryptedData;
+import org.bouncycastle.openpgp.PGPEncryptedDataList;
+import org.bouncycastle.openpgp.PGPException;
+import org.bouncycastle.openpgp.PGPLiteralData;
+import org.bouncycastle.openpgp.PGPObjectFactory;
+import org.bouncycastle.openpgp.PGPPBEEncryptedData;
+import org.bouncycastle.openpgp.PGPPrivateKey;
+import org.bouncycastle.openpgp.PGPPublicKey;
+import org.bouncycastle.openpgp.PGPPublicKeyEncryptedData;
+import org.bouncycastle.openpgp.PGPSecretKey;
+import org.bouncycastle.openpgp.PGPSecretKeyRing;
+import org.bouncycastle.openpgp.jcajce.JcaPGPObjectFactory;
+import org.bouncycastle.openpgp.operator.PBEDataDecryptorFactory;
+import org.bouncycastle.openpgp.operator.PBESecretKeyDecryptor;
+import org.bouncycastle.openpgp.operator.PublicKeyDataDecryptorFactory;
+import org.bouncycastle.openpgp.operator.bc.BcPBEDataDecryptorFactory;
+import org.bouncycastle.openpgp.operator.bc.BcPGPDigestCalculatorProvider;
+import org.bouncycastle.openpgp.operator.bc.BcPublicKeyDataDecryptorFactory;
+import org.bouncycastle.openpgp.operator.jcajce.JcePBESecretKeyDecryptorBuilder;
+
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Optional;
+import java.util.UUID;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.when;
+
+@RunWith(MockitoJUnitRunner.class)
+public class EncryptContentPGPTest {
+    private static final String PASSPHRASE = UUID.randomUUID().toString();
+
+    private static final String DATA = String.class.getName();
+
+    private static final SymmetricKeyAlgorithm DEFAULT_SYMMETRIC_KEY_ALGORITHM = SymmetricKeyAlgorithm.valueOf(EncryptContentPGP.SYMMETRIC_KEY_ALGORITHM.getDefaultValue());
+
+    private static final String SERVICE_ID = PGPPublicKeyService.class.getName();
+
+    private static PGPSecretKey rsaSecretKey;
+
+    private static PGPPrivateKey rsaPrivateKey;
+
+    private static PGPPublicKey elGamalPublicKey;
+
+    private static PGPPrivateKey elGamalPrivateKey;
+
+    private TestRunner runner;
+
+    @Mock
+    private PGPPublicKeyService publicKeyService;
+
+    @BeforeClass
+    public static void setKeys() throws Exception {
+        rsaSecretKey = PGPSecretKeyGenerator.generateRsaSecretKey(PASSPHRASE.toCharArray());
+        final PGPSecretKeyRing dsaElGamalSecretKeyRing = PGPSecretKeyGenerator.generateDsaElGamalSecretKeyRing(PASSPHRASE.toCharArray());
+
+        final PBESecretKeyDecryptor decryptor = new JcePBESecretKeyDecryptorBuilder().build(PASSPHRASE.toCharArray());
+        rsaPrivateKey = rsaSecretKey.extractPrivateKey(decryptor);
+        for (final PGPSecretKey secretKey : dsaElGamalSecretKeyRing) {
+            final PGPPublicKey publicKey = secretKey.getPublicKey();
+            if (PGPPublicKey.ELGAMAL_ENCRYPT == publicKey.getAlgorithm()) {
+                elGamalPrivateKey = secretKey.extractPrivateKey(decryptor);
+                elGamalPublicKey = publicKey;
+            }
+        }
+    }
+
+    @Before
+    public void setRunner() {
+        runner = TestRunners.newTestRunner(new EncryptContentPGP());
+    }
+
+    @Test
+    public void testMissingProperties() {
+        runner.assertNotValid();
+    }
+
+    @Test
+    public void testSuccessPasswordBasedEncryptionDefaultProperties() throws IOException, PGPException {
+        runner.setProperty(EncryptContentPGP.PASSPHRASE, PASSPHRASE);
+        runner.enqueue(DATA);
+        runner.run();
+
+        assertSuccess(DEFAULT_SYMMETRIC_KEY_ALGORITHM, PASSPHRASE.toCharArray());
+    }
+
+    @Test
+    public void testSuccessPasswordBasedEncryptionSymmetricKeyAlgorithms() throws IOException, PGPException {
+        for (final SymmetricKeyAlgorithm symmetricKeyAlgorithm : SymmetricKeyAlgorithm.values()) {
+            runner = TestRunners.newTestRunner(new EncryptContentPGP());
+            runner.setProperty(EncryptContentPGP.PASSPHRASE, PASSPHRASE);
+            runner.setProperty(EncryptContentPGP.SYMMETRIC_KEY_ALGORITHM, symmetricKeyAlgorithm.toString());
+            runner.enqueue(DATA);
+            runner.run();
+            assertSuccess(symmetricKeyAlgorithm, PASSPHRASE.toCharArray());
+        }
+    }
+
+    @Test
+    public void testSuccessPasswordBasedEncryptionCompressionAlgorithms() throws IOException, PGPException {
+        for (final CompressionAlgorithm compressionAlgorithm : CompressionAlgorithm.values()) {
+            runner = TestRunners.newTestRunner(new EncryptContentPGP());
+            runner.setProperty(EncryptContentPGP.PASSPHRASE, PASSPHRASE);
+            runner.setProperty(EncryptContentPGP.COMPRESSION_ALGORITHM, compressionAlgorithm.toString());
+            runner.enqueue(DATA);
+            runner.run();
+            assertSuccess(DEFAULT_SYMMETRIC_KEY_ALGORITHM, PASSPHRASE.toCharArray());
+        }
+    }
+
+    @Test
+    public void testSuccessPasswordBasedEncryptionFileEncodingAscii() throws IOException, PGPException {
+        runner.setProperty(EncryptContentPGP.PASSPHRASE, PASSPHRASE);
+        runner.setProperty(EncryptContentPGP.FILE_ENCODING, FileEncoding.ASCII.toString());
+        runner.enqueue(DATA);
+        runner.run();
+        assertSuccess(DEFAULT_SYMMETRIC_KEY_ALGORITHM, PASSPHRASE.toCharArray());
+    }
+
+    @Test
+    public void testSuccessPublicKeyEncryptionRsaPublicKey() throws IOException, InitializationException, PGPException {
+        final PGPPublicKey publicKey = rsaSecretKey.getPublicKey();
+        setPublicKeyService(publicKey);
+        final String publicKeyIdSearch = Long.toHexString(publicKey.getKeyID()).toUpperCase();
+        when(publicKeyService.findPublicKey(eq(publicKeyIdSearch))).thenReturn(Optional.of(publicKey));
+
+        runner.enqueue(DATA);
+        runner.run();
+        assertSuccess(rsaPrivateKey);
+    }
+

Review comment:
       I'd also like to see a unit test here for using both passphrase and public key service.

##########
File path: nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/DecryptContentPGP.java
##########
@@ -0,0 +1,386 @@
+/*
+ * 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.pgp;
+
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+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.flowfile.FlowFile;
+import org.apache.nifi.pgp.service.api.PGPPrivateKeyService;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.io.StreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.pgp.exception.PGPDecryptionException;
+import org.apache.nifi.processors.pgp.exception.PGPProcessException;
+import org.apache.nifi.stream.io.StreamUtils;
+
+import org.apache.nifi.util.StringUtils;
+import org.bouncycastle.openpgp.PGPCompressedData;
+import org.bouncycastle.openpgp.PGPEncryptedData;
+import org.bouncycastle.openpgp.PGPEncryptedDataList;
+import org.bouncycastle.openpgp.PGPException;
+import org.bouncycastle.openpgp.PGPLiteralData;
+import org.bouncycastle.openpgp.PGPObjectFactory;
+import org.bouncycastle.openpgp.PGPPBEEncryptedData;
+import org.bouncycastle.openpgp.PGPPrivateKey;
+import org.bouncycastle.openpgp.PGPPublicKeyEncryptedData;
+import org.bouncycastle.openpgp.PGPUtil;
+import org.bouncycastle.openpgp.jcajce.JcaPGPObjectFactory;
+import org.bouncycastle.openpgp.operator.PBEDataDecryptorFactory;
+import org.bouncycastle.openpgp.operator.PublicKeyDataDecryptorFactory;
+import org.bouncycastle.openpgp.operator.bc.BcPBEDataDecryptorFactory;
+import org.bouncycastle.openpgp.operator.bc.BcPGPDigestCalculatorProvider;
+import org.bouncycastle.openpgp.operator.bc.BcPublicKeyDataDecryptorFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.UncheckedIOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * Decrypt Content using Open Pretty Good Privacy decryption methods
+ */
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Tags({"PGP", "GPG", "OpenPGP", "Encryption", "RFC 4880"})
+@CapabilityDescription("Decrypt Contents of OpenPGP Messages")
+@WritesAttributes({
+        @WritesAttribute(attribute = PGPAttributeKey.LITERAL_DATA_FILENAME, description = "Filename from decrypted Literal Data"),
+        @WritesAttribute(attribute = PGPAttributeKey.LITERAL_DATA_MODIFIED, description = "Modified Date from decrypted Literal Data"),
+        @WritesAttribute(attribute = PGPAttributeKey.SYMMETRIC_KEY_ALGORITHM_ID, description = "Symmetric-Key Algorithm Identifier")
+})
+public class DecryptContentPGP extends AbstractProcessor {
+
+    public static final Relationship SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Decryption Succeeded")
+            .build();
+
+    public static final Relationship FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Decryption Failed")
+            .build();
+
+    public static final PropertyDescriptor PASSPHRASE = new PropertyDescriptor.Builder()
+            .name("passphrase")
+            .displayName("Passphrase")
+            .description("Passphrase used for decrypting data encrypted with Password-Based Encryption")
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor PRIVATE_KEY_SERVICE = new PropertyDescriptor.Builder()
+            .name("private-key-service")
+            .displayName("Private Key Service")
+            .description("PGP Private Key Service for decrypting data encrypted with Public Key Encryption")
+            .identifiesControllerService(PGPPrivateKeyService.class)
+            .build();
+
+    private static final Set<Relationship> RELATIONSHIPS = new HashSet<>(Arrays.asList(SUCCESS, FAILURE));
+
+    private static final List<PropertyDescriptor> DESCRIPTORS = Arrays.asList(
+            PASSPHRASE,
+            PRIVATE_KEY_SERVICE
+    );
+
+    private static final String PASSWORD_BASED_ENCRYPTION = "Password-Based Encryption";
+
+    private static final String PUBLIC_KEY_ENCRYPTION = "Public Key Encryption";
+
+    /**
+     * Get Relationships
+     *
+     * @return Processor Relationships
+     */
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    /**
+     * Get Supported Property Descriptors
+     *
+     * @return Processor Supported Property Descriptors
+     */
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    /**
+     * On Trigger decrypts Flow File contents using configured properties
+     *
+     * @param context Process Context
+     * @param session Process Session
+     */
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final char[] passphrase = getPassphrase(context);
+        final PGPPrivateKeyService privateKeyService = getPrivateKeyService(context);
+        final DecryptStreamCallback callback = new DecryptStreamCallback(passphrase, privateKeyService);
+
+        try {
+            flowFile = session.write(flowFile, callback);
+            flowFile = session.putAllAttributes(flowFile, callback.attributes);
+            session.transfer(flowFile, SUCCESS);
+        } catch (final RuntimeException e) {
+            getLogger().error("Decryption Failed {}", flowFile, e);
+            session.transfer(flowFile, FAILURE);
+        }
+    }
+
+    /**
+     * Custom Validate requires at least one decryption property to be configured
+     *
+     * @param context Validation Context
+     * @return Collection of Validation Results
+     */
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext context) {
+        final Collection<ValidationResult> results = new ArrayList<>();
+
+        final String passphrase = context.getProperty(PASSPHRASE).getValue();
+        if (StringUtils.isBlank(passphrase)) {
+            final PGPPrivateKeyService privateKeyService = context.getProperty(PRIVATE_KEY_SERVICE).asControllerService(PGPPrivateKeyService.class);
+            if (privateKeyService == null) {
+                final String explanation = String.format("Neither [%s] nor [%s] configured", PASSPHRASE.getDisplayName(), PRIVATE_KEY_SERVICE.getDisplayName());
+                final ValidationResult result = new ValidationResult.Builder()
+                        .valid(false)
+                        .subject(getClass().getSimpleName())
+                        .explanation(explanation)
+                        .build();
+                results.add(result);
+            }
+        }
+
+        return results;
+    }
+
+    private char[] getPassphrase(final ProcessContext context) {
+        char[] passphrase = null;
+        final PropertyValue passphraseProperty = context.getProperty(PASSPHRASE);
+        if (passphraseProperty.isSet()) {
+            passphrase = passphraseProperty.getValue().toCharArray();
+        }
+        return passphrase;
+    }
+
+    private PGPPrivateKeyService getPrivateKeyService(final ProcessContext context) {
+        PGPPrivateKeyService privateKeyService = null;
+        final PropertyValue privateKeyServiceProperty = context.getProperty(PRIVATE_KEY_SERVICE);
+        if (privateKeyServiceProperty.isSet()) {
+            privateKeyService = privateKeyServiceProperty.asControllerService(PGPPrivateKeyService.class);
+        }
+        return privateKeyService;
+    }
+
+    private class DecryptStreamCallback implements StreamCallback {
+        private final char[] passphrase;
+
+        private final PGPPrivateKeyService privateKeyService;
+
+        private final Map<String, String> attributes = new HashMap<>();
+
+        public DecryptStreamCallback(final char[] passphrase, final PGPPrivateKeyService privateKeyService) {
+            this.passphrase = passphrase;
+            this.privateKeyService = privateKeyService;
+        }
+
+        /**
+         * Process Input Stream containing encrypted data and write decrypted contents to Output Stream
+         *
+         * @param inputStream  Input Stream containing encrypted data
+         * @param outputStream Output Stream for decrypted contents
+         * @throws IOException Thrown when unable to read or write streams
+         */
+        @Override
+        public void process(final InputStream inputStream, final OutputStream outputStream) throws IOException {
+            final InputStream decoderInputStream = PGPUtil.getDecoderStream(inputStream);
+            final PGPEncryptedDataList encryptedDataList = getEncryptedDataList(decoderInputStream);
+
+            for (final PGPEncryptedData encryptedData : encryptedDataList) {
+                final PGPLiteralData literalData = getLiteralData(encryptedData);
+
+                attributes.put(PGPAttributeKey.LITERAL_DATA_FILENAME, literalData.getFileName());
+                attributes.put(PGPAttributeKey.LITERAL_DATA_MODIFIED, Long.toString(literalData.getModificationTime().getTime()));
+
+                getLogger().debug("PGP Decrypted File Name [{}] Modified [{}]", literalData.getFileName(), literalData.getModificationTime());
+                StreamUtils.copy(literalData.getInputStream(), outputStream);
+
+                if (isVerified(encryptedData)) {
+                    getLogger().debug("PGP Encrypted Data Verified");
+                } else {
+                    final String message = String.format("PGP Encrypted Data [%s] Not Verified", encryptedData.getClass().getSimpleName());
+                    throw new PGPDecryptionException(message);
+                }
+            }
+        }
+
+        private PGPLiteralData getLiteralData(final PGPEncryptedData encryptedData) {
+            try {
+                final InputStream decryptedDataStream = getDecryptedDataStream(encryptedData);
+                final PGPObjectFactory objectFactory = new JcaPGPObjectFactory(decryptedDataStream);
+                return getLiteralData(objectFactory);
+            } catch (final PGPException e) {
+                final String message = String.format("PGP Decryption Failed [%s]", getEncryptedDataType(encryptedData));
+                throw new PGPDecryptionException(message, e);
+            }
+        }
+
+        private PGPLiteralData getLiteralData(final PGPObjectFactory objectFactory) throws PGPException {
+            PGPLiteralData literalData = null;
+
+            for (final Object object : objectFactory) {
+                if (object instanceof PGPCompressedData) {
+                    final PGPCompressedData compressedData = (PGPCompressedData) object;
+                    getLogger().debug("PGP Compressed Data Algorithm [{}] Found", compressedData.getAlgorithm());
+                    final PGPObjectFactory compressedObjectFactory = new JcaPGPObjectFactory(compressedData.getDataStream());
+                    literalData = getLiteralData(compressedObjectFactory);
+                    break;
+                } else if (object instanceof PGPLiteralData) {
+                    literalData = (PGPLiteralData) object;
+                    break;
+                }
+            }
+
+            if (literalData == null) {
+                throw new PGPProcessException("PGP Literal Data not found");
+            }
+
+            return literalData;
+        }
+
+        private InputStream getDecryptedDataStream(final PGPEncryptedData encryptedData) throws PGPException {
+            getLogger().debug("PGP Encrypted Data [{}] Found", getEncryptedDataType(encryptedData));
+
+            if (encryptedData instanceof PGPPBEEncryptedData) {
+                return getDecryptedDataStream((PGPPBEEncryptedData) encryptedData);
+            } else if (encryptedData instanceof PGPPublicKeyEncryptedData) {
+                return getDecryptedDataStream((PGPPublicKeyEncryptedData) encryptedData);
+            } else {
+                final String message = String.format("PGP Encrypted Data [%s] Not Supported", getEncryptedDataType(encryptedData));
+                throw new UnsupportedOperationException(message);
+            }
+        }
+
+        private InputStream getDecryptedDataStream(final PGPPBEEncryptedData passwordBasedEncryptedData) throws PGPException {
+            if (passphrase == null) {
+                throw new PGPProcessException("PGP Password-Based Encryption Found: Passphrase not configured");
+            } else {
+                final PBEDataDecryptorFactory decryptorFactory = new BcPBEDataDecryptorFactory(passphrase, new BcPGPDigestCalculatorProvider());
+                final int symmetricAlgorithm = passwordBasedEncryptedData.getSymmetricAlgorithm(decryptorFactory);
+                attributes.put(PGPAttributeKey.SYMMETRIC_KEY_ALGORITHM_ID, Integer.toString(symmetricAlgorithm));
+                return passwordBasedEncryptedData.getDataStream(decryptorFactory);
+            }
+        }
+
+        private InputStream getDecryptedDataStream(final PGPPublicKeyEncryptedData publicKeyEncryptedData) throws PGPException {
+            if (privateKeyService == null) {
+                throw new PGPProcessException("PGP Public Key Encryption Found: Private Key Service not configured");
+            } else {
+                final long keyId = publicKeyEncryptedData.getKeyID();
+                final Optional<PGPPrivateKey> foundPrivateKey = privateKeyService.findPrivateKey(keyId);
+                if (foundPrivateKey.isPresent()) {
+                    final PGPPrivateKey privateKey = foundPrivateKey.get();
+                    final PublicKeyDataDecryptorFactory decryptorFactory = new BcPublicKeyDataDecryptorFactory(privateKey);
+                    final int symmetricAlgorithm = publicKeyEncryptedData.getSymmetricAlgorithm(decryptorFactory);
+                    attributes.put(PGPAttributeKey.SYMMETRIC_KEY_ALGORITHM_ID, Integer.toString(symmetricAlgorithm));
+                    return publicKeyEncryptedData.getDataStream(decryptorFactory);

Review comment:
       One note here:
   -  When I specify only valid Public/Private Key services in the EncryptContentPGP and DecryptContentPGP processors but no passphrases, both encryption and decryption work.  Likewise, when I specify only a matching passphrase in both processors, both encryption and decryption work.
   - However, when I use the above working configuration of Public/Private Key services, and then add the same matching passphrase to both processors, I get the following error in DecryptContentPGP:
   `2021-04-02 16:09:17,767 ERROR [Timer-Driven Process Thread-8] o.a.n.processors.pgp.DecryptContentPGP DecryptContentPGP[id=93e3a4e0-0178-1000-a8d0-00468239c9f4] Decryption Failed StandardFlowFileRecord[uuid=9675b046-b929-4b38-81fa-e3e20f8149e8,claim=StandardContentClaim [resourceClaim=StandardResourceClaim[id=1617385030974-1, container=default, section=1], offset=17471, length=672],offset=0,name=9675b046-b929-4b38-81fa-e3e20f8149e8,size=672]: org.apache.nifi.processors.pgp.exception.PGPDecryptionException: PGP Decryption Failed [Public Key Encryption]
   org.apache.nifi.processors.pgp.exception.PGPDecryptionException: PGP Decryption Failed [Public Key Encryption]
   	at org.apache.nifi.processors.pgp.DecryptContentPGP$DecryptStreamCallback.getLiteralData(DecryptContentPGP.java:262)
   	at org.apache.nifi.processors.pgp.DecryptContentPGP$DecryptStreamCallback.process(DecryptContentPGP.java:238)
   	at org.apache.nifi.controller.repository.StandardProcessSession.write(StandardProcessSession.java:3054)
   	at org.apache.nifi.processors.pgp.DecryptContentPGP.onTrigger(DecryptContentPGP.java:159)
   	at org.apache.nifi.processor.AbstractProcessor.onTrigger(AbstractProcessor.java:27)
   	at org.apache.nifi.controller.StandardProcessorNode.onTrigger(StandardProcessorNode.java:1180)
   	at org.apache.nifi.controller.tasks.ConnectableTask.invoke(ConnectableTask.java:214)
   	at org.apache.nifi.controller.scheduling.TimerDrivenSchedulingAgent$1.run(TimerDrivenSchedulingAgent.java:103)
   	at org.apache.nifi.engine.FlowEngine$2.run(FlowEngine.java:110)
   	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
   	at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
   	at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
   	at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
   	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
   	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
   	at java.lang.Thread.run(Thread.java:748)
   Caused by: org.bouncycastle.openpgp.PGPException: Exception starting decryption
   	at org.bouncycastle.openpgp.PGPPublicKeyEncryptedData.getDataStream(Unknown Source)
   	at org.apache.nifi.processors.pgp.DecryptContentPGP$DecryptStreamCallback.getDecryptedDataStream(DecryptContentPGP.java:324)
   	at org.apache.nifi.processors.pgp.DecryptContentPGP$DecryptStreamCallback.getDecryptedDataStream(DecryptContentPGP.java:295)
   	at org.apache.nifi.processors.pgp.DecryptContentPGP$DecryptStreamCallback.getLiteralData(DecryptContentPGP.java:257)
   	... 15 common frames omitted
   Caused by: java.io.EOFException: null
   	at org.bouncycastle.openpgp.PGPEncryptedData$TruncatedStream.<init>(Unknown Source)
   	... 19 common frames omitted`

##########
File path: nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/EncryptContentPGP.java
##########
@@ -0,0 +1,379 @@
+/*
+ * 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.pgp;
+
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+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.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.pgp.service.api.PGPPublicKeyService;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.io.StreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.pgp.attributes.CompressionAlgorithm;
+import org.apache.nifi.processors.pgp.attributes.FileEncoding;
+import org.apache.nifi.processors.pgp.attributes.SymmetricKeyAlgorithm;
+import org.apache.nifi.processors.pgp.exception.PGPEncryptionException;
+import org.apache.nifi.stream.io.StreamUtils;
+import org.apache.nifi.util.StringUtils;
+
+import org.bouncycastle.bcpg.ArmoredOutputStream;
+import org.bouncycastle.openpgp.PGPCompressedDataGenerator;
+import org.bouncycastle.openpgp.PGPEncryptedDataGenerator;
+import org.bouncycastle.openpgp.PGPException;
+import org.bouncycastle.openpgp.PGPLiteralData;
+import org.bouncycastle.openpgp.PGPLiteralDataGenerator;
+import org.bouncycastle.openpgp.PGPPublicKey;
+import org.bouncycastle.openpgp.operator.PGPDataEncryptorBuilder;
+import org.bouncycastle.openpgp.operator.PGPKeyEncryptionMethodGenerator;
+import org.bouncycastle.openpgp.operator.bc.BcPGPDataEncryptorBuilder;
+import org.bouncycastle.openpgp.operator.bc.BcPublicKeyKeyEncryptionMethodGenerator;
+import org.bouncycastle.openpgp.operator.jcajce.JcePBEKeyEncryptionMethodGenerator;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.security.SecureRandom;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * Encrypt Content using Open Pretty Good Privacy encryption methods
+ */
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Tags({"PGP", "GPG", "OpenPGP", "Encryption", "RFC 4880"})
+@CapabilityDescription("Encrypt Contents using OpenPGP")
+@WritesAttributes({
+        @WritesAttribute(attribute = PGPAttributeKey.SYMMETRIC_KEY_ALGORITHM, description = "Symmetric-Key Algorithm"),
+        @WritesAttribute(attribute = PGPAttributeKey.SYMMETRIC_KEY_ALGORITHM_BLOCK_CIPHER, description = "Symmetric-Key Algorithm Block Cipher"),
+        @WritesAttribute(attribute = PGPAttributeKey.SYMMETRIC_KEY_ALGORITHM_KEY_SIZE, description = "Symmetric-Key Algorithm Key Size"),
+        @WritesAttribute(attribute = PGPAttributeKey.SYMMETRIC_KEY_ALGORITHM_ID, description = "Symmetric-Key Algorithm Identifier"),
+        @WritesAttribute(attribute = PGPAttributeKey.FILE_ENCODING, description = "File Encoding"),
+        @WritesAttribute(attribute = PGPAttributeKey.COMPRESS_ALGORITHM, description = "Compression Algorithm"),
+        @WritesAttribute(attribute = PGPAttributeKey.COMPRESS_ALGORITHM_ID, description = "Compression Algorithm Identifier"),
+})
+public class EncryptContentPGP extends AbstractProcessor {
+
+    public static final Relationship SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Encryption Succeeded")
+            .build();
+
+    public static final Relationship FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Encryption Failed")
+            .build();
+
+    public static final PropertyDescriptor SYMMETRIC_KEY_ALGORITHM = new PropertyDescriptor.Builder()
+            .name("symmetric-key-algorithm")
+            .displayName("Symmetric-Key Algorithm")
+            .description("Symmetric-Key Algorithm for encryption")
+            .required(true)
+            .defaultValue(SymmetricKeyAlgorithm.AES_256.toString())
+            .allowableValues(SymmetricKeyAlgorithm.values())
+            .build();
+
+    public static final PropertyDescriptor COMPRESSION_ALGORITHM = new PropertyDescriptor.Builder()
+            .name("compression-algorithm")
+            .displayName("Compression Algorithm")
+            .description("Compression Algorithm for encryption")
+            .required(true)
+            .defaultValue(CompressionAlgorithm.ZIP.toString())
+            .allowableValues(CompressionAlgorithm.values())
+            .build();
+
+    public static final PropertyDescriptor FILE_ENCODING = new PropertyDescriptor.Builder()
+            .name("file-encoding")
+            .displayName("File Encoding")
+            .description("File Encoding for encryption")
+            .required(true)
+            .defaultValue(FileEncoding.BINARY.toString())
+            .allowableValues(FileEncoding.values())
+            .build();
+
+    public static final PropertyDescriptor PASSPHRASE = new PropertyDescriptor.Builder()
+            .name("passphrase")
+            .displayName("Passphrase")
+            .description("Passphrase used for encrypting data with Password-Based Encryption")
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor PUBLIC_KEY_SERVICE = new PropertyDescriptor.Builder()
+            .name("public-key-service")
+            .displayName("Public Key Service")
+            .description("PGP Public Key Service for encrypting data with Public Key Encryption")
+            .identifiesControllerService(PGPPublicKeyService.class)
+            .build();
+
+    public static final PropertyDescriptor PUBLIC_KEY_SEARCH = new PropertyDescriptor.Builder()
+            .name("public-key-search")
+            .displayName("Public Key Search")
+            .description("PGP Public Key Search will be used to match against the User ID or Key ID when formatted as uppercase hexadecimal string of 16 characters")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .addValidator(StandardValidators.ATTRIBUTE_EXPRESSION_LANGUAGE_VALIDATOR)
+            .dependsOn(PUBLIC_KEY_SERVICE)
+            .build();
+
+    /** Enable Integrity Protection as described in RFC 4880 Section 5.13 */
+    private static final boolean ENCRYPTION_INTEGRITY_PACKET_ENABLED = true;
+
+    private static final int OUTPUT_BUFFER_SIZE = 8192;
+
+    private static final Set<Relationship> RELATIONSHIPS = new HashSet<>(Arrays.asList(SUCCESS, FAILURE));
+
+    private static final List<PropertyDescriptor> DESCRIPTORS = Arrays.asList(
+            SYMMETRIC_KEY_ALGORITHM,
+            COMPRESSION_ALGORITHM,
+            FILE_ENCODING,
+            PASSPHRASE,
+            PUBLIC_KEY_SERVICE,
+            PUBLIC_KEY_SEARCH
+    );
+
+    /**
+     * Get Relationships
+     *
+     * @return Processor Relationships
+     */
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    /**
+     * Get Supported Property Descriptors
+     *
+     * @return Processor Supported Property Descriptors
+     */
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    /**
+     * On Trigger encrypts Flow File contents using configured properties
+     *
+     * @param context Process Context
+     * @param session Process Session
+     */
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        try {
+            final SymmetricKeyAlgorithm symmetricKeyAlgorithm = getSymmetricKeyAlgorithm(context);
+            final FileEncoding fileEncoding = getFileEncoding(context);
+            final CompressionAlgorithm compressionAlgorithm = getCompressionAlgorithm(context);
+            final StreamCallback callback = getEncryptStreamCallback(context, flowFile, symmetricKeyAlgorithm, compressionAlgorithm, fileEncoding);
+            flowFile = session.write(flowFile, callback);
+
+            final Map<String, String> attributes = getAttributes(symmetricKeyAlgorithm, fileEncoding, compressionAlgorithm);
+            flowFile = session.putAllAttributes(flowFile, attributes);
+
+            session.transfer(flowFile, SUCCESS);
+        } catch (final RuntimeException e) {
+            getLogger().error("Encryption Failed {}", flowFile, e);
+            session.transfer(flowFile, FAILURE);
+        }
+    }
+
+    /**
+     * Custom Validate requires at least one encryption property to be configured
+     *
+     * @param context Validation Context
+     * @return Collection of Validation Results
+     */
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext context) {
+        final Collection<ValidationResult> results = new ArrayList<>();
+
+        final String passphrase = context.getProperty(PASSPHRASE).getValue();
+        if (StringUtils.isBlank(passphrase)) {
+            final PGPPublicKeyService publicKeyService = context.getProperty(PUBLIC_KEY_SERVICE).asControllerService(PGPPublicKeyService.class);
+            if (publicKeyService == null) {
+                final String explanation = String.format("Neither [%s] nor [%s] configured", PASSPHRASE.getDisplayName(), PUBLIC_KEY_SERVICE.getDisplayName());
+                final ValidationResult result = new ValidationResult.Builder()
+                        .valid(false)
+                        .subject(getClass().getSimpleName())
+                        .explanation(explanation)
+                        .build();
+                results.add(result);
+            }
+        }
+
+        return results;
+    }
+
+    private StreamCallback getEncryptStreamCallback(final ProcessContext context, final FlowFile flowFile,
+                                                    final SymmetricKeyAlgorithm symmetricKeyAlgorithm,
+                                                    final CompressionAlgorithm compressionAlgorithm,
+                                                    final FileEncoding fileEncoding) {
+        final SecureRandom secureRandom = new SecureRandom();
+        final PGPDataEncryptorBuilder dataEncryptorBuilder = new BcPGPDataEncryptorBuilder(symmetricKeyAlgorithm.getId())
+                .setSecureRandom(secureRandom)
+                .setWithIntegrityPacket(ENCRYPTION_INTEGRITY_PACKET_ENABLED);
+        final PGPEncryptedDataGenerator encryptedDataGenerator = new PGPEncryptedDataGenerator(dataEncryptorBuilder);
+        final List<PGPKeyEncryptionMethodGenerator> methodGenerators = getEncryptionMethodGenerators(context, flowFile, secureRandom);
+        methodGenerators.forEach(encryptedDataGenerator::addMethod);
+
+        final String filename = flowFile.getAttribute(CoreAttributes.FILENAME.key());
+        return new EncryptStreamCallback(filename, fileEncoding, encryptedDataGenerator, compressionAlgorithm);
+    }
+
+    private List<PGPKeyEncryptionMethodGenerator> getEncryptionMethodGenerators(final ProcessContext context,
+                                                                                final FlowFile flowFile,
+                                                                                final SecureRandom secureRandom) {
+        final List<PGPKeyEncryptionMethodGenerator> generators = new ArrayList<>();
+
+        final PropertyValue passphraseProperty = context.getProperty(PASSPHRASE);
+        if (passphraseProperty.isSet()) {
+            final char[] passphrase = passphraseProperty.getValue().toCharArray();
+            generators.add(new JcePBEKeyEncryptionMethodGenerator(passphrase).setSecureRandom(secureRandom));
+        }
+
+        final PropertyValue publicKeySearchProperty = context.getProperty(PUBLIC_KEY_SEARCH);
+        if (publicKeySearchProperty.isSet()) {
+            final String publicKeySearch = publicKeySearchProperty.evaluateAttributeExpressions(flowFile).getValue();
+            getLogger().debug("Public Key Search [{}]", publicKeySearch);
+
+            final PGPPublicKeyService publicKeyService = context.getProperty(PUBLIC_KEY_SERVICE).asControllerService(PGPPublicKeyService.class);
+            final Optional<PGPPublicKey> optionalPublicKey = publicKeyService.findPublicKey(publicKeySearch);

Review comment:
       This can throw an NPE if you do the following:
   
   1. Configure the processor with a Public Key Service
   2. Specify a Public Key Search value
   3. Set the Public Key Service to None
   4. Specify a Passphrase
   5. Run a flow file through the processor (this gives you the NPE)




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

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



[GitHub] [nifi] gresockj commented on pull request #4842: NIFI-8251 Added EncryptContentPGP and DecryptContentPGP Processors

Posted by GitBox <gi...@apache.org>.
gresockj commented on pull request #4842:
URL: https://github.com/apache/nifi/pull/4842#issuecomment-814046306


   I just tested it out, and all of the above issues were resolved.


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

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



[GitHub] [nifi] exceptionfactory commented on a change in pull request #4842: NIFI-8251 Added EncryptContentPGP and DecryptContentPGP Processors

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #4842:
URL: https://github.com/apache/nifi/pull/4842#discussion_r607270068



##########
File path: nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-service/src/main/java/org/apache/nifi/pgp/service/standard/StandardPGPPrivateKeyService.java
##########
@@ -0,0 +1,282 @@
+/*
+ * 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.pgp.service.standard;
+
+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.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.pgp.service.api.PGPPrivateKeyService;
+import org.apache.nifi.pgp.service.standard.exception.PGPConfigurationException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.StringUtils;
+
+import org.bouncycastle.openpgp.PGPException;
+import org.bouncycastle.openpgp.PGPPrivateKey;
+import org.bouncycastle.openpgp.PGPSecretKey;
+import org.bouncycastle.openpgp.PGPSecretKeyRing;
+import org.bouncycastle.openpgp.PGPSecretKeyRingCollection;
+import org.bouncycastle.openpgp.PGPUtil;
+import org.bouncycastle.openpgp.operator.KeyFingerPrintCalculator;
+import org.bouncycastle.openpgp.operator.PBESecretKeyDecryptor;
+import org.bouncycastle.openpgp.operator.jcajce.JcaKeyFingerprintCalculator;
+import org.bouncycastle.openpgp.operator.jcajce.JcePBESecretKeyDecryptorBuilder;
+
+import java.io.ByteArrayInputStream;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+/**
+ * Standard Pretty Good Privacy Private Key Service reads Private Keys from configured Keyring files or properties
+ */
+@Tags({"PGP", "GPG", "OpenPGP", "Encryption", "Private", "Key", "RFC 4880"})
+@CapabilityDescription("PGP Private Key Service provides Private Keys loaded from files or properties")
+public class StandardPGPPrivateKeyService extends AbstractControllerService implements PGPPrivateKeyService {
+    public static final PropertyDescriptor KEYRING_FILE = new PropertyDescriptor.Builder()
+            .name("keyring-file")
+            .displayName("Keyring File")
+            .description("File path for PGP Keyring or Secret Key encoded in binary or ASCII Armor")

Review comment:
       Sounds good, will make the adjustment.

##########
File path: nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-service/src/main/java/org/apache/nifi/pgp/service/standard/StandardPGPPublicKeyService.java
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.pgp.service.standard;
+
+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.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.pgp.service.api.PGPPublicKeyService;
+import org.apache.nifi.pgp.service.standard.exception.PGPConfigurationException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.StringUtils;
+
+import org.bouncycastle.openpgp.PGPException;
+import org.bouncycastle.openpgp.PGPPublicKey;
+import org.bouncycastle.openpgp.PGPPublicKeyRingCollection;
+import org.bouncycastle.openpgp.PGPUtil;
+import org.bouncycastle.openpgp.operator.KeyFingerPrintCalculator;
+import org.bouncycastle.openpgp.operator.jcajce.JcaKeyFingerprintCalculator;
+
+import java.io.ByteArrayInputStream;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+
+/**
+ * Standard Pretty Good Privacy Public Key Service reads Public Keys from configured Keyring files
+ */
+@Tags({"PGP", "GPG", "OpenPGP", "Encryption", "Private", "Key", "RFC 4880"})
+@CapabilityDescription("PGP Public Key Service providing Public Keys loaded from files")
+public class StandardPGPPublicKeyService extends AbstractControllerService implements PGPPublicKeyService {
+    public static final PropertyDescriptor KEYRING_FILE = new PropertyDescriptor.Builder()
+            .name("keyring-file")
+            .displayName("Keyring File")
+            .description("File path for PGP Keyring or Public Key encoded in binary or ASCII Armor")

Review comment:
       Sounds good, will make the adjustment.




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

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



[GitHub] [nifi] exceptionfactory commented on a change in pull request #4842: NIFI-8251 Added EncryptContentPGP and DecryptContentPGP Processors

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #4842:
URL: https://github.com/apache/nifi/pull/4842#discussion_r607291909



##########
File path: nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/test/java/org/apache/nifi/processors/pgp/EncryptContentPGPTest.java
##########
@@ -0,0 +1,293 @@
+/*
+ * 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.pgp;
+
+import org.apache.nifi.pgp.service.api.PGPPublicKeyService;
+import org.apache.nifi.processors.pgp.attributes.CompressionAlgorithm;
+import org.apache.nifi.processors.pgp.attributes.FileEncoding;
+import org.apache.nifi.processors.pgp.attributes.SymmetricKeyAlgorithm;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.stream.io.StreamUtils;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.apache.nifi.pgp.util.PGPSecretKeyGenerator;
+
+import org.bouncycastle.bcpg.ArmoredInputStream;
+import org.bouncycastle.openpgp.PGPCompressedData;
+import org.bouncycastle.openpgp.PGPEncryptedData;
+import org.bouncycastle.openpgp.PGPEncryptedDataList;
+import org.bouncycastle.openpgp.PGPException;
+import org.bouncycastle.openpgp.PGPLiteralData;
+import org.bouncycastle.openpgp.PGPObjectFactory;
+import org.bouncycastle.openpgp.PGPPBEEncryptedData;
+import org.bouncycastle.openpgp.PGPPrivateKey;
+import org.bouncycastle.openpgp.PGPPublicKey;
+import org.bouncycastle.openpgp.PGPPublicKeyEncryptedData;
+import org.bouncycastle.openpgp.PGPSecretKey;
+import org.bouncycastle.openpgp.PGPSecretKeyRing;
+import org.bouncycastle.openpgp.jcajce.JcaPGPObjectFactory;
+import org.bouncycastle.openpgp.operator.PBEDataDecryptorFactory;
+import org.bouncycastle.openpgp.operator.PBESecretKeyDecryptor;
+import org.bouncycastle.openpgp.operator.PublicKeyDataDecryptorFactory;
+import org.bouncycastle.openpgp.operator.bc.BcPBEDataDecryptorFactory;
+import org.bouncycastle.openpgp.operator.bc.BcPGPDigestCalculatorProvider;
+import org.bouncycastle.openpgp.operator.bc.BcPublicKeyDataDecryptorFactory;
+import org.bouncycastle.openpgp.operator.jcajce.JcePBESecretKeyDecryptorBuilder;
+
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Optional;
+import java.util.UUID;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.when;
+
+@RunWith(MockitoJUnitRunner.class)
+public class EncryptContentPGPTest {
+    private static final String PASSPHRASE = UUID.randomUUID().toString();
+
+    private static final String DATA = String.class.getName();
+
+    private static final SymmetricKeyAlgorithm DEFAULT_SYMMETRIC_KEY_ALGORITHM = SymmetricKeyAlgorithm.valueOf(EncryptContentPGP.SYMMETRIC_KEY_ALGORITHM.getDefaultValue());
+
+    private static final String SERVICE_ID = PGPPublicKeyService.class.getName();
+
+    private static PGPSecretKey rsaSecretKey;
+
+    private static PGPPrivateKey rsaPrivateKey;
+
+    private static PGPPublicKey elGamalPublicKey;
+
+    private static PGPPrivateKey elGamalPrivateKey;
+
+    private TestRunner runner;
+
+    @Mock
+    private PGPPublicKeyService publicKeyService;
+
+    @BeforeClass
+    public static void setKeys() throws Exception {
+        rsaSecretKey = PGPSecretKeyGenerator.generateRsaSecretKey(PASSPHRASE.toCharArray());
+        final PGPSecretKeyRing dsaElGamalSecretKeyRing = PGPSecretKeyGenerator.generateDsaElGamalSecretKeyRing(PASSPHRASE.toCharArray());
+
+        final PBESecretKeyDecryptor decryptor = new JcePBESecretKeyDecryptorBuilder().build(PASSPHRASE.toCharArray());
+        rsaPrivateKey = rsaSecretKey.extractPrivateKey(decryptor);
+        for (final PGPSecretKey secretKey : dsaElGamalSecretKeyRing) {
+            final PGPPublicKey publicKey = secretKey.getPublicKey();
+            if (PGPPublicKey.ELGAMAL_ENCRYPT == publicKey.getAlgorithm()) {
+                elGamalPrivateKey = secretKey.extractPrivateKey(decryptor);
+                elGamalPublicKey = publicKey;
+            }
+        }
+    }
+
+    @Before
+    public void setRunner() {
+        runner = TestRunners.newTestRunner(new EncryptContentPGP());
+    }
+
+    @Test
+    public void testMissingProperties() {
+        runner.assertNotValid();
+    }
+
+    @Test
+    public void testSuccessPasswordBasedEncryptionDefaultProperties() throws IOException, PGPException {
+        runner.setProperty(EncryptContentPGP.PASSPHRASE, PASSPHRASE);
+        runner.enqueue(DATA);
+        runner.run();
+
+        assertSuccess(DEFAULT_SYMMETRIC_KEY_ALGORITHM, PASSPHRASE.toCharArray());
+    }
+
+    @Test
+    public void testSuccessPasswordBasedEncryptionSymmetricKeyAlgorithms() throws IOException, PGPException {
+        for (final SymmetricKeyAlgorithm symmetricKeyAlgorithm : SymmetricKeyAlgorithm.values()) {
+            runner = TestRunners.newTestRunner(new EncryptContentPGP());
+            runner.setProperty(EncryptContentPGP.PASSPHRASE, PASSPHRASE);
+            runner.setProperty(EncryptContentPGP.SYMMETRIC_KEY_ALGORITHM, symmetricKeyAlgorithm.toString());
+            runner.enqueue(DATA);
+            runner.run();
+            assertSuccess(symmetricKeyAlgorithm, PASSPHRASE.toCharArray());
+        }
+    }
+
+    @Test
+    public void testSuccessPasswordBasedEncryptionCompressionAlgorithms() throws IOException, PGPException {
+        for (final CompressionAlgorithm compressionAlgorithm : CompressionAlgorithm.values()) {
+            runner = TestRunners.newTestRunner(new EncryptContentPGP());
+            runner.setProperty(EncryptContentPGP.PASSPHRASE, PASSPHRASE);
+            runner.setProperty(EncryptContentPGP.COMPRESSION_ALGORITHM, compressionAlgorithm.toString());
+            runner.enqueue(DATA);
+            runner.run();
+            assertSuccess(DEFAULT_SYMMETRIC_KEY_ALGORITHM, PASSPHRASE.toCharArray());
+        }
+    }
+
+    @Test
+    public void testSuccessPasswordBasedEncryptionFileEncodingAscii() throws IOException, PGPException {
+        runner.setProperty(EncryptContentPGP.PASSPHRASE, PASSPHRASE);
+        runner.setProperty(EncryptContentPGP.FILE_ENCODING, FileEncoding.ASCII.toString());
+        runner.enqueue(DATA);
+        runner.run();
+        assertSuccess(DEFAULT_SYMMETRIC_KEY_ALGORITHM, PASSPHRASE.toCharArray());
+    }
+
+    @Test
+    public void testSuccessPublicKeyEncryptionRsaPublicKey() throws IOException, InitializationException, PGPException {
+        final PGPPublicKey publicKey = rsaSecretKey.getPublicKey();
+        setPublicKeyService(publicKey);
+        final String publicKeyIdSearch = Long.toHexString(publicKey.getKeyID()).toUpperCase();
+        when(publicKeyService.findPublicKey(eq(publicKeyIdSearch))).thenReturn(Optional.of(publicKey));
+
+        runner.enqueue(DATA);
+        runner.run();
+        assertSuccess(rsaPrivateKey);
+    }
+

Review comment:
       Thanks for the suggestion, will add a new unit test method for this use case.




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

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



[GitHub] [nifi] exceptionfactory commented on a change in pull request #4842: NIFI-8251 Added EncryptContentPGP and DecryptContentPGP Processors

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #4842:
URL: https://github.com/apache/nifi/pull/4842#discussion_r607277670



##########
File path: nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/EncryptContentPGP.java
##########
@@ -0,0 +1,379 @@
+/*
+ * 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.pgp;
+
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+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.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.pgp.service.api.PGPPublicKeyService;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.io.StreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.pgp.attributes.CompressionAlgorithm;
+import org.apache.nifi.processors.pgp.attributes.FileEncoding;
+import org.apache.nifi.processors.pgp.attributes.SymmetricKeyAlgorithm;
+import org.apache.nifi.processors.pgp.exception.PGPEncryptionException;
+import org.apache.nifi.stream.io.StreamUtils;
+import org.apache.nifi.util.StringUtils;
+
+import org.bouncycastle.bcpg.ArmoredOutputStream;
+import org.bouncycastle.openpgp.PGPCompressedDataGenerator;
+import org.bouncycastle.openpgp.PGPEncryptedDataGenerator;
+import org.bouncycastle.openpgp.PGPException;
+import org.bouncycastle.openpgp.PGPLiteralData;
+import org.bouncycastle.openpgp.PGPLiteralDataGenerator;
+import org.bouncycastle.openpgp.PGPPublicKey;
+import org.bouncycastle.openpgp.operator.PGPDataEncryptorBuilder;
+import org.bouncycastle.openpgp.operator.PGPKeyEncryptionMethodGenerator;
+import org.bouncycastle.openpgp.operator.bc.BcPGPDataEncryptorBuilder;
+import org.bouncycastle.openpgp.operator.bc.BcPublicKeyKeyEncryptionMethodGenerator;
+import org.bouncycastle.openpgp.operator.jcajce.JcePBEKeyEncryptionMethodGenerator;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.security.SecureRandom;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * Encrypt Content using Open Pretty Good Privacy encryption methods
+ */
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Tags({"PGP", "GPG", "OpenPGP", "Encryption", "RFC 4880"})
+@CapabilityDescription("Encrypt Contents using OpenPGP")
+@WritesAttributes({
+        @WritesAttribute(attribute = PGPAttributeKey.SYMMETRIC_KEY_ALGORITHM, description = "Symmetric-Key Algorithm"),
+        @WritesAttribute(attribute = PGPAttributeKey.SYMMETRIC_KEY_ALGORITHM_BLOCK_CIPHER, description = "Symmetric-Key Algorithm Block Cipher"),
+        @WritesAttribute(attribute = PGPAttributeKey.SYMMETRIC_KEY_ALGORITHM_KEY_SIZE, description = "Symmetric-Key Algorithm Key Size"),
+        @WritesAttribute(attribute = PGPAttributeKey.SYMMETRIC_KEY_ALGORITHM_ID, description = "Symmetric-Key Algorithm Identifier"),
+        @WritesAttribute(attribute = PGPAttributeKey.FILE_ENCODING, description = "File Encoding"),
+        @WritesAttribute(attribute = PGPAttributeKey.COMPRESS_ALGORITHM, description = "Compression Algorithm"),
+        @WritesAttribute(attribute = PGPAttributeKey.COMPRESS_ALGORITHM_ID, description = "Compression Algorithm Identifier"),
+})
+public class EncryptContentPGP extends AbstractProcessor {
+
+    public static final Relationship SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Encryption Succeeded")
+            .build();
+
+    public static final Relationship FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Encryption Failed")
+            .build();
+
+    public static final PropertyDescriptor SYMMETRIC_KEY_ALGORITHM = new PropertyDescriptor.Builder()
+            .name("symmetric-key-algorithm")
+            .displayName("Symmetric-Key Algorithm")
+            .description("Symmetric-Key Algorithm for encryption")
+            .required(true)
+            .defaultValue(SymmetricKeyAlgorithm.AES_256.toString())
+            .allowableValues(SymmetricKeyAlgorithm.values())
+            .build();
+
+    public static final PropertyDescriptor COMPRESSION_ALGORITHM = new PropertyDescriptor.Builder()
+            .name("compression-algorithm")
+            .displayName("Compression Algorithm")
+            .description("Compression Algorithm for encryption")
+            .required(true)
+            .defaultValue(CompressionAlgorithm.ZIP.toString())
+            .allowableValues(CompressionAlgorithm.values())
+            .build();
+
+    public static final PropertyDescriptor FILE_ENCODING = new PropertyDescriptor.Builder()
+            .name("file-encoding")
+            .displayName("File Encoding")
+            .description("File Encoding for encryption")
+            .required(true)
+            .defaultValue(FileEncoding.BINARY.toString())
+            .allowableValues(FileEncoding.values())
+            .build();
+
+    public static final PropertyDescriptor PASSPHRASE = new PropertyDescriptor.Builder()
+            .name("passphrase")
+            .displayName("Passphrase")
+            .description("Passphrase used for encrypting data with Password-Based Encryption")
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor PUBLIC_KEY_SERVICE = new PropertyDescriptor.Builder()
+            .name("public-key-service")
+            .displayName("Public Key Service")
+            .description("PGP Public Key Service for encrypting data with Public Key Encryption")
+            .identifiesControllerService(PGPPublicKeyService.class)
+            .build();
+
+    public static final PropertyDescriptor PUBLIC_KEY_SEARCH = new PropertyDescriptor.Builder()
+            .name("public-key-search")
+            .displayName("Public Key Search")
+            .description("PGP Public Key Search will be used to match against the User ID or Key ID when formatted as uppercase hexadecimal string of 16 characters")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .addValidator(StandardValidators.ATTRIBUTE_EXPRESSION_LANGUAGE_VALIDATOR)
+            .dependsOn(PUBLIC_KEY_SERVICE)
+            .build();
+
+    /** Enable Integrity Protection as described in RFC 4880 Section 5.13 */
+    private static final boolean ENCRYPTION_INTEGRITY_PACKET_ENABLED = true;
+
+    private static final int OUTPUT_BUFFER_SIZE = 8192;
+
+    private static final Set<Relationship> RELATIONSHIPS = new HashSet<>(Arrays.asList(SUCCESS, FAILURE));
+
+    private static final List<PropertyDescriptor> DESCRIPTORS = Arrays.asList(
+            SYMMETRIC_KEY_ALGORITHM,
+            COMPRESSION_ALGORITHM,
+            FILE_ENCODING,
+            PASSPHRASE,
+            PUBLIC_KEY_SERVICE,
+            PUBLIC_KEY_SEARCH
+    );
+
+    /**
+     * Get Relationships
+     *
+     * @return Processor Relationships
+     */
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    /**
+     * Get Supported Property Descriptors
+     *
+     * @return Processor Supported Property Descriptors
+     */
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    /**
+     * On Trigger encrypts Flow File contents using configured properties
+     *
+     * @param context Process Context
+     * @param session Process Session
+     */
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        try {
+            final SymmetricKeyAlgorithm symmetricKeyAlgorithm = getSymmetricKeyAlgorithm(context);
+            final FileEncoding fileEncoding = getFileEncoding(context);
+            final CompressionAlgorithm compressionAlgorithm = getCompressionAlgorithm(context);
+            final StreamCallback callback = getEncryptStreamCallback(context, flowFile, symmetricKeyAlgorithm, compressionAlgorithm, fileEncoding);
+            flowFile = session.write(flowFile, callback);
+
+            final Map<String, String> attributes = getAttributes(symmetricKeyAlgorithm, fileEncoding, compressionAlgorithm);
+            flowFile = session.putAllAttributes(flowFile, attributes);
+
+            session.transfer(flowFile, SUCCESS);
+        } catch (final RuntimeException e) {
+            getLogger().error("Encryption Failed {}", flowFile, e);
+            session.transfer(flowFile, FAILURE);
+        }
+    }
+
+    /**
+     * Custom Validate requires at least one encryption property to be configured
+     *
+     * @param context Validation Context
+     * @return Collection of Validation Results
+     */
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext context) {
+        final Collection<ValidationResult> results = new ArrayList<>();
+
+        final String passphrase = context.getProperty(PASSPHRASE).getValue();
+        if (StringUtils.isBlank(passphrase)) {
+            final PGPPublicKeyService publicKeyService = context.getProperty(PUBLIC_KEY_SERVICE).asControllerService(PGPPublicKeyService.class);
+            if (publicKeyService == null) {
+                final String explanation = String.format("Neither [%s] nor [%s] configured", PASSPHRASE.getDisplayName(), PUBLIC_KEY_SERVICE.getDisplayName());
+                final ValidationResult result = new ValidationResult.Builder()
+                        .valid(false)
+                        .subject(getClass().getSimpleName())
+                        .explanation(explanation)
+                        .build();
+                results.add(result);
+            }
+        }
+
+        return results;
+    }
+
+    private StreamCallback getEncryptStreamCallback(final ProcessContext context, final FlowFile flowFile,
+                                                    final SymmetricKeyAlgorithm symmetricKeyAlgorithm,
+                                                    final CompressionAlgorithm compressionAlgorithm,
+                                                    final FileEncoding fileEncoding) {
+        final SecureRandom secureRandom = new SecureRandom();
+        final PGPDataEncryptorBuilder dataEncryptorBuilder = new BcPGPDataEncryptorBuilder(symmetricKeyAlgorithm.getId())
+                .setSecureRandom(secureRandom)
+                .setWithIntegrityPacket(ENCRYPTION_INTEGRITY_PACKET_ENABLED);
+        final PGPEncryptedDataGenerator encryptedDataGenerator = new PGPEncryptedDataGenerator(dataEncryptorBuilder);
+        final List<PGPKeyEncryptionMethodGenerator> methodGenerators = getEncryptionMethodGenerators(context, flowFile, secureRandom);
+        methodGenerators.forEach(encryptedDataGenerator::addMethod);
+
+        final String filename = flowFile.getAttribute(CoreAttributes.FILENAME.key());
+        return new EncryptStreamCallback(filename, fileEncoding, encryptedDataGenerator, compressionAlgorithm);
+    }
+
+    private List<PGPKeyEncryptionMethodGenerator> getEncryptionMethodGenerators(final ProcessContext context,
+                                                                                final FlowFile flowFile,
+                                                                                final SecureRandom secureRandom) {
+        final List<PGPKeyEncryptionMethodGenerator> generators = new ArrayList<>();
+
+        final PropertyValue passphraseProperty = context.getProperty(PASSPHRASE);
+        if (passphraseProperty.isSet()) {
+            final char[] passphrase = passphraseProperty.getValue().toCharArray();
+            generators.add(new JcePBEKeyEncryptionMethodGenerator(passphrase).setSecureRandom(secureRandom));
+        }
+
+        final PropertyValue publicKeySearchProperty = context.getProperty(PUBLIC_KEY_SEARCH);
+        if (publicKeySearchProperty.isSet()) {
+            final String publicKeySearch = publicKeySearchProperty.evaluateAttributeExpressions(flowFile).getValue();
+            getLogger().debug("Public Key Search [{}]", publicKeySearch);
+
+            final PGPPublicKeyService publicKeyService = context.getProperty(PUBLIC_KEY_SERVICE).asControllerService(PGPPublicKeyService.class);
+            final Optional<PGPPublicKey> optionalPublicKey = publicKeyService.findPublicKey(publicKeySearch);

Review comment:
       Thanks for pointing out the problem and providing the steps to reproduce!  Will introduce additional checking to ensure the service is configured before calling `findPublicKey()`




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

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



[GitHub] [nifi] gresockj commented on pull request #4842: NIFI-8251 Added EncryptContentPGP and DecryptContentPGP Processors

Posted by GitBox <gi...@apache.org>.
gresockj commented on pull request #4842:
URL: https://github.com/apache/nifi/pull/4842#issuecomment-812575725


   Minor note -- it looks like you'll need to add the new pgp nars to the nifi-assembly/pom.xml.


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

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



[GitHub] [nifi] thenatog commented on pull request #4842: NIFI-8251 Added EncryptContentPGP and DecryptContentPGP Processors

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


   Tested and looks good to me, +1 will merge. Thanks exceptionfactory, awesome work.


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

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



[GitHub] [nifi] exceptionfactory commented on a change in pull request #4842: NIFI-8251 Added EncryptContentPGP and DecryptContentPGP Processors

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #4842:
URL: https://github.com/apache/nifi/pull/4842#discussion_r607378275



##########
File path: nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/DecryptContentPGP.java
##########
@@ -0,0 +1,386 @@
+/*
+ * 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.pgp;
+
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+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.flowfile.FlowFile;
+import org.apache.nifi.pgp.service.api.PGPPrivateKeyService;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.io.StreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.pgp.exception.PGPDecryptionException;
+import org.apache.nifi.processors.pgp.exception.PGPProcessException;
+import org.apache.nifi.stream.io.StreamUtils;
+
+import org.apache.nifi.util.StringUtils;
+import org.bouncycastle.openpgp.PGPCompressedData;
+import org.bouncycastle.openpgp.PGPEncryptedData;
+import org.bouncycastle.openpgp.PGPEncryptedDataList;
+import org.bouncycastle.openpgp.PGPException;
+import org.bouncycastle.openpgp.PGPLiteralData;
+import org.bouncycastle.openpgp.PGPObjectFactory;
+import org.bouncycastle.openpgp.PGPPBEEncryptedData;
+import org.bouncycastle.openpgp.PGPPrivateKey;
+import org.bouncycastle.openpgp.PGPPublicKeyEncryptedData;
+import org.bouncycastle.openpgp.PGPUtil;
+import org.bouncycastle.openpgp.jcajce.JcaPGPObjectFactory;
+import org.bouncycastle.openpgp.operator.PBEDataDecryptorFactory;
+import org.bouncycastle.openpgp.operator.PublicKeyDataDecryptorFactory;
+import org.bouncycastle.openpgp.operator.bc.BcPBEDataDecryptorFactory;
+import org.bouncycastle.openpgp.operator.bc.BcPGPDigestCalculatorProvider;
+import org.bouncycastle.openpgp.operator.bc.BcPublicKeyDataDecryptorFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.UncheckedIOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * Decrypt Content using Open Pretty Good Privacy decryption methods
+ */
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Tags({"PGP", "GPG", "OpenPGP", "Encryption", "RFC 4880"})
+@CapabilityDescription("Decrypt Contents of OpenPGP Messages")
+@WritesAttributes({
+        @WritesAttribute(attribute = PGPAttributeKey.LITERAL_DATA_FILENAME, description = "Filename from decrypted Literal Data"),
+        @WritesAttribute(attribute = PGPAttributeKey.LITERAL_DATA_MODIFIED, description = "Modified Date from decrypted Literal Data"),
+        @WritesAttribute(attribute = PGPAttributeKey.SYMMETRIC_KEY_ALGORITHM_ID, description = "Symmetric-Key Algorithm Identifier")
+})
+public class DecryptContentPGP extends AbstractProcessor {
+
+    public static final Relationship SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Decryption Succeeded")
+            .build();
+
+    public static final Relationship FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Decryption Failed")
+            .build();
+
+    public static final PropertyDescriptor PASSPHRASE = new PropertyDescriptor.Builder()
+            .name("passphrase")
+            .displayName("Passphrase")
+            .description("Passphrase used for decrypting data encrypted with Password-Based Encryption")
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor PRIVATE_KEY_SERVICE = new PropertyDescriptor.Builder()
+            .name("private-key-service")
+            .displayName("Private Key Service")
+            .description("PGP Private Key Service for decrypting data encrypted with Public Key Encryption")
+            .identifiesControllerService(PGPPrivateKeyService.class)
+            .build();
+
+    private static final Set<Relationship> RELATIONSHIPS = new HashSet<>(Arrays.asList(SUCCESS, FAILURE));
+
+    private static final List<PropertyDescriptor> DESCRIPTORS = Arrays.asList(
+            PASSPHRASE,
+            PRIVATE_KEY_SERVICE
+    );
+
+    private static final String PASSWORD_BASED_ENCRYPTION = "Password-Based Encryption";
+
+    private static final String PUBLIC_KEY_ENCRYPTION = "Public Key Encryption";
+
+    /**
+     * Get Relationships
+     *
+     * @return Processor Relationships
+     */
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    /**
+     * Get Supported Property Descriptors
+     *
+     * @return Processor Supported Property Descriptors
+     */
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    /**
+     * On Trigger decrypts Flow File contents using configured properties
+     *
+     * @param context Process Context
+     * @param session Process Session
+     */
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final char[] passphrase = getPassphrase(context);
+        final PGPPrivateKeyService privateKeyService = getPrivateKeyService(context);
+        final DecryptStreamCallback callback = new DecryptStreamCallback(passphrase, privateKeyService);
+
+        try {
+            flowFile = session.write(flowFile, callback);
+            flowFile = session.putAllAttributes(flowFile, callback.attributes);
+            session.transfer(flowFile, SUCCESS);
+        } catch (final RuntimeException e) {
+            getLogger().error("Decryption Failed {}", flowFile, e);
+            session.transfer(flowFile, FAILURE);
+        }
+    }
+
+    /**
+     * Custom Validate requires at least one decryption property to be configured
+     *
+     * @param context Validation Context
+     * @return Collection of Validation Results
+     */
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext context) {
+        final Collection<ValidationResult> results = new ArrayList<>();
+
+        final String passphrase = context.getProperty(PASSPHRASE).getValue();
+        if (StringUtils.isBlank(passphrase)) {
+            final PGPPrivateKeyService privateKeyService = context.getProperty(PRIVATE_KEY_SERVICE).asControllerService(PGPPrivateKeyService.class);
+            if (privateKeyService == null) {
+                final String explanation = String.format("Neither [%s] nor [%s] configured", PASSPHRASE.getDisplayName(), PRIVATE_KEY_SERVICE.getDisplayName());
+                final ValidationResult result = new ValidationResult.Builder()
+                        .valid(false)
+                        .subject(getClass().getSimpleName())
+                        .explanation(explanation)
+                        .build();
+                results.add(result);
+            }
+        }
+
+        return results;
+    }
+
+    private char[] getPassphrase(final ProcessContext context) {
+        char[] passphrase = null;
+        final PropertyValue passphraseProperty = context.getProperty(PASSPHRASE);
+        if (passphraseProperty.isSet()) {
+            passphrase = passphraseProperty.getValue().toCharArray();
+        }
+        return passphrase;
+    }
+
+    private PGPPrivateKeyService getPrivateKeyService(final ProcessContext context) {
+        PGPPrivateKeyService privateKeyService = null;
+        final PropertyValue privateKeyServiceProperty = context.getProperty(PRIVATE_KEY_SERVICE);
+        if (privateKeyServiceProperty.isSet()) {
+            privateKeyService = privateKeyServiceProperty.asControllerService(PGPPrivateKeyService.class);
+        }
+        return privateKeyService;
+    }
+
+    private class DecryptStreamCallback implements StreamCallback {
+        private final char[] passphrase;
+
+        private final PGPPrivateKeyService privateKeyService;
+
+        private final Map<String, String> attributes = new HashMap<>();
+
+        public DecryptStreamCallback(final char[] passphrase, final PGPPrivateKeyService privateKeyService) {
+            this.passphrase = passphrase;
+            this.privateKeyService = privateKeyService;
+        }
+
+        /**
+         * Process Input Stream containing encrypted data and write decrypted contents to Output Stream
+         *
+         * @param inputStream  Input Stream containing encrypted data
+         * @param outputStream Output Stream for decrypted contents
+         * @throws IOException Thrown when unable to read or write streams
+         */
+        @Override
+        public void process(final InputStream inputStream, final OutputStream outputStream) throws IOException {
+            final InputStream decoderInputStream = PGPUtil.getDecoderStream(inputStream);
+            final PGPEncryptedDataList encryptedDataList = getEncryptedDataList(decoderInputStream);
+
+            for (final PGPEncryptedData encryptedData : encryptedDataList) {
+                final PGPLiteralData literalData = getLiteralData(encryptedData);
+
+                attributes.put(PGPAttributeKey.LITERAL_DATA_FILENAME, literalData.getFileName());
+                attributes.put(PGPAttributeKey.LITERAL_DATA_MODIFIED, Long.toString(literalData.getModificationTime().getTime()));
+
+                getLogger().debug("PGP Decrypted File Name [{}] Modified [{}]", literalData.getFileName(), literalData.getModificationTime());
+                StreamUtils.copy(literalData.getInputStream(), outputStream);
+
+                if (isVerified(encryptedData)) {
+                    getLogger().debug("PGP Encrypted Data Verified");
+                } else {
+                    final String message = String.format("PGP Encrypted Data [%s] Not Verified", encryptedData.getClass().getSimpleName());
+                    throw new PGPDecryptionException(message);
+                }
+            }
+        }
+
+        private PGPLiteralData getLiteralData(final PGPEncryptedData encryptedData) {
+            try {
+                final InputStream decryptedDataStream = getDecryptedDataStream(encryptedData);
+                final PGPObjectFactory objectFactory = new JcaPGPObjectFactory(decryptedDataStream);
+                return getLiteralData(objectFactory);
+            } catch (final PGPException e) {
+                final String message = String.format("PGP Decryption Failed [%s]", getEncryptedDataType(encryptedData));
+                throw new PGPDecryptionException(message, e);
+            }
+        }
+
+        private PGPLiteralData getLiteralData(final PGPObjectFactory objectFactory) throws PGPException {
+            PGPLiteralData literalData = null;
+
+            for (final Object object : objectFactory) {
+                if (object instanceof PGPCompressedData) {
+                    final PGPCompressedData compressedData = (PGPCompressedData) object;
+                    getLogger().debug("PGP Compressed Data Algorithm [{}] Found", compressedData.getAlgorithm());
+                    final PGPObjectFactory compressedObjectFactory = new JcaPGPObjectFactory(compressedData.getDataStream());
+                    literalData = getLiteralData(compressedObjectFactory);
+                    break;
+                } else if (object instanceof PGPLiteralData) {
+                    literalData = (PGPLiteralData) object;
+                    break;
+                }
+            }
+
+            if (literalData == null) {
+                throw new PGPProcessException("PGP Literal Data not found");
+            }
+
+            return literalData;
+        }
+
+        private InputStream getDecryptedDataStream(final PGPEncryptedData encryptedData) throws PGPException {
+            getLogger().debug("PGP Encrypted Data [{}] Found", getEncryptedDataType(encryptedData));
+
+            if (encryptedData instanceof PGPPBEEncryptedData) {
+                return getDecryptedDataStream((PGPPBEEncryptedData) encryptedData);
+            } else if (encryptedData instanceof PGPPublicKeyEncryptedData) {
+                return getDecryptedDataStream((PGPPublicKeyEncryptedData) encryptedData);
+            } else {
+                final String message = String.format("PGP Encrypted Data [%s] Not Supported", getEncryptedDataType(encryptedData));
+                throw new UnsupportedOperationException(message);
+            }
+        }
+
+        private InputStream getDecryptedDataStream(final PGPPBEEncryptedData passwordBasedEncryptedData) throws PGPException {
+            if (passphrase == null) {
+                throw new PGPProcessException("PGP Password-Based Encryption Found: Passphrase not configured");
+            } else {
+                final PBEDataDecryptorFactory decryptorFactory = new BcPBEDataDecryptorFactory(passphrase, new BcPGPDigestCalculatorProvider());
+                final int symmetricAlgorithm = passwordBasedEncryptedData.getSymmetricAlgorithm(decryptorFactory);
+                attributes.put(PGPAttributeKey.SYMMETRIC_KEY_ALGORITHM_ID, Integer.toString(symmetricAlgorithm));
+                return passwordBasedEncryptedData.getDataStream(decryptorFactory);
+            }
+        }
+
+        private InputStream getDecryptedDataStream(final PGPPublicKeyEncryptedData publicKeyEncryptedData) throws PGPException {
+            if (privateKeyService == null) {
+                throw new PGPProcessException("PGP Public Key Encryption Found: Private Key Service not configured");
+            } else {
+                final long keyId = publicKeyEncryptedData.getKeyID();
+                final Optional<PGPPrivateKey> foundPrivateKey = privateKeyService.findPrivateKey(keyId);
+                if (foundPrivateKey.isPresent()) {
+                    final PGPPrivateKey privateKey = foundPrivateKey.get();
+                    final PublicKeyDataDecryptorFactory decryptorFactory = new BcPublicKeyDataDecryptorFactory(privateKey);
+                    final int symmetricAlgorithm = publicKeyEncryptedData.getSymmetricAlgorithm(decryptorFactory);
+                    attributes.put(PGPAttributeKey.SYMMETRIC_KEY_ALGORITHM_ID, Integer.toString(symmetricAlgorithm));
+                    return publicKeyEncryptedData.getDataStream(decryptorFactory);

Review comment:
       Although this is somewhat of an edge case, it is a great example of a message containing multiple Encrypted Data Packets as described in RFC 4880 Section 5.3.  This uncovered an issue with the handling of multiple Encrypted Data elements in the callback.  The `DecryptContentPGP` processor should support decryption using either a Public Key or Passphrase, depending on property configuration and data received, so I will push an update to correct data handling in this scenario.




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

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