You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by alopresto <gi...@git.apache.org> on 2017/04/21 00:35:50 UTC

[GitHub] nifi pull request #1686: NIFI-3594 Encrypted provenance repository implement...

GitHub user alopresto opened a pull request:

    https://github.com/apache/nifi/pull/1686

    NIFI-3594 Encrypted provenance repository implementation

    This is a big PR, and there is some helpful information before delving into the code. 
    
    # What is it?
    
    The `EncryptedWriteAheadProvenanceRepository` is a new implementation of the provenance repository which encrypts all event record information before it is written to the repository. This allows for storage on systems where OS-level access controls are not sufficient to protect the data while still allowing querying and access to the data through the NiFi UI/API. 
    
    # How does it work?
    
    The code will provide more details, and I plan to write extensive documentation for the Admin Guide and User Guide [NIFI-3721](https://issues.apache.org/jira/browse/NIFI-3721), but this will suffice for an overview. 
    
    The `WriteAheadProvenanceRepository` was introduced by @markap14 in [NIFI-3356](https://issues.apache.org/jira/browse/NIFI-3356) and provided a refactored and much faster provenance repository implementation than the previous `PersistentProvenanceRepository`. The encrypted version wraps that implementation with a record writer and reader which encrypt and decrypt the serialized bytes respectively. 
    
    The fully qualified class `org.apache.nifi.provenance.EncryptedWriteAheadProvenanceRepository` is specified as the provenance repository implementation in `nifi.properties` as the value of `nifi.provenance.repository.implementation`. In addition, new properties must be populated to allow successful initialization. 
    
    The simplest configuration is below:
    
    ```
    nifi.provenance.repository.debug.frequency=100
    nifi.provenance.repository.encryption.key.provider.implementation=org.apache.nifi.provenance.StaticKeyProvider
    nifi.provenance.repository.encryption.key.provider.location=
    nifi.provenance.repository.encryption.key.id=Key1
    nifi.provenance.repository.encryption.key=0123456789ABCDEFFEDCBA98765432100123456789ABCDEFFEDCBA9876543210
    ```
    
    * `nifi.provenance.repository.debug.frequency` is a new configuration option to control the rate at which debug messages regarding performance statistics are printed to the logs (in *DEBUG* mode)
    * `nifi.provenance.repository.encryption.key.provider.implementation` is the *Key Provider* implementation. A key provider is the datastore interface for accessing the encryption key to protect the provenance events. There are currently two implementations -- `StaticKeyProvider` which reads a key directly from `nifi.properties`, and `FileBasedKeyProvider` which reads *n* many keys from an encrypted file. The interface is extensible, and HSM-backed or other providers are expected in the future. 
    * `nifi.provenance.repository.encryption.key.provider.location` is the location of the key provider data. For `StaticKeyProvider`, this is left blank. For `FileBasedKeyProvider`, this is a file path to the key provider definition file (e.g. `./keys.nkp`). For an HMS or other provider, this could be a URL, etc. 
    * `nifi.provenance.repository.encryption.key.id` is the *key ID* which is used to encrypt the events. 
    * `nifi.provenance.repository.encryption.key` is the hexadecimal encoding of the key for the `StaticKeyProvider`. For `FileBasedKeyProvider`, this value is left blank. This value can also be encrypted by using the `encrypt-config.sh` tool in the NiFi Toolkit, and is marked as sensitive by default. 
    
    The `FileBasedKeyProvider` implementation reads from an encrypted definition file of the format:
    
    ```
    key1=NGCpDpxBZNN0DBodz0p1SDbTjC2FG5kp1pCmdUKJlxxtcMSo6GC4fMlTyy1mPeKOxzLut3DRX+51j6PCO5SznA==
    key2=GYxPbMMDbnraXs09eGJudAM5jTvVYp05XtImkAg4JY4rIbmHOiVUUI6OeOf7ZW+hH42jtPgNW9pSkkQ9HWY/vQ==
    key3=SFe11xuz7J89Y/IQ7YbJPOL0/YKZRFL/VUxJgEHxxlXpd/8ELA7wwN59K1KTr3BURCcFP5YGmwrSKfr4OE4Vlg==
    key4=kZprfcTSTH69UuOU3jMkZfrtiVR/eqWmmbdku3bQcUJ/+UToecNB5lzOVEMBChyEXppyXXC35Wa6GEXFK6PMKw==
    key5=c6FzfnKm7UR7xqI2NFpZ+fEKBfSU7+1NvRw+XWQ9U39MONWqk5gvoyOCdFR1kUgeg46jrN5dGXk13sRqE0GETQ==
    ```
    
    Each line defines a key ID and then the Base64-encoded cipher text of a 16 byte IV and wrapped AES-128, AES-192, or AES-256 key depending on the JCE policies available. The individual keys are wrapped by AES/GCM encryption using the **master key** defined by `nifi.bootstrap.sensitive.key` in `conf/bootstrap.conf`. 
    
    Once the repository is initialized, all provenance event record write operations are serialized according to the configured schema writer (`EventIdFirstSchemaRecordWriter` by default for `WriteAheadProvenanceRepository`) to a `byte[]`. Those bytes are then encrypted using an implementation of `ProvenanceEventEncryptor` (the only current implementation is `AES/GCM/NoPadding`) and the encryption metadata (`keyId`, `algorithm`, `version`, `IV`) is serialized and prepended. The complete `byte[]` is then written to the repository on disk as normal. 
    
    ![Encrypted provenance repository file on disk](https://i.imgur.com/BCpQoBl.png)
    
    On record read, the process is reversed. The encryption metadata is parsed and used to decrypt the serialized bytes, which are then deserialized into a `ProvenanceEventRecord` object. The delegation to the normal schema record writer/reader allows for "random-access" (i.e. immediate seek without decryption of unnecessary records). 
    
    Within the NiFi UI/API, there is no detectable difference between an encrypted and unencrypted provenance repository. The Provenance Query operations work as expected with no change to the process. 
    
    # Performance
    
    While there is an obvious performance cost to cryptographic operations, I tried to minimize the impact and to provide an estimate of the metrics of this implementation in comparison to existing behavior. 
    
    In general, with low flow event volume, the performance impact is not noticeable -- it is perfectly inline with `WriteAheadProvenanceRepository` and more than twice as fast as the existing `PersistentProvenanceRepository`. 
    
    ![Small event size, low volume](http://i.imgur.com/kc35VJL.png)
    
    With a much higher volume of events, the impact is felt in two ways. First, the throughput of the flow is slower, as more resources are dedicated to encrypting and serializing the events (note the total events processed/events per second). In addition, the provenance queries are slightly slower than the original implementation (1% - 17%), and significantly slower than the new `WriteAheadProvenanceRepository` operating in plaintext (~110%). This is a known trade-off that will need to be evaluated by the deployment administrator given their threat model and risk assessment. 
    
    ![Small event size, high volume](https://i.imgur.com/M8jg75V.png)
    
    # Remaining Efforts
    * Documentation -- as noted above, this effort is captured in [NIFI-3721](https://issues.apache.org/jira/browse/NIFI-3721)
    * Logging data leakage -- in various places, I noted that with logs set to *DEBUG*, the `LuceneEventIndex` printed substantial information from the event record to the log. If the repository is encrypted, an administrator would reasonably expect this potentially-sensitive information not to be printed to the logs. In this specific instance, I changed the log statements to elide this information, but an audit needs to occur for the complete project to detect other instances where this may occur. Ideally, this could be variable depending on the encryption status of the repository, but this would require changing the method signature, and I didn't want to tackle that now 
    * Other implementations -- While AES/GCM is (in my opinion) the best option for event encryption (it is AEAD which provides confidentiality and integrity, very fast, and does not need to be compatible with any external system), users may have requirements/requests for other algorithms
    * Other key providers -- as noted above, HSM is probably the biggest, but other software-based secure data stores like [Vault](https://www.vaultproject.io/) or [KeyWhiz](https://square.github.io/keywhiz/), or JCEKS-backed to be compatible with Hadoop systems may be necessary
    * Refactoring shared code -- as part of the effort to provide encrypted repositories for content and flowfiles, some of this code will likely be moved to other modules
    
    # Potential Issues
    * Key rotation -- If a user wants to rotate the keys used, `StaticKeyProvider` does not provide a mechanism to support this. With `FileBasedKeyProvider`, they can simply specify a new key in the key provider file with `nifi.provenance.repository.encryption.key.id` in `nifi.properties` and future events will be encrypted with that key. Previously-encrypted events can still be decrypted as long as that key is still available in the key definition file
    * Switching between unencrypted and encrypted repositories
        - If a user has an existing repository that is not encrypted and switches their configuration to use an encrypted repository, the application writes an error to the log but starts up. However, previous events are not accessible through the provenance query interface and new events will overwrite the existing events. The same behavior occurs if a user switches from an encrypted repository to an unencrypted repository
        - We should provide logic to handle encrypted -> unencrypted seamlessly as long as the key provider available still has the keys used to encrypt the events (see **Key Rotation**)
        - We should provide logic to handle unencrypted -> encrypted seamlessly as the previously recorded events simply need to be read with a plaintext schema record reader and then written back with the encrypted record writer
        - We should also provide a standalone tool in NiFi Toolkit to encrypt/decrypt an existing provenance repository to make the transition easier. The translation process could take a long time depending on the size of the existing repository, and being able to perform this task outside of application startup would be valuable
    * Multiple repositories -- No additional effort or testing has been applied to multiple repositories at this time. It is possible/likely issues will occur with repositories on different physical devices. There is no option to provide a heterogenous environment (i.e. one encrypted, one plaintext repository). 
    * Corruption -- when a disk is filled or corrupted, there have been reported issues with the repository becoming corrupted and recovery steps are necessary. This is likely to continue to be an issue with the encrypted repository, although still limited in scope to individual records (i.e. an entire repository file won't be irrecoverable due to the encryption)
    * Shutdown -- I noticed that switching from `PersistentProvenanceRepository` to `EncryptedWriteAheadProvenanceRepository` led to slower NiFi app shutdowns [NIFI-3712](https://issues.apache.org/jira/browse/NIFI-3712). This was repeatable with `WriteAheadProvenanceRepository`, so I don't believe it is dependent on the encryption changes
    
    
    ----
    Thank you for submitting a contribution to Apache NiFi.
    
    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 master)?
    
    - [ ] Is your initial contribution a single, squashed commit?
    
    ### For code changes:
    - [ ] 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?
    - [ ] 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?
    - [ ] 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 travis-ci for build issues and submit an update to your PR as soon as possible.


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/alopresto/nifi NIFI-3594

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/nifi/pull/1686.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #1686
    
----
commit d4de39b2505615e83a7c44262c95837a4bcdff48
Author: Andy LoPresto <al...@apache.org>
Date:   2017-03-14T04:53:00Z

    NIFI-3594 Added first unit test for PersistentProvenanceRepository operation.
    Added BC dependency to nifi-persistent-provenance-repository module.

commit 8006d12fa297229438accd617bbb59a9c86ea5f4
Author: Andy LoPresto <al...@apache.org>
Date:   2017-03-14T23:35:18Z

    NIFI-3594 Removed Thread.sleep() from WAPR unit test as per Mark Payne.

commit f6200032207f18fa83d2c4b1ac330f2649d1e7e6
Author: Andy LoPresto <al...@apache.org>
Date:   2017-03-17T03:49:28Z

    NIFI-3594 Fixed WAPR unit test (temporary fix dependent on NIFI-3605).

commit c497644e70e3e9bd766668b2df2d0f454e3d76a5
Author: Andy LoPresto <al...@apache.org>
Date:   2017-03-18T02:08:54Z

    NIFI-3594 Added skeleton of encrypted provenance repository (KeyProvider w/ 2 impls, Encryptor skeleton, and exceptions/utilities).
    Reorganized tests to proper path.

commit 2fdbb233e3332cc5c1205cbbb62b877914a8fc6c
Author: Andy LoPresto <al...@apache.org>
Date:   2017-03-22T03:36:27Z

    NIFI-3594 Added encryption methods and reflective property accessors. Pausing to re-evaluate because work may need to be done at lower level (EventWriter/EventReader -- byte/Object serialization).

commit 5951b78cc92356fa3f20798ba2bb1276b518cb91
Author: Andy LoPresto <al...@apache.org>
Date:   2017-03-24T21:52:13Z

    NIFI-3594 Intermediate changes before discussion with Mark Payne about intercepting SchemaRecordReader/Writer serialization (no updates to schema necessary).

commit 054cdefc0c8bee366670c9ea7d60bcc302a8808c
Author: Andy LoPresto <al...@apache.org>
Date:   2017-03-24T23:02:44Z

    NIFI-3594 Moved (Keyed)CipherProvider classes & tests into nifi-security-utils to include in nifi-data-provenance-utils.

commit 1dfbb5b83178b7e9d190d4c87bb277bb9f5eb6ba
Author: Andy LoPresto <al...@apache.org>
Date:   2017-03-27T23:44:38Z

    NIFI-3594 Working JUnit test with encrypted write and read of PER.

commit 2637a3049b5c6d10e6c42e6f83b7d049159cfca3
Author: Andy LoPresto <al...@apache.org>
Date:   2017-03-28T23:45:18Z

    NIFI-3594 Implemented encrypted read, write, and seek operations.
    Resolved RAT and checkstyle issues.
    All tests pass.

commit 33db1eeae8d48d854c467c91d8c520edce96ad3a
Author: Andy LoPresto <al...@apache.org>
Date:   2017-03-29T00:53:24Z

    NIFI-3594 Changed constant IV (for testing only) to actual random IV.

commit d844c0065062ad5bd181e9e077693d04cde6704d
Author: Andy LoPresto <al...@apache.org>
Date:   2017-03-29T00:55:23Z

    NIFI-3594 Delgated reader and writer to use AESKeyedCipherProvider (enhanced error checking and guard controls).

commit 7ae1a1c1d778d6c8b0ac25fc34b85b5c55c8ca7d
Author: Andy LoPresto <al...@apache.org>
Date:   2017-03-29T02:13:30Z

    NIFI-3594 Refactored to use concatByteArrays() for performance and heap optimization.

commit b912a140eb646f8969b5adb9c7d28b6894316e5d
Author: Andy LoPresto <al...@apache.org>
Date:   2017-04-06T00:25:37Z

    NIFI-3594 Working event encryptor lifecycle unit test with full encryption metadata serialization.

commit 83523ad4e66851da9facaee60ec456ff11ba2615
Author: Andy LoPresto <al...@apache.org>
Date:   2017-04-07T02:17:15Z

    NIFI-3594 Refactored AESProvenanceEventEncryptor implementation (removed cached ciphers to allow non-repeating IVs).
    Added unit tests.

commit 60f4f03b42737028f361621f3eea4c9d54a6971d
Author: Andy LoPresto <al...@apache.org>
Date:   2017-04-07T18:41:17Z

    NIFI-3594 Added forAlgorithm static constructor for EncryptionMethod.
    Added validity checks for algorithm and version in AESProvenanceEventEncryptor.
    Added unit tests.

commit b8314e8ce4e4033bf827f997336a73787fbeffc4
Author: Andy LoPresto <al...@apache.org>
Date:   2017-04-07T18:49:00Z

    NIFI-3594 Included bad keyId scenario in test.

commit 09115b2ad0a63fad8c3f6cdd62ba4d2aa02bc9b3
Author: Andy LoPresto <al...@apache.org>
Date:   2017-04-08T04:15:06Z

    NIFI-3594 Refactored key availability interface contract.
    Refactored encryptor composition.
    Added unit tests.

commit 10b203e187374462ba52f30b8d2c0213ca803f8f
Author: Andy LoPresto <al...@apache.org>
Date:   2017-04-11T23:37:47Z

    NIFI-3594 Began adding configuration properties for encrypted provenance repository.
    Added utility methods for validation.
    Added unit tests.

commit 711bfa4e91b394b2478a5bf5440e24cb0ac25b8d
Author: Andy LoPresto <al...@apache.org>
Date:   2017-04-19T00:14:39Z

    NIFI-3594 Added new NiFi properties keys for provenance repository encryption.
    Added nifi.provenance.repository.encryption.key to default sensitive keys and updated unit tests and test resources.
    Added method to correctly calculate protected percentage of sensitive keys (unpopulated keys are no longer counted against protection %).

commit 092cb2d13ad351761cf2bf3833ae9a01d602b6b7
Author: Andy LoPresto <al...@apache.org>
Date:   2017-04-19T01:21:09Z

    NIFI-3594 Implemented StaticKeyProvider and FileBasedKeyProvider.
    Moved getBestEventIdentifier() from StandardProvenanceEventRecord to ProvenanceEventRecord interface and added delegate in all implementations to avoid ClassCastException from multiple classloaders.
    Initialized IV before cipher to suppress unnecessary warnings.
    Added utility method to read encrypted provenance keys from key provider file.
    Suppressed logging of event record details in LuceneEventIndex.
    Added logic to create EncryptedSchemaRecordReader (if supported) in RecordReaders.
    Cleaned up EncryptedSchemaRecordReader and EncryptedSchemaRecordWriter.
    Added keyProvider, recordReaderFactory, and recordWriterFactory initialization to EncryptedWriteAheadProvenanceRepository to provide complete interceptor implementation.
    Added logic to RepositoryConfiguration to load encryption-related properties if necessary.
    Refactored WriteAheadProvenanceRepository to allow subclass implementation.
    Registered EncryptedWAPR in ProvenanceRepository implementations.
    Added unit tests for EWAPR.
    Added new nifi.properties keys for encrypted provenance repository.

commit 578d0d16097a60576bbb1682880b83040e551e45
Author: Andy LoPresto <al...@apache.org>
Date:   2017-04-20T22:31:12Z

    NIFI-3594 Cleanup of initial efforts (OBE).

commit b44b603b21a3a6b8e110f5d7673e7771aecb70e5
Author: Andy LoPresto <al...@apache.org>
Date:   2017-04-20T22:47:02Z

    NIFI-3594 Continued cleanup of initial efforts (OBE).

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1686: NIFI-3594 Encrypted provenance repository implementation

Posted by alopresto <gi...@git.apache.org>.
Github user alopresto commented on the issue:

    https://github.com/apache/nifi/pull/1686
  
    The "improve switching UX" ticket is [NIFI-3766](https://issues.apache.org/jira/browse/NIFI-3766). I propose merging this as is given the new documentation providing a work-around. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1686: NIFI-3594 Encrypted provenance repository implementation

Posted by alopresto <gi...@git.apache.org>.
Github user alopresto commented on the issue:

    https://github.com/apache/nifi/pull/1686
  
    @markap14 That's not a bad suggestion. I had envisioned the `StaticKeyProvider` as a simple default that only had the one key and didn't revisit it after creating the `FileBasedKeyProvider`. I'll make the change and add a test. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1686: NIFI-3594 Encrypted provenance repository implement...

Posted by alopresto <gi...@git.apache.org>.
Github user alopresto commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1686#discussion_r112980940
  
    --- Diff: nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/CryptoUtils.java ---
    @@ -0,0 +1,228 @@
    +/*
    + * 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.provenance;
    +
    +import java.io.BufferedReader;
    +import java.io.ByteArrayOutputStream;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.security.KeyManagementException;
    +import java.security.NoSuchAlgorithmException;
    +import java.util.Arrays;
    +import java.util.Base64;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.regex.Pattern;
    +import javax.crypto.BadPaddingException;
    +import javax.crypto.Cipher;
    +import javax.crypto.IllegalBlockSizeException;
    +import javax.crypto.SecretKey;
    +import javax.crypto.spec.SecretKeySpec;
    +import org.apache.commons.lang3.StringUtils;
    +import org.apache.nifi.security.util.EncryptionMethod;
    +import org.apache.nifi.security.util.crypto.AESKeyedCipherProvider;
    +import org.apache.nifi.util.NiFiProperties;
    +import org.bouncycastle.util.encoders.Hex;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class CryptoUtils {
    +    private static final Logger logger = LoggerFactory.getLogger(StaticKeyProvider.class);
    +    private static final String STATIC_KEY_PROVIDER_CLASS_NAME = "org.apache.nifi.provenance.StaticKeyProvider";
    +    private static final String FILE_BASED_KEY_PROVIDER_CLASS_NAME = "org.apache.nifi.provenance.FileBasedKeyProvider";
    +    private static final Pattern HEX_PATTERN = Pattern.compile("(?i)^[0-9a-f]+$");
    +
    +    public static final int IV_LENGTH = 16;
    +
    +    public static boolean isUnlimitedStrengthCryptoAvailable() {
    +        try {
    +            return Cipher.getMaxAllowedKeyLength("AES") > 128;
    +        } catch (NoSuchAlgorithmException e) {
    +            logger.warn("Tried to determine if unlimited strength crypto is available but the AES algorithm is not available");
    +            return false;
    +        }
    +    }
    +
    +    /**
    +     * Utility method which returns true if the string is null, empty, or entirely whitespace.
    +     *
    +     * @param src the string to evaluate
    +     * @return true if empty
    +     */
    +    public static boolean isEmpty(String src) {
    +        return src == null || src.trim().isEmpty();
    +    }
    +
    +    /**
    +     * Concatenates multiple byte[] into a single byte[]. Because it uses a {@link ByteArrayOutputStream}
    +     * rather than {@link System#arraycopy(Object, int, Object, int, int)} the performance is much better
    +     * with an arbitrary number of input byte[]s.
    +     *
    +     * @param arrays the component byte[] in order
    +     * @return a concatenated byte[]
    +     * @throws IOException this should never be thrown
    +     */
    +    public static byte[] concatByteArrays(byte[]... arrays) throws IOException {
    +        ByteArrayOutputStream boas = new ByteArrayOutputStream();
    +        for (byte[] arr : arrays) {
    +            boas.write(arr);
    +        }
    +        return boas.toByteArray();
    +    }
    +
    +    public static boolean isValidKeyProvider(String keyProviderImplementation, String keyProviderLocation, String keyId, String encryptionKeyHex) {
    +        if (STATIC_KEY_PROVIDER_CLASS_NAME.equals(keyProviderImplementation)) {
    +            // Ensure the keyId and key are valid
    +            return keyIsValid(encryptionKeyHex) && StringUtils.isNotEmpty(keyId);
    +        } else if (FILE_BASED_KEY_PROVIDER_CLASS_NAME.equals(keyProviderImplementation)) {
    +            // Ensure the file can be read and the keyId is populated (does not read file to validate)
    +            final File kpf = new File(keyProviderLocation);
    +            return kpf.exists() && kpf.canRead() && StringUtils.isNotEmpty(keyId);
    +        } else {
    +            logger.error("The attempt to validate the key provider failed keyProviderImplementation = "
    +                    + keyProviderImplementation + " , keyProviderLocation = "
    +                    + keyProviderLocation + " , keyId = "
    +                    + keyId + " , encryptedKeyHex = "
    +                    + (StringUtils.isNotEmpty(encryptionKeyHex) ? "********" : ""));
    +
    +            return false;
    +        }
    +    }
    +
    +    /**
    +     * Returns true if the provided key is valid hex and is the correct length for the current system's JCE policies.
    +     *
    +     * @param encryptionKeyHex the key in hexadecimal
    +     * @return true if this key is valid
    +     */
    +    public static boolean keyIsValid(String encryptionKeyHex) {
    +        return isHexString(encryptionKeyHex)
    +                && (isUnlimitedStrengthCryptoAvailable()
    +                ? Arrays.asList(32, 48, 64).contains(encryptionKeyHex.length())
    --- End diff --
    
    Will do. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1686: NIFI-3594 Encrypted provenance repository implement...

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1686#discussion_r112956387
  
    --- Diff: nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/AESProvenanceEventEncryptor.java ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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.provenance;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectInputStream;
    +import java.io.ObjectOutputStream;
    +import java.security.KeyManagementException;
    +import java.security.SecureRandom;
    +import java.security.Security;
    +import java.util.Arrays;
    +import java.util.List;
    +import javax.crypto.BadPaddingException;
    +import javax.crypto.Cipher;
    +import javax.crypto.IllegalBlockSizeException;
    +import javax.crypto.SecretKey;
    +import org.apache.commons.lang3.StringUtils;
    +import org.apache.nifi.security.util.EncryptionMethod;
    +import org.apache.nifi.security.util.crypto.AESKeyedCipherProvider;
    +import org.bouncycastle.jce.provider.BouncyCastleProvider;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class AESProvenanceEventEncryptor implements ProvenanceEventEncryptor {
    +    private static final Logger logger = LoggerFactory.getLogger(AESProvenanceEventEncryptor.class);
    +    private static final String ALGORITHM = "AES/GCM/NoPadding";
    +    private static final int IV_LENGTH = 16;
    +    private static final byte[] EMPTY_IV = new byte[IV_LENGTH];
    +    private static final String VERSION = "v1";
    +    private static final List<String> SUPPORTED_VERSIONS = Arrays.asList(VERSION);
    +    private static final int MIN_METADATA_LENGTH = IV_LENGTH + 3 + 3; // 3 delimiters and 3 non-zero elements
    +    private static final int METADATA_DEFAULT_LENGTH = (20 + ALGORITHM.length() + IV_LENGTH + VERSION.length()) * 2; // Default to twice the expected length
    +    private static final byte[] SENTINEL = new byte[]{0x01};
    +
    +    private KeyProvider keyProvider;
    +
    +    private AESKeyedCipherProvider aesKeyedCipherProvider = new AESKeyedCipherProvider();
    +
    +    /**
    +     * Initializes the encryptor with a {@link KeyProvider}.
    +     *
    +     * @param keyProvider the key provider which will be responsible for accessing keys
    +     * @throws KeyManagementException if there is an issue configuring the key provider
    +     */
    +    @Override
    +    public void initialize(KeyProvider keyProvider) throws KeyManagementException {
    +        this.keyProvider = keyProvider;
    +
    +        if (this.aesKeyedCipherProvider == null) {
    +            this.aesKeyedCipherProvider = new AESKeyedCipherProvider();
    +        }
    +
    +        if (Security.getProvider("BC") == null) {
    +            Security.addProvider(new BouncyCastleProvider());
    +        }
    +    }
    +
    +    /**
    +     * Available for dependency injection to override the default {@link AESKeyedCipherProvider} if necessary.
    +     *
    +     * @param cipherProvider the AES cipher provider to use
    +     */
    +    void setCipherProvider(AESKeyedCipherProvider cipherProvider) {
    +        this.aesKeyedCipherProvider = cipherProvider;
    +    }
    +
    +    /**
    +     * Encrypts the provided {@link ProvenanceEventRecord}, serialized to a byte[] by the RecordWriter.
    +     *
    +     * @param plainRecord the plain record, serialized to a byte[]
    +     * @param recordId    an identifier for this record (eventId, generated, etc.)
    +     * @param keyId       the ID of the key to use
    +     * @return the encrypted record
    +     * @throws EncryptionException if there is an issue encrypting this record
    +     */
    +    @Override
    +    public byte[] encrypt(byte[] plainRecord, String recordId, String keyId) throws EncryptionException {
    +        if (plainRecord == null || CryptoUtils.isEmpty(keyId)) {
    +            throw new EncryptionException("The provenance record and key ID cannot be missing");
    +        }
    +
    +        if (keyProvider == null || !keyProvider.keyExists(keyId)) {
    +            throw new EncryptionException("The requested key ID is not available");
    +        } else {
    +            byte[] ivBytes = new byte[IV_LENGTH];
    +            new SecureRandom().nextBytes(ivBytes);
    +            try {
    +                logger.debug("Encrypting provenance record " + recordId + " with key ID " + keyId);
    +                Cipher cipher = initCipher(EncryptionMethod.AES_GCM, Cipher.ENCRYPT_MODE, keyProvider.getKey(keyId), ivBytes);
    +                ivBytes = cipher.getIV();
    +
    +                // Perform the actual encryption
    +                byte[] cipherBytes = cipher.doFinal(plainRecord);
    +
    +                // Serialize and concat encryption details fields (keyId, algo, IV, version, CB length) outside of encryption
    +                EncryptionMetadata metadata = new EncryptionMetadata(keyId, ALGORITHM, ivBytes, VERSION, cipherBytes.length);
    +                byte[] serializedEncryptionMetadata = serializeEncryptionMetadata(metadata);
    +
    +                // Add the sentinel byte of 0x01
    +                logger.debug("Encrypted provenance event record " + recordId + " with key ID " + keyId);
    +                return CryptoUtils.concatByteArrays(SENTINEL, serializedEncryptionMetadata, cipherBytes);
    +            } catch (EncryptionException | BadPaddingException | IllegalBlockSizeException | IOException | KeyManagementException e) {
    +                final String msg = "Encountered an exception encrypting provenance record " + recordId;
    +                logger.error(msg, e);
    +                throw new EncryptionException(msg, e);
    +            }
    +        }
    +    }
    +
    +    private byte[] serializeEncryptionMetadata(EncryptionMetadata metadata) throws IOException {
    +        final ByteArrayOutputStream baos = new ByteArrayOutputStream();
    +        ObjectOutputStream outputStream = new ObjectOutputStream(baos);
    +        outputStream.writeObject(metadata);
    +        outputStream.close();
    +        return baos.toByteArray();
    +    }
    +
    +    private Cipher initCipher(EncryptionMethod method, int mode, SecretKey key, byte[] ivBytes) throws EncryptionException {
    +        try {
    +            if (method == null || key == null || ivBytes == null) {
    +                throw new IllegalArgumentException("Missing critical information");
    +            }
    +            return aesKeyedCipherProvider.getCipher(method, key, ivBytes, mode == Cipher.ENCRYPT_MODE);
    +        } catch (Exception e) {
    +            logger.error("Encountered an exception initializing the cipher", e);
    +            throw new EncryptionException(e);
    +        }
    +    }
    +
    +    /**
    +     * Decrypts the provided byte[] (an encrypted record with accompanying metadata).
    +     *
    +     * @param encryptedRecord the encrypted record in byte[] form
    +     * @param recordId        an identifier for this record (eventId, generated, etc.)
    +     * @return the decrypted record
    +     * @throws EncryptionException if there is an issue decrypting this record
    +     */
    +    @Override
    +    public byte[] decrypt(byte[] encryptedRecord, String recordId) throws EncryptionException {
    +        if (encryptedRecord == null) {
    +            throw new EncryptionException("The encrypted provenance record cannot be missing");
    +        }
    +
    +        EncryptionMetadata metadata;
    +        try {
    +            metadata = extractEncryptionMetadata(encryptedRecord);
    +        } catch (IOException | ClassNotFoundException e) {
    +            final String msg = "Encountered an error reading the encryption metadata: ";
    +            logger.error(msg, e);
    +            throw new EncryptionException(msg, e);
    +        }
    +
    +        if (!SUPPORTED_VERSIONS.contains(metadata.version)) {
    +            throw new EncryptionException("The event was encrypted with version " + metadata.version + " which is not in the list of supported versions " + StringUtils.join(SUPPORTED_VERSIONS, ","));
    +        }
    +
    +        // TODO: Actually use the version to determine schema, etc.
    +
    +        if (keyProvider == null || !keyProvider.keyExists(metadata.keyId) || CryptoUtils.isEmpty(metadata.keyId)) {
    +            throw new EncryptionException("The requested key ID " + metadata.keyId + " is not available");
    +        } else {
    +            try {
    +                logger.debug("Decrypting provenance record " + recordId + " with key ID " + metadata.keyId);
    +                EncryptionMethod method = EncryptionMethod.forAlgorithm(metadata.algorithm);
    +                Cipher cipher = initCipher(method, Cipher.DECRYPT_MODE, keyProvider.getKey(metadata.keyId), metadata.ivBytes);
    +
    +                // Strip the metadata away to get just the cipher bytes
    +                byte[] cipherBytes = extractCipherBytes(encryptedRecord, metadata);
    +
    +                // Perform the actual decryption
    +                byte[] plainBytes = cipher.doFinal(cipherBytes);
    +
    +                logger.debug("Decrypted provenance event record " + recordId + " with key ID " + metadata.keyId);
    +                return plainBytes;
    +            } catch (EncryptionException | BadPaddingException | IllegalBlockSizeException | KeyManagementException e) {
    +                final String msg = "Encountered an exception decrypting provenance record " + recordId;
    +                logger.error(msg, e);
    +                throw new EncryptionException(msg, e);
    +            }
    +        }
    +    }
    +
    +    /**
    +     * Returns a valid key identifier for this encryptor (valid for encryption and decryption) or throws an exception if none are available.
    +     *
    +     * @return the key ID
    +     * @throws KeyManagementException if no available key IDs are valid for both operations
    +     */
    +    @Override
    +    public String getNextKeyId() throws KeyManagementException {
    +        if (keyProvider != null) {
    +            List<String> availableKeyIds = keyProvider.getAvailableKeyIds();
    +            if (!availableKeyIds.isEmpty()) {
    +                return availableKeyIds.get(0);
    +            }
    +        }
    +        throw new KeyManagementException("No available key IDs");
    +    }
    +
    +    private EncryptionMetadata extractEncryptionMetadata(byte[] encryptedRecord) throws EncryptionException, IOException, ClassNotFoundException {
    +        if (encryptedRecord == null || encryptedRecord.length < MIN_METADATA_LENGTH) {
    +            throw new EncryptionException("The encrypted record is too short to contain the metadata");
    +        }
    +
    +        // Skip the first byte (SENTINEL) and don't need to copy all the serialized record
    +        byte[] metadataBytes = Arrays.copyOfRange(encryptedRecord, 1, encryptedRecord.length);
    --- End diff --
    
    Rather than copying the byte[] here and then wrapping in a ByteArrayInputStream, would recommend we instead just wrap the original byte[] in a ByteArrayInputStream, then call in.read() to discard the first byte -- avoids a good bit of garbage creation/collection.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1686: NIFI-3594 Encrypted provenance repository implementation

Posted by YolandaMDavis <gi...@git.apache.org>.
Github user YolandaMDavis commented on the issue:

    https://github.com/apache/nifi/pull/1686
  
    @alopresto Ran a test where I had an existing unencrypted provenance repo and switched to the encrypted provider.  The issues [you've documented](https://github.com/alopresto/nifi/blob/276f31aa85d9ec401fb34f939238c6646e187ae6/nifi-docs/src/main/asciidoc/user-guide.adoc#potential-issues) did occur (where switching between configuration would lead to errors in the log and when querying).  However I'm wondering two things:
    
    1) Can there be a friendlier error message in the logs for those errors? I saw lots of errors around ClassCastException between the old and new Record Readers. Just wondering if NiFi could simply say detected a mismatch and that it could be due to a change in repo providers?
    
    2) I also saw the UI error (see below), however I expected to eventually be able to retrieve the events that were subject to encryption. I have a super short and small flow that runs every 10s. Perhaps provenance hasn't been completely overwritten with the newer encrypted files in my case but each time I attempt to query provenance I get the ClassCastException.  
    
    ![image](https://cloud.githubusercontent.com/assets/1371858/25546889/efea31b6-2c32-11e7-9a98-ac78cc95c1bb.png)
    
    ![image](https://cloud.githubusercontent.com/assets/1371858/25546896/f597cad8-2c32-11e7-8502-18d9af37db32.png)
    
    If clearing the provenance repo when changing configurations would resolve the issue can that step be included in the documentation (if needed I would recommend including that step in the issues section)?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1686: NIFI-3594 Encrypted provenance repository implement...

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1686#discussion_r112966396
  
    --- Diff: nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/groovy/org/apache/nifi/provenance/EncryptedWriteAheadProvenanceRepositoryTest.groovy ---
    @@ -0,0 +1,395 @@
    +/*
    + * 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.provenance
    +
    +import org.apache.nifi.events.EventReporter
    +import org.apache.nifi.flowfile.FlowFile
    +import org.apache.nifi.provenance.serialization.RecordReaders
    +import org.apache.nifi.reporting.Severity
    +import org.apache.nifi.util.file.FileUtils
    +import org.bouncycastle.jce.provider.BouncyCastleProvider
    +import org.junit.After
    +import org.junit.AfterClass
    +import org.junit.Before
    +import org.junit.BeforeClass
    +import org.junit.ClassRule
    +import org.junit.Test
    +import org.junit.rules.TemporaryFolder
    +import org.junit.runner.RunWith
    +import org.junit.runners.JUnit4
    +import org.slf4j.Logger
    +import org.slf4j.LoggerFactory
    +
    +import javax.crypto.Cipher
    +import java.security.Security
    +import java.util.concurrent.TimeUnit
    +import java.util.concurrent.atomic.AtomicLong
    +
    +import static org.apache.nifi.provenance.TestUtil.createFlowFile
    +
    +@RunWith(JUnit4.class)
    +class EncryptedWriteAheadProvenanceRepositoryTest {
    +    private static final Logger logger = LoggerFactory.getLogger(EncryptedWriteAheadProvenanceRepositoryTest.class)
    +
    +    private static final String KEY_HEX_128 = "0123456789ABCDEFFEDCBA9876543210"
    +    private static final String KEY_HEX_256 = KEY_HEX_128 * 2
    +    private static final String KEY_HEX = isUnlimitedStrengthCryptoAvailable() ? KEY_HEX_256 : KEY_HEX_128
    +    private static final int IV_LENGTH = 16
    +    private static final String KEY_ID = "K1"
    +
    +    private static final String TRANSIT_URI = "nifi://unit-test"
    +    private static final String PROCESSOR_TYPE = "Mock Processor"
    +    private static final String COMPONENT_ID = "1234"
    +
    +    private static final AtomicLong recordId = new AtomicLong()
    +
    +//    @Rule
    --- End diff --
    
    There are a handful of places here where lines of code are commented out. Should probably remove.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1686: NIFI-3594 Encrypted provenance repository implement...

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1686#discussion_r113019393
  
    --- Diff: nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/CryptoUtils.java ---
    @@ -0,0 +1,228 @@
    +/*
    + * 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.provenance;
    +
    +import java.io.BufferedReader;
    +import java.io.ByteArrayOutputStream;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.security.KeyManagementException;
    +import java.security.NoSuchAlgorithmException;
    +import java.util.Arrays;
    +import java.util.Base64;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.regex.Pattern;
    +import javax.crypto.BadPaddingException;
    +import javax.crypto.Cipher;
    +import javax.crypto.IllegalBlockSizeException;
    +import javax.crypto.SecretKey;
    +import javax.crypto.spec.SecretKeySpec;
    +import org.apache.commons.lang3.StringUtils;
    +import org.apache.nifi.security.util.EncryptionMethod;
    +import org.apache.nifi.security.util.crypto.AESKeyedCipherProvider;
    +import org.apache.nifi.util.NiFiProperties;
    +import org.bouncycastle.util.encoders.Hex;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class CryptoUtils {
    +    private static final Logger logger = LoggerFactory.getLogger(StaticKeyProvider.class);
    +    private static final String STATIC_KEY_PROVIDER_CLASS_NAME = "org.apache.nifi.provenance.StaticKeyProvider";
    +    private static final String FILE_BASED_KEY_PROVIDER_CLASS_NAME = "org.apache.nifi.provenance.FileBasedKeyProvider";
    +    private static final Pattern HEX_PATTERN = Pattern.compile("(?i)^[0-9a-f]+$");
    +
    +    public static final int IV_LENGTH = 16;
    +
    +    public static boolean isUnlimitedStrengthCryptoAvailable() {
    +        try {
    +            return Cipher.getMaxAllowedKeyLength("AES") > 128;
    +        } catch (NoSuchAlgorithmException e) {
    +            logger.warn("Tried to determine if unlimited strength crypto is available but the AES algorithm is not available");
    +            return false;
    +        }
    +    }
    +
    +    /**
    +     * Utility method which returns true if the string is null, empty, or entirely whitespace.
    +     *
    +     * @param src the string to evaluate
    +     * @return true if empty
    +     */
    +    public static boolean isEmpty(String src) {
    +        return src == null || src.trim().isEmpty();
    +    }
    +
    +    /**
    +     * Concatenates multiple byte[] into a single byte[]. Because it uses a {@link ByteArrayOutputStream}
    +     * rather than {@link System#arraycopy(Object, int, Object, int, int)} the performance is much better
    +     * with an arbitrary number of input byte[]s.
    +     *
    +     * @param arrays the component byte[] in order
    +     * @return a concatenated byte[]
    +     * @throws IOException this should never be thrown
    +     */
    +    public static byte[] concatByteArrays(byte[]... arrays) throws IOException {
    +        ByteArrayOutputStream boas = new ByteArrayOutputStream();
    --- End diff --
    
    So with 100 iterations, you're likely getting a lot of 'jvm warmup time' into your calculations... i'd go for at least 10,000 iterations as a 'warm up' that aren't even counted. Then probably 100,000 - 1 MM iterations to test actual performance...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1686: NIFI-3594 Encrypted provenance repository implement...

Posted by alopresto <gi...@git.apache.org>.
Github user alopresto commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1686#discussion_r113014671
  
    --- Diff: nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/CryptoUtils.java ---
    @@ -0,0 +1,228 @@
    +/*
    + * 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.provenance;
    +
    +import java.io.BufferedReader;
    +import java.io.ByteArrayOutputStream;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.security.KeyManagementException;
    +import java.security.NoSuchAlgorithmException;
    +import java.util.Arrays;
    +import java.util.Base64;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.regex.Pattern;
    +import javax.crypto.BadPaddingException;
    +import javax.crypto.Cipher;
    +import javax.crypto.IllegalBlockSizeException;
    +import javax.crypto.SecretKey;
    +import javax.crypto.spec.SecretKeySpec;
    +import org.apache.commons.lang3.StringUtils;
    +import org.apache.nifi.security.util.EncryptionMethod;
    +import org.apache.nifi.security.util.crypto.AESKeyedCipherProvider;
    +import org.apache.nifi.util.NiFiProperties;
    +import org.bouncycastle.util.encoders.Hex;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class CryptoUtils {
    +    private static final Logger logger = LoggerFactory.getLogger(StaticKeyProvider.class);
    +    private static final String STATIC_KEY_PROVIDER_CLASS_NAME = "org.apache.nifi.provenance.StaticKeyProvider";
    +    private static final String FILE_BASED_KEY_PROVIDER_CLASS_NAME = "org.apache.nifi.provenance.FileBasedKeyProvider";
    +    private static final Pattern HEX_PATTERN = Pattern.compile("(?i)^[0-9a-f]+$");
    +
    +    public static final int IV_LENGTH = 16;
    +
    +    public static boolean isUnlimitedStrengthCryptoAvailable() {
    +        try {
    +            return Cipher.getMaxAllowedKeyLength("AES") > 128;
    +        } catch (NoSuchAlgorithmException e) {
    +            logger.warn("Tried to determine if unlimited strength crypto is available but the AES algorithm is not available");
    +            return false;
    +        }
    +    }
    +
    +    /**
    +     * Utility method which returns true if the string is null, empty, or entirely whitespace.
    +     *
    +     * @param src the string to evaluate
    +     * @return true if empty
    +     */
    +    public static boolean isEmpty(String src) {
    +        return src == null || src.trim().isEmpty();
    +    }
    +
    +    /**
    +     * Concatenates multiple byte[] into a single byte[]. Because it uses a {@link ByteArrayOutputStream}
    +     * rather than {@link System#arraycopy(Object, int, Object, int, int)} the performance is much better
    +     * with an arbitrary number of input byte[]s.
    +     *
    +     * @param arrays the component byte[] in order
    +     * @return a concatenated byte[]
    +     * @throws IOException this should never be thrown
    +     */
    +    public static byte[] concatByteArrays(byte[]... arrays) throws IOException {
    +        ByteArrayOutputStream boas = new ByteArrayOutputStream();
    --- End diff --
    
    I was curious about this, so I added an implementation as you described above and compared them. The BAOS is definitely faster for small byte[], but slower for large ones. Interesting. 
    
    ```java
      public static byte[] concatByteArrays(byte[]... arrays) throws IOException {
            int totalByteLength = 0;
            for (byte[] bytes : arrays) {
                totalByteLength += bytes.length;
            }
            byte[] totalBytes = new byte[totalByteLength];
            int currentLength = 0;
            for (byte[] bytes : arrays) {
                System.arraycopy(bytes, 0, totalBytes, currentLength, bytes.length);
                currentLength += bytes.length;
            }
            return totalBytes;
        }
    
        public static byte[] concatByteArraysWithBAOS(byte[]... arrays) throws IOException {
            ByteArrayOutputStream boas = new ByteArrayOutputStream();
            for (byte[] arr : arrays) {
                boas.write(arr);
            }
            return boas.toByteArray();
        }
    ```
    
    ```
    Calculating small/small -- 3 arrays with avg length 11
    Ran 100 of small/small (traditional) with a total wall time of 28720012 ns and average run of 195354 ns
    Ran 100 of small/small (BAOS) with a total wall time of 6745072 ns and average run of 12300 ns
    Calculating small/large -- 2 arrays with avg length 567
    Ran 100 of small/large (traditional) with a total wall time of 2712642 ns and average run of 5807 ns
    Ran 100 of small/large (BAOS) with a total wall time of 3774826 ns and average run of 11078 ns
    Calculating large/small -- 145 arrays with avg length 8
    Ran 100 of large/small (traditional) with a total wall time of 5173622 ns and average run of 27214 ns
    Ran 100 of large/small (BAOS) with a total wall time of 5120322 ns and average run of 27663 ns
    Calculating large/large -- 182 arrays with avg length 534
    Ran 100 of large/large (traditional) with a total wall time of 11537912 ns and average run of 83769 ns
    Ran 100 of large/large (BAOS) with a total wall time of 65845017 ns and average run of 612915 ns
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1686: NIFI-3594 Encrypted provenance repository implement...

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1686#discussion_r112957440
  
    --- Diff: nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/CryptoUtils.java ---
    @@ -0,0 +1,228 @@
    +/*
    + * 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.provenance;
    +
    +import java.io.BufferedReader;
    +import java.io.ByteArrayOutputStream;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.security.KeyManagementException;
    +import java.security.NoSuchAlgorithmException;
    +import java.util.Arrays;
    +import java.util.Base64;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.regex.Pattern;
    +import javax.crypto.BadPaddingException;
    +import javax.crypto.Cipher;
    +import javax.crypto.IllegalBlockSizeException;
    +import javax.crypto.SecretKey;
    +import javax.crypto.spec.SecretKeySpec;
    +import org.apache.commons.lang3.StringUtils;
    +import org.apache.nifi.security.util.EncryptionMethod;
    +import org.apache.nifi.security.util.crypto.AESKeyedCipherProvider;
    +import org.apache.nifi.util.NiFiProperties;
    +import org.bouncycastle.util.encoders.Hex;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class CryptoUtils {
    +    private static final Logger logger = LoggerFactory.getLogger(StaticKeyProvider.class);
    +    private static final String STATIC_KEY_PROVIDER_CLASS_NAME = "org.apache.nifi.provenance.StaticKeyProvider";
    +    private static final String FILE_BASED_KEY_PROVIDER_CLASS_NAME = "org.apache.nifi.provenance.FileBasedKeyProvider";
    +    private static final Pattern HEX_PATTERN = Pattern.compile("(?i)^[0-9a-f]+$");
    +
    +    public static final int IV_LENGTH = 16;
    +
    +    public static boolean isUnlimitedStrengthCryptoAvailable() {
    +        try {
    +            return Cipher.getMaxAllowedKeyLength("AES") > 128;
    +        } catch (NoSuchAlgorithmException e) {
    +            logger.warn("Tried to determine if unlimited strength crypto is available but the AES algorithm is not available");
    +            return false;
    +        }
    +    }
    +
    +    /**
    +     * Utility method which returns true if the string is null, empty, or entirely whitespace.
    +     *
    +     * @param src the string to evaluate
    +     * @return true if empty
    +     */
    +    public static boolean isEmpty(String src) {
    +        return src == null || src.trim().isEmpty();
    +    }
    +
    +    /**
    +     * Concatenates multiple byte[] into a single byte[]. Because it uses a {@link ByteArrayOutputStream}
    +     * rather than {@link System#arraycopy(Object, int, Object, int, int)} the performance is much better
    +     * with an arbitrary number of input byte[]s.
    +     *
    +     * @param arrays the component byte[] in order
    +     * @return a concatenated byte[]
    +     * @throws IOException this should never be thrown
    +     */
    +    public static byte[] concatByteArrays(byte[]... arrays) throws IOException {
    +        ByteArrayOutputStream boas = new ByteArrayOutputStream();
    --- End diff --
    
    We're doing quite a lot of byte copying here, as the BAOS is potentially resizing a lot... then we call toByteArray() to copy it again. Would recommend calculating size of the arrays and then just creating an array of that size and copying the bytes directly. Alternatively, you could calculate the size of the arrays and pass that as an argument to BAOS so that it doesn't generate so much garbage - but this still has to copy the byte[] when baos.toByteArray() is called.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1686: NIFI-3594 Encrypted provenance repository implement...

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1686#discussion_r112958054
  
    --- Diff: nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/CryptoUtils.java ---
    @@ -0,0 +1,228 @@
    +/*
    + * 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.provenance;
    +
    +import java.io.BufferedReader;
    +import java.io.ByteArrayOutputStream;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.security.KeyManagementException;
    +import java.security.NoSuchAlgorithmException;
    +import java.util.Arrays;
    +import java.util.Base64;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.regex.Pattern;
    +import javax.crypto.BadPaddingException;
    +import javax.crypto.Cipher;
    +import javax.crypto.IllegalBlockSizeException;
    +import javax.crypto.SecretKey;
    +import javax.crypto.spec.SecretKeySpec;
    +import org.apache.commons.lang3.StringUtils;
    +import org.apache.nifi.security.util.EncryptionMethod;
    +import org.apache.nifi.security.util.crypto.AESKeyedCipherProvider;
    +import org.apache.nifi.util.NiFiProperties;
    +import org.bouncycastle.util.encoders.Hex;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class CryptoUtils {
    +    private static final Logger logger = LoggerFactory.getLogger(StaticKeyProvider.class);
    +    private static final String STATIC_KEY_PROVIDER_CLASS_NAME = "org.apache.nifi.provenance.StaticKeyProvider";
    +    private static final String FILE_BASED_KEY_PROVIDER_CLASS_NAME = "org.apache.nifi.provenance.FileBasedKeyProvider";
    +    private static final Pattern HEX_PATTERN = Pattern.compile("(?i)^[0-9a-f]+$");
    +
    +    public static final int IV_LENGTH = 16;
    +
    +    public static boolean isUnlimitedStrengthCryptoAvailable() {
    +        try {
    +            return Cipher.getMaxAllowedKeyLength("AES") > 128;
    +        } catch (NoSuchAlgorithmException e) {
    +            logger.warn("Tried to determine if unlimited strength crypto is available but the AES algorithm is not available");
    +            return false;
    +        }
    +    }
    +
    +    /**
    +     * Utility method which returns true if the string is null, empty, or entirely whitespace.
    +     *
    +     * @param src the string to evaluate
    +     * @return true if empty
    +     */
    +    public static boolean isEmpty(String src) {
    +        return src == null || src.trim().isEmpty();
    +    }
    +
    +    /**
    +     * Concatenates multiple byte[] into a single byte[]. Because it uses a {@link ByteArrayOutputStream}
    +     * rather than {@link System#arraycopy(Object, int, Object, int, int)} the performance is much better
    +     * with an arbitrary number of input byte[]s.
    +     *
    +     * @param arrays the component byte[] in order
    +     * @return a concatenated byte[]
    +     * @throws IOException this should never be thrown
    +     */
    +    public static byte[] concatByteArrays(byte[]... arrays) throws IOException {
    +        ByteArrayOutputStream boas = new ByteArrayOutputStream();
    +        for (byte[] arr : arrays) {
    +            boas.write(arr);
    +        }
    +        return boas.toByteArray();
    +    }
    +
    +    public static boolean isValidKeyProvider(String keyProviderImplementation, String keyProviderLocation, String keyId, String encryptionKeyHex) {
    +        if (STATIC_KEY_PROVIDER_CLASS_NAME.equals(keyProviderImplementation)) {
    +            // Ensure the keyId and key are valid
    +            return keyIsValid(encryptionKeyHex) && StringUtils.isNotEmpty(keyId);
    +        } else if (FILE_BASED_KEY_PROVIDER_CLASS_NAME.equals(keyProviderImplementation)) {
    +            // Ensure the file can be read and the keyId is populated (does not read file to validate)
    +            final File kpf = new File(keyProviderLocation);
    +            return kpf.exists() && kpf.canRead() && StringUtils.isNotEmpty(keyId);
    +        } else {
    +            logger.error("The attempt to validate the key provider failed keyProviderImplementation = "
    +                    + keyProviderImplementation + " , keyProviderLocation = "
    +                    + keyProviderLocation + " , keyId = "
    +                    + keyId + " , encryptedKeyHex = "
    +                    + (StringUtils.isNotEmpty(encryptionKeyHex) ? "********" : ""));
    +
    +            return false;
    +        }
    +    }
    +
    +    /**
    +     * Returns true if the provided key is valid hex and is the correct length for the current system's JCE policies.
    +     *
    +     * @param encryptionKeyHex the key in hexadecimal
    +     * @return true if this key is valid
    +     */
    +    public static boolean keyIsValid(String encryptionKeyHex) {
    +        return isHexString(encryptionKeyHex)
    +                && (isUnlimitedStrengthCryptoAvailable()
    +                ? Arrays.asList(32, 48, 64).contains(encryptionKeyHex.length())
    --- End diff --
    
    This is called a lot - would recommend removing the Arrays.asList(32, 48, 64) and just creating a private static final List or Set, or since there are only 3 possible values, checking them individually


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1686: NIFI-3594 Encrypted provenance repository implement...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/nifi/pull/1686


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1686: NIFI-3594 Encrypted provenance repository implement...

Posted by alopresto <gi...@git.apache.org>.
Github user alopresto commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1686#discussion_r113076158
  
    --- Diff: nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/groovy/org/apache/nifi/provenance/EncryptedWriteAheadProvenanceRepositoryTest.groovy ---
    @@ -0,0 +1,395 @@
    +/*
    + * 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.provenance
    +
    +import org.apache.nifi.events.EventReporter
    +import org.apache.nifi.flowfile.FlowFile
    +import org.apache.nifi.provenance.serialization.RecordReaders
    +import org.apache.nifi.reporting.Severity
    +import org.apache.nifi.util.file.FileUtils
    +import org.bouncycastle.jce.provider.BouncyCastleProvider
    +import org.junit.After
    +import org.junit.AfterClass
    +import org.junit.Before
    +import org.junit.BeforeClass
    +import org.junit.ClassRule
    +import org.junit.Test
    +import org.junit.rules.TemporaryFolder
    +import org.junit.runner.RunWith
    +import org.junit.runners.JUnit4
    +import org.slf4j.Logger
    +import org.slf4j.LoggerFactory
    +
    +import javax.crypto.Cipher
    +import java.security.Security
    +import java.util.concurrent.TimeUnit
    +import java.util.concurrent.atomic.AtomicLong
    +
    +import static org.apache.nifi.provenance.TestUtil.createFlowFile
    +
    +@RunWith(JUnit4.class)
    +class EncryptedWriteAheadProvenanceRepositoryTest {
    +    private static final Logger logger = LoggerFactory.getLogger(EncryptedWriteAheadProvenanceRepositoryTest.class)
    +
    +    private static final String KEY_HEX_128 = "0123456789ABCDEFFEDCBA9876543210"
    +    private static final String KEY_HEX_256 = KEY_HEX_128 * 2
    +    private static final String KEY_HEX = isUnlimitedStrengthCryptoAvailable() ? KEY_HEX_256 : KEY_HEX_128
    +    private static final int IV_LENGTH = 16
    +    private static final String KEY_ID = "K1"
    +
    +    private static final String TRANSIT_URI = "nifi://unit-test"
    +    private static final String PROCESSOR_TYPE = "Mock Processor"
    +    private static final String COMPONENT_ID = "1234"
    +
    +    private static final AtomicLong recordId = new AtomicLong()
    +
    +//    @Rule
    --- End diff --
    
    I cleaned up the unnecessary ones. I think the ones that have the note about [NIFI-3605](https://issues.apache.org/jira/browse/NIFI-3605) can stay because they have the correct code commented out to be switched back when that bug is patched. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1686: NIFI-3594 Encrypted provenance repository implement...

Posted by alopresto <gi...@git.apache.org>.
Github user alopresto commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1686#discussion_r113075612
  
    --- Diff: nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/EncryptedSchemaRecordReader.java ---
    @@ -0,0 +1,154 @@
    +/*
    + * 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.provenance;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.File;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.util.Collection;
    +import java.util.Optional;
    +import java.util.concurrent.TimeUnit;
    +import org.apache.nifi.provenance.schema.LookupTableEventRecord;
    +import org.apache.nifi.provenance.toc.TocReader;
    +import org.apache.nifi.repository.schema.Record;
    +import org.apache.nifi.stream.io.LimitingInputStream;
    +import org.apache.nifi.stream.io.StreamUtils;
    +import org.apache.nifi.util.timebuffer.LongEntityAccess;
    +import org.apache.nifi.util.timebuffer.TimedBuffer;
    +import org.apache.nifi.util.timebuffer.TimestampedLong;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class EncryptedSchemaRecordReader extends EventIdFirstSchemaRecordReader {
    +    private static final Logger logger = LoggerFactory.getLogger(EncryptedSchemaRecordReader.class);
    +
    +    private static final int DEFAULT_DEBUG_FREQUENCY = 1_000_000;
    +
    +    private ProvenanceEventEncryptor provenanceEventEncryptor;
    +
    +    private static final TimedBuffer<TimestampedLong> decryptTimes = new TimedBuffer<>(TimeUnit.SECONDS, 60, new LongEntityAccess());
    +
    +    private int debugFrequency = DEFAULT_DEBUG_FREQUENCY;
    +    public static final int SERIALIZATION_VERSION = 1;
    +
    +    public static final String SERIALIZATION_NAME = "EncryptedSchemaRecordWriter";
    +
    +    public EncryptedSchemaRecordReader(final InputStream inputStream, final String filename, final TocReader tocReader, final int maxAttributeChars,
    +                                       ProvenanceEventEncryptor provenanceEventEncryptor) throws IOException {
    +        this(inputStream, filename, tocReader, maxAttributeChars, provenanceEventEncryptor, DEFAULT_DEBUG_FREQUENCY);
    +    }
    +
    +    public EncryptedSchemaRecordReader(final InputStream inputStream, final String filename, final TocReader tocReader, final int maxAttributeChars,
    +                                       ProvenanceEventEncryptor provenanceEventEncryptor, int debugFrequency) throws IOException {
    +        super(inputStream, filename, tocReader, maxAttributeChars);
    +        this.provenanceEventEncryptor = provenanceEventEncryptor;
    +        this.debugFrequency = debugFrequency;
    +    }
    +
    +    @Override
    +    protected StandardProvenanceEventRecord nextRecord(final DataInputStream in, final int serializationVersion) throws IOException {
    +        verifySerializationVersion(serializationVersion);
    +
    +        final long byteOffset = getBytesConsumed();
    +        final long eventId = in.readInt() + getFirstEventId();
    +        final int recordLength = in.readInt();
    +
    +        return readRecord(in, eventId, byteOffset, recordLength);
    +    }
    +
    +    private StandardProvenanceEventRecord readRecord(final DataInputStream inputStream, final long eventId, final long startOffset, final int recordLength) throws IOException {
    +        try {
    +            final InputStream limitedIn = new LimitingInputStream(inputStream, recordLength);
    +
    +            byte[] encryptedSerializedBytes = new byte[recordLength];
    +            DataInputStream encryptedInputStream = new DataInputStream(limitedIn);
    +            encryptedInputStream.readFully(encryptedSerializedBytes);
    +
    +            byte[] plainSerializedBytes = decrypt(encryptedSerializedBytes, Long.toString(eventId));
    +            InputStream plainStream = new ByteArrayInputStream(plainSerializedBytes);
    +
    +            final Record eventRecord = getRecordReader().readRecord(plainStream);
    +            if (eventRecord == null) {
    +                return null;
    +            }
    +
    +            final StandardProvenanceEventRecord deserializedEvent = LookupTableEventRecord.getEvent(eventRecord, getFilename(), startOffset, getMaxAttributeLength(),
    +                    getFirstEventId(), getSystemTimeOffset(), getComponentIds(), getComponentTypes(), getQueueIds(), getEventTypes());
    +            deserializedEvent.setEventId(eventId);
    +            return deserializedEvent;
    +        } catch (EncryptionException e) {
    +            logger.error("Encountered an error reading the record: ", e);
    +            throw new IOException(e);
    +        }
    +    }
    +
    +    // TODO: Copied from EventIdFirstSchemaRecordReader to force local/overridden readRecord()
    --- End diff --
    
    This was a note to myself/any reviewer about why I copied the entire method body from the other reader implementation. If I just referred to the other instance (i.e. comment out this entire method declaration), the operation will fail (see `EncryptedSchemaRecordReaderWriterTest#testSkipToEvent()`) even though the code is identical. If there is a better suggestion for how to fix this, I am all ears. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1686: NIFI-3594 Encrypted provenance repository implement...

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1686#discussion_r112965910
  
    --- Diff: nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/groovy/org/apache/nifi/provenance/EncryptedSchemaRecordReaderWriterTest.groovy ---
    @@ -0,0 +1,286 @@
    +/*
    + * 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.provenance
    +
    +import org.apache.nifi.flowfile.FlowFile
    +import org.apache.nifi.provenance.serialization.RecordReader
    +import org.apache.nifi.provenance.serialization.RecordWriter
    +import org.apache.nifi.provenance.toc.StandardTocReader
    +import org.apache.nifi.provenance.toc.StandardTocWriter
    +import org.apache.nifi.provenance.toc.TocReader
    +import org.apache.nifi.provenance.toc.TocUtil
    +import org.apache.nifi.provenance.toc.TocWriter
    +import org.apache.nifi.util.file.FileUtils
    +import org.bouncycastle.jce.provider.BouncyCastleProvider
    +import org.bouncycastle.util.encoders.Hex
    +import org.junit.After
    +import org.junit.AfterClass
    +import org.junit.Before
    +import org.junit.BeforeClass
    +import org.junit.ClassRule
    +import org.junit.Test
    +import org.junit.rules.TemporaryFolder
    +import org.junit.runner.RunWith
    +import org.junit.runners.JUnit4
    +import org.slf4j.Logger
    +import org.slf4j.LoggerFactory
    +
    +import javax.crypto.Cipher
    +import javax.crypto.spec.SecretKeySpec
    +import java.security.KeyManagementException
    +import java.security.Security
    +import java.util.concurrent.atomic.AtomicLong
    +
    +import static groovy.test.GroovyAssert.shouldFail
    +import static org.apache.nifi.provenance.TestUtil.createFlowFile
    +
    +@RunWith(JUnit4.class)
    +class EncryptedSchemaRecordReaderWriterTest extends AbstractTestRecordReaderWriter {
    +    private static final Logger logger = LoggerFactory.getLogger(EncryptedSchemaRecordReaderWriterTest.class)
    +
    +    private static final String KEY_HEX_128 = "0123456789ABCDEFFEDCBA9876543210"
    +    private static final String KEY_HEX_256 = KEY_HEX_128 * 2
    +    private static final String KEY_HEX = isUnlimitedStrengthCryptoAvailable() ? KEY_HEX_256 : KEY_HEX_128
    +    private static final int IV_LENGTH = 16
    +    private static final String KEY_ID = "K1"
    +
    +    private static final String TRANSIT_URI = "nifi://unit-test"
    +    private static final String PROCESSOR_TYPE = "Mock Processor"
    +    private static final String COMPONENT_ID = "1234"
    +
    +    private static final int UNCOMPRESSED_BLOCK_SIZE = 1024 * 32
    +    private static final int MAX_ATTRIBUTE_SIZE = 2048
    +
    +    private static final AtomicLong idGenerator = new AtomicLong(0L)
    +    private File journalFile
    +    private File tocFile
    +
    +    private static KeyProvider mockKeyProvider
    +    private static ProvenanceEventEncryptor provenanceEventEncryptor = new AESProvenanceEventEncryptor()
    +
    +    @ClassRule
    +    public static TemporaryFolder tempFolder = new TemporaryFolder()
    +
    +    private static String ORIGINAL_LOG_LEVEL
    +
    +    @BeforeClass
    +    static void setUpOnce() throws Exception {
    +        ORIGINAL_LOG_LEVEL = System.getProperty("org.slf4j.simpleLogger.log.org.apache.nifi.provenance")
    +        System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi.provenance", "DEBUG")
    +
    +        Security.addProvider(new BouncyCastleProvider())
    +
    +        logger.metaClass.methodMissing = { String name, args ->
    +            logger.info("[${name?.toUpperCase()}] ${(args as List).join(" ")}")
    +        }
    +
    +        mockKeyProvider = [
    +                getKey            : { String keyId ->
    +                    logger.mock("Requesting key ID: ${keyId}")
    +                    if (keyId == KEY_ID) {
    +                        new SecretKeySpec(Hex.decode(KEY_HEX), "AES")
    +                    } else {
    +                        throw new KeyManagementException("${keyId} is not available")
    +                    }
    +                },
    +                getAvailableKeyIds: { ->
    +                    logger.mock("Available key IDs: [${KEY_ID}]")
    +                    [KEY_ID]
    +                },
    +                keyExists         : { String keyId ->
    +                    logger.mock("Checking availability of key ID: ${keyId}")
    +                    keyId == KEY_ID
    +                }] as KeyProvider
    +        provenanceEventEncryptor.initialize(mockKeyProvider)
    +//
    --- End diff --
    
    Should this be removed?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1686: NIFI-3594 Encrypted provenance repository implement...

Posted by alopresto <gi...@git.apache.org>.
Github user alopresto commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1686#discussion_r112980590
  
    --- Diff: nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/AESProvenanceEventEncryptor.java ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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.provenance;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectInputStream;
    +import java.io.ObjectOutputStream;
    +import java.security.KeyManagementException;
    +import java.security.SecureRandom;
    +import java.security.Security;
    +import java.util.Arrays;
    +import java.util.List;
    +import javax.crypto.BadPaddingException;
    +import javax.crypto.Cipher;
    +import javax.crypto.IllegalBlockSizeException;
    +import javax.crypto.SecretKey;
    +import org.apache.commons.lang3.StringUtils;
    +import org.apache.nifi.security.util.EncryptionMethod;
    +import org.apache.nifi.security.util.crypto.AESKeyedCipherProvider;
    +import org.bouncycastle.jce.provider.BouncyCastleProvider;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class AESProvenanceEventEncryptor implements ProvenanceEventEncryptor {
    +    private static final Logger logger = LoggerFactory.getLogger(AESProvenanceEventEncryptor.class);
    +    private static final String ALGORITHM = "AES/GCM/NoPadding";
    +    private static final int IV_LENGTH = 16;
    +    private static final byte[] EMPTY_IV = new byte[IV_LENGTH];
    +    private static final String VERSION = "v1";
    +    private static final List<String> SUPPORTED_VERSIONS = Arrays.asList(VERSION);
    +    private static final int MIN_METADATA_LENGTH = IV_LENGTH + 3 + 3; // 3 delimiters and 3 non-zero elements
    +    private static final int METADATA_DEFAULT_LENGTH = (20 + ALGORITHM.length() + IV_LENGTH + VERSION.length()) * 2; // Default to twice the expected length
    +    private static final byte[] SENTINEL = new byte[]{0x01};
    +
    +    private KeyProvider keyProvider;
    +
    +    private AESKeyedCipherProvider aesKeyedCipherProvider = new AESKeyedCipherProvider();
    +
    +    /**
    +     * Initializes the encryptor with a {@link KeyProvider}.
    +     *
    +     * @param keyProvider the key provider which will be responsible for accessing keys
    +     * @throws KeyManagementException if there is an issue configuring the key provider
    +     */
    +    @Override
    +    public void initialize(KeyProvider keyProvider) throws KeyManagementException {
    +        this.keyProvider = keyProvider;
    +
    +        if (this.aesKeyedCipherProvider == null) {
    +            this.aesKeyedCipherProvider = new AESKeyedCipherProvider();
    +        }
    +
    +        if (Security.getProvider("BC") == null) {
    +            Security.addProvider(new BouncyCastleProvider());
    +        }
    +    }
    +
    +    /**
    +     * Available for dependency injection to override the default {@link AESKeyedCipherProvider} if necessary.
    +     *
    +     * @param cipherProvider the AES cipher provider to use
    +     */
    +    void setCipherProvider(AESKeyedCipherProvider cipherProvider) {
    +        this.aesKeyedCipherProvider = cipherProvider;
    +    }
    +
    +    /**
    +     * Encrypts the provided {@link ProvenanceEventRecord}, serialized to a byte[] by the RecordWriter.
    +     *
    +     * @param plainRecord the plain record, serialized to a byte[]
    +     * @param recordId    an identifier for this record (eventId, generated, etc.)
    +     * @param keyId       the ID of the key to use
    +     * @return the encrypted record
    +     * @throws EncryptionException if there is an issue encrypting this record
    +     */
    +    @Override
    +    public byte[] encrypt(byte[] plainRecord, String recordId, String keyId) throws EncryptionException {
    +        if (plainRecord == null || CryptoUtils.isEmpty(keyId)) {
    +            throw new EncryptionException("The provenance record and key ID cannot be missing");
    +        }
    +
    +        if (keyProvider == null || !keyProvider.keyExists(keyId)) {
    +            throw new EncryptionException("The requested key ID is not available");
    +        } else {
    +            byte[] ivBytes = new byte[IV_LENGTH];
    +            new SecureRandom().nextBytes(ivBytes);
    +            try {
    +                logger.debug("Encrypting provenance record " + recordId + " with key ID " + keyId);
    +                Cipher cipher = initCipher(EncryptionMethod.AES_GCM, Cipher.ENCRYPT_MODE, keyProvider.getKey(keyId), ivBytes);
    +                ivBytes = cipher.getIV();
    +
    +                // Perform the actual encryption
    +                byte[] cipherBytes = cipher.doFinal(plainRecord);
    +
    +                // Serialize and concat encryption details fields (keyId, algo, IV, version, CB length) outside of encryption
    +                EncryptionMetadata metadata = new EncryptionMetadata(keyId, ALGORITHM, ivBytes, VERSION, cipherBytes.length);
    +                byte[] serializedEncryptionMetadata = serializeEncryptionMetadata(metadata);
    +
    +                // Add the sentinel byte of 0x01
    +                logger.debug("Encrypted provenance event record " + recordId + " with key ID " + keyId);
    +                return CryptoUtils.concatByteArrays(SENTINEL, serializedEncryptionMetadata, cipherBytes);
    +            } catch (EncryptionException | BadPaddingException | IllegalBlockSizeException | IOException | KeyManagementException e) {
    +                final String msg = "Encountered an exception encrypting provenance record " + recordId;
    +                logger.error(msg, e);
    +                throw new EncryptionException(msg, e);
    +            }
    +        }
    +    }
    +
    +    private byte[] serializeEncryptionMetadata(EncryptionMetadata metadata) throws IOException {
    +        final ByteArrayOutputStream baos = new ByteArrayOutputStream();
    +        ObjectOutputStream outputStream = new ObjectOutputStream(baos);
    +        outputStream.writeObject(metadata);
    +        outputStream.close();
    +        return baos.toByteArray();
    +    }
    +
    +    private Cipher initCipher(EncryptionMethod method, int mode, SecretKey key, byte[] ivBytes) throws EncryptionException {
    +        try {
    +            if (method == null || key == null || ivBytes == null) {
    +                throw new IllegalArgumentException("Missing critical information");
    +            }
    +            return aesKeyedCipherProvider.getCipher(method, key, ivBytes, mode == Cipher.ENCRYPT_MODE);
    +        } catch (Exception e) {
    +            logger.error("Encountered an exception initializing the cipher", e);
    +            throw new EncryptionException(e);
    +        }
    +    }
    +
    +    /**
    +     * Decrypts the provided byte[] (an encrypted record with accompanying metadata).
    +     *
    +     * @param encryptedRecord the encrypted record in byte[] form
    +     * @param recordId        an identifier for this record (eventId, generated, etc.)
    +     * @return the decrypted record
    +     * @throws EncryptionException if there is an issue decrypting this record
    +     */
    +    @Override
    +    public byte[] decrypt(byte[] encryptedRecord, String recordId) throws EncryptionException {
    +        if (encryptedRecord == null) {
    +            throw new EncryptionException("The encrypted provenance record cannot be missing");
    +        }
    +
    +        EncryptionMetadata metadata;
    +        try {
    +            metadata = extractEncryptionMetadata(encryptedRecord);
    +        } catch (IOException | ClassNotFoundException e) {
    +            final String msg = "Encountered an error reading the encryption metadata: ";
    +            logger.error(msg, e);
    +            throw new EncryptionException(msg, e);
    +        }
    +
    +        if (!SUPPORTED_VERSIONS.contains(metadata.version)) {
    +            throw new EncryptionException("The event was encrypted with version " + metadata.version + " which is not in the list of supported versions " + StringUtils.join(SUPPORTED_VERSIONS, ","));
    +        }
    +
    +        // TODO: Actually use the version to determine schema, etc.
    +
    +        if (keyProvider == null || !keyProvider.keyExists(metadata.keyId) || CryptoUtils.isEmpty(metadata.keyId)) {
    +            throw new EncryptionException("The requested key ID " + metadata.keyId + " is not available");
    +        } else {
    +            try {
    +                logger.debug("Decrypting provenance record " + recordId + " with key ID " + metadata.keyId);
    +                EncryptionMethod method = EncryptionMethod.forAlgorithm(metadata.algorithm);
    +                Cipher cipher = initCipher(method, Cipher.DECRYPT_MODE, keyProvider.getKey(metadata.keyId), metadata.ivBytes);
    +
    +                // Strip the metadata away to get just the cipher bytes
    +                byte[] cipherBytes = extractCipherBytes(encryptedRecord, metadata);
    +
    +                // Perform the actual decryption
    +                byte[] plainBytes = cipher.doFinal(cipherBytes);
    +
    +                logger.debug("Decrypted provenance event record " + recordId + " with key ID " + metadata.keyId);
    +                return plainBytes;
    +            } catch (EncryptionException | BadPaddingException | IllegalBlockSizeException | KeyManagementException e) {
    +                final String msg = "Encountered an exception decrypting provenance record " + recordId;
    +                logger.error(msg, e);
    +                throw new EncryptionException(msg, e);
    +            }
    +        }
    +    }
    +
    +    /**
    +     * Returns a valid key identifier for this encryptor (valid for encryption and decryption) or throws an exception if none are available.
    +     *
    +     * @return the key ID
    +     * @throws KeyManagementException if no available key IDs are valid for both operations
    +     */
    +    @Override
    +    public String getNextKeyId() throws KeyManagementException {
    +        if (keyProvider != null) {
    +            List<String> availableKeyIds = keyProvider.getAvailableKeyIds();
    +            if (!availableKeyIds.isEmpty()) {
    +                return availableKeyIds.get(0);
    +            }
    +        }
    +        throw new KeyManagementException("No available key IDs");
    +    }
    +
    +    private EncryptionMetadata extractEncryptionMetadata(byte[] encryptedRecord) throws EncryptionException, IOException, ClassNotFoundException {
    +        if (encryptedRecord == null || encryptedRecord.length < MIN_METADATA_LENGTH) {
    +            throw new EncryptionException("The encrypted record is too short to contain the metadata");
    +        }
    +
    +        // Skip the first byte (SENTINEL) and don't need to copy all the serialized record
    +        byte[] metadataBytes = Arrays.copyOfRange(encryptedRecord, 1, encryptedRecord.length);
    --- End diff --
    
    Good catch. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1686: NIFI-3594 Encrypted provenance repository implementation

Posted by YolandaMDavis <gi...@git.apache.org>.
Github user YolandaMDavis commented on the issue:

    https://github.com/apache/nifi/pull/1686
  
    @alopresto ran tests with the following corner cases:
    -flow running with normal shutdown invoked.  NiFi shutdown and started without issue
    -flow running with kill -9 on process invoked. NiFi process was terminated and I was able to start without issue
    -flow running and disk exceeded space - I was able to shutdown NiFi clear out disk problem and start nifi without issue
    
    Also validated that switch between EWAPR and PPR (and vice versa) along with removal of content, flowfile and provenance repos would prevent classcastexception seen previously.
    
    +1
    
    will merge shortly


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1686: NIFI-3594 Encrypted provenance repository implement...

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1686#discussion_r112963050
  
    --- Diff: nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/EncryptedSchemaRecordReader.java ---
    @@ -0,0 +1,154 @@
    +/*
    + * 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.provenance;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.File;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.util.Collection;
    +import java.util.Optional;
    +import java.util.concurrent.TimeUnit;
    +import org.apache.nifi.provenance.schema.LookupTableEventRecord;
    +import org.apache.nifi.provenance.toc.TocReader;
    +import org.apache.nifi.repository.schema.Record;
    +import org.apache.nifi.stream.io.LimitingInputStream;
    +import org.apache.nifi.stream.io.StreamUtils;
    +import org.apache.nifi.util.timebuffer.LongEntityAccess;
    +import org.apache.nifi.util.timebuffer.TimedBuffer;
    +import org.apache.nifi.util.timebuffer.TimestampedLong;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class EncryptedSchemaRecordReader extends EventIdFirstSchemaRecordReader {
    +    private static final Logger logger = LoggerFactory.getLogger(EncryptedSchemaRecordReader.class);
    +
    +    private static final int DEFAULT_DEBUG_FREQUENCY = 1_000_000;
    +
    +    private ProvenanceEventEncryptor provenanceEventEncryptor;
    +
    +    private static final TimedBuffer<TimestampedLong> decryptTimes = new TimedBuffer<>(TimeUnit.SECONDS, 60, new LongEntityAccess());
    +
    +    private int debugFrequency = DEFAULT_DEBUG_FREQUENCY;
    +    public static final int SERIALIZATION_VERSION = 1;
    +
    +    public static final String SERIALIZATION_NAME = "EncryptedSchemaRecordWriter";
    +
    +    public EncryptedSchemaRecordReader(final InputStream inputStream, final String filename, final TocReader tocReader, final int maxAttributeChars,
    +                                       ProvenanceEventEncryptor provenanceEventEncryptor) throws IOException {
    +        this(inputStream, filename, tocReader, maxAttributeChars, provenanceEventEncryptor, DEFAULT_DEBUG_FREQUENCY);
    +    }
    +
    +    public EncryptedSchemaRecordReader(final InputStream inputStream, final String filename, final TocReader tocReader, final int maxAttributeChars,
    +                                       ProvenanceEventEncryptor provenanceEventEncryptor, int debugFrequency) throws IOException {
    +        super(inputStream, filename, tocReader, maxAttributeChars);
    +        this.provenanceEventEncryptor = provenanceEventEncryptor;
    +        this.debugFrequency = debugFrequency;
    +    }
    +
    +    @Override
    +    protected StandardProvenanceEventRecord nextRecord(final DataInputStream in, final int serializationVersion) throws IOException {
    +        verifySerializationVersion(serializationVersion);
    +
    +        final long byteOffset = getBytesConsumed();
    +        final long eventId = in.readInt() + getFirstEventId();
    +        final int recordLength = in.readInt();
    +
    +        return readRecord(in, eventId, byteOffset, recordLength);
    +    }
    +
    +    private StandardProvenanceEventRecord readRecord(final DataInputStream inputStream, final long eventId, final long startOffset, final int recordLength) throws IOException {
    +        try {
    +            final InputStream limitedIn = new LimitingInputStream(inputStream, recordLength);
    +
    +            byte[] encryptedSerializedBytes = new byte[recordLength];
    +            DataInputStream encryptedInputStream = new DataInputStream(limitedIn);
    +            encryptedInputStream.readFully(encryptedSerializedBytes);
    +
    +            byte[] plainSerializedBytes = decrypt(encryptedSerializedBytes, Long.toString(eventId));
    +            InputStream plainStream = new ByteArrayInputStream(plainSerializedBytes);
    +
    +            final Record eventRecord = getRecordReader().readRecord(plainStream);
    +            if (eventRecord == null) {
    +                return null;
    +            }
    +
    +            final StandardProvenanceEventRecord deserializedEvent = LookupTableEventRecord.getEvent(eventRecord, getFilename(), startOffset, getMaxAttributeLength(),
    +                    getFirstEventId(), getSystemTimeOffset(), getComponentIds(), getComponentTypes(), getQueueIds(), getEventTypes());
    +            deserializedEvent.setEventId(eventId);
    +            return deserializedEvent;
    +        } catch (EncryptionException e) {
    +            logger.error("Encountered an error reading the record: ", e);
    +            throw new IOException(e);
    +        }
    +    }
    +
    +    // TODO: Copied from EventIdFirstSchemaRecordReader to force local/overridden readRecord()
    --- End diff --
    
    Was this TODO intended to remain here? Not sure what is actually to be done...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1686: NIFI-3594 Encrypted provenance repository implementation

Posted by alopresto <gi...@git.apache.org>.
Github user alopresto commented on the issue:

    https://github.com/apache/nifi/pull/1686
  
    @YolandaMDavis Thanks Yolanda. 
    
    During most of my testing, I did clear the repositories between application starts when switching the implementation (`rm -rf provenance_repository/ content_repository/ flowfile_repository/` from `$NIFI_HOME/conf`). I did run tests (at the time) where I left the repository intact (in each direction) and my notes are:
    
    ```
    -Verified switching from encrypted prov repo to plain will not allow access to previously-written events
    		-Error in app log
    			-Suppress stacktrace?
    		-App still starts
    		-UI error dialog on prov query
    		-New provenance events overwrite previous
    		-New prov queries work
    	-Verified switching from plain prov repo to encrypted will not allow access to previously-written events
    		-Error in app log
    			-Suppress stacktrace?
    		-App still starts
    		-UI error dialog on prov query
    		-New provenance events overwrite previous
    		-New prov queries work
    ```
    
    I will re-evaluate those scenarios as soon as I finish reviewing PR 1712 for Bryan. 
    
    I do think suppressing the stacktrace and providing a more descriptive error is a good idea and will tackle that as well. If it is determined there is a simple reason your queries were not working, great. If not, adding documentation instructing repository removal may be necessary. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1686: NIFI-3594 Encrypted provenance repository implementation

Posted by YolandaMDavis <gi...@git.apache.org>.
Github user YolandaMDavis commented on the issue:

    https://github.com/apache/nifi/pull/1686
  
    @alopresto actually Andy just noticed there's a conflict. If you could resolve, squash and push I'll be happy to continue with merging into master.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1686: NIFI-3594 Encrypted provenance repository implementation

Posted by YolandaMDavis <gi...@git.apache.org>.
Github user YolandaMDavis commented on the issue:

    https://github.com/apache/nifi/pull/1686
  
    @alopresto moving to a separate ticket makes sense. I have one more test to run (run out of disk space per @markap14 suggestion) and will provide any feedback shortly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1686: NIFI-3594 Encrypted provenance repository implementation

Posted by YolandaMDavis <gi...@git.apache.org>.
Github user YolandaMDavis commented on the issue:

    https://github.com/apache/nifi/pull/1686
  
    @alopresto happy to do some functional testing of this as well.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1686: NIFI-3594 Encrypted provenance repository implementation

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on the issue:

    https://github.com/apache/nifi/pull/1686
  
    @alopresto @YolandaMDavis just a quick update on my review... I think the code looks good and am a +1 on that front. I had no problem getting this up & running and things seem to work well. I just wanted to verify some corner cases like running out of disk space, kill -9 against NiFI, etc. and ensure that NiFi is still able to restart, as this is an issue that some have hit in the past. Otherwise all looks good to me. Thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1686: NIFI-3594 Encrypted provenance repository implementation

Posted by alopresto <gi...@git.apache.org>.
Github user alopresto commented on the issue:

    https://github.com/apache/nifi/pull/1686
  
    I had to make some substantial changes to enable multiple keys for `StaticKeyProvider`. Please exercise defining multiple keys using the following format in `nifi.properties` (note the format is nifi.provenance.repository.encryption.key.**id**.**keyId**): 
    
    ```
    nifi.provenance.repository.debug.frequency=100
    nifi.provenance.repository.encryption.key.provider.implementation=org.apache.nifi.provenance.StaticKeyProvider
    nifi.provenance.repository.encryption.key.provider.location=
    nifi.provenance.repository.encryption.key.id=Key1
    nifi.provenance.repository.encryption.key=0123456789ABCDEFFEDCBA98765432100123456789ABCDEFFEDCBA9876543210
    nifi.provenance.repository.encryption.key.id.Key2=0000111122223333444455556666777788889999AAAABBBBCCCCDDDDEEEEFFFF
    ```
    
    This should generate a key map of `[Key1:012...210, Key2: 000...FFF]`. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1686: NIFI-3594 Encrypted provenance repository implementation

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on the issue:

    https://github.com/apache/nifi/pull/1686
  
    @alopresto one thought regarding the StaticKeyProvider... rather than having a .key=<hex> perhaps it would make sense to instead use a .key.<key_id>=<hex> property scheme. This would allow multiple Key ID's to be used, which would resolve the 'Potential Issue' listed above, of now allowing the key to rotate, with rather minimal changes to the code??


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1686: NIFI-3594 Encrypted provenance repository implementation

Posted by alopresto <gi...@git.apache.org>.
Github user alopresto commented on the issue:

    https://github.com/apache/nifi/pull/1686
  
    Yolanda,
    
    I can reproduce your described issue. I believe the difference is that when I tested and did not clear the existing provenance repository, I was switching between `EncryptedWriteAheadProvenanceRepository` and `WriteAheadProvenanceRepository` which both use a `SchemaRecordReader` that can be cross-cast (`EncryptedRecordSchemaReader` extends `EventIdFirstRecordSchemaReader`), so the class cast exception wasn't occurring. However, if you test by first using `PersistentProvenanceRepository`, this uses `ByteArraySchemaRecordReader`, which implements the same `RecordReader` interface but cannot be cast. As `PersistentProvenanceRepository` is the existing option that most people will be using, production switch over will encounter this issue. I will try to improve this user experience (opening a separate sub-task to do that). For now, I've included documentation in [NIFI-3721] stating that the existing repository should be erased when switching (as this is a new feature and backward-com
 patibility is not yet provided). 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1686: NIFI-3594 Encrypted provenance repository implementation

Posted by alopresto <gi...@git.apache.org>.
Github user alopresto commented on the issue:

    https://github.com/apache/nifi/pull/1686
  
    I rebased and resolved the conflict. There were test errors in `nifi-gcp-processors` and `nifi-poi-processors` where `MockComponentLog#getErrorMessages()` was returning twice the number of expected errors. With @bbende , we determined this was because `src/test/resources/logback-test.xml` in `nifi-data-provenance-utils` set the log level to `DEBUG`, and `MockComponentLog` double records the error message if `DEBUG` is enabled. I provided custom `src/test/resources/logback-test.xml` files in the two offending modules to ensure that the log level is set as expected when those tests run. 
    
    Merged. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1686: NIFI-3594 Encrypted provenance repository implement...

Posted by alopresto <gi...@git.apache.org>.
Github user alopresto commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1686#discussion_r113066789
  
    --- Diff: nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/CryptoUtils.java ---
    @@ -0,0 +1,228 @@
    +/*
    + * 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.provenance;
    +
    +import java.io.BufferedReader;
    +import java.io.ByteArrayOutputStream;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.security.KeyManagementException;
    +import java.security.NoSuchAlgorithmException;
    +import java.util.Arrays;
    +import java.util.Base64;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.regex.Pattern;
    +import javax.crypto.BadPaddingException;
    +import javax.crypto.Cipher;
    +import javax.crypto.IllegalBlockSizeException;
    +import javax.crypto.SecretKey;
    +import javax.crypto.spec.SecretKeySpec;
    +import org.apache.commons.lang3.StringUtils;
    +import org.apache.nifi.security.util.EncryptionMethod;
    +import org.apache.nifi.security.util.crypto.AESKeyedCipherProvider;
    +import org.apache.nifi.util.NiFiProperties;
    +import org.bouncycastle.util.encoders.Hex;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class CryptoUtils {
    +    private static final Logger logger = LoggerFactory.getLogger(StaticKeyProvider.class);
    +    private static final String STATIC_KEY_PROVIDER_CLASS_NAME = "org.apache.nifi.provenance.StaticKeyProvider";
    +    private static final String FILE_BASED_KEY_PROVIDER_CLASS_NAME = "org.apache.nifi.provenance.FileBasedKeyProvider";
    +    private static final Pattern HEX_PATTERN = Pattern.compile("(?i)^[0-9a-f]+$");
    +
    +    public static final int IV_LENGTH = 16;
    +
    +    public static boolean isUnlimitedStrengthCryptoAvailable() {
    +        try {
    +            return Cipher.getMaxAllowedKeyLength("AES") > 128;
    +        } catch (NoSuchAlgorithmException e) {
    +            logger.warn("Tried to determine if unlimited strength crypto is available but the AES algorithm is not available");
    +            return false;
    +        }
    +    }
    +
    +    /**
    +     * Utility method which returns true if the string is null, empty, or entirely whitespace.
    +     *
    +     * @param src the string to evaluate
    +     * @return true if empty
    +     */
    +    public static boolean isEmpty(String src) {
    +        return src == null || src.trim().isEmpty();
    +    }
    +
    +    /**
    +     * Concatenates multiple byte[] into a single byte[]. Because it uses a {@link ByteArrayOutputStream}
    +     * rather than {@link System#arraycopy(Object, int, Object, int, int)} the performance is much better
    +     * with an arbitrary number of input byte[]s.
    +     *
    +     * @param arrays the component byte[] in order
    +     * @return a concatenated byte[]
    +     * @throws IOException this should never be thrown
    +     */
    +    public static byte[] concatByteArrays(byte[]... arrays) throws IOException {
    +        ByteArrayOutputStream boas = new ByteArrayOutputStream();
    --- End diff --
    
    Yep, looks like with the higher iteration count the array copy is faster. 
    
    ```
    Ran 1000 iterations in 34702232 ns to warm up the JVM
    Calculating small/small -- 2 arrays with avg length 5
    Ran 1000000 of small/small (traditional) with a total wall time of 1339514066 ns and average run of 131 ns
    Ran 1000000 of small/small (BAOS) with a total wall time of 1008624897 ns and average run of 83 ns
    Calculating small/large -- 5 arrays with avg length 484
    Ran 1000000 of small/large (traditional) with a total wall time of 1699096744 ns and average run of 805 ns
    Ran 1000000 of small/large (BAOS) with a total wall time of 2978547636 ns and average run of 1975 ns
    Calculating large/small -- 173 arrays with avg length 8
    Ran 1000000 of large/small (traditional) with a total wall time of 2692231144 ns and average run of 1521 ns
    Ran 1000000 of large/small (BAOS) with a total wall time of 3514575357 ns and average run of 2582 ns
    Calculating large/large -- 147 arrays with avg length 579
    Ran 1000000 of large/large (traditional) with a total wall time of 16443696576 ns and average run of 15418 ns
    Ran 1000000 of large/large (BAOS) with a total wall time of 61356984740 ns and average run of 59737 ns
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1686: NIFI-3594 Encrypted provenance repository implement...

Posted by alopresto <gi...@git.apache.org>.
Github user alopresto commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1686#discussion_r113075785
  
    --- Diff: nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/groovy/org/apache/nifi/provenance/EncryptedSchemaRecordReaderWriterTest.groovy ---
    @@ -0,0 +1,286 @@
    +/*
    + * 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.provenance
    +
    +import org.apache.nifi.flowfile.FlowFile
    +import org.apache.nifi.provenance.serialization.RecordReader
    +import org.apache.nifi.provenance.serialization.RecordWriter
    +import org.apache.nifi.provenance.toc.StandardTocReader
    +import org.apache.nifi.provenance.toc.StandardTocWriter
    +import org.apache.nifi.provenance.toc.TocReader
    +import org.apache.nifi.provenance.toc.TocUtil
    +import org.apache.nifi.provenance.toc.TocWriter
    +import org.apache.nifi.util.file.FileUtils
    +import org.bouncycastle.jce.provider.BouncyCastleProvider
    +import org.bouncycastle.util.encoders.Hex
    +import org.junit.After
    +import org.junit.AfterClass
    +import org.junit.Before
    +import org.junit.BeforeClass
    +import org.junit.ClassRule
    +import org.junit.Test
    +import org.junit.rules.TemporaryFolder
    +import org.junit.runner.RunWith
    +import org.junit.runners.JUnit4
    +import org.slf4j.Logger
    +import org.slf4j.LoggerFactory
    +
    +import javax.crypto.Cipher
    +import javax.crypto.spec.SecretKeySpec
    +import java.security.KeyManagementException
    +import java.security.Security
    +import java.util.concurrent.atomic.AtomicLong
    +
    +import static groovy.test.GroovyAssert.shouldFail
    +import static org.apache.nifi.provenance.TestUtil.createFlowFile
    +
    +@RunWith(JUnit4.class)
    +class EncryptedSchemaRecordReaderWriterTest extends AbstractTestRecordReaderWriter {
    +    private static final Logger logger = LoggerFactory.getLogger(EncryptedSchemaRecordReaderWriterTest.class)
    +
    +    private static final String KEY_HEX_128 = "0123456789ABCDEFFEDCBA9876543210"
    +    private static final String KEY_HEX_256 = KEY_HEX_128 * 2
    +    private static final String KEY_HEX = isUnlimitedStrengthCryptoAvailable() ? KEY_HEX_256 : KEY_HEX_128
    +    private static final int IV_LENGTH = 16
    +    private static final String KEY_ID = "K1"
    +
    +    private static final String TRANSIT_URI = "nifi://unit-test"
    +    private static final String PROCESSOR_TYPE = "Mock Processor"
    +    private static final String COMPONENT_ID = "1234"
    +
    +    private static final int UNCOMPRESSED_BLOCK_SIZE = 1024 * 32
    +    private static final int MAX_ATTRIBUTE_SIZE = 2048
    +
    +    private static final AtomicLong idGenerator = new AtomicLong(0L)
    +    private File journalFile
    +    private File tocFile
    +
    +    private static KeyProvider mockKeyProvider
    +    private static ProvenanceEventEncryptor provenanceEventEncryptor = new AESProvenanceEventEncryptor()
    +
    +    @ClassRule
    +    public static TemporaryFolder tempFolder = new TemporaryFolder()
    +
    +    private static String ORIGINAL_LOG_LEVEL
    +
    +    @BeforeClass
    +    static void setUpOnce() throws Exception {
    +        ORIGINAL_LOG_LEVEL = System.getProperty("org.slf4j.simpleLogger.log.org.apache.nifi.provenance")
    +        System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi.provenance", "DEBUG")
    +
    +        Security.addProvider(new BouncyCastleProvider())
    +
    +        logger.metaClass.methodMissing = { String name, args ->
    +            logger.info("[${name?.toUpperCase()}] ${(args as List).join(" ")}")
    +        }
    +
    +        mockKeyProvider = [
    +                getKey            : { String keyId ->
    +                    logger.mock("Requesting key ID: ${keyId}")
    +                    if (keyId == KEY_ID) {
    +                        new SecretKeySpec(Hex.decode(KEY_HEX), "AES")
    +                    } else {
    +                        throw new KeyManagementException("${keyId} is not available")
    +                    }
    +                },
    +                getAvailableKeyIds: { ->
    +                    logger.mock("Available key IDs: [${KEY_ID}]")
    +                    [KEY_ID]
    +                },
    +                keyExists         : { String keyId ->
    +                    logger.mock("Checking availability of key ID: ${keyId}")
    +                    keyId == KEY_ID
    +                }] as KeyProvider
    +        provenanceEventEncryptor.initialize(mockKeyProvider)
    +//
    --- End diff --
    
    Yes. Thanks. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1686: NIFI-3594 Encrypted provenance repository implement...

Posted by alopresto <gi...@git.apache.org>.
Github user alopresto commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1686#discussion_r112980831
  
    --- Diff: nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/CryptoUtils.java ---
    @@ -0,0 +1,228 @@
    +/*
    + * 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.provenance;
    +
    +import java.io.BufferedReader;
    +import java.io.ByteArrayOutputStream;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.security.KeyManagementException;
    +import java.security.NoSuchAlgorithmException;
    +import java.util.Arrays;
    +import java.util.Base64;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.regex.Pattern;
    +import javax.crypto.BadPaddingException;
    +import javax.crypto.Cipher;
    +import javax.crypto.IllegalBlockSizeException;
    +import javax.crypto.SecretKey;
    +import javax.crypto.spec.SecretKeySpec;
    +import org.apache.commons.lang3.StringUtils;
    +import org.apache.nifi.security.util.EncryptionMethod;
    +import org.apache.nifi.security.util.crypto.AESKeyedCipherProvider;
    +import org.apache.nifi.util.NiFiProperties;
    +import org.bouncycastle.util.encoders.Hex;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class CryptoUtils {
    +    private static final Logger logger = LoggerFactory.getLogger(StaticKeyProvider.class);
    +    private static final String STATIC_KEY_PROVIDER_CLASS_NAME = "org.apache.nifi.provenance.StaticKeyProvider";
    +    private static final String FILE_BASED_KEY_PROVIDER_CLASS_NAME = "org.apache.nifi.provenance.FileBasedKeyProvider";
    +    private static final Pattern HEX_PATTERN = Pattern.compile("(?i)^[0-9a-f]+$");
    +
    +    public static final int IV_LENGTH = 16;
    +
    +    public static boolean isUnlimitedStrengthCryptoAvailable() {
    +        try {
    +            return Cipher.getMaxAllowedKeyLength("AES") > 128;
    +        } catch (NoSuchAlgorithmException e) {
    +            logger.warn("Tried to determine if unlimited strength crypto is available but the AES algorithm is not available");
    +            return false;
    +        }
    +    }
    +
    +    /**
    +     * Utility method which returns true if the string is null, empty, or entirely whitespace.
    +     *
    +     * @param src the string to evaluate
    +     * @return true if empty
    +     */
    +    public static boolean isEmpty(String src) {
    +        return src == null || src.trim().isEmpty();
    +    }
    +
    +    /**
    +     * Concatenates multiple byte[] into a single byte[]. Because it uses a {@link ByteArrayOutputStream}
    +     * rather than {@link System#arraycopy(Object, int, Object, int, int)} the performance is much better
    +     * with an arbitrary number of input byte[]s.
    +     *
    +     * @param arrays the component byte[] in order
    +     * @return a concatenated byte[]
    +     * @throws IOException this should never be thrown
    +     */
    +    public static byte[] concatByteArrays(byte[]... arrays) throws IOException {
    +        ByteArrayOutputStream boas = new ByteArrayOutputStream();
    --- End diff --
    
    Ok. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---