[
https://issues.apache.org/jira/browse/NIFI-3388?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andy LoPresto updated NIFI-3388:
--------------------------------
Issue Type: Epic (was: New Feature)
> Provide encrypted repository implementations
> --------------------------------------------
>
> Key: NIFI-3388
> URL: https://issues.apache.org/jira/browse/NIFI-3388
> Project: Apache NiFi
> Issue Type: Epic
> Components: Core Framework
> Affects Versions: 1.1.1
> Reporter: Andy LoPresto
> Assignee: Andy LoPresto
> Priority: Critical
> Labels: data-integrity, encryption, repository, security
>
> Apache NiFi secures data within the application but the various
> *repositories* -- content, provenance, flowfile (aka attribute), and to a
> lesser extent bulletin, counter, component status, and log -- are stored
> unencrypted on disk. Thus far, OS-level access control policies and full disk
> encryption (FDE) have been recommended to secure these repositories. However,
> the underlying raw data can be viewed, and possibly manipulated, by a user
> with access to the repository files.
> With additional focus on data security (confidentiality, integrity, *and*
> authentication), especially as more users intend to deploy NiFi on
> third-party provisioned hardware and operating systems (AWS, Azure, etc.),
> further steps should be taken to secure the repository data which NiFi writes
> and reads.
> Each of the repository implementations adheres to an interface definition:
> * Content: {{ContentRepository}}
> ** {{FileSystemRepository}} *
> ** {{VolatileContentRepository}}
> ** {{MockContentRepository}}
> * Provenance: {{ProvenanceRepository extends ProvenanceEventRepository}}
> ** {{PersistentProvenanceRepository}} (to be deprecated via
> [NIFI-3356|https://issues.apache.org/jira/browse/NIFI-3356])*
> ** {{WriteAheadProvenanceRepository}} (to be introduced via NIFI-3356) *
> ** {{VolatileProvenanceRepository}}
> ** {{MockProvenanceRepository}}
> * Flowfile: {{FlowFileRepository}} / {{FlowFileEventRepository}}
> ** {{WriteAheadFlowFileRepository implements FlowFileRepository}} *
> ** {{VolatileFlowFileRepository implements FlowFileRepository}}
> ** {{MockFlowFileRepository implements FlowFileRepository}}
> ** {{RingBufferEventRepository implements FlowFileEventRepository}}
> The bulletin, counter, component status, and log repositories currently have
> only *volatile* implementations, and are not addressed in this ticket. The
> repository implementations noted above with an asterisk will have new
> implementations provided of the form
> {{EncryptedWriteAheadFlowFileRepository}}, extending the behavior of the
> existing repository and adding transparent encryption/decryption on
> serialization, following the existing interface contracts and thus invisible
> to the higher level code delegating these operations, aside from
> configuration requirements.
> There are substantial concerns to address in this approach.
> * Should the various repositories all be required to have the same
> encrypted/plaintext status (i.e. can they be encrypted independently)?
> * Should all encrypted repositories use the same encryption key, or should it
> be segmented by repository?
> * If a content or provenance repository has multiple shards, do they all
> require the same level of encryption? If not, can "sensitive" records be
> routed to an encrypted repository and "normal" records to a plaintext
> repository for performance reasons?
> * Can a plaintext repository have encryption enabled at any time? Can an
> encrypted repository have decryption removed?
> * Performance impact on reading and writing from the repositories is not yet
> captured
> ** The provenance repository requires many fast writes and reads during high
> performance and query operations
> ** The flowfile repository requires many fast writes and reads
> ** The content repository requires fewer reads and writes, but the current
> content repository stores multiple flowfile contents in the same ["sections"
> of the
> "containers"|https://nifi.apache.org/docs/nifi-docs/html/nifi-in-depth.html#deeper-view-content-claim]
> that make up the repository, so content claims will need to be encrypted
> separately to allow random-access seek and retrieve (i.e. if a 10 byte claim
> and a 10 GB claim are stored in the same section, the 10 B claim must be
> retrievable without reading 10 GB into memory to decrypt)
> * The provenance repository uses GZip compression to improve the use of disk
> space. Compression over encrypted data will yield close to zero improvement
> (as encryption intentionally generates near-random output, which means
> pattern recognition/entropy removal will have no effect), and compression
> before encryption can cause security vulnerabilities (see
> [CRIME|http://security.stackexchange.com/a/19914/16485], [Compression and
> Information Leakage of Plaintext by John
> Kelsey|https://dl.acm.org/citation.cfm?id=741226], etc.)
> * The provenance repository event records are indexed by Lucene to allow
> retrieval through the provenance query system, but encrypted fields cannot be
> indexed. HMAC/CHF (Hash-based message authentication codes/cryptographic hash
> functions) may provide an alternative for non-fuzzy matching for information
> retrieval
> * The flowfile repository implementation uses swap files to maintain flowfile
> state if too many are loaded into memory -- these swap files (and anything
> else persisted to disk) will need to be encrypted
> * The content and provenance repositories can be spread across multiple
> physical volumes. In that case, should data stored on different disks be
> encrypted with the same key or unique keys (perhaps derived from a master key
> using a disk identifier)? If a constituent disk is swapped out, will that
> data be recoverable?
> * How is the configuration of encrypted repositories handled (i.e. new
> properties in {{nifi.properties}})?
> ** How are the keys generated and secured?
> ** What permissions/policies are required to configure these properties?
> ** What UI / API signals (if any) are provided to inform users of the
> encrypted/plaintext status of the repositories?
> * What encryption algorithm(s) is/are used?
> ** Performance considerations (CTR vs. CBC)
> ** AEAD considerations (GCM, CCM, vs. CBC/CTR + HMAC/SHA-256)
> * What actions should be taken if the encrypted data cannot be read
> (authentication tag corrupted, cipher text malformed, etc.)? These are risk
> vectors for DoS attacks
> I am currently working on this issue (planning & architecture stages), so I
> would appreciate community feedback in order to provide the best possible
> solution that balances the security, performance, and usability needs of
> everyone. I will likely break the work into the following subtasks:
> * Build/consume encapsulated encryption service layer (see
> [{{AESKeyedCipherProvider}}|https://github.com/apache/nifi/blob/master/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/crypto/AESKeyedCipherProvider.java]
> and
> [{{KeyedEncryptor}}|https://github.com/apache/nifi/blob/master/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/crypto/KeyedEncryptor.java])
> * Build {{EncryptedWriteAheadFlowFileRepository}}
> * Build {{EncryptedWriteAheadProvenanceRepository}}
> * Build {{EncryptedFileSystemContentRepository}} (order may change depending
> on further investigation)
> For anyone interested in further detail on the existing repository design,
> implementations, and use, see:
> * [NiFi's Write-Ahead Log Implementation (FlowFile
> Repository)|https://cwiki.apache.org/confluence/display/NIFI/NiFi%27s+Write-Ahead+Log+Implementation]
> * [Persistent Provenance Repository
> Design|https://cwiki.apache.org/confluence/display/NIFI/Persistent+Provenance+Repository+Design]
> * [NiFi
> In-Depth|https://nifi.apache.org/docs/nifi-docs/html/nifi-in-depth.html]
> * [Admin
> Guide|https://nifi.apache.org/docs/nifi-docs/html/administration-guide.html]
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)