You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by jg...@apache.org on 2021/10/19 10:47:56 UTC

[nifi] branch main updated: NIFI-7322 Added SignContentPGP and VerifyContentPGP

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

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


The following commit(s) were added to refs/heads/main by this push:
     new da1b1df  NIFI-7322 Added SignContentPGP and VerifyContentPGP
da1b1df is described below

commit da1b1dfd4be1e7e2ac5096eedbc954da95f713a8
Author: exceptionfactory <ex...@apache.org>
AuthorDate: Wed Oct 6 21:03:25 2021 -0500

    NIFI-7322 Added SignContentPGP and VerifyContentPGP
    
    - Added Decryption Strategy property to DecryptContentPGP
    - Added OpenPGP Packet detection to EncryptContentPGP to avoid unnecessary packaging
    - Refactored shared processing to EncodingStreamCallback
    
    Signed-off-by: Joe Gresock <jg...@gmail.com>
    
    This closes #5457.
---
 .../nifi/processors/pgp/DecryptContentPGP.java     |  77 +++-
 .../nifi/processors/pgp/EncryptContentPGP.java     | 118 +++---
 .../nifi/processors/pgp/PGPAttributeKey.java       |  14 +
 .../apache/nifi/processors/pgp/SignContentPGP.java | 408 +++++++++++++++++++++
 .../nifi/processors/pgp/VerifyContentPGP.java      | 321 ++++++++++++++++
 .../pgp/attributes/DecryptionStrategy.java         |  36 ++
 .../processors/pgp/attributes/HashAlgorithm.java   |  40 ++
 .../processors/pgp/attributes/SigningStrategy.java |  36 ++
 .../processors/pgp/io/EncodingStreamCallback.java  | 126 +++++++
 .../processors/pgp/io/KeyIdentifierConverter.java  |  47 +++
 .../services/org.apache.nifi.processor.Processor   |   4 +-
 .../nifi/processors/pgp/DecryptContentPGPTest.java | 113 ++++--
 .../nifi/processors/pgp/EncryptContentPGPTest.java |  75 ++--
 .../nifi/processors/pgp/SignContentPGPTest.java    | 277 ++++++++++++++
 .../nifi/processors/pgp/VerifyContentPGPTest.java  | 187 ++++++++++
 .../pgp/io/EncodingStreamCallbackTest.java         |  96 +++++
 .../pgp/io/KeyIdentifierConverterTest.java         |  47 +++
 .../apache/nifi/pgp/util/PGPOperationUtils.java    | 108 ++++++
 18 files changed, 2008 insertions(+), 122 deletions(-)

diff --git a/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/DecryptContentPGP.java b/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/DecryptContentPGP.java
index 1e570fc..048d483 100644
--- a/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/DecryptContentPGP.java
+++ b/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/DecryptContentPGP.java
@@ -22,6 +22,7 @@ import org.apache.nifi.annotation.behavior.WritesAttributes;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.SeeAlso;
 import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.PropertyValue;
 import org.apache.nifi.components.ValidationContext;
@@ -34,8 +35,10 @@ 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.DecryptionStrategy;
 import org.apache.nifi.processors.pgp.exception.PGPDecryptionException;
 import org.apache.nifi.processors.pgp.exception.PGPProcessException;
+import org.apache.nifi.processors.pgp.io.KeyIdentifierConverter;
 import org.apache.nifi.stream.io.StreamUtils;
 
 import org.apache.nifi.util.StringUtils;
@@ -76,11 +79,12 @@ import java.util.Set;
  */
 @InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
 @Tags({"PGP", "GPG", "OpenPGP", "Encryption", "RFC 4880"})
