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

[GitHub] [nifi] exceptionfactory opened a new pull request #5457: NIFI-7322 Add SignContentPGP and VerifyContentPGP

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


   #### Description of PR
   
   NIFI-7322 Adds `SignContentPGP` and `VerifyContentPGP` Processors to the `nifi-pgp-processors` module and includes related updates to `DecryptContentPGP` and `EncryptContentPGP`.
   
   The `SignContentPGP` Processor supports the following features:
   
   - Configurable `Signing Strategy` property supporting `SIGNED` messages with signature and message, or `DETACHED` signatures
   - Configurable `Compression Algorithm` supporting OpenPGP compression options
   - Configurable `File Encoding` support `BINARY` or `ASCII` Armor encoding
   - Configurable `Hash Algorithm` for signature generation supporting a subset of RFC 4880 options based on current security best practices: `SHA256`, `SHA384`, and `SHA512`, with `SHA512` as the default value
   - Configurable `Private Key Service` requiring a PGP Private Key Service
   - Configurable `Private Key ID` supporting standard 16 character hexadecimal key identifiers
   - Writing PGP signature algorithm FlowFile attributes based on configured values
   
   The `VerifyContentPGP` Processor supports the following features:
   
   - Configurable `Public Key Service` which the processor uses to find matching PGP Public Keys using the key identifier found on OpenPGP signature packets
   - Writing PGP signature algorithm FlowFile attributes based on parsed values
   
   The `DecryptContentPGP` Processor includes the following changes:
   
   - New `Decryption Strategy` property, defaulting to `DECRYPTED`, maintaining the current default behavior of decrypting content and unpacking literal data packets without signature verification
   - `PACKAGED` setting for `Decryption Strategy` instructing the processor to write decrypted contents as an OpenPGP message to support sending to `VerifyContentPGP` when signature verification is expected after decryption
   
   The `EncryptContentPGP` Processor includes the following changes:
   
   - Internal detection of OpenPGP packets in FlowFile contents to support encryption after signing without wrapping the content inside a new PGP Literal Data packet
   
   All new processors and features include unit tests to demonstrate expected behavior.
   
   NIFI-7322 included a request to support the Cleartext Signature Framework described in [RFC 4880 Section 7](https://datatracker.ietf.org/doc/html/rfc4880#section-7), however this capability is not included for several reasons.  The Cleartext Signature Framework, also described as [clearsigning](https://www.gnupg.org/gph/en/manual/r684.html) in GPG documentation, is not suitable for generalized processing.  Clearsigning requires text input and involves specialized end-of-line handling. Clearsigning also appends the signature to the end of the message in a format specific to GPG processing.  As mentioned in RFC 4880 Section 7, [RFC 3156](https://datatracker.ietf.org/doc/html/rfc3156) describes a more generalized standard for packaging signature and content using MIME formatting.  Using standard OpenPGP message formatting for signed information provides the widest compatibility across OpenPGP implementations, and also supports any handling any type in input contents.
   
   In order to streamline the review of the contribution we ask you
   to ensure the following steps have been taken:
   
   ### For all changes:
   - [X] Is there a JIRA ticket associated with this PR? Is it referenced 
        in the commit message?
   
   - [X] Does your PR title start with **NIFI-XXXX** where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
   
   - [X] Has your PR been rebased against the latest commit within the target branch (typically `main`)?
   
   - [X] Is your initial contribution a single, squashed commit? _Additional commits in response to PR reviewer feedback should be made on this branch and pushed to allow change tracking. Do not `squash` or use `--force` when pushing to allow for clean monitoring of changes._
   
   ### For code changes:
   - [X] Have you ensured that the full suite of tests is executed via `mvn -Pcontrib-check clean install` at the root `nifi` folder?
   - [X] Have you written or updated unit tests to verify your changes?
   - [X] Have you verified that the full build is successful on JDK 8?
   - [ ] Have you verified that the full build is successful on JDK 11?
   - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)?
   - [ ] If applicable, have you updated the `LICENSE` file, including the main `LICENSE` file under `nifi-assembly`?
   - [ ] If applicable, have you updated the `NOTICE` file, including the main `NOTICE` file found under `nifi-assembly`?
   - [X] If adding new Properties, have you added `.displayName` in addition to .name (programmatic access) for each of the new properties?
   
   ### For documentation related changes:
   - [ ] Have you ensured that format looks appropriate for the output in which it is rendered?
   
   ### Note:
   Please ensure that once the PR is submitted, you check GitHub Actions CI for build issues and submit an update to your PR as soon as possible.
   


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

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

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



