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 2020/06/24 08:11:28 UTC

[GitHub] [nifi] sasqnz commented on a change in pull request #4077: NIFI-5346 Introduces new PGP controller service and PGP processors.

sasqnz commented on a change in pull request #4077:
URL: https://github.com/apache/nifi/pull/4077#discussion_r444720358



##########
File path: nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/pgp/StandardPGPOperator.java
##########
@@ -0,0 +1,852 @@
+/*
+ * 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.security.pgp;
+
+import org.apache.commons.codec.DecoderException;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.pgp.controllerservices.PGPKeyMaterialService;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.bouncycastle.bcpg.ArmoredOutputStream;
+import org.bouncycastle.bcpg.HashAlgorithmTags;
+import org.bouncycastle.bcpg.PublicKeyAlgorithmTags;
+import org.bouncycastle.jce.provider.BouncyCastleProvider;
+import org.bouncycastle.openpgp.PGPCompressedData;
+import org.bouncycastle.openpgp.PGPCompressedDataGenerator;
+import org.bouncycastle.openpgp.PGPEncryptedData;
+import org.bouncycastle.openpgp.PGPEncryptedDataGenerator;
+import org.bouncycastle.openpgp.PGPEncryptedDataList;
+import org.bouncycastle.openpgp.PGPException;
+import org.bouncycastle.openpgp.PGPLiteralData;
+import org.bouncycastle.openpgp.PGPLiteralDataGenerator;
+import org.bouncycastle.openpgp.PGPObjectFactory;
+import org.bouncycastle.openpgp.PGPPBEEncryptedData;
+import org.bouncycastle.openpgp.PGPPrivateKey;
+import org.bouncycastle.openpgp.PGPPublicKey;
+import org.bouncycastle.openpgp.PGPPublicKeyEncryptedData;
+import org.bouncycastle.openpgp.PGPPublicKeyRing;
+import org.bouncycastle.openpgp.PGPSecretKey;
+import org.bouncycastle.openpgp.PGPSecretKeyRing;
+import org.bouncycastle.openpgp.PGPSecretKeyRingCollection;
+import org.bouncycastle.openpgp.PGPSignature;
+import org.bouncycastle.openpgp.PGPSignatureGenerator;
+import org.bouncycastle.openpgp.PGPSignatureList;
+import org.bouncycastle.openpgp.PGPUtil;
+import org.bouncycastle.openpgp.jcajce.JcaPGPObjectFactory;
+import org.bouncycastle.openpgp.jcajce.JcaPGPPublicKeyRingCollection;
+import org.bouncycastle.openpgp.operator.KeyFingerPrintCalculator;
+import org.bouncycastle.openpgp.operator.PBEDataDecryptorFactory;
+import org.bouncycastle.openpgp.operator.PBESecretKeyDecryptor;
+import org.bouncycastle.openpgp.operator.PGPDigestCalculatorProvider;
+import org.bouncycastle.openpgp.operator.PublicKeyDataDecryptorFactory;
+import org.bouncycastle.openpgp.operator.bc.BcKeyFingerprintCalculator;
+import org.bouncycastle.openpgp.operator.bc.BcPGPDigestCalculatorProvider;
+import org.bouncycastle.openpgp.operator.jcajce.JcaPGPContentSignerBuilder;
+import org.bouncycastle.openpgp.operator.jcajce.JcaPGPContentVerifierBuilderProvider;
+import org.bouncycastle.openpgp.operator.jcajce.JcaPGPDigestCalculatorProviderBuilder;
+import org.bouncycastle.openpgp.operator.jcajce.JcePBEDataDecryptorFactoryBuilder;
+import org.bouncycastle.openpgp.operator.jcajce.JcePBEKeyEncryptionMethodGenerator;
+import org.bouncycastle.openpgp.operator.jcajce.JcePBESecretKeyDecryptorBuilder;
+import org.bouncycastle.openpgp.operator.jcajce.JcePGPDataEncryptorBuilder;
+import org.bouncycastle.openpgp.operator.jcajce.JcePublicKeyDataDecryptorFactoryBuilder;
+import org.bouncycastle.openpgp.operator.jcajce.JcePublicKeyKeyEncryptionMethodGenerator;
+import org.bouncycastle.util.io.Streams;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.charset.StandardCharsets;
+import java.security.Security;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.zip.Deflater;
+
+
+public class StandardPGPOperator implements PGPOperator {
+    static {
+        Security.addProvider(new BouncyCastleProvider());
+    }
+
+    private static final Logger logger = LoggerFactory.getLogger(StandardPGPOperator.class);
+    public static final String DEFAULT_SIGNATURE_ATTRIBUTE = "content-signature";
+
+    public static final PropertyDescriptor PGP_KEY_SERVICE = new PropertyDescriptor.Builder()
+            .name("pgp-key-service")
+            .displayName("PGP Key Material Service")
+            .description("PGP key material service for using cryptographic keys and passphrases.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .identifiesControllerService(PGPKeyMaterialService.class)
+            .build();
+
+    public static final PropertyDescriptor PUBLIC_KEYRING_FILE = new PropertyDescriptor.Builder()
+            .name("public-keyring-file")
+            .displayName("Public Key or Keyring File")
+            .description("PGP public key or keyring file.")
+            .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor PUBLIC_KEYRING_TEXT = new PropertyDescriptor.Builder()
+            .name("public-keyring-text")
+            .displayName("Public Key or Keyring Text")
+            .description("PGP public key or keyring as text (also called Armored text or ASCII text).")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor PUBLIC_KEY_USER_ID = new PropertyDescriptor.Builder()
+            .name("public-key-user-id")
+            .displayName("Public Key User ID")
+            .description("Public Key user ID (also called ID or Name) for the key within the public key keyring.")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor SECRET_KEYRING_FILE = new PropertyDescriptor.Builder()
+            .name("secret-keyring-file")
+            .displayName("Secret Key or Keyring File")
+            .description("PGP secret key or keyring file.")
+            .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor SECRET_KEYRING_TEXT = new PropertyDescriptor.Builder()
+            .name("secret-keyring-text")
+            .displayName("Secret Key or Keyring Text")
+            .description("PGP secret key or keyring as text (also called Armored text or ASCII text).")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor SECRET_KEY_USER_ID = new PropertyDescriptor.Builder()
+            .name("secret-key-user-id")
+            .displayName("Secret Key User ID")
+            .description("Secret Key user ID (also called ID or Name) for the key within the secret keyring.")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor PRIVATE_KEY_PASSPHRASE = new PropertyDescriptor.Builder()
+            .name("private-key-passphrase")
+            .displayName("Private Key Passphrase")
+            .description("This is the passphrase for the specified secret key.")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor PBE_PASSPHRASE = new PropertyDescriptor.Builder()
+            .name("pbe-passphrase")
+            .displayName("Encryption Passphrase")
+            .description("This is the passphrase for password-based encryption and decryption.")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor ENCRYPT_ALGORITHM = new PropertyDescriptor.Builder()
+            .name("encrypt-algorithm")
+            .displayName("Encryption Cipher Algorithm")
+            .description("The cipher algorithm used when encrypting data.")
+            .allowableValues(getCipherAllowableValues())
+            .defaultValue(getCipherDefaultValue())
+            .build();
+
+    public static final PropertyDescriptor ENCRYPT_ENCODING = new PropertyDescriptor.Builder()
+            .name("encrypt-encoding")
+            .displayName("Encryption Data Encoding")
+            .description("The data encoding method used when writing encrypting data.")
+            .allowableValues(
+                    new AllowableValue("0", "Raw (bytes with no encoding)"),
+                    new AllowableValue("1", "PGP Armor (encoded text)"))
+            .defaultValue("0")
+            .build();
+
+    public static final PropertyDescriptor SIGNATURE_HASH_ALGORITHM = new PropertyDescriptor.Builder()
+            .name("signature-hash-algorithm")
+            .displayName("Signature Hash Function")
+            .description("The hash function used when signing data.")
+            .allowableValues(getSignatureHashAllowableValues())
+            .defaultValue(getSignatureHashDefaultValue())
+            .build();
+
+    public static final PropertyDescriptor SIGNATURE_ATTRIBUTE = new PropertyDescriptor.Builder()
+            .name("signature-attribute")
+            .displayName("Signature Attribute")
+            .description("The name of the FlowFile Attribute for the signature to write during Sign operations and to read during Verify operations.")
+            .defaultValue(DEFAULT_SIGNATURE_ATTRIBUTE)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    protected final KeyCache<PGPPublicKeys> publicKeyCache = new KeyCache<>(); // Operator instances have a public key cache
+    protected final KeyCache<PGPSecretKeys> secretKeyCache = new KeyCache<>(); // and a private key cache
+
+
+    public static String getSignatureHashDefaultValue() {
+        return String.valueOf(PGPUtil.SHA256);
+    }
+
+
+    // Values match integer values in org.bouncycastle.bcpg.HashAlgorithmTags
+    static AllowableValue[] getSignatureHashAllowableValues() {
+        return new AllowableValue[]{
+                new AllowableValue("1", "MD5"),
+                new AllowableValue("2", "SHA1"),
+                new AllowableValue("6", "TIGER 192"),
+                new AllowableValue("8", "SHA 256"),
+                new AllowableValue("9", "SHA 384"),
+                new AllowableValue("10", "SHA 512"),
+        };
+    }
+
+    // Values match integer values in org.bouncycastle.bcpg.SymmetricKeyAlgorithmTags
+    static AllowableValue[] getCipherAllowableValues() {
+        return new AllowableValue[]{
+                // 0 - NULL not supported
+                new AllowableValue("1", "IDEA"),
+                new AllowableValue("2", "TRIPLE DES"),
+                new AllowableValue("3", "CAST5"),
+                new AllowableValue("4", "BLOWFISH"),
+                new AllowableValue("6", "DES"),
+                // 6 - SAFER not supported
+                new AllowableValue("7", "AES 128"),
+                new AllowableValue("8", "AES 192"),
+                new AllowableValue("9", "AES 256"),
+                new AllowableValue("10", "TWOFISH"),
+                new AllowableValue("11", "CAMELLIA 128"),
+                new AllowableValue("12", "CAMELLIA 192"),
+                new AllowableValue("13", "CAMELLIA 256")};
+    }
+
+    public static String getCipherDefaultValue() {
+        return String.valueOf(PGPEncryptedData.AES_128);
+    }
+
+    public char[] getPBEPassphrase(PropertyContext context) {
+        if (context.getProperty(PBE_PASSPHRASE).isSet()) {
+            return context.getProperty(PBE_PASSPHRASE).getValue().toCharArray();
+        }
+        return null;
+    }
+
+    public InputStream getSignature(PropertyContext context, FlowFile flowFile) throws PGPException {
+        if (!context.getProperty(SIGNATURE_ATTRIBUTE).isSet()) {
+            throw new PGPException("Signature attribute not set.");
+        }
+
+        final String signature = flowFile.getAttribute(context.getProperty(SIGNATURE_ATTRIBUTE).getValue());
+        try {
+            return new ByteArrayInputStream(Hex.decodeHex(signature));
+        } catch (final DecoderException e) {
+            throw new PGPException("Unable to decode signature.", e);
+        }
+    }
+
+
+    /**
+     * Creates map of public keys from the input stream.
+     *
+     * @param input public key or key ring stream
+     * @param hash value to use as cache key
+     * @return public keys
+     */
+    PGPPublicKeys readPublicKeys(InputStream input, Integer hash) {
+         if (hash != null && publicKeyCache.containsKey(hash)) {
+            return publicKeyCache.get(hash);
+        }
+
+        final PGPPublicKeys keys = new PGPPublicKeys();
+        JcaPGPPublicKeyRingCollection rings;
+
+        try {
+            InputStream decoderStream = PGPUtil.getDecoderStream(input);
+            rings = new JcaPGPPublicKeyRingCollection(decoderStream);
+            input.close();
+        } catch (final IOException | PGPException ignored) {
+            return null;
+        }
+
+        for (final PGPPublicKeyRing ring : rings) {
+            for (final PGPPublicKey key : ring) {
+                for (final Iterator<String> it = key.getUserIDs(); it.hasNext(); ) {

Review comment:
       When using a ECDSA/ECDH pgp key, the public key is created with ECDSA as primary, and ECDH as a sub key. This method fails to detect the ECDH sub key as it does not contain a user id; which means that the keys hash only contains the signing key, not the encryption key. This then fails at ln 742. 
   Steps to repeat 
   gpg --full-gen-key --expert
   Please select what kind of key you want:
      (1) RSA and RSA (default)
      (2) DSA and Elgamal
      (3) DSA (sign only)
      (4) RSA (sign only)
      (7) DSA (set your own capabilities)
      (8) RSA (set your own capabilities)
      (9) ECC and ECC
     (10) ECC (sign only)
     (11) ECC (set your own capabilities)
     (13) Existing key
     (14) Existing key from card
   **Your selection? 9**
   Please select which elliptic curve you want:
      (1) Curve 25519
      (3) NIST P-256
      (4) NIST P-384
      (5) NIST P-521
      (9) secp256k1
   **Your selection? 4**
   Please specify how long the key should be valid.
            0 = key does not expire
         <n>  = key expires in n days
         <n>w = key expires in n weeks
         <n>m = key expires in n months
         <n>y = key expires in n years
   Key is valid for? (0) 0
   Key does not expire at all
   Is this correct? (y/N) y
   
   gpg --export --armor --output public-key.asc <key id>




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

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