-@CapabilityDescription("Decrypt Contents of OpenPGP Messages")
-@SeeAlso(EncryptContentPGP.class)
+@CapabilityDescription("Decrypt contents of OpenPGP messages. Using the Packaged Decryption Strategy preserves OpenPGP encoding to support subsequent signature verification.")
+@SeeAlso({EncryptContentPGP.class, SignContentPGP.class, VerifyContentPGP.class})
 @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_BLOCK_CIPHER, description = "Symmetric-Key Algorithm Block Cipher"),
         @WritesAttribute(attribute = PGPAttributeKey.SYMMETRIC_KEY_ALGORITHM_ID, description = "Symmetric-Key Algorithm Identifier")
 })
 public class DecryptContentPGP extends AbstractProcessor {
@@ -95,6 +99,19 @@ public class DecryptContentPGP extends AbstractProcessor {
             .description("Decryption Failed")
             .build();
 
+    public static final PropertyDescriptor DECRYPTION_STRATEGY = new PropertyDescriptor.Builder()
+            .name("decryption-strategy")
+            .displayName("Decryption Strategy")
+            .description("Strategy for writing files to success after decryption")
+            .required(true)
+            .defaultValue(DecryptionStrategy.DECRYPTED.name())
+            .allowableValues(
+                    Arrays.stream(DecryptionStrategy.values()).map(strategy ->
+                            new AllowableValue(strategy.name(), strategy.name(), strategy.getDescription())
+                    ).toArray(AllowableValue[]::new)
+            )
+            .build();
+
     public static final PropertyDescriptor PASSPHRASE = new PropertyDescriptor.Builder()
             .name("passphrase")
             .displayName("Passphrase")
@@ -113,6 +130,7 @@ public class DecryptContentPGP extends AbstractProcessor {
     private static final Set<Relationship> RELATIONSHIPS = new HashSet<>(Arrays.asList(SUCCESS, FAILURE));
 
     private static final List<PropertyDescriptor> DESCRIPTORS = Arrays.asList(
+            DECRYPTION_STRATEGY,
             PASSPHRASE,
             PRIVATE_KEY_SERVICE
     );
@@ -156,7 +174,8 @@ public class DecryptContentPGP extends AbstractProcessor {
 
         final char[] passphrase = getPassphrase(context);
         final PGPPrivateKeyService privateKeyService = getPrivateKeyService(context);
-        final DecryptStreamCallback callback = new DecryptStreamCallback(passphrase, privateKeyService);
+        final DecryptionStrategy decryptionStrategy = getDecryptionStrategy(context);
+        final DecryptStreamCallback callback = new DecryptStreamCallback(passphrase, privateKeyService, decryptionStrategy);
 
         try {
             flowFile = session.write(flowFile, callback);
@@ -213,16 +232,26 @@ public class DecryptContentPGP extends AbstractProcessor {
         return privateKeyService;
     }
 
+    private DecryptionStrategy getDecryptionStrategy(final ProcessContext context) {
+        final String strategy = context.getProperty(DECRYPTION_STRATEGY).getValue();
+        return DecryptionStrategy.valueOf(strategy);
+    }
+
     private class DecryptStreamCallback implements StreamCallback {
         private final char[] passphrase;
 
         private final PGPPrivateKeyService privateKeyService;
 
+        private final DecryptionStrategy decryptionStrategy;
+
         private final Map<String, String> attributes = new HashMap<>();
 
-        public DecryptStreamCallback(final char[] passphrase, final PGPPrivateKeyService privateKeyService) {
+        public DecryptStreamCallback(final char[] passphrase,
+                                     final PGPPrivateKeyService privateKeyService,
+                                     final DecryptionStrategy decryptionStrategy) {
             this.passphrase = passphrase;
             this.privateKeyService = privateKeyService;
+            this.decryptionStrategy = decryptionStrategy;
         }
 
         /**
@@ -236,13 +265,23 @@ public class DecryptContentPGP extends AbstractProcessor {
         public void process(final InputStream inputStream, final OutputStream outputStream) throws IOException {
             final PGPEncryptedDataList encryptedDataList = getEncryptedDataList(inputStream);
             final PGPEncryptedData encryptedData = findSupportedEncryptedData(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()));
+            if (DecryptionStrategy.PACKAGED == decryptionStrategy) {
+                try {
+                    final InputStream decryptedDataStream = getDecryptedDataStream(encryptedData);
+                    StreamUtils.copy(decryptedDataStream, outputStream);
+                } catch (final PGPException e) {
+                    final String message = String.format("PGP Decryption Failed [%s]", getEncryptedDataType(encryptedData));
+                    throw new PGPDecryptionException(message, e);
+                }
+            } else {
+                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);
+                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");
@@ -286,7 +325,7 @@ public class DecryptContentPGP extends AbstractProcessor {
                     final Optional<PGPPrivateKey> privateKey = privateKeyService.findPrivateKey(keyId);
                     if (privateKey.isPresent()) {
                         supportedEncryptedData = publicKeyEncryptedData;
-                        final String keyIdentifier = Long.toHexString(keyId).toUpperCase();
+                        final String keyIdentifier = KeyIdentifierConverter.format(keyId);
                         getLogger().debug("PGP Private Key [{}] Found for Public Key Encrypted Data", keyIdentifier);
                         break;
                     }
@@ -354,7 +393,7 @@ public class DecryptContentPGP extends AbstractProcessor {
             } 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));
+                setSymmetricKeyAlgorithmAttributes(symmetricAlgorithm);
                 return passwordBasedEncryptedData.getDataStream(decryptorFactory);
             }
         }
@@ -369,16 +408,22 @@ public class DecryptContentPGP extends AbstractProcessor {
                     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));
+                    setSymmetricKeyAlgorithmAttributes(symmetricAlgorithm);
                     return publicKeyEncryptedData.getDataStream(decryptorFactory);
                 } else {
-                    final String keyIdentifier = Long.toHexString(keyId).toUpperCase();
+                    final String keyIdentifier = KeyIdentifierConverter.format(keyId);
                     final String message = String.format("PGP Private Key [%s] not found for Public Key Encryption", keyIdentifier);
                     throw new PGPDecryptionException(message);
                 }
             }
         }
 
+        private void setSymmetricKeyAlgorithmAttributes(final int symmetricAlgorithm) {
+            final String blockCipher = PGPUtil.getSymmetricCipherName(symmetricAlgorithm);
+            attributes.put(PGPAttributeKey.SYMMETRIC_KEY_ALGORITHM_BLOCK_CIPHER, blockCipher);
+            attributes.put(PGPAttributeKey.SYMMETRIC_KEY_ALGORITHM_ID, Integer.toString(symmetricAlgorithm));
+        }
+
         private boolean isVerified(final PGPEncryptedData encryptedData) {
             boolean verified;
 
@@ -399,8 +444,7 @@ public class DecryptContentPGP extends AbstractProcessor {
 
         private PGPEncryptedDataList getEncryptedDataList(final InputStream inputStream) throws IOException {
             final InputStream decoderInputStream = PGPUtil.getDecoderStream(inputStream);
-            final PGPObjectFactory encryptedObjectFactory = new JcaPGPObjectFactory(decoderInputStream);
-            final PGPEncryptedDataList encryptedDataList = findEncryptedDataList(encryptedObjectFactory);
+            final PGPEncryptedDataList encryptedDataList = findEncryptedDataList(decoderInputStream);
             if (encryptedDataList == null) {
                 throw new PGPProcessException("PGP Encrypted Data Packets not found");
             } else {
@@ -409,9 +453,10 @@ public class DecryptContentPGP extends AbstractProcessor {
             }
         }
 
-        private PGPEncryptedDataList findEncryptedDataList(final PGPObjectFactory objectFactory) {
+        private PGPEncryptedDataList findEncryptedDataList(final InputStream inputStream) {
             PGPEncryptedDataList encryptedDataList = null;
 
+            final PGPObjectFactory objectFactory = new JcaPGPObjectFactory(inputStream);
             for (final Object object : objectFactory) {
                 getLogger().debug("PGP Object Read [{}]", object.getClass().getSimpleName());
                 if (object instanceof PGPEncryptedDataList) {
diff --git a/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/EncryptContentPGP.java b/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/EncryptContentPGP.java
index 49a345f..a45d90d 100644
--- a/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/EncryptContentPGP.java
+++ b/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/EncryptContentPGP.java
@@ -34,22 +34,23 @@ 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.InputStreamCallback;
 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.processors.pgp.io.EncodingStreamCallback;
 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.bcpg.BCPGInputStream;
+import org.bouncycastle.bcpg.Packet;
 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.PGPUtil;
 import org.bouncycastle.openpgp.operator.PGPDataEncryptorBuilder;
 import org.bouncycastle.openpgp.operator.PGPKeyEncryptionMethodGenerator;
 import org.bouncycastle.openpgp.operator.bc.BcPGPDataEncryptorBuilder;
@@ -63,7 +64,6 @@ 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;
@@ -76,8 +76,8 @@ import java.util.Set;
  */
 @InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
 @Tags({"PGP", "GPG", "OpenPGP", "Encryption", "RFC 4880"})
-@CapabilityDescription("Encrypt Contents using OpenPGP")
-@SeeAlso(DecryptContentPGP.class)
+@CapabilityDescription("Encrypt contents using OpenPGP. The processor reads input and detects OpenPGP messages to avoid unnecessary additional wrapping in Literal Data packets.")
+@SeeAlso({ DecryptContentPGP.class, SignContentPGP.class, VerifyContentPGP.class })
 @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"),
@@ -153,8 +153,6 @@ public class EncryptContentPGP extends AbstractProcessor {
     /** 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(
@@ -200,10 +198,14 @@ public class EncryptContentPGP extends AbstractProcessor {
         }
 
         try {
+            final PacketReadInputStreamCallback packetCallback = new PacketReadInputStreamCallback();
+            session.read(flowFile, packetCallback);
+
             final SymmetricKeyAlgorithm symmetricKeyAlgorithm = getSymmetricKeyAlgorithm(context);
             final FileEncoding fileEncoding = getFileEncoding(context);
             final CompressionAlgorithm compressionAlgorithm = getCompressionAlgorithm(context);
-            final StreamCallback callback = getEncryptStreamCallback(context, flowFile, symmetricKeyAlgorithm, compressionAlgorithm, fileEncoding);
+            final StreamCallback callback = getEncryptStreamCallback(context, flowFile, symmetricKeyAlgorithm,
+                    compressionAlgorithm, fileEncoding, packetCallback.packetFound);
             flowFile = session.write(flowFile, callback);
 
             final Map<String, String> attributes = getAttributes(symmetricKeyAlgorithm, fileEncoding, compressionAlgorithm);
@@ -261,10 +263,12 @@ public class EncryptContentPGP extends AbstractProcessor {
         return results;
     }
 
-    private StreamCallback getEncryptStreamCallback(final ProcessContext context, final FlowFile flowFile,
+    private StreamCallback getEncryptStreamCallback(final ProcessContext context,
+                                                    final FlowFile flowFile,
                                                     final SymmetricKeyAlgorithm symmetricKeyAlgorithm,
                                                     final CompressionAlgorithm compressionAlgorithm,
-                                                    final FileEncoding fileEncoding) {
+                                                    final FileEncoding fileEncoding,
+                                                    final boolean packetFound) {
         final SecureRandom secureRandom = new SecureRandom();
         final PGPDataEncryptorBuilder dataEncryptorBuilder = new BcPGPDataEncryptorBuilder(symmetricKeyAlgorithm.getId())
                 .setSecureRandom(secureRandom)
@@ -274,7 +278,7 @@ public class EncryptContentPGP extends AbstractProcessor {
         methodGenerators.forEach(encryptedDataGenerator::addMethod);
 
         final String filename = flowFile.getAttribute(CoreAttributes.FILENAME.key());
-        return new EncryptStreamCallback(filename, fileEncoding, encryptedDataGenerator, compressionAlgorithm);
+        return new EncryptStreamCallback(fileEncoding, compressionAlgorithm, filename, packetFound, encryptedDataGenerator);
     }
 
     private List<PGPKeyEncryptionMethodGenerator> getEncryptionMethodGenerators(final ProcessContext context,
@@ -334,65 +338,65 @@ public class EncryptContentPGP extends AbstractProcessor {
         return attributes;
     }
 
-    private static class EncryptStreamCallback implements StreamCallback {
-        private static final char DATA_FORMAT = PGPLiteralData.BINARY;
+    private class PacketReadInputStreamCallback implements InputStreamCallback {
+        private boolean packetFound;
 
-        private final String filename;
+        /**
+         * Process Input Stream and attempt to read OpenPGP Packet for content detection
+         *
+         * @param inputStream Input Stream to be read
+         */
+        @Override
+        public void process(final InputStream inputStream) {
+            try {
+                final InputStream decodedInputStream = PGPUtil.getDecoderStream(inputStream);
+                final BCPGInputStream packetInputStream = new BCPGInputStream(decodedInputStream);
+                final Packet packet = packetInputStream.readPacket();
+                if (packet == null) {
+                    getLogger().debug("PGP Packet not found");
+                } else {
+                    packetFound = true;
+                }
+            } catch (final IOException e) {
+                getLogger().debug("PGP Packet read failed", e);
+            }
+        }
+    }
 
-        private final FileEncoding fileEncoding;
+    private static class EncryptStreamCallback extends EncodingStreamCallback {
+        private final boolean packetFound;
 
         private final PGPEncryptedDataGenerator encryptedDataGenerator;
 
-        private final CompressionAlgorithm compressionAlgorithm;
-
-        public EncryptStreamCallback(final String filename, final FileEncoding fileEncoding, final PGPEncryptedDataGenerator encryptedDataGenerator, final CompressionAlgorithm compressionAlgorithm) {
-            this.filename = filename;
-            this.fileEncoding = fileEncoding;
+        public EncryptStreamCallback(final FileEncoding fileEncoding,
+                                     final CompressionAlgorithm compressionAlgorithm,
+                                     final String filename,
+                                     final boolean packetFound,
+                                     final PGPEncryptedDataGenerator encryptedDataGenerator) {
+            super(fileEncoding, compressionAlgorithm, filename);
+            this.packetFound = packetFound;
             this.encryptedDataGenerator = encryptedDataGenerator;
-            this.compressionAlgorithm = compressionAlgorithm;
         }
 
         /**
-         * Process Input Stream and write encrypted contents to Output Stream
+         * Process Encoding Output Stream using Encrypted Data Generator with for subsequent processing
          *
-         * @param inputStream  Input Stream
-         * @param outputStream Output Stream for encrypted contents
-         * @throws IOException Thrown when unable to read or write streams
+         * @param inputStream          Input Stream
+         * @param encodingOutputStream Output Stream to be processed for encryption
+         * @throws IOException Thrown when failing to read or write streams
+         * @throws PGPException Thrown when failing to perform encryption operations
          */
         @Override
-        public void process(final InputStream inputStream, final OutputStream outputStream) throws IOException {
-            try (final OutputStream encodingOutputStream = getEncodingOutputStream(outputStream)) {
-                processEncoding(inputStream, encodingOutputStream);
-            } catch (final PGPException e) {
-                throw new PGPEncryptionException("PGP Encryption Stream Processing Failed", e);
-            }
-        }
-
-        private OutputStream getEncodingOutputStream(final OutputStream outputStream) throws PGPException {
-            OutputStream encodingOutputStream = outputStream;
-            if (FileEncoding.ASCII.equals(fileEncoding)) {
-                encodingOutputStream = new ArmoredOutputStream(outputStream);
-            }
-            return encodingOutputStream;
-        }
-
-        private void processEncoding(final InputStream inputStream, final OutputStream encodingOutputStream) throws IOException, PGPException {
-            try (final OutputStream encryptedOutputStream = encryptedDataGenerator.open(encodingOutputStream, createBuffer())) {
-                final PGPCompressedDataGenerator compressedDataGenerator = new PGPCompressedDataGenerator(compressionAlgorithm.getId());
-                try (final OutputStream compressedOutputStream = compressedDataGenerator.open(encryptedOutputStream, createBuffer())) {
-                    final PGPLiteralDataGenerator literalDataGenerator = new PGPLiteralDataGenerator();
-                    try (final OutputStream literalOutputStream = literalDataGenerator.open(compressedOutputStream, DATA_FORMAT, filename, new Date(), createBuffer())) {
-                        StreamUtils.copy(inputStream, literalOutputStream);
-                    }
-                    literalDataGenerator.close();
+        protected void processEncoding(final InputStream inputStream, final OutputStream encodingOutputStream) throws IOException, PGPException {
+            try (final OutputStream encryptedOutputStream = encryptedDataGenerator.open(encodingOutputStream, createOutputBuffer())) {
+                if (packetFound) {
+                    // Write OpenPGP packets to encrypted stream without additional encoding
+                    StreamUtils.copy(inputStream, encryptedOutputStream);
+                } else {
+                    super.processEncoding(inputStream, encryptedOutputStream);
                 }
-                compressedDataGenerator.close();
             }
             encryptedDataGenerator.close();
         }
-
-        private byte[] createBuffer() {
-            return new byte[OUTPUT_BUFFER_SIZE];
-        }
     }
 }
diff --git a/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/PGPAttributeKey.java b/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/PGPAttributeKey.java
index 17b17dc..459b158 100644
--- a/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/PGPAttributeKey.java
+++ b/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/PGPAttributeKey.java
@@ -37,4 +37,18 @@ class PGPAttributeKey {
     static final String COMPRESS_ALGORITHM = "pgp.compression.algorithm";
 
     static final String COMPRESS_ALGORITHM_ID = "pgp.compression.algorithm.id";
+
+    static final String SIGNATURE_CREATED = "pgp.signature.created";
+
+    static final String SIGNATURE_ALGORITHM = "pgp.signature.algorithm";
+
+    static final String SIGNATURE_HASH_ALGORITHM_ID = "pgp.signature.hash.algorithm.id";
+
+    static final String SIGNATURE_KEY_ALGORITHM_ID = "pgp.signature.key.algorithm.id";
+
+    static final String SIGNATURE_KEY_ID = "pgp.signature.key.id";
+
+    static final String SIGNATURE_TYPE_ID = "pgp.signature.type.id";
+
+    static final String SIGNATURE_VERSION = "pgp.signature.version";
 }
diff --git a/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/SignContentPGP.java b/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/SignContentPGP.java
new file mode 100644
index 0000000..cdee4be
--- /dev/null
+++ b/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/SignContentPGP.java
@@ -0,0 +1,408 @@
+/*
+ * 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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+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.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.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.HashAlgorithm;
+import org.apache.nifi.processors.pgp.attributes.SigningStrategy;
+import org.apache.nifi.processors.pgp.exception.PGPProcessException;
+import org.apache.nifi.processors.pgp.io.EncodingStreamCallback;
+import org.apache.nifi.processors.pgp.io.KeyIdentifierConverter;
+import org.bouncycastle.openpgp.PGPException;
+import org.bouncycastle.openpgp.PGPLiteralDataGenerator;
+import org.bouncycastle.openpgp.PGPOnePassSignature;
+import org.bouncycastle.openpgp.PGPPrivateKey;
+import org.bouncycastle.openpgp.PGPSignature;
+import org.bouncycastle.openpgp.PGPSignatureGenerator;
+import org.bouncycastle.openpgp.PGPUtil;
+import org.bouncycastle.openpgp.operator.jcajce.JcaPGPContentSignerBuilder;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.security.SecureRandom;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * Sign Content using Open Pretty Good Privacy Private Keys
+ */
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Tags({"PGP", "GPG", "OpenPGP", "Encryption", "Signing", "RFC 4880"})
+@CapabilityDescription("Sign content using OpenPGP Private Keys")
+@SeeAlso({DecryptContentPGP.class, EncryptContentPGP.class, VerifyContentPGP.class})
+@WritesAttributes({
+        @WritesAttribute(attribute = PGPAttributeKey.COMPRESS_ALGORITHM, description = "Compression Algorithm"),
+        @WritesAttribute(attribute = PGPAttributeKey.COMPRESS_ALGORITHM_ID, description = "Compression Algorithm Identifier"),
+        @WritesAttribute(attribute = PGPAttributeKey.FILE_ENCODING, description = "File Encoding"),
+        @WritesAttribute(attribute = PGPAttributeKey.SIGNATURE_ALGORITHM, description = "Signature Algorithm including key and hash algorithm names"),
+        @WritesAttribute(attribute = PGPAttributeKey.SIGNATURE_HASH_ALGORITHM_ID, description = "Signature Hash Algorithm Identifier"),
+        @WritesAttribute(attribute = PGPAttributeKey.SIGNATURE_KEY_ALGORITHM_ID, description = "Signature Key Algorithm Identifier"),
+        @WritesAttribute(attribute = PGPAttributeKey.SIGNATURE_KEY_ID, description = "Signature Public Key Identifier"),
+        @WritesAttribute(attribute = PGPAttributeKey.SIGNATURE_TYPE_ID, description = "Signature Type Identifier"),
+        @WritesAttribute(attribute = PGPAttributeKey.SIGNATURE_VERSION, description = "Signature Version Number"),
+})
+public class SignContentPGP extends AbstractProcessor {
+
+    public static final Relationship SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Content signing succeeded")
+            .build();
+
+    public static final Relationship FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Content signing failed")
+            .build();
+
+    public static final PropertyDescriptor COMPRESSION_ALGORITHM = new PropertyDescriptor.Builder()
+            .name("compression-algorithm")
+            .displayName("Compression Algorithm")
+            .description("Compression Algorithm for signing")
+            .required(true)
+            .defaultValue(CompressionAlgorithm.ZIP.name())
+            .allowableValues(CompressionAlgorithm.values())
+            .build();
+
+    public static final PropertyDescriptor FILE_ENCODING = new PropertyDescriptor.Builder()
+            .name("file-encoding")
+            .displayName("File Encoding")
+            .description("File Encoding for signing")
+            .required(true)
+            .defaultValue(FileEncoding.BINARY.name())
+            .allowableValues(FileEncoding.values())
+            .build();
+
+    public static final PropertyDescriptor HASH_ALGORITHM = new PropertyDescriptor.Builder()
+            .name("hash-algorithm")
+            .displayName("Hash Algorithm")
+            .description("Hash Algorithm for signing")
+            .required(true)
+            .defaultValue(HashAlgorithm.SHA512.name())
+            .allowableValues(HashAlgorithm.values())
+            .build();
+
+    public static final PropertyDescriptor SIGNING_STRATEGY = new PropertyDescriptor.Builder()
+            .name("signing-strategy")
+            .displayName("Signing Strategy")
+            .description("Strategy for writing files to success after signing")
+            .required(true)
+            .defaultValue(SigningStrategy.SIGNED.name())
+            .allowableValues(
+                    Arrays.stream(SigningStrategy.values()).map(strategy ->
+                            new AllowableValue(strategy.name(), strategy.name(), strategy.getDescription())
+                    ).toArray(AllowableValue[]::new)
+            )
+            .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 generating content signatures")
+            .identifiesControllerService(PGPPrivateKeyService.class)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor PRIVATE_KEY_ID = new PropertyDescriptor.Builder()
+            .name("private-key-id")
+            .displayName("Private Key ID")
+            .description("PGP Private Key Identifier formatted as uppercase hexadecimal string of 16 characters used for signing")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
+            .required(true)
+            .build();
+
+    private static final Set<Relationship> RELATIONSHIPS = new HashSet<>(Arrays.asList(SUCCESS, FAILURE));
+
+    private static final List<PropertyDescriptor> DESCRIPTORS = Arrays.asList(
+            COMPRESSION_ALGORITHM,
+            FILE_ENCODING,
+            HASH_ALGORITHM,
+            SIGNING_STRATEGY,
+            PRIVATE_KEY_SERVICE,
+            PRIVATE_KEY_ID
+    );
+
+    private static final boolean NESTED_SIGNATURE_DISABLED = false;
+
+    /**
+     * 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 generates signatures for Flow File contents using private keys
+     *
+     * @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 SignatureStreamCallback callback = getStreamCallback(context, flowFile);
+            flowFile = session.write(flowFile, callback);
+            flowFile = session.putAllAttributes(flowFile, callback.attributes);
+            session.transfer(flowFile, SUCCESS);
+        } catch (final RuntimeException e) {
+            getLogger().error("Signing Failed {}", flowFile, e);
+            session.transfer(flowFile, FAILURE);
+        }
+    }
+
+    private SignatureStreamCallback getStreamCallback(final ProcessContext context, final FlowFile flowFile) {
+        final FileEncoding fileEncoding = getFileEncoding(context);
+        final CompressionAlgorithm compressionAlgorithm = getCompressionAlgorithm(context);
+        final HashAlgorithm hashAlgorithm = getHashAlgorithm(context);
+        final String filename = flowFile.getAttribute(CoreAttributes.FILENAME.key());
+        final SigningStrategy signingStrategy = getSigningStrategy(context);
+        final PGPPrivateKey privateKey = getPrivateKey(context, flowFile);
+        return SigningStrategy.SIGNED.equals(signingStrategy)
+                ? new SignedStreamCallback(fileEncoding, compressionAlgorithm, filename, hashAlgorithm, privateKey)
+                : new DetachedStreamCallback(fileEncoding, compressionAlgorithm, filename, hashAlgorithm, privateKey);
+    }
+
+    private PGPPrivateKey getPrivateKey(final ProcessContext context, final FlowFile flowFile) {
+        final PGPPrivateKeyService privateKeyService = context.getProperty(PRIVATE_KEY_SERVICE).asControllerService(PGPPrivateKeyService.class);
+        final long privateKeyId = getPrivateKeyId(context, flowFile);
+        final Optional<PGPPrivateKey> optionalPrivateKey = privateKeyService.findPrivateKey(privateKeyId);
+
+        return optionalPrivateKey.orElseThrow(() -> {
+            final String message = String.format("Private Key ID [%s] not found", KeyIdentifierConverter.format(privateKeyId));
+            return new PGPProcessException(message);
+        });
+    }
+
+    private long getPrivateKeyId(final ProcessContext context, final FlowFile flowFile) {
+        final String privateKeyId = context.getProperty(PRIVATE_KEY_ID).evaluateAttributeExpressions(flowFile).getValue();
+        try {
+            return KeyIdentifierConverter.parse(privateKeyId);
+        } catch (final NumberFormatException e) {
+            throw new PGPProcessException(String.format("Private Key ID [%s] Hexadecimal Parsing Failed", privateKeyId), e);
+        }
+    }
+
+    private CompressionAlgorithm getCompressionAlgorithm(final ProcessContext context) {
+        final String algorithm = context.getProperty(COMPRESSION_ALGORITHM).getValue();
+        return CompressionAlgorithm.valueOf(algorithm);
+    }
+
+    private FileEncoding getFileEncoding(final ProcessContext context) {
+        final String encoding = context.getProperty(FILE_ENCODING).getValue();
+        return FileEncoding.valueOf(encoding);
+    }
+
+    private HashAlgorithm getHashAlgorithm(final ProcessContext context) {
+        final String algorithm = context.getProperty(HASH_ALGORITHM).getValue();
+        return HashAlgorithm.valueOf(algorithm);
+    }
+
+    private SigningStrategy getSigningStrategy(final ProcessContext context) {
+        final String strategy = context.getProperty(SIGNING_STRATEGY).getValue();
+        return SigningStrategy.valueOf(strategy);
+    }
+
+    private class SignatureStreamCallback extends EncodingStreamCallback {
+        private final PGPPrivateKey privateKey;
+
+        private final HashAlgorithm hashAlgorithm;
+
+        private final Map<String, String> attributes = new HashMap<>();
+
+        protected SignatureStreamCallback(final FileEncoding fileEncoding,
+                                   final CompressionAlgorithm compressionAlgorithm,
+                                   final String filename,
+                                   final HashAlgorithm hashAlgorithm,
+                                   final PGPPrivateKey privateKey
+        ) {
+            super(fileEncoding, compressionAlgorithm, filename);
+            this.hashAlgorithm = hashAlgorithm;
+            this.privateKey = privateKey;
+
+            attributes.put(PGPAttributeKey.COMPRESS_ALGORITHM, compressionAlgorithm.toString());
+            attributes.put(PGPAttributeKey.COMPRESS_ALGORITHM_ID, Integer.toString(compressionAlgorithm.getId()));
+            attributes.put(PGPAttributeKey.FILE_ENCODING, fileEncoding.toString());
+        }
+
+        /***
+         * Write Signature to Output Stream
+         *
+         * @param signatureGenerator Signature Generator initialized with Private Key
+         * @param outputStream Output Stream for writing encoded signature
+         * @throws PGPException Thrown when failing to generate signature
+         * @throws IOException Thrown when failing to write signature
+         */
+        protected void writeSignature(final PGPSignatureGenerator signatureGenerator, final OutputStream outputStream) throws PGPException, IOException {
+            final PGPSignature signature = signatureGenerator.generate();
+            signature.encode(outputStream);
+            setSignatureAttributes(signature);
+        }
+
+        /**
+         * Get Signature Generator initialized using configuration properties and Private Key
+         * @return Initialized Signature Generator
+         * @throws PGPException Thrown when failing to initialize signature generator
+         */
+        protected PGPSignatureGenerator getSignatureGenerator() throws PGPException {
+            final int keyAlgorithm = privateKey.getPublicKeyPacket().getAlgorithm();
+            final SecureRandom secureRandom = new SecureRandom();
+            final JcaPGPContentSignerBuilder builder = new JcaPGPContentSignerBuilder(keyAlgorithm, hashAlgorithm.getId()).setSecureRandom(secureRandom);
+            final PGPSignatureGenerator signatureGenerator = new PGPSignatureGenerator(builder);
+            signatureGenerator.init(PGPSignature.BINARY_DOCUMENT, privateKey);
+            return signatureGenerator;
+        }
+
+        private void setSignatureAttributes(final PGPSignature signature) {
+            setSignatureAlgorithm(signature.getKeyAlgorithm(), signature.getHashAlgorithm());
+            attributes.put(PGPAttributeKey.SIGNATURE_CREATED, Long.toString(signature.getCreationTime().getTime()));
+            attributes.put(PGPAttributeKey.SIGNATURE_KEY_ID, KeyIdentifierConverter.format(signature.getKeyID()));
+            attributes.put(PGPAttributeKey.SIGNATURE_TYPE_ID, Integer.toString(signature.getSignatureType()));
+            attributes.put(PGPAttributeKey.SIGNATURE_VERSION, Integer.toString(signature.getVersion()));
+        }
+
+        private void setSignatureAlgorithm(final int keyAlgorithm, final int hashAlgorithm) {
+            attributes.put(PGPAttributeKey.SIGNATURE_HASH_ALGORITHM_ID, Integer.toString(hashAlgorithm));
+            attributes.put(PGPAttributeKey.SIGNATURE_KEY_ALGORITHM_ID, Integer.toString(keyAlgorithm));
+            try {
+                final String algorithm = PGPUtil.getSignatureName(keyAlgorithm, hashAlgorithm);
+                attributes.put(PGPAttributeKey.SIGNATURE_ALGORITHM, algorithm);
+            } catch (final PGPException e) {
+                getLogger().debug("Signature Algorithm Key Identifier [{}] Hash Identifier [{}] not found", keyAlgorithm, hashAlgorithm);
+            }
+        }
+    }
+
+    private class DetachedStreamCallback extends SignatureStreamCallback {
+        private DetachedStreamCallback(final FileEncoding fileEncoding,
+                                        final CompressionAlgorithm compressionAlgorithm,
+                                        final String filename,
+                                        final HashAlgorithm hashAlgorithm,
+                                        final PGPPrivateKey privateKey
+        ) {
+            super(fileEncoding, compressionAlgorithm, filename, hashAlgorithm, privateKey);
+        }
+
+        /**
+         * Process Encoding writes detached signature through Compression Output Stream
+         *
+         * @param inputStream          Input Stream
+         * @param encodingOutputStream Output Stream configured according to File Encoding
+         * @throws IOException  Thrown when unable to read or write streams
+         * @throws PGPException Thrown when unable to process compression
+         */
+        @Override
+        protected void processEncoding(final InputStream inputStream, final OutputStream encodingOutputStream) throws IOException, PGPException {
+            processDetached(inputStream, encodingOutputStream);
+        }
+
+        private void processDetached(final InputStream inputStream, final OutputStream outputStream) throws IOException, PGPException {
+            final PGPSignatureGenerator signatureGenerator = getSignatureGenerator();
+            final byte[] buffer = createOutputBuffer();
+            int read;
+            while ((read = inputStream.read(buffer)) >= 0) {
+                signatureGenerator.update(buffer, 0, read);
+            }
+            writeSignature(signatureGenerator, outputStream);
+        }
+    }
+
+    private class SignedStreamCallback extends SignatureStreamCallback {
+
+        private SignedStreamCallback(final FileEncoding fileEncoding,
+                                   final CompressionAlgorithm compressionAlgorithm,
+                                   final String filename,
+                                   final HashAlgorithm hashAlgorithm,
+                                   final PGPPrivateKey privateKey
+        ) {
+            super(fileEncoding, compressionAlgorithm, filename, hashAlgorithm, privateKey);
+        }
+
+        /**
+         * Process Compression passing Input Stream through Literal Data Output Stream prepended with One-Pass Signature and followed with Signature
+         *
+         * @param inputStream            Input Stream
+         * @param compressedOutputStream Output Stream configured according to Compression Algorithm
+         * @throws IOException  Thrown when unable to read or write streams
+         * @throws PGPException Thrown when unable to generate signatures
+         */
+        @Override
+        protected void processCompression(final InputStream inputStream, final OutputStream compressedOutputStream) throws IOException, PGPException {
+            final PGPSignatureGenerator signatureGenerator = getSignatureGenerator();
+
+            final PGPOnePassSignature onePassSignature = signatureGenerator.generateOnePassVersion(NESTED_SIGNATURE_DISABLED);
+            onePassSignature.encode(compressedOutputStream);
+
+            final PGPLiteralDataGenerator literalDataGenerator = new PGPLiteralDataGenerator();
+            try (final OutputStream literalOutputStream = openLiteralOutputStream(literalDataGenerator, compressedOutputStream)) {
+                processSigned(inputStream, literalOutputStream, signatureGenerator);
+            }
+            literalDataGenerator.close();
+
+            writeSignature(signatureGenerator, compressedOutputStream);
+        }
+
+        private void processSigned(final InputStream inputStream, final OutputStream outputStream, final PGPSignatureGenerator signatureGenerator) throws IOException {
+            final byte[] buffer = createOutputBuffer();
+            int read;
+            while ((read = inputStream.read(buffer)) >= 0) {
+                outputStream.write(buffer, 0, read);
+                signatureGenerator.update(buffer, 0, read);
+            }
+        }
+    }
+}
diff --git a/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/VerifyContentPGP.java b/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/VerifyContentPGP.java
new file mode 100644
index 0000000..115ffdd
--- /dev/null
+++ b/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/VerifyContentPGP.java
@@ -0,0 +1,321 @@
+/*
+ * 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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+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.processors.pgp.exception.PGPProcessException;
+import org.apache.nifi.processors.pgp.io.KeyIdentifierConverter;
+import org.apache.nifi.stream.io.StreamUtils;
+import org.bouncycastle.openpgp.PGPCompressedData;
+import org.bouncycastle.openpgp.PGPException;
+import org.bouncycastle.openpgp.PGPLiteralData;
+import org.bouncycastle.openpgp.PGPObjectFactory;
+import org.bouncycastle.openpgp.PGPOnePassSignature;
+import org.bouncycastle.openpgp.PGPOnePassSignatureList;
+import org.bouncycastle.openpgp.PGPPublicKey;
+import org.bouncycastle.openpgp.PGPSignature;
+import org.bouncycastle.openpgp.PGPSignatureList;
+import org.bouncycastle.openpgp.PGPUtil;
+import org.bouncycastle.openpgp.jcajce.JcaPGPObjectFactory;
+import org.bouncycastle.openpgp.operator.jcajce.JcaPGPContentVerifierBuilderProvider;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * Verify Content using Open Pretty Good Privacy Public Keys
+ */
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Tags({"PGP", "GPG", "OpenPGP", "Encryption", "Signing", "RFC 4880"})
+@CapabilityDescription("Verify signatures using OpenPGP Public Keys")
+@SeeAlso({ DecryptContentPGP.class, EncryptContentPGP.class, SignContentPGP.class })
+@WritesAttributes({
+        @WritesAttribute(attribute = PGPAttributeKey.LITERAL_DATA_FILENAME, description = "Filename from Literal Data"),
+        @WritesAttribute(attribute = PGPAttributeKey.LITERAL_DATA_MODIFIED, description = "Modified Date Time from Literal Data in milliseconds"),
+        @WritesAttribute(attribute = PGPAttributeKey.SIGNATURE_CREATED, description = "Signature Creation Time in milliseconds"),
+        @WritesAttribute(attribute = PGPAttributeKey.SIGNATURE_ALGORITHM, description = "Signature Algorithm including key and hash algorithm names"),
+        @WritesAttribute(attribute = PGPAttributeKey.SIGNATURE_HASH_ALGORITHM_ID, description = "Signature Hash Algorithm Identifier"),
+        @WritesAttribute(attribute = PGPAttributeKey.SIGNATURE_KEY_ALGORITHM_ID, description = "Signature Key Algorithm Identifier"),
+        @WritesAttribute(attribute = PGPAttributeKey.SIGNATURE_KEY_ID, description = "Signature Public Key Identifier"),
+        @WritesAttribute(attribute = PGPAttributeKey.SIGNATURE_TYPE_ID, description = "Signature Type Identifier"),
+        @WritesAttribute(attribute = PGPAttributeKey.SIGNATURE_VERSION, description = "Signature Version Number"),
+})
+public class VerifyContentPGP extends AbstractProcessor {
+
+    public static final Relationship SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Signature Verification Succeeded")
+            .build();
+
+    public static final Relationship FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Signature Verification Failed")
+            .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 verifying signatures with Public Key Encryption")
+            .identifiesControllerService(PGPPublicKeyService.class)
+            .required(true)
+            .build();
+
+    private static final Set<Relationship> RELATIONSHIPS = new HashSet<>(Arrays.asList(SUCCESS, FAILURE));
+
+    private static final List<PropertyDescriptor> DESCRIPTORS = Collections.singletonList(
+            PUBLIC_KEY_SERVICE
+    );
+
+    private static final int BUFFER_SIZE = 8192;
+
+    private static final String KEY_ID_UNKNOWN = "UNKNOWN";
+
+    /**
+     * 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 verifies signatures found in 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 PGPPublicKeyService publicKeyService = context.getProperty(PUBLIC_KEY_SERVICE).asControllerService(PGPPublicKeyService.class);
+        final VerifyStreamCallback callback = new VerifyStreamCallback(publicKeyService);
+        try {
+            flowFile = session.write(flowFile, callback);
+            flowFile = session.putAllAttributes(flowFile, callback.attributes);
+            final String keyId = flowFile.getAttribute(PGPAttributeKey.SIGNATURE_KEY_ID);
+            getLogger().info("Signature Key ID [{}] Verification Completed {}", keyId, flowFile);
+            session.transfer(flowFile, SUCCESS);
+        } catch (final RuntimeException e) {
+            flowFile = session.putAllAttributes(flowFile, callback.attributes);
+            getLogger().error("Processing Failed {}", flowFile, e);
+            session.transfer(flowFile, FAILURE);
+        }
+    }
+
+    private class VerifyStreamCallback implements StreamCallback {
+        private final PGPPublicKeyService publicKeyService;
+
+        private final Map<String, String> attributes = new HashMap<>();
+
+        private boolean verified;
+
+        private VerifyStreamCallback(final PGPPublicKeyService publicKeyService) {
+            this.publicKeyService = publicKeyService;
+        }
+
+        /**
+         * Process Input Stream containing binary or ASCII Armored OpenPGP messages and write literal data after verification
+         *
+         * @param inputStream Input Stream to be read
+         * @param outputStream Output Stream for literal data 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 PGPObjectFactory pgpObjectFactory = new JcaPGPObjectFactory(decoderInputStream);
+            final Iterator<?> objects = pgpObjectFactory.iterator();
+            if (objects.hasNext()) {
+                processObjectFactory(objects, outputStream);
+            }
+
+            if (verified) {
+                getLogger().debug("One-Pass Signature Algorithm [{}] Verified", attributes.get(PGPAttributeKey.SIGNATURE_ALGORITHM));
+            } else {
+                final String keyId = attributes.getOrDefault(PGPAttributeKey.SIGNATURE_KEY_ID, KEY_ID_UNKNOWN);
+                throw new PGPProcessException(String.format("Signature Key ID [%s] Verification Failed", keyId));
+            }
+        }
+
+        private void processObjectFactory(final Iterator<?> objects, final OutputStream outputStream) throws IOException {
+            PGPOnePassSignature onePassSignature = null;
+
+            while (objects.hasNext()) {
+                final Object object = objects.next();
+                getLogger().debug("PGP Object Read [{}]", object.getClass().getSimpleName());
+
+                if (object instanceof PGPCompressedData) {
+                    final PGPCompressedData compressedData = (PGPCompressedData) object;
+                    try {
+                        final PGPObjectFactory compressedObjectFactory = new JcaPGPObjectFactory(compressedData.getDataStream());
+                        processObjectFactory(compressedObjectFactory.iterator(), outputStream);
+                    } catch (final PGPException e) {
+                        throw new PGPProcessException("Read Compressed Data Failed", e);
+                    }
+                } else if (object instanceof PGPOnePassSignatureList) {
+                    final PGPOnePassSignatureList onePassSignatureList = (PGPOnePassSignatureList) object;
+                    onePassSignature = processOnePassSignatures(onePassSignatureList);
+                } else if (object instanceof PGPLiteralData) {
+                    final PGPLiteralData literalData = (PGPLiteralData) object;
+                    processLiteralData(literalData, outputStream, onePassSignature);
+                } else if (object instanceof PGPSignatureList) {
+                    final PGPSignatureList signatureList = (PGPSignatureList) object;
+                    processSignatures(signatureList, onePassSignature);
+                }
+            }
+        }
+
+        private PGPOnePassSignature processOnePassSignatures(final PGPOnePassSignatureList onePassSignatureList) {
+            getLogger().debug("One-Pass Signatures Found [{}]", onePassSignatureList.size());
+
+            PGPOnePassSignature initializedOnePassSignature = null;
+            final Iterator<PGPOnePassSignature> onePassSignatures = onePassSignatureList.iterator();
+            if (onePassSignatures.hasNext()) {
+                final PGPOnePassSignature onePassSignature = onePassSignatures.next();
+                setOnePassSignatureAttributes(onePassSignature);
+
+                final String keyId = KeyIdentifierConverter.format(onePassSignature.getKeyID());
+                final Optional<PGPPublicKey> optionalPublicKey = publicKeyService.findPublicKey(keyId);
+                if (optionalPublicKey.isPresent()) {
+                    getLogger().debug("One-Pass Signature Key ID [{}] found", keyId);
+                    final PGPPublicKey publicKey = optionalPublicKey.get();
+                    try {
+                        onePassSignature.init(new JcaPGPContentVerifierBuilderProvider(), publicKey);
+                        initializedOnePassSignature = onePassSignature;
+                    } catch (final PGPException e) {
+                        throw new PGPProcessException(String.format("One-Pass Signature Key ID [%s] Initialization Failed", keyId), e);
+                    }
+                } else {
+                    getLogger().warn("One-Pass Signature Key ID [{}] not found in Public Key Service", keyId);
+                }
+            }
+            return initializedOnePassSignature;
+        }
+
+        private void processLiteralData(final PGPLiteralData literalData,
+                                        final OutputStream outputStream,
+                                        final PGPOnePassSignature onePassSignature) throws IOException {
+            setLiteralDataAttributes(literalData);
+            final InputStream literalInputStream = literalData.getInputStream();
+            if (onePassSignature == null) {
+                StreamUtils.copy(literalInputStream, outputStream);
+            } else {
+                processSignedStream(literalInputStream, outputStream, onePassSignature);
+            }
+        }
+
+        private void processSignatures(final PGPSignatureList signatureList, final PGPOnePassSignature onePassSignature) {
+            getLogger().debug("Signatures Found [{}]", signatureList.size());
+            final Iterator<PGPSignature> signatures = signatureList.iterator();
+            if (signatures.hasNext()) {
+                final PGPSignature signature = signatures.next();
+                setSignatureAttributes(signature);
+
+                if (onePassSignature == null) {
+                    getLogger().debug("One-Pass Signature not found: Verification Failed");
+                } else {
+                    try {
+                        verified = onePassSignature.verify(signature);
+                    } catch (final PGPException e) {
+                        final String keyId = KeyIdentifierConverter.format(onePassSignature.getKeyID());
+                        throw new PGPProcessException(String.format("One-Pass Signature Key ID [%s] Verification Failed", keyId), e);
+                    }
+                }
+            }
+        }
+
+        private void processSignedStream(final InputStream inputStream, final OutputStream outputStream, final PGPOnePassSignature onePassSignature) throws IOException {
+            final String keyId = KeyIdentifierConverter.format(onePassSignature.getKeyID());
+            getLogger().debug("Processing Data for One-Pass Signature with Key ID [{}]", keyId);
+            final byte[] buffer = new byte[BUFFER_SIZE];
+            int read;
+            while ((read = inputStream.read(buffer)) >= 0) {
+                onePassSignature.update(buffer, 0, read);
+                outputStream.write(buffer, 0, read);
+            }
+        }
+
+        private void setOnePassSignatureAttributes(final PGPOnePassSignature onePassSignature) {
+            setSignatureAlgorithm(onePassSignature.getKeyAlgorithm(), onePassSignature.getHashAlgorithm());
+            attributes.put(PGPAttributeKey.SIGNATURE_KEY_ID, KeyIdentifierConverter.format(onePassSignature.getKeyID()));
+            attributes.put(PGPAttributeKey.SIGNATURE_TYPE_ID, Integer.toString(onePassSignature.getSignatureType()));
+        }
+
+        private void setSignatureAttributes(final PGPSignature signature) {
+            setSignatureAlgorithm(signature.getKeyAlgorithm(), signature.getHashAlgorithm());
+            attributes.put(PGPAttributeKey.SIGNATURE_CREATED, Long.toString(signature.getCreationTime().getTime()));
+            attributes.put(PGPAttributeKey.SIGNATURE_KEY_ID, KeyIdentifierConverter.format(signature.getKeyID()));
+            attributes.put(PGPAttributeKey.SIGNATURE_TYPE_ID, Integer.toString(signature.getSignatureType()));
+            attributes.put(PGPAttributeKey.SIGNATURE_VERSION, Integer.toString(signature.getVersion()));
+        }
+
+        private void setLiteralDataAttributes(final PGPLiteralData literalData) {
+            attributes.put(PGPAttributeKey.LITERAL_DATA_FILENAME, literalData.getFileName());
+            attributes.put(PGPAttributeKey.LITERAL_DATA_MODIFIED, Long.toString(literalData.getModificationTime().getTime()));
+        }
+
+        private void setSignatureAlgorithm(final int keyAlgorithm, final int hashAlgorithm) {
+            attributes.put(PGPAttributeKey.SIGNATURE_HASH_ALGORITHM_ID, Integer.toString(hashAlgorithm));
+            attributes.put(PGPAttributeKey.SIGNATURE_KEY_ALGORITHM_ID, Integer.toString(keyAlgorithm));
+            try {
+                final String algorithm = PGPUtil.getSignatureName(keyAlgorithm, hashAlgorithm);
+                attributes.put(PGPAttributeKey.SIGNATURE_ALGORITHM, algorithm);
+            } catch (final PGPException e) {
+                getLogger().debug("Signature Algorithm Key Identifier [{}] Hash Identifier [{}] not found", keyAlgorithm, hashAlgorithm);
+            }
+        }
+    }
+}
diff --git a/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/attributes/DecryptionStrategy.java b/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/attributes/DecryptionStrategy.java
new file mode 100644
index 0000000..54bf2d5
--- /dev/null
+++ b/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/attributes/DecryptionStrategy.java
@@ -0,0 +1,36 @@
+/*
+ * 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.attributes;
+
+/**
+ * Pretty Good Privacy Decryption Strategy
+ */
+public enum DecryptionStrategy {
+    DECRYPTED("Produce decrypted content read from literal data ignoring signatures"),
+
+    PACKAGED("Produce decrypted content packaged as an OpenPGP message for additional processing");
+
+    private final String description;
+
+    DecryptionStrategy(final String description) {
+        this.description = description;
+    }
+
+    public String getDescription() {
+        return description;
+    }
+}
diff --git a/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/attributes/HashAlgorithm.java b/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/attributes/HashAlgorithm.java
new file mode 100644
index 0000000..9549147
--- /dev/null
+++ b/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/attributes/HashAlgorithm.java
@@ -0,0 +1,40 @@
+/*
+ * 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.attributes;
+
+import org.bouncycastle.bcpg.HashAlgorithmTags;
+
+/**
+ * PGP Hash Algorithm Definitions supported for Signing
+ */
+public enum HashAlgorithm {
+    SHA256(HashAlgorithmTags.SHA256),
+
+    SHA384(HashAlgorithmTags.SHA384),
+
+    SHA512(HashAlgorithmTags.SHA512);
+
+    private int id;
+
+    HashAlgorithm(final int id) {
+        this.id = id;
+    }
+
+    public int getId() {
+        return id;
+    }
+}
diff --git a/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/attributes/SigningStrategy.java b/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/attributes/SigningStrategy.java
new file mode 100644
index 0000000..06ba4da
--- /dev/null
+++ b/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/attributes/SigningStrategy.java
@@ -0,0 +1,36 @@
+/*
+ * 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.attributes;
+
+/**
+ * Pretty Good Privacy File Signing Strategy
+ */
+public enum SigningStrategy {
+    SIGNED("Produce signed content packaged as an OpenPGP message"),
+
+    DETACHED("Produce detached signature based on associated content packaged according to OpenPGP encoding");
+
+    private final String description;
+
+    SigningStrategy(final String description) {
+        this.description = description;
+    }
+
+    public String getDescription() {
+        return description;
+    }
+}
diff --git a/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/io/EncodingStreamCallback.java b/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/io/EncodingStreamCallback.java
new file mode 100644
index 0000000..1377e4e
--- /dev/null
+++ b/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/io/EncodingStreamCallback.java
@@ -0,0 +1,126 @@
+/*
+ * 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.io;
+
+import org.apache.nifi.processor.io.StreamCallback;
+import org.apache.nifi.processors.pgp.attributes.CompressionAlgorithm;
+import org.apache.nifi.processors.pgp.attributes.FileEncoding;
+import org.apache.nifi.processors.pgp.exception.PGPProcessException;
+import org.apache.nifi.stream.io.StreamUtils;
+import org.bouncycastle.bcpg.ArmoredOutputStream;
+import org.bouncycastle.openpgp.PGPCompressedDataGenerator;
+import org.bouncycastle.openpgp.PGPException;
+import org.bouncycastle.openpgp.PGPLiteralData;
+import org.bouncycastle.openpgp.PGPLiteralDataGenerator;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Date;
+import java.util.Objects;
+
+/**
+ * Encoding Stream Callback handles writing PGP messages using configured properties
+ */
+public class EncodingStreamCallback implements StreamCallback {
+    private static final int OUTPUT_BUFFER_SIZE = 8192;
+
+    private final FileEncoding fileEncoding;
+
+    private final CompressionAlgorithm compressionAlgorithm;
+
+    private final String filename;
+
+    public EncodingStreamCallback(final FileEncoding fileEncoding, final CompressionAlgorithm compressionAlgorithm, final String filename) {
+        this.fileEncoding = Objects.requireNonNull(fileEncoding, "File Encoding required");
+        this.compressionAlgorithm = Objects.requireNonNull(compressionAlgorithm, "Compression Algorithm required");
+        this.filename = Objects.requireNonNull(filename, "Filename required");
+    }
+
+    /**
+     * Process Input Stream and write encoded contents to Output Stream
+     *
+     * @param inputStream  Input Stream
+     * @param outputStream Output Stream for encrypted contents
+     * @throws IOException Thrown when unable to read or write streams
+     */
+    @Override
+    public void process(final InputStream inputStream, final OutputStream outputStream) throws IOException {
+        try (final OutputStream encodingOutputStream = getEncodingOutputStream(outputStream)) {
+            processEncoding(inputStream, encodingOutputStream);
+        } catch (final PGPException e) {
+            throw new PGPProcessException("PGP Stream Processing Failed", e);
+        }
+    }
+
+    /**
+     * Create Output Buffer byte array with size of 8192
+     *
+     * @return New empty array of 8192 bytes
+     */
+    protected byte[] createOutputBuffer() {
+        return new byte[OUTPUT_BUFFER_SIZE];
+    }
+
+    /**
+     * Process Encoding passing Input Stream through Compression Output Stream
+     *
+     * @param inputStream          Input Stream
+     * @param encodingOutputStream Output Stream configured according to File Encoding
+     * @throws IOException  Thrown when unable to read or write streams
+     * @throws PGPException Thrown when unable to process compression
+     */
+    protected void processEncoding(final InputStream inputStream, final OutputStream encodingOutputStream) throws IOException, PGPException {
+        final PGPCompressedDataGenerator compressedDataGenerator = new PGPCompressedDataGenerator(compressionAlgorithm.getId());
+        try (final OutputStream compressedOutputStream = compressedDataGenerator.open(encodingOutputStream, createOutputBuffer())) {
+            processCompression(inputStream, compressedOutputStream);
+        }
+        compressedDataGenerator.close();
+    }
+
+    /**
+     * Process Compression passing Input Stream through Literal Data Output Stream
+     *
+     * @param inputStream            Input Stream
+     * @param compressedOutputStream Output Stream configured according to Compression Algorithm
+     * @throws IOException Thrown when unable to read or write streams
+     * @throws PGPException Thrown when unable to process streams using PGP operations
+     */
+    protected void processCompression(final InputStream inputStream, final OutputStream compressedOutputStream) throws IOException, PGPException {
+        final PGPLiteralDataGenerator generator = new PGPLiteralDataGenerator();
+        try (final OutputStream literalOutputStream = openLiteralOutputStream(generator, compressedOutputStream)) {
+            StreamUtils.copy(inputStream, literalOutputStream);
+        }
+        generator.close();
+    }
+
+    /**
+     * Open Literal Data Output Stream using binary indicator with configured filename and current date indicating modification
+     *
+     * @param generator              PGP Literal Data Generator
+     * @param compressedOutputStream Output Stream configured according to Compression Algorithm
+     * @return Literal Data Output Stream
+     * @throws IOException Thrown when unable to open Literal Data Output Stream
+     */
+    protected OutputStream openLiteralOutputStream(final PGPLiteralDataGenerator generator, final OutputStream compressedOutputStream) throws IOException {
+        return generator.open(compressedOutputStream, PGPLiteralData.BINARY, filename, new Date(), createOutputBuffer());
+    }
+
+    private OutputStream getEncodingOutputStream(final OutputStream outputStream) {
+        return FileEncoding.ASCII.equals(fileEncoding) ? new ArmoredOutputStream(outputStream) : outputStream;
+    }
+}
diff --git a/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/io/KeyIdentifierConverter.java b/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/io/KeyIdentifierConverter.java
new file mode 100644
index 0000000..25e5d91
--- /dev/null
+++ b/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/io/KeyIdentifierConverter.java
@@ -0,0 +1,47 @@
+/*
+ * 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.io;
+
+import java.math.BigInteger;
+
+/**
+ * Key Identifier Converter from number to hexadecimal string
+ */
+public class KeyIdentifierConverter {
+    private static final int HEXADECIMAL_RADIX = 16;
+
+    /**
+     * Format numeric key identifier as uppercase hexadecimal string
+     *
+     * @param keyId Key Identifier
+     * @return Uppercase hexadecimal string
+     */
+    public static String format(final long keyId) {
+        return Long.toHexString(keyId).toUpperCase();
+    }
+
+    /**
+     * Parse hexadecimal key identifier to numeric key identifier
+     *
+     * @param keyId Hexadecimal string
+     * @return Key Identifier
+     */
+    public static long parse(final String keyId) {
+        final BigInteger parsed = new BigInteger(keyId, HEXADECIMAL_RADIX);
+        return parsed.longValue();
+    }
+}
diff --git a/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
index 8ce19cd..e8c6858 100644
--- a/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
+++ b/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
@@ -13,4 +13,6 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 org.apache.nifi.processors.pgp.EncryptContentPGP
-org.apache.nifi.processors.pgp.DecryptContentPGP
\ No newline at end of file
+org.apache.nifi.processors.pgp.DecryptContentPGP
+org.apache.nifi.processors.pgp.SignContentPGP
+org.apache.nifi.processors.pgp.VerifyContentPGP
diff --git a/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/test/java/org/apache/nifi/processors/pgp/DecryptContentPGPTest.java b/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/test/java/org/apache/nifi/processors/pgp/DecryptContentPGPTest.java
index dcff039..c58080c 100644
--- a/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/test/java/org/apache/nifi/processors/pgp/DecryptContentPGPTest.java
+++ b/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/test/java/org/apache/nifi/processors/pgp/DecryptContentPGPTest.java
@@ -18,9 +18,12 @@ 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.pgp.util.PGPOperationUtils;
+import org.apache.nifi.processors.pgp.attributes.DecryptionStrategy;
 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.stream.io.StreamUtils;
 import org.apache.nifi.util.LogMessage;
 import org.apache.nifi.util.MockFlowFile;
 import org.apache.nifi.util.TestRunner;
@@ -30,20 +33,23 @@ import org.bouncycastle.openpgp.PGPCompressedData;
 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.PGPObjectFactory;
+import org.bouncycastle.openpgp.PGPOnePassSignature;
+import org.bouncycastle.openpgp.PGPOnePassSignatureList;
 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.PGPSignatureList;
 import org.bouncycastle.openpgp.PGPUtil;
+import org.bouncycastle.openpgp.jcajce.JcaPGPObjectFactory;
 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.jupiter.api.BeforeAll;
@@ -55,7 +61,9 @@ import org.mockito.junit.jupiter.MockitoExtension;
 
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
+import java.io.InputStream;
 import java.io.OutputStream;
+import java.io.UncheckedIOException;
 import java.nio.charset.Charset;
 import java.nio.charset.StandardCharsets;
 import java.util.Arrays;
@@ -66,6 +74,7 @@ import java.util.UUID;
 
 import static org.hamcrest.CoreMatchers.hasItem;
 import static org.hamcrest.CoreMatchers.isA;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.mockito.ArgumentMatchers.eq;
@@ -95,8 +104,6 @@ public class DecryptContentPGPTest {
 
     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;
@@ -265,6 +272,23 @@ public class DecryptContentPGPTest {
     }
 
     @Test
+    public void testSuccessPublicKeyEncryptionRsaPrivateKeyPackaged() throws InitializationException, IOException, PGPException {
+        setPrivateKeyService();
+        final PGPPublicKey publicKey = rsaSecretKey.getPublicKey();
+        when(privateKeyService.findPrivateKey(eq(publicKey.getKeyID()))).thenReturn(Optional.of(rsaPrivateKey));
+
+        runner.setProperty(DecryptContentPGP.DECRYPTION_STRATEGY, DecryptionStrategy.PACKAGED.toString());
+
+        final byte[] contents = DATA.getBytes(DATA_CHARSET);
+        final byte[] signedData = PGPOperationUtils.getOnePassSignedLiteralData(contents, rsaPrivateKey);
+        final byte[] encryptedData = getPublicKeyEncryptedData(signedData, publicKey);
+        runner.enqueue(encryptedData);
+        runner.run();
+
+        assertSuccess(ENCRYPTION_ALGORITHM, DecryptionStrategy.PACKAGED);
+    }
+
+    @Test
     public void testSuccessPublicKeyEncryptionElGamalPrivateKey() throws InitializationException, IOException, PGPException {
         setPrivateKeyService();
         when(privateKeyService.findPrivateKey(eq(elGamalPrivateKey.getKeyID()))).thenReturn(Optional.of(elGamalPrivateKey));
@@ -281,8 +305,9 @@ public class DecryptContentPGPTest {
         final PGPPublicKey publicKey = rsaSecretKey.getPublicKey();
         when(privateKeyService.findPrivateKey(eq(publicKey.getKeyID()))).thenReturn(Optional.of(rsaPrivateKey));
 
-        final byte[] encryptedData = getPublicKeyEncryptedData(getLiteralData(), publicKey);
-        final byte[] signedData = getSignedData(encryptedData, publicKey, rsaPrivateKey);
+        final byte[] literalData = getLiteralData();
+        final byte[] encrypted = getPublicKeyEncryptedData(literalData, publicKey);
+        final byte[] signedData = PGPOperationUtils.getOnePassSignedData(encrypted, rsaPrivateKey);
         runner.enqueue(signedData);
         runner.run();
 
@@ -325,18 +350,71 @@ public class DecryptContentPGPTest {
     }
 
     private void assertSuccess() {
-        assertSuccess(ENCRYPTION_ALGORITHM);
+        assertSuccess(ENCRYPTION_ALGORITHM, DecryptionStrategy.DECRYPTED);
     }
 
     private void assertSuccess(final int encryptionAlgorithm) {
+        assertSuccess(encryptionAlgorithm, DecryptionStrategy.DECRYPTED);
+    }
+
+    private void assertSuccess(final int encryptionAlgorithm, final DecryptionStrategy decryptionStrategy) {
         runner.assertAllFlowFilesTransferred(DecryptContentPGP.SUCCESS);
         final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(DecryptContentPGP.SUCCESS);
         final MockFlowFile flowFile = flowFiles.iterator().next();
-        flowFile.assertContentEquals(DATA, DATA_CHARSET);
 
-        flowFile.assertAttributeEquals(PGPAttributeKey.LITERAL_DATA_FILENAME, FILE_NAME);
-        flowFile.assertAttributeEquals(PGPAttributeKey.LITERAL_DATA_MODIFIED, Long.toString(MODIFIED_MILLISECONDS));
+        if (DecryptionStrategy.PACKAGED == decryptionStrategy) {
+            assertSuccessPackaged(flowFile.getContentStream());
+        } else {
+            flowFile.assertContentEquals(DATA, DATA_CHARSET);
+            flowFile.assertAttributeEquals(PGPAttributeKey.LITERAL_DATA_FILENAME, FILE_NAME);
+            flowFile.assertAttributeEquals(PGPAttributeKey.LITERAL_DATA_MODIFIED, Long.toString(MODIFIED_MILLISECONDS));
+        }
+
         flowFile.assertAttributeEquals(PGPAttributeKey.SYMMETRIC_KEY_ALGORITHM_ID, Integer.toString(encryptionAlgorithm));
+        flowFile.assertAttributeEquals(PGPAttributeKey.SYMMETRIC_KEY_ALGORITHM_BLOCK_CIPHER, PGPUtil.getSymmetricCipherName(encryptionAlgorithm));
+    }
+
+    private void assertSuccessPackaged(final InputStream inputStream) {
+        final PGPObjectFactory objectFactory = new JcaPGPObjectFactory(inputStream);
+        try {
+            final Object firstObject = objectFactory.nextObject();
+            assertOnePassSignatureEquals(firstObject);
+
+            final Object secondObject = objectFactory.nextObject();
+            assertLiteralDataEquals(secondObject);
+
+            final Object thirdObject = objectFactory.nextObject();
+            assertSignatureEquals(thirdObject);
+        } catch (final IOException e) {
+            throw new UncheckedIOException(e);
+        }
+    }
+
+    private void assertOnePassSignatureEquals(final Object object) {
+        assertTrue(object instanceof PGPOnePassSignatureList);
+        final PGPOnePassSignatureList onePassSignatureList = (PGPOnePassSignatureList) object;
+        final PGPOnePassSignature onePassSignature = onePassSignatureList.iterator().next();
+        assertEquals(onePassSignature.getKeyID(), rsaPrivateKey.getKeyID());
+    }
+
+    private void assertLiteralDataEquals(final Object object) throws IOException {
+        assertTrue(object instanceof PGPLiteralData);
+        final PGPLiteralData literalData = (PGPLiteralData) object;
+        assertEquals(FILE_NAME, literalData.getFileName());
+        assertEquals(MODIFIED, literalData.getModificationTime());
+
+        final ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+        StreamUtils.copy(literalData.getDataStream(), outputStream);
+        final byte[] literalBinary = outputStream.toByteArray();
+        final String literal = new String(literalBinary, DATA_CHARSET);
+        assertEquals(DATA, literal);
+    }
+
+    private void assertSignatureEquals(final Object object) {
+        assertTrue(object instanceof PGPSignatureList);
+        final PGPSignatureList signatureList = (PGPSignatureList) object;
+        final PGPSignature signature = signatureList.iterator().next();
+        assertEquals(rsaPrivateKey.getKeyID(), signature.getKeyID());
     }
 
     private void assertFailureExceptionLogged(final Class<? extends Exception> exceptionClass) {
@@ -347,19 +425,6 @@ public class DecryptContentPGPTest {
         assertThat(Arrays.asList(logMessage.getArgs()), hasItem(isA(exceptionClass)));
     }
 
-    private byte[] getSignedData(final byte[] contents, final PGPPublicKey publicKey, final PGPPrivateKey privateKey) throws PGPException, IOException {
-        final PGPContentSignerBuilder contentSignerBuilder = new JcaPGPContentSignerBuilder(publicKey.getAlgorithm(), PGPUtil.SHA1);
-        final PGPSignatureGenerator signatureGenerator = new PGPSignatureGenerator(contentSignerBuilder);
-        signatureGenerator.init(PGPSignature.BINARY_DOCUMENT, privateKey);
-
-        final ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
-        signatureGenerator.generateOnePassVersion(NESTED_SIGNATURE_DISABLED).encode(outputStream);
-        outputStream.write(contents);
-        signatureGenerator.update(contents);
-        signatureGenerator.generate().encode(outputStream);
-        return outputStream.toByteArray();
-    }
-
     private byte[] getPublicKeyEncryptedData(final byte[] contents, final PGPPublicKey publicKey) throws IOException, PGPException {
         final PGPDataEncryptorBuilder builder = new BcPGPDataEncryptorBuilder(ENCRYPTION_ALGORITHM).setWithIntegrityPacket(INTEGRITY_ENABLED);
         final PGPEncryptedDataGenerator generator = new PGPEncryptedDataGenerator(builder);
diff --git a/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/test/java/org/apache/nifi/processors/pgp/EncryptContentPGPTest.java b/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/test/java/org/apache/nifi/processors/pgp/EncryptContentPGPTest.java
index 5fb2be7..78a31d9 100644
--- a/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/test/java/org/apache/nifi/processors/pgp/EncryptContentPGPTest.java
+++ b/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/test/java/org/apache/nifi/processors/pgp/EncryptContentPGPTest.java
@@ -17,9 +17,12 @@
 package org.apache.nifi.processors.pgp;
 
 import org.apache.nifi.pgp.service.api.PGPPublicKeyService;
+import org.apache.nifi.pgp.util.PGPOperationUtils;
 import org.apache.nifi.processors.pgp.attributes.CompressionAlgorithm;
+import org.apache.nifi.processors.pgp.attributes.DecryptionStrategy;
 import org.apache.nifi.processors.pgp.attributes.FileEncoding;
 import org.apache.nifi.processors.pgp.attributes.SymmetricKeyAlgorithm;
+import org.apache.nifi.processors.pgp.io.KeyIdentifierConverter;
 import org.apache.nifi.reporting.InitializationException;
 import org.apache.nifi.stream.io.StreamUtils;
 import org.apache.nifi.util.MockFlowFile;
@@ -59,10 +62,12 @@ import org.mockito.junit.jupiter.MockitoExtension;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
+import java.nio.charset.StandardCharsets;
 import java.util.Optional;
 import java.util.UUID;
 import java.util.stream.StreamSupport;
 
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.mockito.ArgumentMatchers.eq;
@@ -74,6 +79,8 @@ public class EncryptContentPGPTest {
 
     private static final String DATA = String.class.getName();
 
+    private static final byte[] DATA_BINARY = DATA.getBytes(StandardCharsets.UTF_8);
+
     private static final SymmetricKeyAlgorithm DEFAULT_SYMMETRIC_KEY_ALGORITHM = SymmetricKeyAlgorithm.valueOf(EncryptContentPGP.SYMMETRIC_KEY_ALGORITHM.getDefaultValue());
 
     private static final String SERVICE_ID = PGPPublicKeyService.class.getName();
@@ -180,19 +187,34 @@ public class EncryptContentPGPTest {
     public void testSuccessPublicKeyEncryptionRsaPublicKey() throws IOException, InitializationException, PGPException {
         final PGPPublicKey publicKey = rsaSecretKey.getPublicKey();
         setPublicKeyService(publicKey);
-        final String publicKeyIdSearch = Long.toHexString(publicKey.getKeyID()).toUpperCase();
+        final String publicKeyIdSearch = KeyIdentifierConverter.format(publicKey.getKeyID());
         when(publicKeyService.findPublicKey(eq(publicKeyIdSearch))).thenReturn(Optional.of(publicKey));
 
         runner.enqueue(DATA);
         runner.run();
-        assertSuccess(rsaPrivateKey);
+        assertSuccess(rsaPrivateKey, DecryptionStrategy.DECRYPTED, DATA_BINARY);
+    }
+
+    @Test
+    public void testSuccessPublicKeyEncryptionRsaPublicKeySignedDataPackaged() throws IOException, InitializationException, PGPException {
+        final PGPPublicKey publicKey = rsaSecretKey.getPublicKey();
+        setPublicKeyService(publicKey);
+        final String publicKeyIdSearch = KeyIdentifierConverter.format(publicKey.getKeyID());
+        when(publicKeyService.findPublicKey(eq(publicKeyIdSearch))).thenReturn(Optional.of(publicKey));
+
+        final byte[] contents = DATA.getBytes(StandardCharsets.UTF_8);
+        final byte[] signedData = PGPOperationUtils.getOnePassSignedLiteralData(contents, rsaPrivateKey);
+
+        runner.enqueue(signedData);
+        runner.run();
+        assertSuccess(rsaPrivateKey, DecryptionStrategy.PACKAGED, signedData);
     }
 
     @Test
     public void testSuccessPasswordBasedAndPublicKeyEncryptionRsaPublicKey() throws IOException, InitializationException, PGPException {
         final PGPPublicKey publicKey = rsaSecretKey.getPublicKey();
         setPublicKeyService(publicKey);
-        final String publicKeyIdSearch = Long.toHexString(publicKey.getKeyID()).toUpperCase();
+        final String publicKeyIdSearch = KeyIdentifierConverter.format(publicKey.getKeyID());
         when(publicKeyService.findPublicKey(eq(publicKeyIdSearch))).thenReturn(Optional.of(publicKey));
 
         runner.setProperty(EncryptContentPGP.PASSPHRASE, PASSPHRASE);
@@ -200,19 +222,19 @@ public class EncryptContentPGPTest {
         runner.enqueue(DATA);
         runner.run();
 
-        assertSuccess(rsaPrivateKey);
+        assertSuccess(rsaPrivateKey, DecryptionStrategy.DECRYPTED, DATA_BINARY);
         assertSuccess(DEFAULT_SYMMETRIC_KEY_ALGORITHM, PASSPHRASE.toCharArray());
     }
 
     @Test
     public void testSuccessPublicKeyEncryptionElGamalPublicKey() throws IOException, InitializationException, PGPException {
         setPublicKeyService(elGamalPublicKey);
-        final String publicKeyIdSearch = Long.toHexString(elGamalPublicKey.getKeyID()).toUpperCase();
+        final String publicKeyIdSearch = KeyIdentifierConverter.format(elGamalPublicKey.getKeyID());
         when(publicKeyService.findPublicKey(eq(publicKeyIdSearch))).thenReturn(Optional.of(elGamalPublicKey));
 
         runner.enqueue(DATA);
         runner.run();
-        assertSuccess(elGamalPrivateKey);
+        assertSuccess(elGamalPrivateKey, DecryptionStrategy.DECRYPTED, DATA_BINARY);
     }
 
     @Test
@@ -220,7 +242,7 @@ public class EncryptContentPGPTest {
         final PGPPublicKey publicKey = rsaSecretKey.getPublicKey();
         setPublicKeyService(publicKey);
 
-        final String publicKeyIdNotFound = Long.toHexString(Long.MAX_VALUE).toUpperCase();
+        final String publicKeyIdNotFound = KeyIdentifierConverter.format(Long.MAX_VALUE);
         runner.setProperty(EncryptContentPGP.PUBLIC_KEY_SEARCH, publicKeyIdNotFound);
 
         runner.enqueue(DATA);
@@ -234,12 +256,11 @@ public class EncryptContentPGPTest {
         runner.enableControllerService(publicKeyService);
 
         runner.setProperty(EncryptContentPGP.PUBLIC_KEY_SERVICE, SERVICE_ID);
-        final long publicKeyId = publicKey.getKeyID();
-        final String publicKeyIdLong = Long.toHexString(publicKeyId).toUpperCase();
-        runner.setProperty(EncryptContentPGP.PUBLIC_KEY_SEARCH, publicKeyIdLong);
+        final String publicKeyId = KeyIdentifierConverter.format(publicKey.getKeyID());
+        runner.setProperty(EncryptContentPGP.PUBLIC_KEY_SEARCH, publicKeyId);
     }
 
-    private void assertSuccess(final PGPPrivateKey privateKey) throws IOException, PGPException {
+    private void assertSuccess(final PGPPrivateKey privateKey, final DecryptionStrategy decryptionStrategy, final byte[] expected) throws IOException, PGPException {
         runner.assertAllFlowFilesTransferred(EncryptContentPGP.SUCCESS);
         final MockFlowFile flowFile = runner.getFlowFilesForRelationship(EncryptContentPGP.SUCCESS).iterator().next();
         assertAttributesFound(DEFAULT_SYMMETRIC_KEY_ALGORITHM, flowFile);
@@ -251,8 +272,8 @@ public class EncryptContentPGPTest {
         assertTrue(encryptedData.isPresent(), "Public Key Encrypted Data not found");
 
         final PGPPublicKeyEncryptedData publicKeyEncryptedData = (PGPPublicKeyEncryptedData) encryptedData.get();
-        final String decryptedData = getDecryptedData(publicKeyEncryptedData, privateKey);
-        assertEquals(DATA, decryptedData);
+        final byte[] decryptedData = getDecryptedData(publicKeyEncryptedData, privateKey, decryptionStrategy);
+        assertArrayEquals(expected, decryptedData);
     }
 
     private void assertSuccess(final SymmetricKeyAlgorithm symmetricKeyAlgorithm, final char[] passphrase) throws IOException, PGPException {
@@ -267,8 +288,8 @@ public class EncryptContentPGPTest {
         assertTrue(encryptedData.isPresent(), "Password Based Encrypted Data not found");
 
         final PGPPBEEncryptedData passwordBasedEncryptedData = (PGPPBEEncryptedData) encryptedData.get();
-        final String decryptedData = getDecryptedData(passwordBasedEncryptedData, passphrase);
-        assertEquals(DATA, decryptedData);
+        final byte[] decryptedData = getDecryptedData(passwordBasedEncryptedData, passphrase);
+        assertArrayEquals(DATA_BINARY, decryptedData);
     }
 
     private void assertAttributesFound(final SymmetricKeyAlgorithm symmetricKeyAlgorithm, final MockFlowFile flowFile) {
@@ -295,24 +316,30 @@ public class EncryptContentPGPTest {
         return (PGPEncryptedDataList) firstObject;
     }
 
-    private String getDecryptedData(final PGPPBEEncryptedData passwordBasedEncryptedData, final char[] passphrase) throws PGPException, IOException {
+    private byte[] getDecryptedData(final PGPPBEEncryptedData passwordBasedEncryptedData, final char[] passphrase) throws PGPException, IOException {
         final PBEDataDecryptorFactory decryptorFactory = new BcPBEDataDecryptorFactory(passphrase, new BcPGPDigestCalculatorProvider());
         final InputStream decryptedDataStream = passwordBasedEncryptedData.getDataStream(decryptorFactory);
-        return getDecryptedData(decryptedDataStream);
+        return getDecryptedData(decryptedDataStream, DecryptionStrategy.DECRYPTED);
     }
 
-    private String getDecryptedData(final PGPPublicKeyEncryptedData publicKeyEncryptedData, final PGPPrivateKey privateKey) throws PGPException, IOException {
+    private byte[] getDecryptedData(final PGPPublicKeyEncryptedData publicKeyEncryptedData,
+                                    final PGPPrivateKey privateKey,
+                                    final DecryptionStrategy decryptionStrategy) throws PGPException, IOException {
         final PublicKeyDataDecryptorFactory decryptorFactory = new BcPublicKeyDataDecryptorFactory(privateKey);
         final InputStream decryptedDataStream = publicKeyEncryptedData.getDataStream(decryptorFactory);
-        return getDecryptedData(decryptedDataStream);
+        return getDecryptedData(decryptedDataStream, decryptionStrategy);
     }
 
-    private String getDecryptedData(final InputStream decryptedDataStream) throws PGPException, IOException {
+    private byte[] getDecryptedData(final InputStream decryptedDataStream, final DecryptionStrategy decryptionStrategy) throws PGPException, IOException {
         final ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
-        final PGPObjectFactory objectFactory = new JcaPGPObjectFactory(decryptedDataStream);
-        final PGPLiteralData literalData = getLiteralData(objectFactory);
-        StreamUtils.copy(literalData.getDataStream(), outputStream);
-        return outputStream.toString();
+        if (DecryptionStrategy.PACKAGED == decryptionStrategy) {
+            StreamUtils.copy(decryptedDataStream, outputStream);
+        } else {
+            final PGPObjectFactory objectFactory = new JcaPGPObjectFactory(decryptedDataStream);
+            final PGPLiteralData literalData = getLiteralData(objectFactory);
+            StreamUtils.copy(literalData.getDataStream(), outputStream);
+        }
+        return outputStream.toByteArray();
     }
 
     private PGPLiteralData getLiteralData(final PGPObjectFactory objectFactory) throws PGPException {
diff --git a/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/test/java/org/apache/nifi/processors/pgp/SignContentPGPTest.java b/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/test/java/org/apache/nifi/processors/pgp/SignContentPGPTest.java
new file mode 100644
index 0000000..0b92b3d
--- /dev/null
+++ b/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/test/java/org/apache/nifi/processors/pgp/SignContentPGPTest.java
@@ -0,0 +1,277 @@
+/*
+ * 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.attributes.CompressionAlgorithm;
+import org.apache.nifi.processors.pgp.attributes.FileEncoding;
+import org.apache.nifi.processors.pgp.attributes.HashAlgorithm;
+import org.apache.nifi.processors.pgp.attributes.SigningStrategy;
+import org.apache.nifi.processors.pgp.io.KeyIdentifierConverter;
+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.openpgp.PGPCompressedData;
+import org.bouncycastle.openpgp.PGPException;
+import org.bouncycastle.openpgp.PGPLiteralData;
+import org.bouncycastle.openpgp.PGPObjectFactory;
+import org.bouncycastle.openpgp.PGPOnePassSignature;
+import org.bouncycastle.openpgp.PGPOnePassSignatureList;
+import org.bouncycastle.openpgp.PGPPrivateKey;
+import org.bouncycastle.openpgp.PGPPublicKey;
+import org.bouncycastle.openpgp.PGPSecretKey;
+import org.bouncycastle.openpgp.PGPSignature;
+import org.bouncycastle.openpgp.PGPSignatureList;
+import org.bouncycastle.openpgp.PGPUtil;
+import org.bouncycastle.openpgp.jcajce.JcaPGPObjectFactory;
+import org.bouncycastle.openpgp.operator.PBESecretKeyDecryptor;
+import org.bouncycastle.openpgp.operator.jcajce.JcaPGPContentVerifierBuilderProvider;
+import org.bouncycastle.openpgp.operator.jcajce.JcePBESecretKeyDecryptorBuilder;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.StandardCharsets;
+import java.util.Optional;
+import java.util.UUID;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.when;
+
+@ExtendWith(MockitoExtension.class)
+public class SignContentPGPTest {
+    private static final String PASSPHRASE = UUID.randomUUID().toString();
+
+    private static final String DATA = String.class.getName();
+
+    private static final byte[] DATA_BINARY = DATA.getBytes(StandardCharsets.UTF_8);
+
+    private static final String SERVICE_ID = PGPPrivateKeyService.class.getName();
+
+    private static final int SIGNATURE_VERSION = 4;
+
+    private static PGPPrivateKey rsaPrivateKey;
+
+    private static PGPPublicKey rsaPublicKey;
+
+    @Mock
+    private PGPPrivateKeyService privateKeyService;
+
+    private TestRunner runner;
+
+    @BeforeAll
+    public static void setKeys() throws Exception {
+        final PGPSecretKey rsaSecretKey = PGPSecretKeyGenerator.generateRsaSecretKey(PASSPHRASE.toCharArray());
+        rsaPublicKey = rsaSecretKey.getPublicKey();
+
+        final PBESecretKeyDecryptor decryptor = new JcePBESecretKeyDecryptorBuilder().build(PASSPHRASE.toCharArray());
+        rsaPrivateKey = rsaSecretKey.extractPrivateKey(decryptor);
+    }
+
+    @BeforeEach
+    public void setRunner() throws InitializationException {
+        runner = TestRunners.newTestRunner(new SignContentPGP());
+
+        when(privateKeyService.getIdentifier()).thenReturn(SERVICE_ID);
+        runner.addControllerService(SERVICE_ID, privateKeyService);
+        runner.enableControllerService(privateKeyService);
+        runner.setProperty(SignContentPGP.PRIVATE_KEY_SERVICE, SERVICE_ID);
+    }
+
+    @Test
+    public void testFailurePrivateKeyIdParsingException() {
+        runner.setProperty(SignContentPGP.PRIVATE_KEY_ID, String.class.getSimpleName());
+
+        runner.enqueue(DATA);
+        runner.run();
+
+        assertFailureErrorLogged();
+    }
+
+    @Test
+    public void testFailureServiceException() {
+        final String privateKeyId = KeyIdentifierConverter.format((rsaPrivateKey.getKeyID()));
+        runner.setProperty(SignContentPGP.PRIVATE_KEY_ID, privateKeyId);
+        when(privateKeyService.findPrivateKey(eq(rsaPrivateKey.getKeyID()))).thenThrow(new RuntimeException());
+
+        runner.enqueue(DATA);
+        runner.run();
+
+        assertFailureErrorLogged();
+    }
+
+    @Test
+    public void testFailurePrivateKeyIdNotFound() {
+        final String privateKeyId = KeyIdentifierConverter.format((rsaPrivateKey.getKeyID()));
+        runner.setProperty(SignContentPGP.PRIVATE_KEY_ID, privateKeyId);
+        when(privateKeyService.findPrivateKey(eq(rsaPrivateKey.getKeyID()))).thenReturn(Optional.empty());
+
+        runner.enqueue(DATA);
+        runner.run();
+
+        assertFailureErrorLogged();
+    }
+
+    @Test
+    public void testSuccessFileEncodingAsciiHashAlgorithmSha512() throws PGPException, IOException {
+        assertSuccess(FileEncoding.ASCII, HashAlgorithm.SHA512, SigningStrategy.SIGNED);
+    }
+
+    @Test
+    public void testSuccessFileEncodingBinaryHashAlgorithmSha512() throws PGPException, IOException {
+        assertSuccess(FileEncoding.BINARY, HashAlgorithm.SHA512, SigningStrategy.SIGNED);
+    }
+
+    @Test
+    public void testSuccessFileEncodingBinaryUncompressedHashAlgorithmSha256() throws PGPException, IOException {
+        runner.setProperty(SignContentPGP.COMPRESSION_ALGORITHM, CompressionAlgorithm.UNCOMPRESSED.toString());
+        assertSuccess(FileEncoding.BINARY, HashAlgorithm.SHA256, SigningStrategy.SIGNED);
+    }
+
+    @Test
+    public void testSuccessFileEncodingBinaryHashAlgorithmSha256() throws PGPException, IOException {
+        assertSuccess(FileEncoding.BINARY, HashAlgorithm.SHA256, SigningStrategy.SIGNED);
+    }
+
+    @Test
+    public void testSuccessDetachedFileEncodingBinaryHashAlgorithmSha256() throws PGPException, IOException {
+        assertSuccess(FileEncoding.BINARY, HashAlgorithm.SHA256, SigningStrategy.DETACHED);
+    }
+
+    private void setPrivateKey() {
+        final String privateKeyId = KeyIdentifierConverter.format((rsaPrivateKey.getKeyID()));
+        runner.setProperty(SignContentPGP.PRIVATE_KEY_ID, privateKeyId);
+        when(privateKeyService.findPrivateKey(eq(rsaPrivateKey.getKeyID()))).thenReturn(Optional.of(rsaPrivateKey));
+    }
+
+    private void assertSuccess(final FileEncoding fileEncoding, final HashAlgorithm hashAlgorithm, final SigningStrategy signingStrategy) throws PGPException, IOException {
+        setPrivateKey();
+
+        runner.setProperty(SignContentPGP.FILE_ENCODING, fileEncoding.toString());
+        runner.setProperty(SignContentPGP.HASH_ALGORITHM, hashAlgorithm.toString());
+        runner.setProperty(SignContentPGP.SIGNING_STRATEGY, signingStrategy.toString());
+
+        runner.enqueue(DATA);
+        runner.run();
+
+        runner.assertTransferCount(SignContentPGP.SUCCESS, 1);
+        final MockFlowFile flowFile = runner.getFlowFilesForRelationship(SignContentPGP.SUCCESS).iterator().next();
+        assertFlowFileAttributesFound(flowFile, fileEncoding, hashAlgorithm);
+
+        if (SigningStrategy.DETACHED == signingStrategy) {
+            assertDetachedSignatureVerified(flowFile);
+        } else {
+            assertSignatureVerified(flowFile);
+        }
+    }
+
+    private void assertFailureErrorLogged() {
+        runner.assertAllFlowFilesTransferred(SignContentPGP.FAILURE);
+        final Optional<LogMessage> optionalLogMessage = runner.getLogger().getErrorMessages().stream().findFirst();
+        assertTrue(optionalLogMessage.isPresent());
+    }
+
+    private void assertFlowFileAttributesFound(final MockFlowFile flowFile, final FileEncoding fileEncoding, final HashAlgorithm hashAlgorithm) throws PGPException {
+        flowFile.assertAttributeEquals(PGPAttributeKey.FILE_ENCODING, fileEncoding.toString());
+        flowFile.assertAttributeExists(PGPAttributeKey.COMPRESS_ALGORITHM);
+        flowFile.assertAttributeExists(PGPAttributeKey.COMPRESS_ALGORITHM_ID);
+
+        final String signatureAlgorithm = PGPUtil.getSignatureName(rsaPrivateKey.getPublicKeyPacket().getAlgorithm(), hashAlgorithm.getId());
+        final String keyId = KeyIdentifierConverter.format((rsaPrivateKey.getKeyID()));
+
+        flowFile.assertAttributeExists(PGPAttributeKey.SIGNATURE_CREATED);
+        flowFile.assertAttributeEquals(PGPAttributeKey.SIGNATURE_ALGORITHM, signatureAlgorithm);
+        flowFile.assertAttributeEquals(PGPAttributeKey.SIGNATURE_HASH_ALGORITHM_ID, Integer.toString(hashAlgorithm.getId()));
+        flowFile.assertAttributeEquals(PGPAttributeKey.SIGNATURE_KEY_ALGORITHM_ID, Integer.toString(rsaPrivateKey.getPublicKeyPacket().getAlgorithm()));
+        flowFile.assertAttributeEquals(PGPAttributeKey.SIGNATURE_KEY_ID, keyId);
+        flowFile.assertAttributeEquals(PGPAttributeKey.SIGNATURE_TYPE_ID, Integer.toString(PGPSignature.BINARY_DOCUMENT));
+        flowFile.assertAttributeEquals(PGPAttributeKey.SIGNATURE_VERSION, Integer.toString(SIGNATURE_VERSION));
+    }
+
+    private void assertDetachedSignatureVerified(final MockFlowFile signatureFlowFile) throws IOException, PGPException {
+        final InputStream signatureContentStream = PGPUtil.getDecoderStream(signatureFlowFile.getContentStream());
+        final PGPObjectFactory objectFactory = new JcaPGPObjectFactory(signatureContentStream);
+
+        final PGPSignatureList signatureList = (PGPSignatureList) objectFactory.nextObject();
+        final PGPSignature signature = signatureList.iterator().next();
+
+        signature.init(new JcaPGPContentVerifierBuilderProvider(), rsaPublicKey);
+
+        signature.update(DATA_BINARY);
+
+        final boolean verified = signature.verify();
+        assertTrue(verified);
+    }
+
+    private void assertSignatureVerified(final MockFlowFile flowFile) throws IOException, PGPException {
+        final InputStream flowFileContentStream = PGPUtil.getDecoderStream(flowFile.getContentStream());
+        final PGPObjectFactory objectFactory = new JcaPGPObjectFactory(flowFileContentStream);
+
+        final PGPCompressedData compressedData = (PGPCompressedData) objectFactory.nextObject();
+        final InputStream dataInputStream = compressedData.getDataStream();
+        final PGPObjectFactory dataObjectFactory = new JcaPGPObjectFactory(dataInputStream);
+
+        final ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+        PGPOnePassSignature onePassSignature = null;
+        boolean verified = false;
+        for (final Object object : dataObjectFactory) {
+            if (object instanceof PGPOnePassSignatureList) {
+                final PGPOnePassSignatureList onePassSignatureList = (PGPOnePassSignatureList) object;
+                onePassSignature = onePassSignatureList.iterator().next();
+                onePassSignature.init(new JcaPGPContentVerifierBuilderProvider(), rsaPublicKey);
+            } else if (object instanceof PGPLiteralData) {
+                if (onePassSignature == null) {
+                    throw new IllegalStateException("One-Pass Signature not found before Literal Data");
+                }
+
+                final PGPLiteralData literalData = (PGPLiteralData) object;
+                final InputStream literalInputStream = literalData.getDataStream();
+                int read;
+                while ((read = literalInputStream.read()) >= 0) {
+                    onePassSignature.update((byte) read);
+                    outputStream.write(read);
+                }
+            } else if (object instanceof PGPSignatureList) {
+                if (onePassSignature == null) {
+                    throw new IllegalStateException("One-Pass Signature not found before Signature");
+                }
+
+                final PGPSignatureList signatureList = (PGPSignatureList) object;
+                final PGPSignature signature = signatureList.iterator().next();
+                verified = onePassSignature.verify(signature);
+            } else {
+                throw new IllegalStateException(String.format("Unexpected PGP Object Found [%s]", object.getClass()));
+            }
+        }
+
+        assertTrue(verified);
+
+        final String literal = new String(outputStream.toByteArray(), StandardCharsets.UTF_8);
+        assertEquals(DATA, literal);
+    }
+}
diff --git a/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/test/java/org/apache/nifi/processors/pgp/VerifyContentPGPTest.java b/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/test/java/org/apache/nifi/processors/pgp/VerifyContentPGPTest.java
new file mode 100644
index 0000000..33e74c3
--- /dev/null
+++ b/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/test/java/org/apache/nifi/processors/pgp/VerifyContentPGPTest.java
@@ -0,0 +1,187 @@
+/*
+ * 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.pgp.util.PGPFileUtils;
+import org.apache.nifi.pgp.util.PGPSecretKeyGenerator;
+import org.apache.nifi.pgp.util.PGPOperationUtils;
+import org.apache.nifi.processors.pgp.io.KeyIdentifierConverter;
+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.HashAlgorithmTags;
+import org.bouncycastle.openpgp.PGPException;
+import org.bouncycastle.openpgp.PGPPrivateKey;
+import org.bouncycastle.openpgp.PGPPublicKey;
+import org.bouncycastle.openpgp.PGPSecretKey;
+import org.bouncycastle.openpgp.PGPSignature;
+import org.bouncycastle.openpgp.PGPUtil;
+import org.bouncycastle.openpgp.operator.PBESecretKeyDecryptor;
+import org.bouncycastle.openpgp.operator.jcajce.JcePBESecretKeyDecryptorBuilder;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Optional;
+import java.util.UUID;
+
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.when;
+
+@ExtendWith(MockitoExtension.class)
+public class VerifyContentPGPTest {
+    private static final String PASSPHRASE = UUID.randomUUID().toString();
+
+    private static final String SERVICE_ID = PGPPublicKeyService.class.getName();
+
+    private static final int SIGNATURE_VERSION = 4;
+
+    private static final int HASH_ALGORITHM_ID = HashAlgorithmTags.SHA512;
+
+    private static final String DATA = VerifyContentPGPTest.class.getName();
+
+    private static final byte[] DATA_BINARY = DATA.getBytes(StandardCharsets.UTF_8);
+
+    private static PGPPrivateKey rsaPrivateKey;
+
+    private static PGPPublicKey rsaPublicKey;
+
+    private TestRunner runner;
+
+    @Mock
+    private PGPPublicKeyService publicKeyService;
+
+    @BeforeAll
+    public static void setKeys() throws Exception {
+        final PGPSecretKey rsaSecretKey = PGPSecretKeyGenerator.generateRsaSecretKey(PASSPHRASE.toCharArray());
+        final PBESecretKeyDecryptor decryptor = new JcePBESecretKeyDecryptorBuilder().build(PASSPHRASE.toCharArray());
+        rsaPrivateKey = rsaSecretKey.extractPrivateKey(decryptor);
+        rsaPublicKey = rsaSecretKey.getPublicKey();
+    }
+
+    @BeforeEach
+    public void setRunner() throws InitializationException {
+        runner = TestRunners.newTestRunner(new VerifyContentPGP());
+
+        when(publicKeyService.getIdentifier()).thenReturn(SERVICE_ID);
+        runner.addControllerService(SERVICE_ID, publicKeyService);
+        runner.enableControllerService(publicKeyService);
+        runner.setProperty(EncryptContentPGP.PUBLIC_KEY_SERVICE, SERVICE_ID);
+    }
+
+    @Test
+    public void testFailureDataNotFound() {
+        runner.enqueue(new byte[]{});
+        runner.run();
+
+        assertFailureErrorLogged();
+    }
+
+    @Test
+    public void testFailureFlowFileUnchanged() {
+        runner.enqueue(DATA);
+        runner.run();
+
+        assertFailureErrorLogged();
+        final MockFlowFile flowFile = runner.getFlowFilesForRelationship(VerifyContentPGP.FAILURE).iterator().next();
+        flowFile.assertContentEquals(DATA);
+    }
+
+    @Test
+    public void testFailurePublicKeyNotFoundDataUnchanged() throws PGPException, IOException {
+        final byte[] signed = PGPOperationUtils.getOnePassSignedLiteralData(DATA_BINARY, rsaPrivateKey);
+
+        final String publicKeyIdSearch = KeyIdentifierConverter.format((rsaPublicKey.getKeyID()));
+        when(publicKeyService.findPublicKey(eq(publicKeyIdSearch))).thenReturn(Optional.empty());
+
+        runner.enqueue(signed);
+        runner.run();
+
+        assertFailureErrorLogged();
+        final MockFlowFile flowFile = runner.getFlowFilesForRelationship(VerifyContentPGP.FAILURE).iterator().next();
+        flowFile.assertContentEquals(signed);
+        assertFlowFileAttributesFound(flowFile);
+    }
+
+    @Test
+    public void testSuccessAsciiDataUnpacked() throws PGPException, IOException {
+        final byte[] signed = PGPOperationUtils.getOnePassSignedLiteralData(DATA_BINARY, rsaPrivateKey);
+        final String armored = PGPFileUtils.getArmored(signed);
+
+        setPublicKeyId();
+        runner.enqueue(armored);
+        runner.run();
+
+        final MockFlowFile flowFile = assertSuccess();
+        flowFile.assertContentEquals(DATA);
+    }
+
+    @Test
+    public void testSuccessBinaryDataUnpacked() throws PGPException, IOException {
+        final byte[] signed = PGPOperationUtils.getOnePassSignedLiteralData(DATA_BINARY, rsaPrivateKey);
+
+        setPublicKeyId();
+        runner.enqueue(signed);
+        runner.run();
+
+        final MockFlowFile flowFile = assertSuccess();
+        flowFile.assertContentEquals(DATA);
+    }
+
+    private void setPublicKeyId() {
+        final String publicKeyIdSearch = KeyIdentifierConverter.format((rsaPrivateKey.getKeyID()));
+        when(publicKeyService.findPublicKey(eq(publicKeyIdSearch))).thenReturn(Optional.of(rsaPublicKey));
+    }
+
+    private MockFlowFile assertSuccess() throws PGPException {
+        runner.assertAllFlowFilesTransferred(VerifyContentPGP.SUCCESS);
+        final MockFlowFile flowFile = runner.getFlowFilesForRelationship(VerifyContentPGP.SUCCESS).iterator().next();
+        assertFlowFileAttributesFound(flowFile);
+        return flowFile;
+    }
+
+    private void assertFailureErrorLogged() {
+        runner.assertAllFlowFilesTransferred(VerifyContentPGP.FAILURE);
+        final Optional<LogMessage> optionalLogMessage = runner.getLogger().getErrorMessages().stream().findFirst();
+        assertTrue(optionalLogMessage.isPresent());
+    }
+
+    private void assertFlowFileAttributesFound(final MockFlowFile flowFile) throws PGPException {
+        flowFile.assertAttributeExists(PGPAttributeKey.LITERAL_DATA_FILENAME);
+        flowFile.assertAttributeExists(PGPAttributeKey.LITERAL_DATA_MODIFIED);
+
+        final String signatureAlgorithm = PGPUtil.getSignatureName(rsaPublicKey.getAlgorithm(), HASH_ALGORITHM_ID);
+        final String keyId = KeyIdentifierConverter.format((rsaPrivateKey.getKeyID()));
+
+        flowFile.assertAttributeExists(PGPAttributeKey.SIGNATURE_CREATED);
+        flowFile.assertAttributeEquals(PGPAttributeKey.SIGNATURE_ALGORITHM, signatureAlgorithm);
+        flowFile.assertAttributeEquals(PGPAttributeKey.SIGNATURE_HASH_ALGORITHM_ID, Integer.toString(HASH_ALGORITHM_ID));
+        flowFile.assertAttributeEquals(PGPAttributeKey.SIGNATURE_KEY_ALGORITHM_ID, Integer.toString(rsaPublicKey.getAlgorithm()));
+        flowFile.assertAttributeEquals(PGPAttributeKey.SIGNATURE_KEY_ID, keyId);
+        flowFile.assertAttributeEquals(PGPAttributeKey.SIGNATURE_TYPE_ID, Integer.toString(PGPSignature.BINARY_DOCUMENT));
+        flowFile.assertAttributeEquals(PGPAttributeKey.SIGNATURE_VERSION, Integer.toString(SIGNATURE_VERSION));
+    }
+}
diff --git a/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/test/java/org/apache/nifi/processors/pgp/io/EncodingStreamCallbackTest.java b/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/test/java/org/apache/nifi/processors/pgp/io/EncodingStreamCallbackTest.java
new file mode 100644
index 0000000..3926a0b
--- /dev/null
+++ b/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/test/java/org/apache/nifi/processors/pgp/io/EncodingStreamCallbackTest.java
@@ -0,0 +1,96 @@
+/*
+ * 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.io;
+
+import org.apache.nifi.processors.pgp.attributes.CompressionAlgorithm;
+import org.apache.nifi.processors.pgp.attributes.FileEncoding;
+import org.apache.nifi.stream.io.StreamUtils;
+import org.bouncycastle.openpgp.PGPCompressedData;
+import org.bouncycastle.openpgp.PGPException;
+import org.bouncycastle.openpgp.PGPLiteralData;
+import org.bouncycastle.openpgp.PGPObjectFactory;
+import org.bouncycastle.openpgp.PGPUtil;
+import org.bouncycastle.openpgp.jcajce.JcaPGPObjectFactory;
+import org.junit.jupiter.api.Test;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.StandardCharsets;
+
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+
+public class EncodingStreamCallbackTest {
+    private static final String FILENAME = String.class.getName();
+
+    private static final byte[] DATA = String.class.getSimpleName().getBytes(StandardCharsets.UTF_8);
+
+    @Test
+    public void testProcessBinaryCompressionZip() throws IOException, PGPException {
+        final CompressionAlgorithm compressionAlgorithm = CompressionAlgorithm.ZIP;
+        final EncodingStreamCallback callback = new EncodingStreamCallback(FileEncoding.BINARY, compressionAlgorithm, FILENAME);
+        final ByteArrayInputStream inputStream = new ByteArrayInputStream(DATA);
+        final ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+        callback.process(inputStream, outputStream);
+
+        final InputStream processed = new ByteArrayInputStream(outputStream.toByteArray());
+        final InputStream compressedInputStream = assertCompressDataEquals(processed, compressionAlgorithm);
+        assertLiteralDataEquals(compressedInputStream);
+    }
+
+    @Test
+    public void testProcessAsciiCompressionBzip2() throws IOException, PGPException {
+        final CompressionAlgorithm compressionAlgorithm = CompressionAlgorithm.BZIP2;
+        final EncodingStreamCallback callback = new EncodingStreamCallback(FileEncoding.ASCII, compressionAlgorithm, FILENAME);
+        final ByteArrayInputStream inputStream = new ByteArrayInputStream(DATA);
+        final ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+        callback.process(inputStream, outputStream);
+
+        final InputStream processed = PGPUtil.getDecoderStream(new ByteArrayInputStream(outputStream.toByteArray()));
+        final InputStream compressedInputStream = assertCompressDataEquals(processed, compressionAlgorithm);
+        assertLiteralDataEquals(compressedInputStream);
+    }
+
+    private InputStream assertCompressDataEquals(final InputStream processed, final CompressionAlgorithm compressionAlgorithm) throws IOException, PGPException {
+        final PGPObjectFactory objectFactory = new JcaPGPObjectFactory(processed);
+        final Object firstObject = objectFactory.nextObject();
+        assertNotNull(firstObject);
+        assertEquals(PGPCompressedData.class, firstObject.getClass());
+
+        final PGPCompressedData compressedData = (PGPCompressedData) firstObject;
+        assertEquals(compressionAlgorithm.getId(), compressedData.getAlgorithm());
+        return compressedData.getDataStream();
+    }
+
+    private void assertLiteralDataEquals(final InputStream inputStream) throws IOException {
+        final PGPObjectFactory compressedObjectFactory = new JcaPGPObjectFactory(inputStream);
+        final Object firstCompressedObject = compressedObjectFactory.nextObject();
+        assertNotNull(firstCompressedObject);
+        assertEquals(PGPLiteralData.class, firstCompressedObject.getClass());
+
+        final PGPLiteralData literalData = (PGPLiteralData) firstCompressedObject;
+        assertEquals(FILENAME, literalData.getFileName());
+        assertEquals(PGPLiteralData.BINARY, literalData.getFormat());
+
+        final ByteArrayOutputStream literalOutputStream = new ByteArrayOutputStream();
+        StreamUtils.copy(literalData.getDataStream(), literalOutputStream);
+        assertArrayEquals(DATA, literalOutputStream.toByteArray());
+    }
+}
diff --git a/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/test/java/org/apache/nifi/processors/pgp/io/KeyIdentifierConverterTest.java b/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/test/java/org/apache/nifi/processors/pgp/io/KeyIdentifierConverterTest.java
new file mode 100644
index 0000000..7c047b3
--- /dev/null
+++ b/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/test/java/org/apache/nifi/processors/pgp/io/KeyIdentifierConverterTest.java
@@ -0,0 +1,47 @@
+/*
+ * 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.io;
+
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+public class KeyIdentifierConverterTest {
+    private static final long KEY_ID = Long.MAX_VALUE;
+
+    private static final String KEY_ID_FORMATTED = "7FFFFFFFFFFFFFFF";
+
+    private static final String INVALID = Long.class.getSimpleName();
+
+    @Test
+    public void testFormat() {
+        final String formatted = KeyIdentifierConverter.format(KEY_ID);
+        assertEquals(KEY_ID_FORMATTED, formatted);
+    }
+
+    @Test
+    public void testParse() {
+        final long parsed = KeyIdentifierConverter.parse(KEY_ID_FORMATTED);
+        assertEquals(KEY_ID, parsed);
+    }
+
+    @Test
+    public void testParseNumberFormatException() {
+        assertThrows(NumberFormatException.class, () -> KeyIdentifierConverter.parse(INVALID));
+    }
+}
diff --git a/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-test-utils/src/main/java/org/apache/nifi/pgp/util/PGPOperationUtils.java b/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-test-utils/src/main/java/org/apache/nifi/pgp/util/PGPOperationUtils.java
new file mode 100644
index 0000000..63e1fe7
--- /dev/null
+++ b/nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-test-utils/src/main/java/org/apache/nifi/pgp/util/PGPOperationUtils.java
@@ -0,0 +1,108 @@
+/*
+ * 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.util;
+
+import org.bouncycastle.bcpg.HashAlgorithmTags;
+import org.bouncycastle.openpgp.PGPException;
+import org.bouncycastle.openpgp.PGPLiteralDataGenerator;
+import org.bouncycastle.openpgp.PGPOnePassSignature;
+import org.bouncycastle.openpgp.PGPPrivateKey;
+import org.bouncycastle.openpgp.PGPSignature;
+import org.bouncycastle.openpgp.PGPSignatureGenerator;
+import org.bouncycastle.openpgp.operator.PGPContentSignerBuilder;
+import org.bouncycastle.openpgp.operator.jcajce.JcaPGPContentSignerBuilder;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Date;
+
+/**
+ * Pretty Good Privacy Operation Utilities
+ */
+public class PGPOperationUtils {
+    private static final boolean NESTED_SIGNATURE_DISABLED = false;
+
+    private static final int BUFFER_SIZE = 2048;
+
+    private static final long MODIFIED_MILLISECONDS = 86400000;
+
+    private static final Date MODIFIED = new Date(MODIFIED_MILLISECONDS);
+
+    private static final String FILE_NAME = String.class.getSimpleName();
+
+    private static final char FILE_TYPE = PGPLiteralDataGenerator.BINARY;
+
+    /**
+     * Get data signed using one-pass signature generator
+     *
+     * @param contents Byte array contents to be signed
+     * @param privateKey Private Key used for signing
+     * @return Signed byte array
+     * @throws PGPException Thrown when signature initialization failed
+     * @throws IOException Thrown when signature generation failed
+     */
+    public static byte[] getOnePassSignedData(final byte[] contents, final PGPPrivateKey privateKey) throws IOException, PGPException {
+        final PGPSignatureGenerator signatureGenerator = getSignatureGenerator(privateKey);
+
+        final ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+        final PGPOnePassSignature onePassSignature = signatureGenerator.generateOnePassVersion(NESTED_SIGNATURE_DISABLED);
+        onePassSignature.encode(outputStream);
+
+        outputStream.write(contents);
+        signatureGenerator.update(contents);
+
+        final PGPSignature signature = signatureGenerator.generate();
+        signature.encode(outputStream);
+        return outputStream.toByteArray();
+    }
+
+    /**
+     * Get data signed using one-pass signature generator wrapping literal data
+     *
+     * @param contents Byte array contents to be signed
+     * @param privateKey Private Key used for signing
+     * @return Signed byte array
+     * @throws PGPException Thrown when signature initialization failed
+     * @throws IOException Thrown when signature generation failed
+     */
+    public static byte[] getOnePassSignedLiteralData(final byte[] contents, final PGPPrivateKey privateKey) throws IOException, PGPException {
+        final PGPSignatureGenerator signatureGenerator = getSignatureGenerator(privateKey);
+
+        final ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+        final PGPOnePassSignature onePassSignature = signatureGenerator.generateOnePassVersion(NESTED_SIGNATURE_DISABLED);
+        onePassSignature.encode(outputStream);
+
+        final PGPLiteralDataGenerator generator = new PGPLiteralDataGenerator();
+        final byte[] buffer = new byte[BUFFER_SIZE];
+        try (final OutputStream literalStream = generator.open(outputStream, FILE_TYPE, FILE_NAME, MODIFIED, buffer)) {
+            literalStream.write(contents);
+            signatureGenerator.update(contents);
+        }
+
+        final PGPSignature signature = signatureGenerator.generate();
+        signature.encode(outputStream);
+        return outputStream.toByteArray();
+    }
+
+    private static PGPSignatureGenerator getSignatureGenerator(final PGPPrivateKey privateKey) throws PGPException {
+        final PGPContentSignerBuilder contentSignerBuilder = new JcaPGPContentSignerBuilder(privateKey.getPublicKeyPacket().getAlgorithm(), HashAlgorithmTags.SHA512);
+        final PGPSignatureGenerator signatureGenerator = new PGPSignatureGenerator(contentSignerBuilder);
+        signatureGenerator.init(PGPSignature.BINARY_DOCUMENT, privateKey);
+        return signatureGenerator;
+    }
+}