[ 
https://issues.apache.org/jira/browse/NIFI-2961?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15724321#comment-15724321
 ] 

ASF GitHub Bot commented on NIFI-2961:
--------------------------------------

Github user alopresto commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1294#discussion_r90929270
  
    --- Diff: 
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EncryptAttributes.java
 ---
    @@ -0,0 +1,611 @@
    +/*
    + * 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.standard;
    +
    +import org.apache.commons.codec.DecoderException;
    +import org.apache.commons.codec.binary.Base64;
    +import org.apache.commons.codec.binary.Hex;
    +import org.apache.commons.lang3.StringUtils;
    +import org.apache.nifi.annotation.behavior.EventDriven;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.SideEffectFree;
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.components.AllowableValue;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.flowfile.attributes.CoreAttributes;
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.processor.AbstractProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.standard.util.crypto.CipherUtility;
    +import org.apache.nifi.processors.standard.util.crypto.KeyedEncryptor;
    +import 
org.apache.nifi.processors.standard.util.crypto.OpenPGPKeyBasedEncryptor;
    +import 
org.apache.nifi.processors.standard.util.crypto.OpenPGPPasswordBasedEncryptor;
    +import 
org.apache.nifi.processors.standard.util.crypto.PasswordBasedEncryptor;
    +import org.apache.nifi.security.util.EncryptionMethod;
    +import org.apache.nifi.security.util.KeyDerivationFunction;
    +import org.bouncycastle.jce.provider.BouncyCastleProvider;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.nio.charset.StandardCharsets;
    +import java.security.Security;
    +import java.text.Normalizer;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +/**
    + * Provides functionality of encrypting attributes with various algorithms.
    + * Note. It'll not modify filename or uuid as they are sensitive and are
    + * internally used by either Algorithm itself or FlowFile repo.
    + */
    +@EventDriven
    +@SideEffectFree
    +@SupportsBatching
    +@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
    +@Tags({"encryption", "decryption", "password", "JCE", "OpenPGP", "PGP", 
"GPG"})
    +@CapabilityDescription("Encrypts or Decrypts a FlowFile attributes using 
either symmetric encryption with a password " +
    +        "and randomly generated salt, or asymmetric encryption using a 
public and secret key.")
    +public class EncryptAttributes extends AbstractProcessor {
    +
    +    public static final String ENCRYPT_MODE = "Encrypt";
    +    public static final String DECRYPT_MODE = "Decrypt";
    +
    +    public static final String WEAK_CRYPTO_ALLOWED_NAME = "allowed";
    +    public static final String WEAK_CRYPTO_NOT_ALLOWED_NAME = 
"not-allowed";
    +
    +    public static final PropertyDescriptor ATTRIBUTES_TO_ENCRYPT = new 
PropertyDescriptor.Builder()
    +            .name("Attributes to encrypt")
    +            .description("Comma separated list of attributes to encrypt, 
if empty then it'll encrypt all the " +
    +                    "attributes including CoreAttributes EXCEPT filename 
and uuid. " +
    +                    "This list is case sensitive and if attribute is not 
found " +
    +                    "then the value will be ignored. " )
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +    public static final PropertyDescriptor MODE = new 
PropertyDescriptor.Builder()
    +            .name("Mode")
    +            .description("Specifies whether the content should be 
encrypted or decrypted")
    +            .required(true)
    +            .allowableValues(ENCRYPT_MODE, DECRYPT_MODE)
    +            .defaultValue(ENCRYPT_MODE)
    +            .build();
    +    public static final PropertyDescriptor KEY_DERIVATION_FUNCTION = new 
PropertyDescriptor.Builder()
    +            .name("key-derivation-function")
    +            .displayName("Key Derivation Function")
    +            .description("Specifies the key derivation function to 
generate the key from the password (and salt)")
    +            .required(true)
    +            .allowableValues(buildKeyDerivationFunctionAllowableValues())
    +            .defaultValue(KeyDerivationFunction.BCRYPT.name())
    +            .build();
    +    public static final PropertyDescriptor ENCRYPTION_ALGORITHM = new 
PropertyDescriptor.Builder()
    +            .name("Encryption Algorithm")
    +            .description("The Encryption Algorithm to use")
    +            .required(true)
    +            .allowableValues(buildEncryptionMethodAllowableValues())
    +            .defaultValue(EncryptionMethod.MD5_128AES.name())
    +            .build();
    +    public static final PropertyDescriptor PASSWORD = new 
PropertyDescriptor.Builder()
    +            .name("Password")
    +            .description("The Password to use for encrypting or decrypting 
the data")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .sensitive(true)
    +            .build();
    +    public static final PropertyDescriptor PUBLIC_KEYRING = new 
PropertyDescriptor.Builder()
    +            .name("public-keyring-file")
    +            .displayName("Public Keyring File")
    +            .description("In a PGP encrypt mode, this keyring contains the 
public key of the recipient")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +    public static final PropertyDescriptor PUBLIC_KEY_USERID = new 
PropertyDescriptor.Builder()
    +            .name("public-key-user-id")
    +            .displayName("Public Key User Id")
    +            .description("In a PGP encrypt mode, this user id of the 
recipient")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +    public static final PropertyDescriptor PRIVATE_KEYRING = new 
PropertyDescriptor.Builder()
    +            .name("private-keyring-file")
    +            .displayName("Private Keyring File")
    +            .description("In a PGP decrypt mode, this keyring contains the 
private key of the recipient")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +    public static final PropertyDescriptor PRIVATE_KEYRING_PASSPHRASE = 
new PropertyDescriptor.Builder()
    +            .name("private-keyring-passphrase")
    +            .displayName("Private Keyring Passphrase")
    +            .description("In a PGP decrypt mode, this is the private 
keyring passphrase")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .sensitive(true)
    +            .build();
    +    public static final PropertyDescriptor RAW_KEY_HEX = new 
PropertyDescriptor.Builder()
    +            .name("raw-key-hex")
    +            .displayName("Raw Key (hexadecimal)")
    +            .description("In keyed encryption, this is the raw key, 
encoded in hexadecimal")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .sensitive(true)
    +            .build();
    +    public static final PropertyDescriptor ALLOW_WEAK_CRYPTO = new 
PropertyDescriptor.Builder()
    +            .name("allow-weak-crypto")
    +            .displayName("Allow insecure cryptographic modes")
    +            .description("Overrides the default behavior to prevent unsafe 
combinations of encryption algorithms and short passwords on JVMs with limited 
strength cryptographic jurisdiction policies")
    +            .required(true)
    +            .allowableValues(buildWeakCryptoAllowableValues())
    +            
.defaultValue(buildDefaultWeakCryptoAllowableValue().getValue())
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new 
Relationship.Builder().name("success")
    +            .description("Any FlowFile that is successfully encrypted or 
decrypted will be routed to success").build();
    +
    +    public static final Relationship REL_FAILURE = new 
Relationship.Builder().name("failure")
    +            .description("Any FlowFile that cannot be encrypted or 
decrypted will be routed to failure").build();
    +    private List<PropertyDescriptor> properties;
    +
    +    private Set<Relationship> relationships;
    +
    +    static {
    +        // add BouncyCastle encryption providers
    +        Security.addProvider(new BouncyCastleProvider());
    +    }
    +
    +    private static AllowableValue[] 
buildKeyDerivationFunctionAllowableValues() {
    +        final KeyDerivationFunction[] keyDerivationFunctions = 
KeyDerivationFunction.values();
    +        List<AllowableValue> allowableValues = new 
ArrayList<>(keyDerivationFunctions.length);
    +        for (KeyDerivationFunction kdf : keyDerivationFunctions) {
    +            allowableValues.add(new AllowableValue(kdf.name(), 
kdf.getName(), kdf.getDescription()));
    +        }
    +
    +        return allowableValues.toArray(new AllowableValue[0]);
    +    }
    +
    +    private static AllowableValue[] buildEncryptionMethodAllowableValues() 
{
    +        final EncryptionMethod[] encryptionMethods = 
EncryptionMethod.values();
    +        List<AllowableValue> allowableValues = new 
ArrayList<>(encryptionMethods.length);
    +        for (EncryptionMethod em : encryptionMethods) {
    +            allowableValues.add(new AllowableValue(em.name(), em.name(), 
em.toString()));
    +        }
    +
    +        return allowableValues.toArray(new AllowableValue[0]);
    +    }
    +
    +    private static AllowableValue[] buildWeakCryptoAllowableValues() {
    +        List<AllowableValue> allowableValues = new ArrayList<>();
    +        allowableValues.add(new AllowableValue(WEAK_CRYPTO_ALLOWED_NAME, 
"Allowed", "Operation will not be blocked and no alerts will be presented " +
    +                "when unsafe combinations of encryption algorithms and 
passwords are provided"));
    +        allowableValues.add(buildDefaultWeakCryptoAllowableValue());
    +        return allowableValues.toArray(new AllowableValue[0]);
    +    }
    +
    +    private static AllowableValue buildDefaultWeakCryptoAllowableValue() {
    --- End diff --
    
    This method should be collapsed into `buildWeakCryptoAllowableValues()`. 


> Create EncryptAttribute processor
> ---------------------------------
>
>                 Key: NIFI-2961
>                 URL: https://issues.apache.org/jira/browse/NIFI-2961
>             Project: Apache NiFi
>          Issue Type: Improvement
>          Components: Extensions
>    Affects Versions: 1.0.0
>            Reporter: Andy LoPresto
>              Labels: attributes, encryption, security
>
> Similar to {{EncryptContent}}, the {{EncryptAttribute}} processor would allow 
> individual (and multiple) flowfile attributes to be encrypted (either 
> in-place or to a new attribute key) with various encryption algorithms (AES, 
> RSA, PBE, and PGP). 
> Specific compatibility with the {{OpenSSL EVP_BytesToKey}}, {{PBKDF2}}, 
> {{scrypt}}, and {{bcrypt}} key derivation functions should be included. 
> The processor should provide the boolean option to encrypt or decrypt (only 
> one operation per instance of the processor). The processor should also allow 
> Base64 encoding (aka ASCII armor) for the encrypted attributes to prevent 
> byte escaping/data loss. 
> If [dangerous processor 
> annotations|https://cwiki.apache.org/confluence/display/NIFI/Security+Feature+Roadmap]
>  are introduced, this processor should be marked as such and the 
> corresponding attribute protection (i.e. provenance before/after, etc.) 
> should be applied. 
> Originally requested in this [Stack Overflow 
> question|https://stackoverflow.com/questions/40294945/nifi-encrypt-json].  



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to