[GitHub] [nifi] gresockj commented on pull request #5457: NIFI-7322 Add SignContentPGP and VerifyContentPGP

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


   
   > I will take another look at verifying the detached signature using `gpg`.
   
   Disregard my original comment here -- I ended up getting this working ("Good signature from...") by downloading the actual flow file from NiFi instead of trying to reproduce the original file.  I suspect it was an issue with hidden line breaks or something subtle, but once I used the actual original file, the verification of the detached worked.


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

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

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



[GitHub] [nifi] exceptionfactory commented on pull request #5457: NIFI-7322 Add SignContentPGP and VerifyContentPGP

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


   Thanks for the feedback @gresockj! I pushed an update to address your recommendations.
   
   Regarding sending the detached signature through `VerifyContentPGP`, this is the result of attempting to processing the input as an OpenPGP message.  The behavior could be changed to avoid unwrapping OpenPGP messages when verification fails, but that would probably break other use cases if not implemented correctly.  The logic could be more granular to detect the presence of a Signature or One-Pass Signature packet, what do you think?
   
   I added property description documentation and an additional note in the Capability Description of `EncryptContentPGP` and `DecryptContentPGP`.
   
   I will take another look at verifying the detached signature using `gpg`.


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

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

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



[GitHub] [nifi] exceptionfactory commented on a change in pull request #5457: NIFI-7322 Add SignContentPGP and VerifyContentPGP

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



##########
File path: nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/SignContentPGP.java
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.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.toString())
+            .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.toString())
+            .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.toString())
+            .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.toString())
+            .allowableValues(SigningStrategy.values())

Review comment:
       Thanks for the suggestion, adding a description sounds good.




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

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

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



[GitHub] [nifi] exceptionfactory commented on a change in pull request #5457: NIFI-7322 Add SignContentPGP and VerifyContentPGP

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



