You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by al...@apache.org on 2019/09/03 18:00:45 UTC

[nifi] branch master updated: NIFI-6536 EncryptContent accepts configurable PGP symmetric cipher Additional test case in TestEncryptContent Apply suggestions from code review nit picking Apply suggestions from code review nitpicking from mgaido91 Small fixes Additional test case to check the provided cipher is actually used by inferring it from the ciphertext Updated `EncryptContent` to use a pre-defined list of allowable PGP ciphers Updated `EncryptContent` to check the validity of `PGP_SYMMETRIC_ENCRYPTION_CIPHER` Up [...]

This is an automated email from the ASF dual-hosted git repository.

alopresto pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/nifi.git


The following commit(s) were added to refs/heads/master by this push:
     new 4c6c1cb  NIFI-6536 EncryptContent accepts configurable PGP symmetric cipher Additional test case in TestEncryptContent Apply suggestions from code review nit picking Apply suggestions from code review nitpicking from mgaido91 Small fixes Additional test case to check the provided cipher is actually used by inferring it from the ciphertext Updated `EncryptContent` to use a pre-defined list of allowable PGP ciphers Updated `EncryptContent` to check the validity of `PGP_SYMMETRIC_EN [...]
4c6c1cb is described below

commit 4c6c1cbb14aaedda649981f76d71ced2e8a36eac
Author: Alessandro D'Armiento <al...@agilelab.it>
AuthorDate: Sat Aug 17 18:30:24 2019 +0200

    NIFI-6536 EncryptContent accepts configurable PGP symmetric cipher
    Additional test case in TestEncryptContent
    Apply suggestions from code review
    nit picking
    Apply suggestions from code review
    nitpicking from mgaido91
    Small fixes
    Additional test case to check the provided cipher is actually used by inferring it from the ciphertext
    Updated `EncryptContent` to use a pre-defined list of allowable PGP ciphers
    Updated `EncryptContent` to check the validity of `PGP_SYMMETRIC_ENCRYPTION_CIPHER`
    Updated test cases
    Minor fixes
    NIFI-6536 Fixed typo and strengthened validity checking for PGP symmetric cipher.
    
    This closes #3664.
    
    Co-Authored-By: Marco Gaido <ma...@gmail.com>
    
    Signed-off-by: Andy LoPresto <al...@apache.org>
---
 .../nifi/processors/standard/EncryptContent.java   |  67 +++++++++--
 .../util/crypto/OpenPGPKeyBasedEncryptor.java      |  15 ++-
 .../util/crypto/OpenPGPPasswordBasedEncryptor.java |  13 +-
 .../processors/standard/TestEncryptContent.java    | 131 +++++++++++++++++++++
 .../util/crypto/OpenPGPKeyBasedEncryptorTest.java  | 128 +++++++++++---------
 .../crypto/OpenPGPPasswordBasedEncryptorTest.java  | 107 ++++++++++-------
 6 files changed, 340 insertions(+), 121 deletions(-)

diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EncryptContent.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EncryptContent.java
index a3249ba..b59b5fa 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EncryptContent.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EncryptContent.java
@@ -63,6 +63,7 @@ import org.apache.nifi.security.util.crypto.OpenPGPPasswordBasedEncryptor;
 import org.apache.nifi.security.util.crypto.PasswordBasedEncryptor;
 import org.apache.nifi.util.StopWatch;
 import org.bouncycastle.jce.provider.BouncyCastleProvider;
+import org.bouncycastle.openpgp.PGPEncryptedData;
 
 @EventDriven
 @SideEffectFree
@@ -138,6 +139,18 @@ public class EncryptContent extends AbstractProcessor {
             .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .sensitive(true)
             .build();
+
+    public static final PropertyDescriptor PGP_SYMMETRIC_ENCRYPTION_CIPHER = new PropertyDescriptor.Builder()
+            .name("pgp-symmetric-cipher")
+            .displayName("PGP Symmetric Cipher")
+            .description("When using PGP encryption, this is the symmetric cipher to be used. This property is ignored if "
+                    + "Encryption Algorithm is not PGP or PGP-ASCII-ARMOR\nNote that the provided cipher is only used during"
+                    + "the encryption phase, while it is inferred from the ciphertext in the decryption phase")
+            .required(false)
+            .allowableValues(buildPGPSymmetricCipherAllowableValues())
+            .defaultValue(String.valueOf(PGPEncryptedData.AES_128))
+            .build();
+
     public static final PropertyDescriptor RAW_KEY_HEX = new PropertyDescriptor.Builder()
             .name("raw-key-hex")
             .displayName("Raw Key (hexadecimal)")
@@ -202,6 +215,23 @@ public class EncryptContent extends AbstractProcessor {
                 "if unsafe combinations of encryption algorithms and passwords are provided on a JVM with limited strength crypto. To fix this, see the Admin Guide.");
     }
 
+    private static AllowableValue[] buildPGPSymmetricCipherAllowableValues() {
+        // Allowed values are inferred from SymmetricKeyAlgorithmTags. Note that NULL and SAFER cipher are not supported and therefore not listed
+        return new AllowableValue[] {
+                new AllowableValue("1", "IDEA"),
+                new AllowableValue("2", "TRIPLE_DES"),
+                new AllowableValue("3", "CAST5"),
+                new AllowableValue("4", "BLOWFISH"),
+                new AllowableValue("6", "DES"),
+                new AllowableValue("7", "AES_128"),
+                new AllowableValue("8", "AES_192"),
+                new AllowableValue("9", "AES_256"),
+                new AllowableValue("10", "TWOFISH"),
+                new AllowableValue("11", "CAMELLIA_128"),
+                new AllowableValue("12", "CAMELLIA_192"),
+                new AllowableValue("13", "CAMELLIA_256") };
+    }
+
     @Override
     protected void init(final ProcessorInitializationContext context) {
         final List<PropertyDescriptor> properties = new ArrayList<>();
@@ -215,6 +245,7 @@ public class EncryptContent extends AbstractProcessor {
         properties.add(PUBLIC_KEY_USERID);
         properties.add(PRIVATE_KEYRING);
         properties.add(PRIVATE_KEYRING_PASSPHRASE);
+        properties.add(PGP_SYMMETRIC_ENCRYPTION_CIPHER);
         this.properties = Collections.unmodifiableList(properties);
 
         final Set<Relationship> relationships = new HashSet<>();
@@ -256,7 +287,9 @@ public class EncryptContent extends AbstractProcessor {
             final String publicUserId = context.getProperty(PUBLIC_KEY_USERID).getValue();
             final String privateKeyring = context.getProperty(PRIVATE_KEYRING).getValue();
             final String privateKeyringPassphrase = context.getProperty(PRIVATE_KEYRING_PASSPHRASE).evaluateAttributeExpressions().getValue();
-            validationResults.addAll(validatePGP(encryptionMethod, password, encrypt, publicKeyring, publicUserId, privateKeyring, privateKeyringPassphrase));
+            final Integer cipher = context.getProperty(PGP_SYMMETRIC_ENCRYPTION_CIPHER).asInteger();
+            validationResults.addAll(validatePGP(encryptionMethod, password, encrypt, publicKeyring, publicUserId,
+                    privateKeyring, privateKeyringPassphrase, cipher));
         } else { // Not PGP
             if (encryptionMethod.isKeyedCipher()) { // Raw key
                 validationResults.addAll(validateKeyed(encryptionMethod, kdf, keyHex));
@@ -268,10 +301,28 @@ public class EncryptContent extends AbstractProcessor {
         return validationResults;
     }
 
-    private List<ValidationResult> validatePGP(EncryptionMethod encryptionMethod, String password, boolean encrypt, String publicKeyring, String publicUserId, String privateKeyring,
-                                               String privateKeyringPassphrase) {
+    /**
+     * Returns true if the integer value provided maps to a valid {@code cipher} as contained in the {@code PGP_SYMMETRIC_ENCRYPTION_CIPHER}.
+     *
+     * @param cipher an integer indicating a particular cipher
+     * @return true if the cipher is supported
+     */
+    private static boolean isValidCipher(int cipher) {
+        return PGP_SYMMETRIC_ENCRYPTION_CIPHER.getAllowableValues().stream().anyMatch(av -> av.getValue().equals(String.valueOf(cipher)));
+    }
+
+    private List<ValidationResult> validatePGP(EncryptionMethod encryptionMethod, String password, boolean encrypt,
+                                               String publicKeyring, String publicUserId, String privateKeyring,
+                                               String privateKeyringPassphrase, int cipher) {
         List<ValidationResult> validationResults = new ArrayList<>();
 
+        if(encrypt && password != null && !isValidCipher(cipher)) {
+            validationResults.add(new ValidationResult.Builder().subject(PGP_SYMMETRIC_ENCRYPTION_CIPHER.getDisplayName())
+                    .explanation("When performing an encryption with " + encryptionMethod.getAlgorithm() + " and a symmetric " +
+                            PASSWORD.getDisplayName() + ", a" + PGP_SYMMETRIC_ENCRYPTION_CIPHER.getDisplayName() + " is required")
+                    .build());
+        }
+
         if (password == null) {
             if (encrypt) {
                 // If encrypting without a password, require both public-keyring-file and public-key-user-id
@@ -463,6 +514,7 @@ public class EncryptContent extends AbstractProcessor {
         final EncryptionMethod encryptionMethod = EncryptionMethod.valueOf(method);
         final String providerName = encryptionMethod.getProvider();
         final String algorithm = encryptionMethod.getAlgorithm();
+        final Integer pgpCipher = context.getProperty(PGP_SYMMETRIC_ENCRYPTION_CIPHER).asInteger();
         final String password = context.getProperty(PASSWORD).getValue();
         final KeyDerivationFunction kdf = KeyDerivationFunction.valueOf(context.getProperty(KEY_DERIVATION_FUNCTION).getValue());
         final boolean encrypt = context.getProperty(MODE).getValue().equalsIgnoreCase(ENCRYPT_MODE);
@@ -476,14 +528,13 @@ public class EncryptContent extends AbstractProcessor {
                 final String privateKeyring = context.getProperty(PRIVATE_KEYRING).getValue();
                 if (encrypt && publicKeyring != null) {
                     final String publicUserId = context.getProperty(PUBLIC_KEY_USERID).getValue();
-                    encryptor = new OpenPGPKeyBasedEncryptor(algorithm, providerName, publicKeyring, publicUserId, null, filename);
+                    encryptor = new OpenPGPKeyBasedEncryptor(algorithm, pgpCipher, providerName, publicKeyring, publicUserId, null, filename);
                 } else if (!encrypt && privateKeyring != null) {
                     final char[] keyringPassphrase = context.getProperty(PRIVATE_KEYRING_PASSPHRASE).evaluateAttributeExpressions().getValue().toCharArray();
-                    encryptor = new OpenPGPKeyBasedEncryptor(algorithm, providerName, privateKeyring, null, keyringPassphrase,
-                            filename);
+                    encryptor = new OpenPGPKeyBasedEncryptor(algorithm, pgpCipher, providerName, privateKeyring, null, keyringPassphrase, filename);
                 } else {
                     final char[] passphrase = Normalizer.normalize(password, Normalizer.Form.NFC).toCharArray();
-                    encryptor = new OpenPGPPasswordBasedEncryptor(algorithm, providerName, passphrase, filename);
+                    encryptor = new OpenPGPPasswordBasedEncryptor(algorithm, pgpCipher, providerName, passphrase, filename);
                 }
             } else if (kdf.equals(KeyDerivationFunction.NONE)) { // Raw key
                 final String keyHex = context.getProperty(RAW_KEY_HEX).getValue();
@@ -524,4 +575,4 @@ public class EncryptContent extends AbstractProcessor {
         StreamCallback getDecryptionCallback() throws Exception;
     }
 
-}
\ No newline at end of file
+}
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/security/util/crypto/OpenPGPKeyBasedEncryptor.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/security/util/crypto/OpenPGPKeyBasedEncryptor.java
index 6b6c2fc..6b5e780 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/security/util/crypto/OpenPGPKeyBasedEncryptor.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/security/util/crypto/OpenPGPKeyBasedEncryptor.java
@@ -35,7 +35,6 @@ import org.apache.nifi.processors.standard.EncryptContent.Encryptor;
 import org.bouncycastle.bcpg.ArmoredOutputStream;
 import org.bouncycastle.openpgp.PGPCompressedData;
 import org.bouncycastle.openpgp.PGPCompressedDataGenerator;
-import org.bouncycastle.openpgp.PGPEncryptedData;
 import org.bouncycastle.openpgp.PGPEncryptedDataGenerator;
 import org.bouncycastle.openpgp.PGPEncryptedDataList;
 import org.bouncycastle.openpgp.PGPException;
@@ -67,6 +66,7 @@ public class OpenPGPKeyBasedEncryptor implements Encryptor {
     private static final Logger logger = LoggerFactory.getLogger(OpenPGPPasswordBasedEncryptor.class);
 
     private String algorithm;
+    private Integer cipher;
     private String provider;
     // TODO: This can hold either the secret or public keyring path
     private String keyring;
@@ -74,8 +74,10 @@ public class OpenPGPKeyBasedEncryptor implements Encryptor {
     private char[] passphrase;
     private String filename;
 
-    public OpenPGPKeyBasedEncryptor(final String algorithm, final String provider, final String keyring, final String userId, final char[] passphrase, final String filename) {
+    public OpenPGPKeyBasedEncryptor(final String algorithm, final Integer cipher, final String provider, final String keyring,
+                                    final String userId, final char[] passphrase, final String filename) {
         this.algorithm = algorithm;
+        this.cipher = cipher;
         this.provider = provider;
         this.keyring = keyring;
         this.userId = userId;
@@ -85,7 +87,7 @@ public class OpenPGPKeyBasedEncryptor implements Encryptor {
 
     @Override
     public StreamCallback getEncryptionCallback() throws Exception {
-        return new OpenPGPEncryptCallback(algorithm, provider, keyring, userId, filename);
+        return new OpenPGPEncryptCallback(algorithm, cipher, provider, keyring, userId, filename);
     }
 
     @Override
@@ -314,13 +316,15 @@ public class OpenPGPKeyBasedEncryptor implements Encryptor {
     private static class OpenPGPEncryptCallback implements StreamCallback {
 
         private String algorithm;
+        private Integer cipher;
         private String provider;
         private String publicKeyring;
         private String userId;
         private String filename;
 
-        OpenPGPEncryptCallback(final String algorithm, final String provider, final String keyring, final String userId, final String filename) {
+        OpenPGPEncryptCallback(final String algorithm, final Integer cipher, final String provider, final String keyring, final String userId, final String filename) {
             this.algorithm = algorithm;
+            this.cipher = cipher;
             this.provider = provider;
             this.publicKeyring = keyring;
             this.userId = userId;
@@ -345,9 +349,8 @@ public class OpenPGPKeyBasedEncryptor implements Encryptor {
                 }
 
                 try {
-                    // TODO: Refactor internal symmetric encryption algorithm to be customizable
                     PGPEncryptedDataGenerator encryptedDataGenerator = new PGPEncryptedDataGenerator(
-                            new JcePGPDataEncryptorBuilder(PGPEncryptedData.AES_128).setWithIntegrityPacket(true).setSecureRandom(new SecureRandom()).setProvider(provider));
+                            new JcePGPDataEncryptorBuilder(cipher).setWithIntegrityPacket(true).setSecureRandom(new SecureRandom()).setProvider(provider));
 
                     encryptedDataGenerator.addMethod(new JcePublicKeyKeyEncryptionMethodGenerator(publicKey).setProvider(provider));
 
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/security/util/crypto/OpenPGPPasswordBasedEncryptor.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/security/util/crypto/OpenPGPPasswordBasedEncryptor.java
index 6d5bb6d..1e843a7 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/security/util/crypto/OpenPGPPasswordBasedEncryptor.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/security/util/crypto/OpenPGPPasswordBasedEncryptor.java
@@ -25,7 +25,6 @@ import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.io.StreamCallback;
 import org.apache.nifi.processors.standard.EncryptContent.Encryptor;
 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;
@@ -47,17 +46,19 @@ public class OpenPGPPasswordBasedEncryptor implements Encryptor {
     private String provider;
     private char[] password;
     private String filename;
+    private Integer cipher;
 
-    public OpenPGPPasswordBasedEncryptor(final String algorithm, final String provider, final char[] passphrase, final String filename) {
+    public OpenPGPPasswordBasedEncryptor(final String algorithm, final Integer cipher, final String provider, final char[] passphrase, final String filename) {
         this.algorithm = algorithm;
         this.provider = provider;
         this.password = passphrase;
         this.filename = filename;
+        this.cipher = cipher;
     }
 
     @Override
     public StreamCallback getEncryptionCallback() throws Exception {
-        return new OpenPGPEncryptCallback(algorithm, provider, password, filename);
+        return new OpenPGPEncryptCallback(algorithm, cipher, provider, password, filename);
     }
 
     @Override
@@ -136,19 +137,21 @@ public class OpenPGPPasswordBasedEncryptor implements Encryptor {
         private String provider;
         private char[] password;
         private String filename;
+        private Integer cipher;
 
-        OpenPGPEncryptCallback(final String algorithm, final String provider, final char[] password, final String filename) {
+        OpenPGPEncryptCallback(final String algorithm, final Integer cipher, final String provider, final char[] password, final String filename) {
             this.algorithm = algorithm;
             this.provider = provider;
             this.password = password;
             this.filename = filename;
+            this.cipher = cipher;
         }
 
         @Override
         public void process(InputStream in, OutputStream out) throws IOException {
             try {
                 PGPKeyEncryptionMethodGenerator encryptionMethodGenerator = new JcePBEKeyEncryptionMethodGenerator(password).setProvider(provider);
-                org.apache.nifi.processors.standard.util.PGPUtil.encrypt(in, out, algorithm, provider, PGPEncryptedData.AES_128, filename, encryptionMethodGenerator);
+                org.apache.nifi.processors.standard.util.PGPUtil.encrypt(in, out, algorithm, provider, cipher, filename, encryptionMethodGenerator);
             } catch (Exception e) {
                 throw new ProcessException(e.getMessage());
             }
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEncryptContent.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEncryptContent.java
index 063652b..6464665 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEncryptContent.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEncryptContent.java
@@ -16,12 +16,20 @@
  */
 package org.apache.nifi.processors.standard;
 
+import java.io.ByteArrayInputStream;
 import java.io.File;
+import java.io.InputStream;
 import java.io.IOException;
+import java.lang.reflect.Method;
 import java.nio.file.Paths;
 import java.security.Security;
+import java.util.ArrayList;
 import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+
 import org.apache.commons.codec.binary.Hex;
+import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.security.util.EncryptionMethod;
 import org.apache.nifi.security.util.KeyDerivationFunction;
@@ -31,6 +39,8 @@ import org.apache.nifi.util.MockFlowFile;
 import org.apache.nifi.util.MockProcessContext;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
+import org.bouncycastle.bcpg.BCPGInputStream;
+import org.bouncycastle.bcpg.SymmetricKeyEncSessionPacket;
 import org.bouncycastle.jce.provider.BouncyCastleProvider;
 import org.junit.Assert;
 import org.junit.Assume;
@@ -39,10 +49,25 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.bouncycastle.openpgp.PGPUtil.getDecoderStream;
+import static org.junit.Assert.fail;
+
 public class TestEncryptContent {
 
     private static final Logger logger = LoggerFactory.getLogger(TestEncryptContent.class);
 
+    private static AllowableValue[] getPGPCipherList() {
+        try{
+            Method method = EncryptContent.class.getDeclaredMethod("buildPGPSymmetricCipherAllowableValues");
+            method.setAccessible(true);
+            return ((AllowableValue[]) method.invoke(null));
+        } catch (Exception e){
+            logger.error("Cannot access buildPGPSymmetricCipherAllowableValues", e);
+            fail("Cannot access buildPGPSymmetricCipherAllowableValues");
+        }
+        return null;
+    }
+
     @Before
     public void setUp() {
         Security.addProvider(new BouncyCastleProvider());
@@ -93,6 +118,83 @@ public class TestEncryptContent {
     }
 
     @Test
+    public void testPGPCiphersRoundTrip() {
+        final TestRunner testRunner = TestRunners.newTestRunner(new EncryptContent());
+        testRunner.setProperty(EncryptContent.PASSWORD, "passwordpassword"); // a >=16 characters password
+        testRunner.setProperty(EncryptContent.KEY_DERIVATION_FUNCTION, KeyDerivationFunction.NONE.name());
+
+        List<String> pgpAlgorithms = new ArrayList<>();
+        pgpAlgorithms.add("PGP");
+        pgpAlgorithms.add("PGP_ASCII_ARMOR");
+
+        for (String algorithm : pgpAlgorithms) {
+            testRunner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, algorithm);
+            for (AllowableValue cipher : Objects.requireNonNull(getPGPCipherList())) {
+                testRunner.setProperty(EncryptContent.PGP_SYMMETRIC_ENCRYPTION_CIPHER, cipher.getValue());
+                testRunner.setProperty(EncryptContent.MODE, EncryptContent.ENCRYPT_MODE);
+
+                testRunner.enqueue("A cool plaintext!");
+                testRunner.clearTransferState();
+                testRunner.run();
+
+                testRunner.assertAllFlowFilesTransferred(EncryptContent.REL_SUCCESS, 1);
+
+                MockFlowFile flowFile = testRunner.getFlowFilesForRelationship(EncryptContent.REL_SUCCESS).get(0);
+                testRunner.assertQueueEmpty();
+
+                testRunner.setProperty(EncryptContent.MODE, EncryptContent.DECRYPT_MODE);
+                // Encryption cipher is inferred from ciphertext, this property deliberately set a fixed cipher to prove
+                // the output will still be correct
+                testRunner.setProperty(EncryptContent.PGP_SYMMETRIC_ENCRYPTION_CIPHER, "1");
+
+                testRunner.enqueue(flowFile);
+                testRunner.clearTransferState();
+                testRunner.run();
+                testRunner.assertAllFlowFilesTransferred(EncryptContent.REL_SUCCESS, 1);
+
+                flowFile = testRunner.getFlowFilesForRelationship(EncryptContent.REL_SUCCESS).get(0);
+                flowFile.assertContentEquals("A cool plaintext!");
+            }
+        }
+    }
+
+    @Test
+    public void testPGPCiphers() throws Exception {
+        final TestRunner testRunner = TestRunners.newTestRunner(new EncryptContent());
+        testRunner.setProperty(EncryptContent.PASSWORD, "passwordpassword"); // a >= 16 characters password
+        testRunner.setProperty(EncryptContent.KEY_DERIVATION_FUNCTION, KeyDerivationFunction.NONE.name());
+
+        List<String> pgpAlgorithms = new ArrayList<>();
+        pgpAlgorithms.add("PGP");
+        pgpAlgorithms.add("PGP_ASCII_ARMOR");
+
+        for (String algorithm : pgpAlgorithms) {
+
+            testRunner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, algorithm);
+            for (AllowableValue cipher : Objects.requireNonNull(getPGPCipherList())) {
+                testRunner.setProperty(EncryptContent.PGP_SYMMETRIC_ENCRYPTION_CIPHER, cipher.getValue());
+                testRunner.setProperty(EncryptContent.MODE, EncryptContent.ENCRYPT_MODE);
+
+                testRunner.enqueue("A cool plaintext!");
+                testRunner.clearTransferState();
+                testRunner.run();
+
+                testRunner.assertAllFlowFilesTransferred(EncryptContent.REL_SUCCESS, 1);
+
+                MockFlowFile flowFile = testRunner.getFlowFilesForRelationship(EncryptContent.REL_SUCCESS).get(0);
+                testRunner.assertQueueEmpty();
+
+                // Other than the round trip, checks that the provided cipher is actually used, inferring it from the ciphertext
+                InputStream ciphertext = new ByteArrayInputStream(flowFile.toByteArray());
+                BCPGInputStream pgpin = new BCPGInputStream(getDecoderStream(ciphertext));
+                assert pgpin.nextPacketTag() == 3;
+                assert ((SymmetricKeyEncSessionPacket) pgpin.readPacket()).getEncAlgorithm() == Integer.valueOf(cipher.getValue());
+                pgpin.close();
+            }
+        }
+    }
+
+    @Test
     public void testShouldDetermineMaxKeySizeForAlgorithms() throws IOException {
         // Arrange
         final String AES_ALGORITHM = EncryptionMethod.MD5_256AES.getAlgorithm();
@@ -416,5 +518,34 @@ public class TestEncryptContent {
                     " could not be opened with the provided " + EncryptContent.PRIVATE_KEYRING_PASSPHRASE.getDisplayName()));
 
         }
+        runner.removeProperty(EncryptContent.PRIVATE_KEYRING_PASSPHRASE);
+
+        // This configuration is invalid because PGP_SYMMETRIC_ENCRYPTION_CIPHER is outside the allowed [1-13] interval
+        runner.setProperty(EncryptContent.MODE, EncryptContent.ENCRYPT_MODE);
+        runner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, "PGP");
+        runner.setProperty(EncryptContent.PASSWORD, "PASSWORD");
+        runner.setProperty(EncryptContent.PGP_SYMMETRIC_ENCRYPTION_CIPHER, "256");
+        runner.assertNotValid();
+
+        // This configuration is invalid because PGP_SYMMETRIC_ENCRYPTION_CIPHER points to SAFER cipher which is unsupported
+        runner.setProperty(EncryptContent.MODE, EncryptContent.ENCRYPT_MODE);
+        runner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, "PGP");
+        runner.setProperty(EncryptContent.PASSWORD, "PASSWORD");
+        runner.setProperty(EncryptContent.PGP_SYMMETRIC_ENCRYPTION_CIPHER, "5");
+        runner.assertNotValid();
+
+        // This configuration is valid
+        runner.setProperty(EncryptContent.MODE, EncryptContent.DECRYPT_MODE);
+        runner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, "PGP");
+        runner.setProperty(EncryptContent.PASSWORD, "PASSWORD");
+        runner.removeProperty(EncryptContent.PGP_SYMMETRIC_ENCRYPTION_CIPHER);
+        runner.assertValid();
+
+        // This configuration is valid because the default value will be used for PGP_SYMMETRIC_ENCRYPTION_CIPHER
+        runner.setProperty(EncryptContent.MODE, EncryptContent.ENCRYPT_MODE);
+        runner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, "PGP");
+        runner.setProperty(EncryptContent.PASSWORD, "PASSWORD");
+        runner.removeProperty(EncryptContent.PGP_SYMMETRIC_ENCRYPTION_CIPHER);
+        runner.assertValid();
     }
 }
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/security/util/crypto/OpenPGPKeyBasedEncryptorTest.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/security/util/crypto/OpenPGPKeyBasedEncryptorTest.java
index c823e61..dd10550 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/security/util/crypto/OpenPGPKeyBasedEncryptorTest.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/security/util/crypto/OpenPGPKeyBasedEncryptorTest.java
@@ -29,6 +29,8 @@ import org.apache.commons.codec.binary.Hex;
 import org.apache.nifi.processor.io.StreamCallback;
 import org.apache.nifi.security.util.EncryptionMethod;
 import org.bouncycastle.jce.provider.BouncyCastleProvider;
+import org.bouncycastle.openpgp.PGPEncryptedData;
+import org.bouncycastle.openpgp.PGPUtil;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -67,65 +69,77 @@ public class OpenPGPKeyBasedEncryptorTest {
 
     @Test
     public void testShouldEncryptAndDecrypt() throws Exception {
-        // Arrange
-        final String PLAINTEXT = "This is a plaintext message.";
-        logger.info("Plaintext: {}", PLAINTEXT);
-        InputStream plainStream = new ByteArrayInputStream(PLAINTEXT.getBytes("UTF-8"));
-        OutputStream cipherStream = new ByteArrayOutputStream();
-        OutputStream recoveredStream = new ByteArrayOutputStream();
-
-        // No file, just streams
-        String filename = "tempFile.txt";
-
-        // Encryptor does not require password
-        OpenPGPKeyBasedEncryptor encryptor = new OpenPGPKeyBasedEncryptor(
-            EncryptionMethod.PGP.getAlgorithm(), EncryptionMethod.PGP.getProvider(), PUBLIC_KEYRING_PATH, USER_ID, new char[0], filename);
-        StreamCallback encryptionCallback = encryptor.getEncryptionCallback();
-
-        OpenPGPKeyBasedEncryptor decryptor = new OpenPGPKeyBasedEncryptor(
-            EncryptionMethod.PGP.getAlgorithm(), EncryptionMethod.PGP.getProvider(), SECRET_KEYRING_PATH, USER_ID, PASSWORD.toCharArray(), filename);
-        StreamCallback decryptionCallback = decryptor.getDecryptionCallback();
-
-        // Act
-        encryptionCallback.process(plainStream, cipherStream);
-
-        final byte[] cipherBytes = ((ByteArrayOutputStream) cipherStream).toByteArray();
-        logger.info("Encrypted: {}", Hex.encodeHexString(cipherBytes));
-        InputStream cipherInputStream = new ByteArrayInputStream(cipherBytes);
-
-        decryptionCallback.process(cipherInputStream, recoveredStream);
-
-        // Assert
-        byte[] recoveredBytes = ((ByteArrayOutputStream) recoveredStream).toByteArray();
-        String recovered = new String(recoveredBytes, "UTF-8");
-        logger.info("Recovered: {}", recovered);
-        assert PLAINTEXT.equals(recovered);
+        for (int i = 1; i < 14; i++) {
+            if (PGPEncryptedData.SAFER != i) { // SAFER cipher is not supported and therefore its test is skipped
+                Integer cipher = i;
+                logger.info("Testing PGP encryption with " + PGPUtil.getSymmetricCipherName(cipher) + " cipher.");
+                // Arrange
+                final String PLAINTEXT = "This is a plaintext message.";
+                logger.info("Plaintext: {}", PLAINTEXT);
+                InputStream plainStream = new ByteArrayInputStream(PLAINTEXT.getBytes("UTF-8"));
+                OutputStream cipherStream = new ByteArrayOutputStream();
+                OutputStream recoveredStream = new ByteArrayOutputStream();
+
+                // No file, just streams
+                String filename = "tempFile.txt";
+
+
+                // Encryptor does not require password
+                OpenPGPKeyBasedEncryptor encryptor = new OpenPGPKeyBasedEncryptor(
+                        EncryptionMethod.PGP.getAlgorithm(), cipher, EncryptionMethod.PGP.getProvider(), PUBLIC_KEYRING_PATH, USER_ID, new char[0], filename);
+                StreamCallback encryptionCallback = encryptor.getEncryptionCallback();
+
+                OpenPGPKeyBasedEncryptor decryptor = new OpenPGPKeyBasedEncryptor(
+                        EncryptionMethod.PGP.getAlgorithm(), cipher, EncryptionMethod.PGP.getProvider(), SECRET_KEYRING_PATH, USER_ID, PASSWORD.toCharArray(), filename);
+                StreamCallback decryptionCallback = decryptor.getDecryptionCallback();
+
+                // Act
+                encryptionCallback.process(plainStream, cipherStream);
+
+                final byte[] cipherBytes = ((ByteArrayOutputStream) cipherStream).toByteArray();
+                logger.info("Encrypted: {}", Hex.encodeHexString(cipherBytes));
+                InputStream cipherInputStream = new ByteArrayInputStream(cipherBytes);
+
+                decryptionCallback.process(cipherInputStream, recoveredStream);
+
+                // Assert
+                byte[] recoveredBytes = ((ByteArrayOutputStream) recoveredStream).toByteArray();
+                String recovered = new String(recoveredBytes, "UTF-8");
+                logger.info("Recovered: {}", recovered);
+                assert PLAINTEXT.equals(recovered);
+            }
+        }
     }
 
     @Test
     public void testShouldDecryptExternalFile() throws Exception {
-        // Arrange
-        byte[] plainBytes = Files.readAllBytes(Paths.get(plainFile.getPath()));
-        final String PLAINTEXT = new String(plainBytes, "UTF-8");
-
-        InputStream cipherStream = new FileInputStream(unsignedFile);
-        OutputStream recoveredStream = new ByteArrayOutputStream();
-
-        // No file, just streams
-        String filename = unsignedFile.getName();
-
-        OpenPGPKeyBasedEncryptor encryptor = new OpenPGPKeyBasedEncryptor(
-            EncryptionMethod.PGP.getAlgorithm(), EncryptionMethod.PGP.getProvider(), SECRET_KEYRING_PATH, USER_ID, PASSWORD.toCharArray(), filename);
-
-        StreamCallback decryptionCallback = encryptor.getDecryptionCallback();
-
-        // Act
-        decryptionCallback.process(cipherStream, recoveredStream);
-
-        // Assert
-        byte[] recoveredBytes = ((ByteArrayOutputStream) recoveredStream).toByteArray();
-        String recovered = new String(recoveredBytes, "UTF-8");
-        logger.info("Recovered: {}", recovered);
-        Assert.assertEquals("Recovered text", PLAINTEXT, recovered);
+        for (int i = 1; i<14; i++) {
+            if (PGPEncryptedData.SAFER != i) { // SAFER cipher is not supported and therefore its test is skipped
+                Integer cipher = i;
+                // Arrange
+                byte[] plainBytes = Files.readAllBytes(Paths.get(plainFile.getPath()));
+                final String PLAINTEXT = new String(plainBytes, "UTF-8");
+
+                InputStream cipherStream = new FileInputStream(unsignedFile);
+                OutputStream recoveredStream = new ByteArrayOutputStream();
+
+                // No file, just streams
+                String filename = unsignedFile.getName();
+
+                OpenPGPKeyBasedEncryptor encryptor = new OpenPGPKeyBasedEncryptor(
+                        EncryptionMethod.PGP.getAlgorithm(), cipher, EncryptionMethod.PGP.getProvider(), SECRET_KEYRING_PATH, USER_ID, PASSWORD.toCharArray(), filename);
+
+                StreamCallback decryptionCallback = encryptor.getDecryptionCallback();
+
+                // Act
+                decryptionCallback.process(cipherStream, recoveredStream);
+
+                // Assert
+                byte[] recoveredBytes = ((ByteArrayOutputStream) recoveredStream).toByteArray();
+                String recovered = new String(recoveredBytes, "UTF-8");
+                logger.info("Recovered: {}", recovered);
+                Assert.assertEquals("Recovered text", PLAINTEXT, recovered);
+            }
+        }
     }
-}
\ No newline at end of file
+}
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/security/util/crypto/OpenPGPPasswordBasedEncryptorTest.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/security/util/crypto/OpenPGPPasswordBasedEncryptorTest.java
index 2e1cd5f..3ad926b 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/security/util/crypto/OpenPGPPasswordBasedEncryptorTest.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/security/util/crypto/OpenPGPPasswordBasedEncryptorTest.java
@@ -29,6 +29,8 @@ import org.apache.commons.codec.binary.Hex;
 import org.apache.nifi.processor.io.StreamCallback;
 import org.apache.nifi.security.util.EncryptionMethod;
 import org.bouncycastle.jce.provider.BouncyCastleProvider;
+import org.bouncycastle.openpgp.PGPEncryptedData;
+import org.bouncycastle.openpgp.PGPUtil;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -63,60 +65,75 @@ public class OpenPGPPasswordBasedEncryptorTest {
 
     @Test
     public void testShouldEncryptAndDecrypt() throws Exception {
-        // Arrange
-        final String PLAINTEXT = "This is a plaintext message.";
-        logger.info("Plaintext: {}", PLAINTEXT);
-        InputStream plainStream = new java.io.ByteArrayInputStream(PLAINTEXT.getBytes("UTF-8"));
-        OutputStream cipherStream = new ByteArrayOutputStream();
-        OutputStream recoveredStream = new ByteArrayOutputStream();
 
-        // No file, just streams
-        String filename = "tempFile.txt";
+        for (int i = 1; i<14; i++) {
+            if (PGPEncryptedData.SAFER != i) { // SAFER cipher is not supported and therefore its test is skipped
+                Integer cipher = i;
+                logger.info("Testing PGP encryption with " + PGPUtil.getSymmetricCipherName(cipher) + " cipher.");
 
-        OpenPGPPasswordBasedEncryptor encryptor = new OpenPGPPasswordBasedEncryptor(EncryptionMethod.PGP.getAlgorithm(), EncryptionMethod.PGP.getProvider(), PASSWORD.toCharArray(), filename);
+                // Arrange
+                final String PLAINTEXT = "This is a plaintext message.";
+                logger.info("Plaintext: {}", PLAINTEXT);
+                InputStream plainStream = new java.io.ByteArrayInputStream(PLAINTEXT.getBytes("UTF-8"));
+                OutputStream cipherStream = new ByteArrayOutputStream();
+                OutputStream recoveredStream = new ByteArrayOutputStream();
 
-        StreamCallback encryptionCallback = encryptor.getEncryptionCallback();
-        StreamCallback decryptionCallback = encryptor.getDecryptionCallback();
+                // No file, just streams
+                String filename = "tempFile.txt";
 
-        // Act
-        encryptionCallback.process(plainStream, cipherStream);
+                OpenPGPPasswordBasedEncryptor encryptor = new OpenPGPPasswordBasedEncryptor(EncryptionMethod.PGP.getAlgorithm(),
+                        cipher, EncryptionMethod.PGP.getProvider(), PASSWORD.toCharArray(), filename);
 
-        final byte[] cipherBytes = ((ByteArrayOutputStream) cipherStream).toByteArray();
-        logger.info("Encrypted: {}", Hex.encodeHexString(cipherBytes));
-        InputStream cipherInputStream = new ByteArrayInputStream(cipherBytes);
+                StreamCallback encryptionCallback = encryptor.getEncryptionCallback();
+                StreamCallback decryptionCallback = encryptor.getDecryptionCallback();
 
-        decryptionCallback.process(cipherInputStream, recoveredStream);
+                // Act
+                encryptionCallback.process(plainStream, cipherStream);
 
-        // Assert
-        byte[] recoveredBytes = ((ByteArrayOutputStream) recoveredStream).toByteArray();
-        String recovered = new String(recoveredBytes, "UTF-8");
-        logger.info("Recovered: {}", recovered);
-        assert PLAINTEXT.equals(recovered);
+                final byte[] cipherBytes = ((ByteArrayOutputStream) cipherStream).toByteArray();
+                logger.info("Encrypted: {}", Hex.encodeHexString(cipherBytes));
+                InputStream cipherInputStream = new ByteArrayInputStream(cipherBytes);
+
+                decryptionCallback.process(cipherInputStream, recoveredStream);
+
+                // Assert
+                byte[] recoveredBytes = ((ByteArrayOutputStream) recoveredStream).toByteArray();
+                String recovered = new String(recoveredBytes, "UTF-8");
+                logger.info("Recovered: {}", recovered);
+                assert PLAINTEXT.equals(recovered);
+            }
+        }
     }
 
     @Test
     public void testShouldDecryptExternalFile() throws Exception {
-        // Arrange
-        byte[] plainBytes = Files.readAllBytes(Paths.get(plainFile.getPath()));
-        final String PLAINTEXT = new String(plainBytes, "UTF-8");
-
-        InputStream cipherStream = new FileInputStream(encryptedFile);
-        OutputStream recoveredStream = new ByteArrayOutputStream();
-
-        // No file, just streams
-        String filename = encryptedFile.getName();
-
-        OpenPGPPasswordBasedEncryptor encryptor = new OpenPGPPasswordBasedEncryptor(EncryptionMethod.PGP.getAlgorithm(), EncryptionMethod.PGP.getProvider(), LEGACY_PASSWORD.toCharArray(), filename);
-
-        StreamCallback decryptionCallback = encryptor.getDecryptionCallback();
-
-        // Act
-        decryptionCallback.process(cipherStream, recoveredStream);
-
-        // Assert
-        byte[] recoveredBytes = ((ByteArrayOutputStream) recoveredStream).toByteArray();
-        String recovered = new String(recoveredBytes, "UTF-8");
-        logger.info("Recovered: {}", recovered);
-        Assert.assertEquals("Recovered text", PLAINTEXT, recovered);
+        for (int i = 1; i<14; i++) {
+            if (PGPEncryptedData.SAFER != i) {  // SAFER cipher is not supported and therefore its test is skipped
+                Integer cipher = i;
+                // Arrange
+                byte[] plainBytes = Files.readAllBytes(Paths.get(plainFile.getPath()));
+                final String PLAINTEXT = new String(plainBytes, "UTF-8");
+
+                InputStream cipherStream = new FileInputStream(encryptedFile);
+                OutputStream recoveredStream = new ByteArrayOutputStream();
+
+                // No file, just streams
+                String filename = encryptedFile.getName();
+
+                OpenPGPPasswordBasedEncryptor encryptor = new OpenPGPPasswordBasedEncryptor(EncryptionMethod.PGP.getAlgorithm(), cipher,
+                        EncryptionMethod.PGP.getProvider(), LEGACY_PASSWORD.toCharArray(), filename);
+
+                StreamCallback decryptionCallback = encryptor.getDecryptionCallback();
+
+                // Act
+                decryptionCallback.process(cipherStream, recoveredStream);
+
+                // Assert
+                byte[] recoveredBytes = ((ByteArrayOutputStream) recoveredStream).toByteArray();
+                String recovered = new String(recoveredBytes, "UTF-8");
+                logger.info("Recovered: {}", recovered);
+                Assert.assertEquals("Recovered text", PLAINTEXT, recovered);
+            }
+        }
     }
-}
\ No newline at end of file
+}