##########
File path: nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/SignContentPGP.java
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.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.toString())
+            .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.toString())
+            .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.toString())
+            .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.toString())
+            .allowableValues(SigningStrategy.values())
+            .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 SignStreamCallback callback = getSignStreamCallback(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 SignStreamCallback getSignStreamCallback(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 new SignStreamCallback(fileEncoding, compressionAlgorithm, filename, hashAlgorithm, signingStrategy, 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 SignStreamCallback extends EncodingStreamCallback {
+        private final PGPPrivateKey privateKey;
+
+        private final HashAlgorithm hashAlgorithm;
+
+        private final SigningStrategy signingStrategy;
+
+        private final Map<String, String> attributes = new HashMap<>();
+
+        private SignStreamCallback(final FileEncoding fileEncoding,
+                                   final CompressionAlgorithm compressionAlgorithm,
+                                   final String filename,
+                                   final HashAlgorithm hashAlgorithm,
+                                   final SigningStrategy signingStrategy,
+                                   final PGPPrivateKey privateKey
+        ) {
+            super(fileEncoding, compressionAlgorithm, filename);
+            this.hashAlgorithm = hashAlgorithm;
+            this.signingStrategy = signingStrategy;
+            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());
+        }
+
+        /**
+         * 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
+         */
+        @Override
+        protected void processEncoding(final InputStream inputStream, final OutputStream encodingOutputStream) throws IOException, PGPException {
+            if (SigningStrategy.DETACHED == signingStrategy) {
+                processDetached(inputStream, encodingOutputStream);
+            } else {
+                super.processEncoding(inputStream, encodingOutputStream);
+            }
+        }
+
+        /**
+         * 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 processDetached(final InputStream inputStream, final OutputStream outputStream) throws IOException, PGPException {
+            final PGPSignatureGenerator signatureGenerator = getSignatureGenerator();
+            int read;
+            while ((read = inputStream.read()) >= 0) {
+                signatureGenerator.update((byte) read);

Review comment:
       I considered this as well, but the `update(byte[], int, int)` method is just a loop that calls `update(byte)`, so changing this does not appear to provide any optimization:
   
   https://github.com/bcgit/bc-java/blob/bc3b92f1f0e78b82e2584c5fb4b226a13e7f8b3b/pg/src/main/java/org/bouncycastle/openpgp/PGPSignatureGenerator.java#L108




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

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

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



[GitHub] [nifi] gresockj commented on a change in pull request #5457: NIFI-7322 Add SignContentPGP and VerifyContentPGP

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



##########
File path: nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/VerifyContentPGP.java
##########
@@ -0,0 +1,320 @@
+/*
+ * 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;
+
+    /**
+     * 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);
+        try {
+            final VerifyStreamCallback callback = new VerifyStreamCallback(publicKeyService);
+            flowFile = session.write(flowFile, callback);
+            flowFile = session.putAllAttributes(flowFile, callback.attributes);
+            final String keyId = flowFile.getAttribute(PGPAttributeKey.SIGNATURE_KEY_ID);
+            if (callback.verified) {
+                getLogger().info("Signature Key ID [{}] Verification Completed {}", keyId, flowFile);
+                session.transfer(flowFile, SUCCESS);
+            } else {
+                getLogger().warn("Signature Key ID [{}] Verification Failed {}", keyId, flowFile);
+                session.transfer(flowFile, FAILURE);
+            }
+        } catch (final RuntimeException e) {
+            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);
+            } else {
+                getLogger().debug("PGP Objects not found");
+                inputStream.reset();
+                StreamUtils.copy(inputStream, outputStream);
+            }

Review comment:
       Regarding my latest review comment, what I'm wondering is if something like this would work:
   ```suggestion
               if (objects.hasNext()) {
                   processObjectFactory(objects, outputStream);
               } else {
                   getLogger().debug("PGP Objects not found");
               }
               if (!verified) {
                   inputStream.reset();
                   StreamUtils.copy(inputStream, outputStream);
               }
   ```




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

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

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



[GitHub] [nifi] gresockj edited a comment on pull request #5457: NIFI-7322 Add SignContentPGP and VerifyContentPGP

Posted by GitBox <gi...@apache.org>.
gresockj edited a comment on pull request #5457:
URL: https://github.com/apache/nifi/pull/5457#issuecomment-946006858


   > I will take another look at verifying the detached signature using `gpg`.
   
   Disregard my original comment here -- I ended up getting this working ("Good signature from...") by downloading the actual flow file from NiFi instead of trying to reproduce the original file.  I suspect it was an issue with hidden line breaks or something subtle, but once I used the actual original file, the verification of the detached signature worked.


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

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

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



[GitHub] [nifi] exceptionfactory commented on a change in pull request #5457: NIFI-7322 Add SignContentPGP and VerifyContentPGP

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



##########
File path: nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/SignContentPGP.java
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.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.toString())
+            .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.toString())
+            .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.toString())
+            .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.toString())
+            .allowableValues(SigningStrategy.values())
+            .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 SignStreamCallback callback = getSignStreamCallback(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 SignStreamCallback getSignStreamCallback(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 new SignStreamCallback(fileEncoding, compressionAlgorithm, filename, hashAlgorithm, signingStrategy, 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 SignStreamCallback extends EncodingStreamCallback {
+        private final PGPPrivateKey privateKey;
+
+        private final HashAlgorithm hashAlgorithm;
+
+        private final SigningStrategy signingStrategy;
+
+        private final Map<String, String> attributes = new HashMap<>();
+
+        private SignStreamCallback(final FileEncoding fileEncoding,
+                                   final CompressionAlgorithm compressionAlgorithm,
+                                   final String filename,
+                                   final HashAlgorithm hashAlgorithm,
+                                   final SigningStrategy signingStrategy,
+                                   final PGPPrivateKey privateKey
+        ) {
+            super(fileEncoding, compressionAlgorithm, filename);
+            this.hashAlgorithm = hashAlgorithm;
+            this.signingStrategy = signingStrategy;
+            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());
+        }
+
+        /**
+         * 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
+         */
+        @Override
+        protected void processEncoding(final InputStream inputStream, final OutputStream encodingOutputStream) throws IOException, PGPException {
+            if (SigningStrategy.DETACHED == signingStrategy) {
+                processDetached(inputStream, encodingOutputStream);
+            } else {
+                super.processEncoding(inputStream, encodingOutputStream);
+            }
+        }
+
+        /**
+         * 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 processDetached(final InputStream inputStream, final OutputStream outputStream) throws IOException, PGPException {
+            final PGPSignatureGenerator signatureGenerator = getSignatureGenerator();
+            int read;
+            while ((read = inputStream.read()) >= 0) {
+                signatureGenerator.update((byte) read);

Review comment:
       On further review of the [update() method](https://github.com/bcgit/bc-java/blob/bc3b92f1f0e78b82e2584c5fb4b226a13e7f8b3b/pg/src/main/java/org/bouncycastle/openpgp/PGPSignatureGenerator.java#L108), this could provide some optimization, so I will look at making the change.




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

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

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



[GitHub] [nifi] exceptionfactory commented on a change in pull request #5457: NIFI-7322 Add SignContentPGP and VerifyContentPGP

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



##########
File path: nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/VerifyContentPGP.java
##########
@@ -0,0 +1,320 @@
+/*
+ * 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;
+
+    /**
+     * 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);
+        try {
+            final VerifyStreamCallback callback = new VerifyStreamCallback(publicKeyService);
+            flowFile = session.write(flowFile, callback);
+            flowFile = session.putAllAttributes(flowFile, callback.attributes);
+            final String keyId = flowFile.getAttribute(PGPAttributeKey.SIGNATURE_KEY_ID);
+            if (callback.verified) {
+                getLogger().info("Signature Key ID [{}] Verification Completed {}", keyId, flowFile);
+                session.transfer(flowFile, SUCCESS);
+            } else {
+                getLogger().warn("Signature Key ID [{}] Verification Failed {}", keyId, flowFile);
+                session.transfer(flowFile, FAILURE);
+            }
+        } catch (final RuntimeException e) {
+            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);
+            } else {
+                getLogger().debug("PGP Objects not found");
+                inputStream.reset();
+                StreamUtils.copy(inputStream, outputStream);
+            }

Review comment:
       Thanks for the suggestions @gresockj. I updated the `VerifyStreamCallback` to throw an exception on verification failures, and simplified the handling in `onTrigger()` in order to avoid changing the FlowFile contents on verification failures.  This avoids potential issues with `InputStream.reset()` and also streamlines the `VerifyContentPGP` behavior so that the content is not modified under any failure conditions.




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

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

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



[GitHub] [nifi] exceptionfactory commented on pull request #5457: NIFI-7322 Add SignContentPGP and VerifyContentPGP

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


   Thanks for additional suggestions @gresockj, please let me know if you have any other feedback.


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

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

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



[GitHub] [nifi] gresockj commented on a change in pull request #5457: NIFI-7322 Add SignContentPGP and VerifyContentPGP

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



##########
File path: nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/VerifyContentPGP.java
##########
@@ -0,0 +1,320 @@
+/*
+ * 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;
+
+    /**
+     * 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);
+        try {
+            final VerifyStreamCallback callback = new VerifyStreamCallback(publicKeyService);
+            flowFile = session.write(flowFile, callback);
+            flowFile = session.putAllAttributes(flowFile, callback.attributes);
+            final String keyId = flowFile.getAttribute(PGPAttributeKey.SIGNATURE_KEY_ID);
+            if (callback.verified) {
+                getLogger().info("Signature Key ID [{}] Verification Completed {}", keyId, flowFile);
+                session.transfer(flowFile, SUCCESS);
+            } else {
+                getLogger().warn("Signature Key ID [{}] Verification Failed {}", keyId, flowFile);
+                session.transfer(flowFile, FAILURE);
+            }
+        } catch (final RuntimeException e) {
+            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);
+            } else {
+                getLogger().debug("PGP Objects not found");
+                inputStream.reset();
+                StreamUtils.copy(inputStream, outputStream);
+            }

Review comment:
       Regarding my latest review comment, what I'm wondering is if something like this would work:
   ```suggestion
               if (objects.hasNext()) {
                   processObjectFactory(objects, outputStream);
               }
               if (!verified) {
                   getLogger().debug("PGP Objects not found");
                   inputStream.reset();
                   StreamUtils.copy(inputStream, outputStream);
               }
   ```




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

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

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



[GitHub] [nifi] asfgit closed pull request #5457: NIFI-7322 Add SignContentPGP and VerifyContentPGP

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


   


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

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

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



[GitHub] [nifi] exceptionfactory commented on a change in pull request #5457: NIFI-7322 Add SignContentPGP and VerifyContentPGP

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



##########
File path: nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/SignContentPGP.java
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.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.toString())
+            .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.toString())
+            .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.toString())
+            .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.toString())
+            .allowableValues(SigningStrategy.values())
+            .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 SignStreamCallback callback = getSignStreamCallback(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 SignStreamCallback getSignStreamCallback(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 new SignStreamCallback(fileEncoding, compressionAlgorithm, filename, hashAlgorithm, signingStrategy, 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 SignStreamCallback extends EncodingStreamCallback {
+        private final PGPPrivateKey privateKey;
+
+        private final HashAlgorithm hashAlgorithm;
+
+        private final SigningStrategy signingStrategy;
+
+        private final Map<String, String> attributes = new HashMap<>();
+
+        private SignStreamCallback(final FileEncoding fileEncoding,
+                                   final CompressionAlgorithm compressionAlgorithm,
+                                   final String filename,
+                                   final HashAlgorithm hashAlgorithm,
+                                   final SigningStrategy signingStrategy,
+                                   final PGPPrivateKey privateKey
+        ) {
+            super(fileEncoding, compressionAlgorithm, filename);
+            this.hashAlgorithm = hashAlgorithm;
+            this.signingStrategy = signingStrategy;
+            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());
+        }
+
+        /**
+         * 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
+         */
+        @Override
+        protected void processEncoding(final InputStream inputStream, final OutputStream encodingOutputStream) throws IOException, PGPException {
+            if (SigningStrategy.DETACHED == signingStrategy) {

Review comment:
       Thanks for the suggestion, having separate sub-classes seems like it could be helpful, so I will make the adjustments.




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

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

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



[GitHub] [nifi] gresockj commented on pull request #5457: NIFI-7322 Add SignContentPGP and VerifyContentPGP

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


   > Regarding sending the detached signature through `VerifyContentPGP`, this is the result of attempting to processing the input as an OpenPGP message. The behavior could be changed to avoid unwrapping OpenPGP messages when verification fails, but that would probably break other use cases if not implemented correctly. The logic could be more granular to detect the presence of a Signature or One-Pass Signature packet, what do you think?
   
   This would probably resolve the issue I described, but I wonder if it would be possible to call the code at [Line 186-188](https://github.com/apache/nifi/pull/5457/files#diff-ebe449a7623de3f323fc99a1de34903509cffa5e2a7d1c434f9a6653741282ccR186) in any failure scenario, rather than trying to handle each case.
   


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

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

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



[GitHub] [nifi] gresockj commented on a change in pull request #5457: NIFI-7322 Add SignContentPGP and VerifyContentPGP

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



##########
File path: nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/DecryptContentPGP.java
##########
@@ -95,6 +99,19 @@
             .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.toString())

Review comment:
       I think `.name()` would be slightly preferred here: since it can't be overridden, its meaning is immediately clear.

##########
File path: nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/io/EncodingStreamCallback.java
##########
@@ -0,0 +1,130 @@
+/*
+ * 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) {
+        OutputStream encodingOutputStream = outputStream;
+        if (FileEncoding.ASCII.equals(fileEncoding)) {
+            encodingOutputStream = new ArmoredOutputStream(outputStream);
+        }
+        return encodingOutputStream;

Review comment:
       What about:
   ```suggestion
           return FileEncoding.ASCII.equals(fileEncoding) ? new ArmoredOutputStream(outputStream) : outputStream;
   ```

##########
File path: nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/SignContentPGP.java
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.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.toString())
+            .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.toString())
+            .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.toString())
+            .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.toString())
+            .allowableValues(SigningStrategy.values())
+            .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 SignStreamCallback callback = getSignStreamCallback(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 SignStreamCallback getSignStreamCallback(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 new SignStreamCallback(fileEncoding, compressionAlgorithm, filename, hashAlgorithm, signingStrategy, 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 SignStreamCallback extends EncodingStreamCallback {
+        private final PGPPrivateKey privateKey;
+
+        private final HashAlgorithm hashAlgorithm;
+
+        private final SigningStrategy signingStrategy;
+
+        private final Map<String, String> attributes = new HashMap<>();
+
+        private SignStreamCallback(final FileEncoding fileEncoding,
+                                   final CompressionAlgorithm compressionAlgorithm,
+                                   final String filename,
+                                   final HashAlgorithm hashAlgorithm,
+                                   final SigningStrategy signingStrategy,
+                                   final PGPPrivateKey privateKey
+        ) {
+            super(fileEncoding, compressionAlgorithm, filename);
+            this.hashAlgorithm = hashAlgorithm;
+            this.signingStrategy = signingStrategy;
+            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());
+        }
+
+        /**
+         * 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
+         */
+        @Override
+        protected void processEncoding(final InputStream inputStream, final OutputStream encodingOutputStream) throws IOException, PGPException {
+            if (SigningStrategy.DETACHED == signingStrategy) {

Review comment:
       What do you think about moving this check up to `#getSignStreamCallback`, and instead creating a `DetachedSignatureStreamCallback extends EncodingStreamCallback`?  The `DetachedSignatureStreamCallback` could implement the logic from `#processDetached` directly in `#processEncoding`, and your implementation code wouldn't have to swap out the implementation based on a conditional.  This way, the separate strategies are handled by separate callback classes, making for a cleaner logical flow.

##########
File path: nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/SignContentPGP.java
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.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.toString())
+            .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.toString())
+            .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.toString())
+            .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.toString())
+            .allowableValues(SigningStrategy.values())

Review comment:
       When using this processor, I found that I wanted a description for the allowable values in this property.

##########
File path: nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/SignContentPGP.java
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.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.toString())
+            .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.toString())
+            .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.toString())
+            .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.toString())
+            .allowableValues(SigningStrategy.values())
+            .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 SignStreamCallback callback = getSignStreamCallback(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 SignStreamCallback getSignStreamCallback(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 new SignStreamCallback(fileEncoding, compressionAlgorithm, filename, hashAlgorithm, signingStrategy, 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 SignStreamCallback extends EncodingStreamCallback {
+        private final PGPPrivateKey privateKey;
+
+        private final HashAlgorithm hashAlgorithm;
+
+        private final SigningStrategy signingStrategy;
+
+        private final Map<String, String> attributes = new HashMap<>();
+
+        private SignStreamCallback(final FileEncoding fileEncoding,
+                                   final CompressionAlgorithm compressionAlgorithm,
+                                   final String filename,
+                                   final HashAlgorithm hashAlgorithm,
+                                   final SigningStrategy signingStrategy,
+                                   final PGPPrivateKey privateKey
+        ) {
+            super(fileEncoding, compressionAlgorithm, filename);
+            this.hashAlgorithm = hashAlgorithm;
+            this.signingStrategy = signingStrategy;
+            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());
+        }
+
+        /**
+         * 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
+         */
+        @Override
+        protected void processEncoding(final InputStream inputStream, final OutputStream encodingOutputStream) throws IOException, PGPException {
+            if (SigningStrategy.DETACHED == signingStrategy) {
+                processDetached(inputStream, encodingOutputStream);
+            } else {
+                super.processEncoding(inputStream, encodingOutputStream);
+            }
+        }
+
+        /**
+         * 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 processDetached(final InputStream inputStream, final OutputStream outputStream) throws IOException, PGPException {
+            final PGPSignatureGenerator signatureGenerator = getSignatureGenerator();
+            int read;
+            while ((read = inputStream.read()) >= 0) {
+                signatureGenerator.update((byte) read);
+            }
+            writeSignature(signatureGenerator, outputStream);
+        }
+
+        private void processSigned(final InputStream inputStream, final OutputStream outputStream, final PGPSignatureGenerator signatureGenerator) throws IOException {
+            int read;
+            while ((read = inputStream.read()) >= 0) {
+                outputStream.write(read);
+                signatureGenerator.update((byte) read);

Review comment:
       Same comment as above

##########
File path: nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/VerifyContentPGP.java
##########
@@ -0,0 +1,317 @@
+/*
+ * 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
+    );
+
+    /**
+     * 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);
+        try {
+            final VerifyStreamCallback callback = new VerifyStreamCallback(publicKeyService);
+            flowFile = session.write(flowFile, callback);
+            flowFile = session.putAllAttributes(flowFile, callback.attributes);
+            final String keyId = flowFile.getAttribute(PGPAttributeKey.SIGNATURE_KEY_ID);
+            if (callback.verified) {
+                getLogger().info("Signature Key ID [{}] Verification Completed {}", keyId, flowFile);
+                session.transfer(flowFile, SUCCESS);
+            } else {
+                getLogger().warn("Signature Key ID [{}] Verification Failed {}", keyId, flowFile);
+                session.transfer(flowFile, FAILURE);
+            }
+        } catch (final RuntimeException e) {
+            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);
+            } else {
+                getLogger().debug("PGP Objects not found");
+                inputStream.reset();
+                StreamUtils.copy(inputStream, outputStream);
+            }
+        }
+
+        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);
+            int read;
+            while ((read = inputStream.read()) >= 0) {
+                onePassSignature.update((byte) read);

Review comment:
       Same comment as above with `#read()` vs `#read(byte[], int, int)`

##########
File path: nifi-nar-bundles/nifi-pgp-bundle/nifi-pgp-processors/src/main/java/org/apache/nifi/processors/pgp/SignContentPGP.java
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.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.toString())
+            .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.toString())
+            .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.toString())
+            .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.toString())
+            .allowableValues(SigningStrategy.values())
+            .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 SignStreamCallback callback = getSignStreamCallback(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 SignStreamCallback getSignStreamCallback(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 new SignStreamCallback(fileEncoding, compressionAlgorithm, filename, hashAlgorithm, signingStrategy, 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 SignStreamCallback extends EncodingStreamCallback {
+        private final PGPPrivateKey privateKey;
+
+        private final HashAlgorithm hashAlgorithm;
+
+        private final SigningStrategy signingStrategy;
+
+        private final Map<String, String> attributes = new HashMap<>();
+
+        private SignStreamCallback(final FileEncoding fileEncoding,
+                                   final CompressionAlgorithm compressionAlgorithm,
+                                   final String filename,
+                                   final HashAlgorithm hashAlgorithm,
+                                   final SigningStrategy signingStrategy,
+                                   final PGPPrivateKey privateKey
+        ) {
+            super(fileEncoding, compressionAlgorithm, filename);
+            this.hashAlgorithm = hashAlgorithm;
+            this.signingStrategy = signingStrategy;
+            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());
+        }
+
+        /**
+         * 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
+         */
+        @Override
+        protected void processEncoding(final InputStream inputStream, final OutputStream encodingOutputStream) throws IOException, PGPException {
+            if (SigningStrategy.DETACHED == signingStrategy) {
+                processDetached(inputStream, encodingOutputStream);
+            } else {
+                super.processEncoding(inputStream, encodingOutputStream);
+            }
+        }
+
+        /**
+         * 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 processDetached(final InputStream inputStream, final OutputStream outputStream) throws IOException, PGPException {
+            final PGPSignatureGenerator signatureGenerator = getSignatureGenerator();
+            int read;
+            while ((read = inputStream.read()) >= 0) {
+                signatureGenerator.update((byte) read);

Review comment:
       I notice that `PGPSignatureGenerator` also has the method `#update(byte[] buf, int offset, int len)`, which I feel may have a performance benefit over writing byte by byte.  What do you think about using the latter method?




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

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

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