mirror of
https://github.com/apache/nifi.git
synced 2025-02-09 11:35:05 +00:00
NIFI-10344 Refactored Groovy tests in standard-processor to Java and JUnit 5
This closes #6321 Signed-off-by: David Handermann <exceptionfactory@apache.org>
This commit is contained in:
parent
d698e7af1a
commit
f5fee4dda3
@ -952,7 +952,7 @@ public class PutDatabaseRecord extends AbstractProcessor {
|
||||
}
|
||||
}
|
||||
|
||||
private String generateTableName(final DMLSettings settings, final String catalog, final String schemaName, final String tableName, final TableSchema tableSchema) {
|
||||
String generateTableName(final DMLSettings settings, final String catalog, final String schemaName, final String tableName, final TableSchema tableSchema) {
|
||||
final StringBuilder tableNameBuilder = new StringBuilder();
|
||||
if (catalog != null) {
|
||||
if (settings.quoteTableName) {
|
||||
@ -1399,7 +1399,7 @@ public class PutDatabaseRecord extends AbstractProcessor {
|
||||
private Map<String, ColumnDescription> columns;
|
||||
private String quotedIdentifierString;
|
||||
|
||||
private TableSchema(final List<ColumnDescription> columnDescriptions, final boolean translateColumnNames,
|
||||
TableSchema(final List<ColumnDescription> columnDescriptions, final boolean translateColumnNames,
|
||||
final Set<String> primaryKeyColumnNames, final String quotedIdentifierString) {
|
||||
this.columns = new LinkedHashMap<>();
|
||||
this.primaryKeyColumnNames = primaryKeyColumnNames;
|
||||
@ -1688,7 +1688,7 @@ public class PutDatabaseRecord extends AbstractProcessor {
|
||||
// Quote table name?
|
||||
private final boolean quoteTableName;
|
||||
|
||||
private DMLSettings(ProcessContext context) {
|
||||
DMLSettings(ProcessContext context) {
|
||||
translateFieldNames = context.getProperty(TRANSLATE_FIELD_NAMES).asBoolean();
|
||||
ignoreUnmappedFields = IGNORE_UNMATCHED_FIELD.getValue().equalsIgnoreCase(context.getProperty(UNMATCHED_FIELD_BEHAVIOR).getValue());
|
||||
|
||||
|
@ -1,396 +0,0 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License") you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.processors.standard
|
||||
|
||||
import org.apache.nifi.components.PropertyDescriptor
|
||||
import org.apache.nifi.flowfile.FlowFile
|
||||
import org.apache.nifi.util.MockComponentLog
|
||||
import org.apache.nifi.util.MockProcessSession
|
||||
import org.apache.nifi.util.TestRunner
|
||||
import org.apache.nifi.util.TestRunners
|
||||
import org.bouncycastle.jce.provider.BouncyCastleProvider
|
||||
import org.junit.After
|
||||
import org.junit.Before
|
||||
import org.junit.BeforeClass
|
||||
import org.junit.Test
|
||||
import org.junit.runner.RunWith
|
||||
import org.junit.runners.JUnit4
|
||||
import org.mockito.Mockito
|
||||
import org.slf4j.Logger
|
||||
import org.slf4j.LoggerFactory
|
||||
|
||||
import java.nio.charset.StandardCharsets
|
||||
import java.security.Security
|
||||
|
||||
import static org.mockito.ArgumentMatchers.anyBoolean
|
||||
import static org.mockito.ArgumentMatchers.anyString
|
||||
import static org.mockito.Mockito.doReturn
|
||||
import static org.mockito.Mockito.spy
|
||||
import static org.mockito.Mockito.when
|
||||
|
||||
@RunWith(JUnit4.class)
|
||||
class CountTextTest extends GroovyTestCase {
|
||||
private static final Logger logger = LoggerFactory.getLogger(CountTextTest.class)
|
||||
|
||||
private static final String TLC = "text.line.count"
|
||||
private static final String TLNEC = "text.line.nonempty.count"
|
||||
private static final String TWC = "text.word.count"
|
||||
private static final String TCC = "text.character.count"
|
||||
|
||||
|
||||
@BeforeClass
|
||||
static void setUpOnce() throws Exception {
|
||||
Security.addProvider(new BouncyCastleProvider())
|
||||
|
||||
logger.metaClass.methodMissing = { String name, args ->
|
||||
logger.info("[${name?.toUpperCase()}] ${(args as List).join(" ")}")
|
||||
}
|
||||
}
|
||||
|
||||
@Before
|
||||
void setUp() throws Exception {
|
||||
}
|
||||
|
||||
@After
|
||||
void tearDown() throws Exception {
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldCountAllMetrics() throws Exception {
|
||||
// Arrange
|
||||
final TestRunner runner = TestRunners.newTestRunner(CountText.class)
|
||||
|
||||
runner.setProperty(CountText.TEXT_LINE_COUNT_PD, "true")
|
||||
runner.setProperty(CountText.TEXT_LINE_NONEMPTY_COUNT_PD, "true")
|
||||
runner.setProperty(CountText.TEXT_WORD_COUNT_PD, "true")
|
||||
runner.setProperty(CountText.TEXT_CHARACTER_COUNT_PD, "true")
|
||||
|
||||
// This text is the same as in src/test/resources/TestCountText/jabberwocky.txt but is copied here
|
||||
// to ensure that reading from a file vs. static text doesn't cause line break issues
|
||||
String INPUT_TEXT = """’Twas brillig, and the slithy toves
|
||||
Did gyre and gimble in the wade;
|
||||
All mimsy were the borogoves,
|
||||
And the mome raths outgrabe.
|
||||
|
||||
"Beware the Jabberwock, my son!
|
||||
The jaws that bite, the claws that catch!
|
||||
Beware the Jubjub bird, and shun
|
||||
The frumious Bandersnatch!"
|
||||
|
||||
He took his vorpal sword in hand:
|
||||
Long time the manxome foe he sought—
|
||||
So rested he by the Tumtum tree,
|
||||
And stood awhile in thought.
|
||||
|
||||
And as in uffish thought he stood,
|
||||
The Jabberwock, with eyes of flame,
|
||||
Came whiffling through the tulgey wood.
|
||||
And burbled as it came!
|
||||
|
||||
One, two! One, two! And through and through
|
||||
The vorpal blade went snicker-snack!
|
||||
He left it dead, and with its head
|
||||
He went galumphing back.
|
||||
|
||||
"And hast thou slain the Jabberwock?
|
||||
Come to my arms, my beamish boy!
|
||||
O frabjous day! Callooh! Callay!"
|
||||
He chortled in his joy.
|
||||
|
||||
’Twas brillig, and the slithy toves
|
||||
Did gyre and gimble in the wabe;
|
||||
All mimsy were the borogoves,
|
||||
And the mome raths outgrabe."""
|
||||
|
||||
runner.enqueue(INPUT_TEXT.bytes)
|
||||
|
||||
// Act
|
||||
runner.run()
|
||||
|
||||
// Assert
|
||||
runner.assertAllFlowFilesTransferred(CountText.REL_SUCCESS, 1)
|
||||
FlowFile flowFile = runner.getFlowFilesForRelationship(CountText.REL_SUCCESS).first()
|
||||
assert flowFile.attributes."$TLC" == 34 as String
|
||||
assert flowFile.attributes."$TLNEC" == 28 as String
|
||||
assert flowFile.attributes."$TWC" == 166 as String
|
||||
assert flowFile.attributes."$TCC" == 900 as String
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldCountEachMetric() throws Exception {
|
||||
// Arrange
|
||||
final TestRunner runner = TestRunners.newTestRunner(CountText.class)
|
||||
String INPUT_TEXT = new File("src/test/resources/TestCountText/jabberwocky.txt").text
|
||||
|
||||
final def EXPECTED_VALUES = [
|
||||
(TLC) : 34,
|
||||
(TLNEC): 28,
|
||||
(TWC) : 166,
|
||||
(TCC) : 900,
|
||||
]
|
||||
|
||||
def linesOnly = [(CountText.TEXT_LINE_COUNT_PD): "true"]
|
||||
def linesNonEmptyOnly = [(CountText.TEXT_LINE_NONEMPTY_COUNT_PD): "true"]
|
||||
def wordsOnly = [(CountText.TEXT_WORD_COUNT_PD): "true"]
|
||||
def charactersOnly = [(CountText.TEXT_CHARACTER_COUNT_PD): "true"]
|
||||
|
||||
final List<Map<PropertyDescriptor, String>> SCENARIOS = [linesOnly, linesNonEmptyOnly, wordsOnly, charactersOnly]
|
||||
|
||||
SCENARIOS.each { map ->
|
||||
// Reset the processor properties
|
||||
runner.setProperty(CountText.TEXT_LINE_COUNT_PD, "false")
|
||||
runner.setProperty(CountText.TEXT_LINE_NONEMPTY_COUNT_PD, "false")
|
||||
runner.setProperty(CountText.TEXT_WORD_COUNT_PD, "false")
|
||||
runner.setProperty(CountText.TEXT_CHARACTER_COUNT_PD, "false")
|
||||
|
||||
// Apply the scenario-specific properties
|
||||
map.each { key, value ->
|
||||
runner.setProperty(key, value)
|
||||
}
|
||||
|
||||
runner.clearProvenanceEvents()
|
||||
runner.clearTransferState()
|
||||
runner.enqueue(INPUT_TEXT.bytes)
|
||||
|
||||
// Act
|
||||
runner.run()
|
||||
|
||||
// Assert
|
||||
runner.assertAllFlowFilesTransferred(CountText.REL_SUCCESS, 1)
|
||||
FlowFile flowFile = runner.getFlowFilesForRelationship(CountText.REL_SUCCESS).first()
|
||||
logger.info("Generated flowfile: ${flowFile} | ${flowFile.attributes}")
|
||||
EXPECTED_VALUES.each { key, value ->
|
||||
if (flowFile.attributes.containsKey(key)) {
|
||||
assert flowFile.attributes.get(key) == value as String
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldCountWordsSplitOnSymbol() throws Exception {
|
||||
// Arrange
|
||||
final TestRunner runner = TestRunners.newTestRunner(CountText.class)
|
||||
String INPUT_TEXT = new File("src/test/resources/TestCountText/jabberwocky.txt").text
|
||||
|
||||
final int EXPECTED_WORD_COUNT = 167
|
||||
|
||||
// Reset the processor properties
|
||||
runner.setProperty(CountText.TEXT_LINE_COUNT_PD, "false")
|
||||
runner.setProperty(CountText.TEXT_LINE_NONEMPTY_COUNT_PD, "false")
|
||||
runner.setProperty(CountText.TEXT_WORD_COUNT_PD, "true")
|
||||
runner.setProperty(CountText.TEXT_CHARACTER_COUNT_PD, "false")
|
||||
runner.setProperty(CountText.SPLIT_WORDS_ON_SYMBOLS_PD, "true")
|
||||
|
||||
runner.clearProvenanceEvents()
|
||||
runner.clearTransferState()
|
||||
runner.enqueue(INPUT_TEXT.bytes)
|
||||
|
||||
// Act
|
||||
runner.run()
|
||||
|
||||
// Assert
|
||||
runner.assertAllFlowFilesTransferred(CountText.REL_SUCCESS, 1)
|
||||
FlowFile flowFile = runner.getFlowFilesForRelationship(CountText.REL_SUCCESS).first()
|
||||
logger.info("Generated flowfile: ${flowFile} | ${flowFile.attributes}")
|
||||
assert flowFile.attributes.get(CountText.TEXT_WORD_COUNT) == EXPECTED_WORD_COUNT as String
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldCountIndependentlyPerFlowFile() throws Exception {
|
||||
// Arrange
|
||||
final TestRunner runner = TestRunners.newTestRunner(CountText.class)
|
||||
String INPUT_TEXT = new File("src/test/resources/TestCountText/jabberwocky.txt").text
|
||||
|
||||
final def EXPECTED_VALUES = [
|
||||
(TLC) : 34,
|
||||
(TLNEC): 28,
|
||||
(TWC) : 166,
|
||||
(TCC) : 900,
|
||||
]
|
||||
|
||||
// Reset the processor properties
|
||||
runner.setProperty(CountText.TEXT_LINE_COUNT_PD, "true")
|
||||
runner.setProperty(CountText.TEXT_LINE_NONEMPTY_COUNT_PD, "true")
|
||||
runner.setProperty(CountText.TEXT_WORD_COUNT_PD, "true")
|
||||
runner.setProperty(CountText.TEXT_CHARACTER_COUNT_PD, "true")
|
||||
|
||||
2.times { int i ->
|
||||
runner.clearProvenanceEvents()
|
||||
runner.clearTransferState()
|
||||
runner.enqueue(INPUT_TEXT.bytes)
|
||||
|
||||
// Act
|
||||
runner.run()
|
||||
|
||||
// Assert
|
||||
runner.assertAllFlowFilesTransferred(CountText.REL_SUCCESS, 1)
|
||||
FlowFile flowFile = runner.getFlowFilesForRelationship(CountText.REL_SUCCESS).first()
|
||||
logger.info("Generated flowfile: ${flowFile} | ${flowFile.attributes}")
|
||||
EXPECTED_VALUES.each { key, value ->
|
||||
if (flowFile.attributes.containsKey(key)) {
|
||||
assert flowFile.attributes.get(key) == value as String
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldTrackSessionCountersAcrossMultipleFlowfiles() throws Exception {
|
||||
// Arrange
|
||||
final TestRunner runner = TestRunners.newTestRunner(CountText.class)
|
||||
String INPUT_TEXT = new File("src/test/resources/TestCountText/jabberwocky.txt").text
|
||||
|
||||
final def EXPECTED_VALUES = [
|
||||
(TLC) : 34,
|
||||
(TLNEC): 28,
|
||||
(TWC) : 166,
|
||||
(TCC) : 900,
|
||||
]
|
||||
|
||||
// Reset the processor properties
|
||||
runner.setProperty(CountText.TEXT_LINE_COUNT_PD, "true")
|
||||
runner.setProperty(CountText.TEXT_LINE_NONEMPTY_COUNT_PD, "true")
|
||||
runner.setProperty(CountText.TEXT_WORD_COUNT_PD, "true")
|
||||
runner.setProperty(CountText.TEXT_CHARACTER_COUNT_PD, "true")
|
||||
|
||||
MockProcessSession mockPS = runner.processSessionFactory.createSession()
|
||||
def sessionCounters = mockPS.sharedState.counterMap
|
||||
logger.info("Session counters (0): ${sessionCounters}")
|
||||
|
||||
int n = 2
|
||||
|
||||
n.times { int i ->
|
||||
runner.clearTransferState()
|
||||
runner.enqueue(INPUT_TEXT.bytes)
|
||||
|
||||
// Act
|
||||
runner.run()
|
||||
logger.info("Session counters (${i + 1}): ${sessionCounters}")
|
||||
|
||||
// Assert
|
||||
runner.assertAllFlowFilesTransferred(CountText.REL_SUCCESS, 1)
|
||||
FlowFile flowFile = runner.getFlowFilesForRelationship(CountText.REL_SUCCESS).first()
|
||||
logger.info("Generated flowfile: ${flowFile} | ${flowFile.attributes}")
|
||||
EXPECTED_VALUES.each { key, value ->
|
||||
if (flowFile.attributes.containsKey(key)) {
|
||||
assert flowFile.attributes.get(key) == value as String
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
assert sessionCounters.get("Lines Counted").get() == EXPECTED_VALUES[TLC] * n as long
|
||||
assert sessionCounters.get("Lines (non-empty) Counted").get() == EXPECTED_VALUES[TLNEC] * n as long
|
||||
assert sessionCounters.get("Words Counted").get() == EXPECTED_VALUES[TWC] * n as long
|
||||
assert sessionCounters.get("Characters Counted").get() == EXPECTED_VALUES[TCC] * n as long
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldHandleInternalError() throws Exception {
|
||||
// Arrange
|
||||
CountText ct = new CountText()
|
||||
ct.countLines = true
|
||||
ct.countLinesNonEmpty = true
|
||||
ct.countWords = true
|
||||
ct.countCharacters = true
|
||||
|
||||
CountText ctSpy = Mockito.spy(ct)
|
||||
when(ctSpy.countWordsInLine(anyString(), anyBoolean())).thenThrow(new IOException("Expected exception"))
|
||||
|
||||
final TestRunner runner = TestRunners.newTestRunner(ctSpy)
|
||||
final String INPUT_TEXT = "This flowfile should throw an error"
|
||||
|
||||
// Reset the processor properties
|
||||
runner.setProperty(CountText.TEXT_LINE_COUNT_PD, "true")
|
||||
runner.setProperty(CountText.TEXT_LINE_NONEMPTY_COUNT_PD, "true")
|
||||
runner.setProperty(CountText.TEXT_WORD_COUNT_PD, "true")
|
||||
runner.setProperty(CountText.TEXT_CHARACTER_COUNT_PD, "true")
|
||||
runner.setProperty(CountText.CHARACTER_ENCODING_PD, StandardCharsets.US_ASCII.displayName())
|
||||
|
||||
runner.enqueue(INPUT_TEXT.bytes)
|
||||
|
||||
// Act
|
||||
// Need initialize = true to run #onScheduled()
|
||||
runner.run(1, true, true)
|
||||
|
||||
// Assert
|
||||
runner.assertAllFlowFilesTransferred(CountText.REL_FAILURE, 1)
|
||||
FlowFile flowFile = runner.getFlowFilesForRelationship(CountText.REL_FAILURE).first()
|
||||
logger.info("Generated flowfile: ${flowFile} | ${flowFile.attributes}")
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldIgnoreWhitespaceWordsWhenCounting() throws Exception {
|
||||
// Arrange
|
||||
final TestRunner runner = TestRunners.newTestRunner(CountText.class)
|
||||
String INPUT_TEXT = "a b c"
|
||||
|
||||
final int EXPECTED_WORD_COUNT = 3
|
||||
|
||||
// Reset the processor properties
|
||||
runner.setProperty(CountText.TEXT_LINE_COUNT_PD, "false")
|
||||
runner.setProperty(CountText.TEXT_LINE_NONEMPTY_COUNT_PD, "false")
|
||||
runner.setProperty(CountText.TEXT_WORD_COUNT_PD, "true")
|
||||
runner.setProperty(CountText.TEXT_CHARACTER_COUNT_PD, "false")
|
||||
runner.setProperty(CountText.SPLIT_WORDS_ON_SYMBOLS_PD, "true")
|
||||
|
||||
runner.clearProvenanceEvents()
|
||||
runner.clearTransferState()
|
||||
runner.enqueue(INPUT_TEXT.bytes)
|
||||
|
||||
// Act
|
||||
runner.run()
|
||||
|
||||
// Assert
|
||||
runner.assertAllFlowFilesTransferred(CountText.REL_SUCCESS, 1)
|
||||
FlowFile flowFile = runner.getFlowFilesForRelationship(CountText.REL_SUCCESS).first()
|
||||
logger.info("Generated flowfile: ${flowFile} | ${flowFile.attributes}")
|
||||
assert flowFile.attributes.get(CountText.TEXT_WORD_COUNT) == EXPECTED_WORD_COUNT as String
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldIgnoreWhitespaceWordsWhenCountingDebugMode() throws Exception {
|
||||
// Arrange
|
||||
MockComponentLog componentLogger = spy(new MockComponentLog("processorId", new CountText()))
|
||||
doReturn(true).when(componentLogger).isDebugEnabled()
|
||||
final TestRunner runner = TestRunners.newTestRunner(CountText.class, componentLogger)
|
||||
String INPUT_TEXT = "a b c"
|
||||
|
||||
final int EXPECTED_WORD_COUNT = 3
|
||||
|
||||
// Reset the processor properties
|
||||
runner.setProperty(CountText.TEXT_LINE_COUNT_PD, "false")
|
||||
runner.setProperty(CountText.TEXT_LINE_NONEMPTY_COUNT_PD, "false")
|
||||
runner.setProperty(CountText.TEXT_WORD_COUNT_PD, "true")
|
||||
runner.setProperty(CountText.TEXT_CHARACTER_COUNT_PD, "false")
|
||||
runner.setProperty(CountText.SPLIT_WORDS_ON_SYMBOLS_PD, "true")
|
||||
|
||||
runner.clearProvenanceEvents()
|
||||
runner.clearTransferState()
|
||||
runner.enqueue(INPUT_TEXT.bytes)
|
||||
|
||||
// Act
|
||||
runner.run()
|
||||
|
||||
// Assert
|
||||
runner.assertAllFlowFilesTransferred(CountText.REL_SUCCESS, 1)
|
||||
FlowFile flowFile = runner.getFlowFilesForRelationship(CountText.REL_SUCCESS).first()
|
||||
logger.info("Generated flowfile: ${flowFile} | ${flowFile.attributes}")
|
||||
assert flowFile.attributes.get(CountText.TEXT_WORD_COUNT) == EXPECTED_WORD_COUNT as String
|
||||
}
|
||||
|
||||
}
|
@ -1,416 +0,0 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License") you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.processors.standard
|
||||
|
||||
|
||||
import org.apache.nifi.security.util.crypto.HashAlgorithm
|
||||
import org.apache.nifi.security.util.crypto.HashService
|
||||
import org.apache.nifi.util.MockFlowFile
|
||||
import org.apache.nifi.util.TestRunner
|
||||
import org.apache.nifi.util.TestRunners
|
||||
import org.bouncycastle.jce.provider.BouncyCastleProvider
|
||||
import org.junit.After
|
||||
import org.junit.Before
|
||||
import org.junit.BeforeClass
|
||||
import org.junit.Test
|
||||
import org.junit.runner.RunWith
|
||||
import org.junit.runners.JUnit4
|
||||
import org.slf4j.Logger
|
||||
import org.slf4j.LoggerFactory
|
||||
|
||||
import java.nio.charset.Charset
|
||||
import java.nio.charset.StandardCharsets
|
||||
import java.security.Security
|
||||
import java.time.ZonedDateTime
|
||||
import java.time.format.DateTimeFormatter
|
||||
|
||||
@RunWith(JUnit4.class)
|
||||
class CryptographicHashAttributeTest extends GroovyTestCase {
|
||||
private static final Logger logger = LoggerFactory.getLogger(CryptographicHashAttributeTest.class)
|
||||
|
||||
|
||||
@BeforeClass
|
||||
static void setUpOnce() throws Exception {
|
||||
Security.addProvider(new BouncyCastleProvider())
|
||||
|
||||
logger.metaClass.methodMissing = { String name, args ->
|
||||
logger.info("[${name?.toUpperCase()}] ${(args as List).join(" ")}")
|
||||
}
|
||||
}
|
||||
|
||||
@Before
|
||||
void setUp() throws Exception {
|
||||
}
|
||||
|
||||
@After
|
||||
void tearDown() throws Exception {
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldCalculateHashOfPresentAttribute() {
|
||||
// Arrange
|
||||
def algorithms = HashAlgorithm.values()
|
||||
|
||||
final TestRunner runner = TestRunners.newTestRunner(new CryptographicHashAttribute())
|
||||
|
||||
// Create attributes for username and date
|
||||
def attributes = [
|
||||
username: "alopresto",
|
||||
// FIXME groovy-datetime dependency not providing the "format" method for new Date().format("YYYY-MM-dd HH:mm:ss.SSS Z")
|
||||
// adding the following workaround temporarily
|
||||
date : ZonedDateTime.now().format(DateTimeFormatter.ofPattern('YYYY-MM-dd HH:mm:ss.SSS Z'))
|
||||
]
|
||||
def attributeKeys = attributes.keySet()
|
||||
|
||||
algorithms.each { HashAlgorithm algorithm ->
|
||||
final EXPECTED_USERNAME_HASH = HashService.hashValue(algorithm, attributes["username"])
|
||||
logger.expected("${algorithm.name.padLeft(11)}(${attributes["username"]}) = ${EXPECTED_USERNAME_HASH}")
|
||||
final EXPECTED_DATE_HASH = HashService.hashValue(algorithm, attributes["date"])
|
||||
logger.expected("${algorithm.name.padLeft(11)}(${attributes["date"]}) = ${EXPECTED_DATE_HASH}")
|
||||
|
||||
// Reset the processor
|
||||
runner.clearProperties()
|
||||
runner.clearProvenanceEvents()
|
||||
runner.clearTransferState()
|
||||
|
||||
// Set the algorithm
|
||||
logger.info("Setting hash algorithm to ${algorithm.name}")
|
||||
runner.setProperty(CryptographicHashAttribute.HASH_ALGORITHM, algorithm.name)
|
||||
|
||||
// Add the desired dynamic properties
|
||||
attributeKeys.each { String attr ->
|
||||
runner.setProperty(attr, "${attr}_${algorithm.name}")
|
||||
}
|
||||
|
||||
// Insert the attributes in the mock flowfile
|
||||
runner.enqueue(new byte[0], attributes)
|
||||
|
||||
// Act
|
||||
runner.run(1)
|
||||
|
||||
// Assert
|
||||
runner.assertTransferCount(CryptographicHashAttribute.REL_FAILURE, 0)
|
||||
runner.assertTransferCount(CryptographicHashAttribute.REL_SUCCESS, 1)
|
||||
|
||||
final List<MockFlowFile> successfulFlowfiles = runner.getFlowFilesForRelationship(CryptographicHashAttribute.REL_SUCCESS)
|
||||
|
||||
// Extract the generated attributes from the flowfile
|
||||
MockFlowFile flowFile = successfulFlowfiles.first()
|
||||
String hashedUsername = flowFile.getAttribute("username_${algorithm.name}")
|
||||
logger.info("flowfile.username_${algorithm.name} = ${hashedUsername}")
|
||||
String hashedDate = flowFile.getAttribute("date_${algorithm.name}")
|
||||
logger.info("flowfile.date_${algorithm.name} = ${hashedDate}")
|
||||
|
||||
assert hashedUsername == EXPECTED_USERNAME_HASH
|
||||
assert hashedDate == EXPECTED_DATE_HASH
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldCalculateHashOfMissingAttribute() {
|
||||
// Arrange
|
||||
def algorithms = HashAlgorithm.values()
|
||||
|
||||
final TestRunner runner = TestRunners.newTestRunner(new CryptographicHashAttribute())
|
||||
|
||||
// Create attributes for username (empty string) and date (null)
|
||||
def attributes = [
|
||||
username: "",
|
||||
date : null
|
||||
]
|
||||
def attributeKeys = attributes.keySet()
|
||||
|
||||
algorithms.each { HashAlgorithm algorithm ->
|
||||
final EXPECTED_USERNAME_HASH = HashService.hashValue(algorithm, attributes["username"])
|
||||
logger.expected("${algorithm.name.padLeft(11)}(${attributes["username"]}) = ${EXPECTED_USERNAME_HASH}")
|
||||
final EXPECTED_DATE_HASH = null
|
||||
logger.expected("${algorithm.name.padLeft(11)}(${attributes["date"]}) = ${EXPECTED_DATE_HASH}")
|
||||
|
||||
// Reset the processor
|
||||
runner.clearProperties()
|
||||
runner.clearProvenanceEvents()
|
||||
runner.clearTransferState()
|
||||
|
||||
// Set the algorithm
|
||||
logger.info("Setting hash algorithm to ${algorithm.name}")
|
||||
runner.setProperty(CryptographicHashAttribute.HASH_ALGORITHM, algorithm.name)
|
||||
|
||||
// Add the desired dynamic properties
|
||||
attributeKeys.each { String attr ->
|
||||
runner.setProperty(attr, "${attr}_${algorithm.name}")
|
||||
}
|
||||
|
||||
// Insert the attributes in the mock flowfile
|
||||
runner.enqueue(new byte[0], attributes)
|
||||
|
||||
// Act
|
||||
runner.run(1)
|
||||
|
||||
// Assert
|
||||
runner.assertTransferCount(CryptographicHashAttribute.REL_FAILURE, 0)
|
||||
runner.assertTransferCount(CryptographicHashAttribute.REL_SUCCESS, 1)
|
||||
|
||||
final List<MockFlowFile> successfulFlowfiles = runner.getFlowFilesForRelationship(CryptographicHashAttribute.REL_SUCCESS)
|
||||
|
||||
// Extract the generated attributes from the flowfile
|
||||
MockFlowFile flowFile = successfulFlowfiles.first()
|
||||
String hashedUsername = flowFile.getAttribute("username_${algorithm.name}")
|
||||
logger.info("flowfile.username_${algorithm.name} = ${hashedUsername}")
|
||||
String hashedDate = flowFile.getAttribute("date_${algorithm.name}")
|
||||
logger.info("flowfile.date_${algorithm.name} = ${hashedDate}")
|
||||
|
||||
assert hashedUsername == EXPECTED_USERNAME_HASH
|
||||
assert hashedDate == EXPECTED_DATE_HASH
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldRouteToFailureOnProhibitedMissingAttribute() {
|
||||
// Arrange
|
||||
def algorithms = HashAlgorithm.values()
|
||||
|
||||
final TestRunner runner = TestRunners.newTestRunner(new CryptographicHashAttribute())
|
||||
|
||||
// Create attributes for username (empty string) and date (null)
|
||||
def attributes = [
|
||||
username: "",
|
||||
date : null
|
||||
]
|
||||
def attributeKeys = attributes.keySet()
|
||||
|
||||
algorithms.each { HashAlgorithm algorithm ->
|
||||
final EXPECTED_USERNAME_HASH = HashService.hashValue(algorithm, attributes["username"])
|
||||
logger.expected("${algorithm.name.padLeft(11)}(${attributes["username"]}) = ${EXPECTED_USERNAME_HASH}")
|
||||
final EXPECTED_DATE_HASH = null
|
||||
logger.expected("${algorithm.name.padLeft(11)}(${attributes["date"]}) = ${EXPECTED_DATE_HASH}")
|
||||
|
||||
// Reset the processor
|
||||
runner.clearProperties()
|
||||
runner.clearProvenanceEvents()
|
||||
runner.clearTransferState()
|
||||
|
||||
// Set the algorithm
|
||||
logger.info("Setting hash algorithm to ${algorithm.name}")
|
||||
runner.setProperty(CryptographicHashAttribute.HASH_ALGORITHM, algorithm.name)
|
||||
|
||||
// Set to fail if there are missing attributes
|
||||
runner.setProperty(CryptographicHashAttribute.PARTIAL_ATTR_ROUTE_POLICY, CryptographicHashAttribute.PartialAttributePolicy.PROHIBIT.name())
|
||||
|
||||
// Add the desired dynamic properties
|
||||
attributeKeys.each { String attr ->
|
||||
runner.setProperty(attr, "${attr}_${algorithm.name}")
|
||||
}
|
||||
|
||||
// Insert the attributes in the mock flowfile
|
||||
runner.enqueue(new byte[0], attributes)
|
||||
|
||||
// Act
|
||||
runner.run(1)
|
||||
|
||||
// Assert
|
||||
runner.assertTransferCount(CryptographicHashAttribute.REL_FAILURE, 1)
|
||||
runner.assertTransferCount(CryptographicHashAttribute.REL_SUCCESS, 0)
|
||||
|
||||
final List<MockFlowFile> failedFlowFiles = runner.getFlowFilesForRelationship(CryptographicHashAttribute.REL_FAILURE)
|
||||
|
||||
// Extract the generated attributes from the flowfile
|
||||
MockFlowFile flowFile = failedFlowFiles.first()
|
||||
logger.info("Failed flowfile has attributes ${flowFile.attributes}")
|
||||
attributeKeys.each { String missingAttribute ->
|
||||
flowFile.assertAttributeNotExists("${missingAttribute}_${algorithm.name}")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldRouteToFailureOnEmptyAttributes() {
|
||||
// Arrange
|
||||
def algorithms = HashAlgorithm.values()
|
||||
|
||||
final TestRunner runner = TestRunners.newTestRunner(new CryptographicHashAttribute())
|
||||
|
||||
// Create attributes for username (empty string) and date (null)
|
||||
def attributes = [
|
||||
username: "",
|
||||
date : null
|
||||
]
|
||||
def attributeKeys = attributes.keySet()
|
||||
|
||||
algorithms.each { HashAlgorithm algorithm ->
|
||||
// Reset the processor
|
||||
runner.clearProperties()
|
||||
runner.clearProvenanceEvents()
|
||||
runner.clearTransferState()
|
||||
|
||||
// Set the algorithm
|
||||
logger.info("Setting hash algorithm to ${algorithm.name}")
|
||||
runner.setProperty(CryptographicHashAttribute.HASH_ALGORITHM, algorithm.name)
|
||||
|
||||
// Set to fail if all attributes are missing
|
||||
runner.setProperty(CryptographicHashAttribute.FAIL_WHEN_EMPTY, "true")
|
||||
|
||||
// Insert the attributes in the mock flowfile
|
||||
runner.enqueue(new byte[0], attributes)
|
||||
|
||||
// Act
|
||||
runner.run(1)
|
||||
|
||||
// Assert
|
||||
runner.assertTransferCount(CryptographicHashAttribute.REL_FAILURE, 1)
|
||||
runner.assertTransferCount(CryptographicHashAttribute.REL_SUCCESS, 0)
|
||||
|
||||
final List<MockFlowFile> failedFlowFiles = runner.getFlowFilesForRelationship(CryptographicHashAttribute.REL_FAILURE)
|
||||
|
||||
// Extract the generated attributes from the flowfile
|
||||
MockFlowFile flowFile = failedFlowFiles.first()
|
||||
logger.info("Failed flowfile has attributes ${flowFile.attributes}")
|
||||
attributeKeys.each { String missingAttribute ->
|
||||
flowFile.assertAttributeNotExists("${missingAttribute}_${algorithm.name}")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldRouteToSuccessOnAllowPartial() {
|
||||
// Arrange
|
||||
def algorithms = HashAlgorithm.values()
|
||||
|
||||
final TestRunner runner = TestRunners.newTestRunner(new CryptographicHashAttribute())
|
||||
|
||||
// Create attributes for username (empty string) and date (null)
|
||||
def attributes = [
|
||||
username: ""
|
||||
]
|
||||
def attributeKeys = attributes.keySet()
|
||||
|
||||
algorithms.each { HashAlgorithm algorithm ->
|
||||
final EXPECTED_USERNAME_HASH = HashService.hashValue(algorithm, attributes["username"])
|
||||
logger.expected("${algorithm.name.padLeft(11)}(${attributes["username"]}) = ${EXPECTED_USERNAME_HASH}")
|
||||
final EXPECTED_DATE_HASH = null
|
||||
logger.expected("${algorithm.name.padLeft(11)}(${attributes["date"]}) = ${EXPECTED_DATE_HASH}")
|
||||
|
||||
// Reset the processor
|
||||
runner.clearProperties()
|
||||
runner.clearProvenanceEvents()
|
||||
runner.clearTransferState()
|
||||
|
||||
// Set the algorithm
|
||||
logger.info("Setting hash algorithm to ${algorithm.name}")
|
||||
runner.setProperty(CryptographicHashAttribute.HASH_ALGORITHM, algorithm.name)
|
||||
|
||||
// Set to fail if there are missing attributes
|
||||
runner.setProperty(CryptographicHashAttribute.PARTIAL_ATTR_ROUTE_POLICY, CryptographicHashAttribute.PartialAttributePolicy.ALLOW.name())
|
||||
|
||||
// Add the desired dynamic properties
|
||||
attributeKeys.each { String attr ->
|
||||
runner.setProperty(attr, "${attr}_${algorithm.name}")
|
||||
}
|
||||
|
||||
// Insert the attributes in the mock flowfile
|
||||
runner.enqueue(new byte[0], attributes)
|
||||
|
||||
// Act
|
||||
runner.run(1)
|
||||
|
||||
// Assert
|
||||
runner.assertTransferCount(CryptographicHashAttribute.REL_FAILURE, 0)
|
||||
runner.assertTransferCount(CryptographicHashAttribute.REL_SUCCESS, 1)
|
||||
|
||||
final List<MockFlowFile> successfulFlowFiles = runner.getFlowFilesForRelationship(CryptographicHashAttribute.REL_SUCCESS)
|
||||
|
||||
// Extract the generated attributes from the flowfile
|
||||
MockFlowFile flowFile = successfulFlowFiles.first()
|
||||
logger.info("Successful flowfile has attributes ${flowFile.attributes}")
|
||||
attributeKeys.each { String attribute ->
|
||||
flowFile.assertAttributeExists("${attribute}_${algorithm.name}")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldCalculateHashWithVariousCharacterEncodings() {
|
||||
// Arrange
|
||||
final TestRunner runner = TestRunners.newTestRunner(new CryptographicHashAttribute())
|
||||
|
||||
// Create attributes
|
||||
def attributes = [test_attribute: "apachenifi"]
|
||||
def attributeKeys = attributes.keySet()
|
||||
|
||||
HashAlgorithm algorithm = HashAlgorithm.MD5
|
||||
|
||||
List<Charset> charsets = [StandardCharsets.UTF_8, StandardCharsets.UTF_16, StandardCharsets.UTF_16LE, StandardCharsets.UTF_16BE]
|
||||
|
||||
final def EXPECTED_MD5_HASHES = [
|
||||
"utf_8" : "a968b5ec1d52449963dcc517789baaaf",
|
||||
"utf_16" : "b8413d18f7e64042bb0322a1cd61eba2",
|
||||
"utf_16be": "b8413d18f7e64042bb0322a1cd61eba2",
|
||||
"utf_16le": "91c3b67f9f8ae77156f21f271cc09121",
|
||||
]
|
||||
EXPECTED_MD5_HASHES.each { k, hash ->
|
||||
logger.expected("MD5(${k.padLeft(9)}(${attributes["test_attribute"]})) = ${hash}")
|
||||
}
|
||||
|
||||
charsets.each { Charset charset ->
|
||||
// Calculate the expected hash value given the character set
|
||||
final EXPECTED_HASH = HashService.hashValue(algorithm, attributes["test_attribute"], charset)
|
||||
logger.expected("${algorithm.name}(${attributes["test_attribute"]}, ${charset.name()}) = ${EXPECTED_HASH}")
|
||||
|
||||
// Sanity check
|
||||
assert EXPECTED_HASH == EXPECTED_MD5_HASHES[translateEncodingToMapKey(charset.name())]
|
||||
|
||||
// Reset the processor
|
||||
runner.clearProperties()
|
||||
runner.clearProvenanceEvents()
|
||||
runner.clearTransferState()
|
||||
|
||||
// Set the properties
|
||||
logger.info("Setting hash algorithm to ${algorithm.name}")
|
||||
runner.setProperty(CryptographicHashAttribute.HASH_ALGORITHM, algorithm.name)
|
||||
|
||||
logger.info("Setting character set to ${charset.name()}")
|
||||
runner.setProperty(CryptographicHashAttribute.CHARACTER_SET, charset.name())
|
||||
|
||||
// Add the desired dynamic properties
|
||||
attributeKeys.each { String attr ->
|
||||
runner.setProperty(attr, "${attr}_${algorithm.name}")
|
||||
}
|
||||
|
||||
// Insert the attributes in the mock flowfile
|
||||
runner.enqueue(new byte[0], attributes)
|
||||
|
||||
// Act
|
||||
runner.run(1)
|
||||
|
||||
// Assert
|
||||
runner.assertTransferCount(CryptographicHashAttribute.REL_FAILURE, 0)
|
||||
runner.assertTransferCount(CryptographicHashAttribute.REL_SUCCESS, 1)
|
||||
|
||||
final List<MockFlowFile> successfulFlowfiles = runner.getFlowFilesForRelationship(CryptographicHashAttribute.REL_SUCCESS)
|
||||
|
||||
// Extract the generated attributes from the flowfile
|
||||
MockFlowFile flowFile = successfulFlowfiles.first()
|
||||
String hashedAttribute = flowFile.getAttribute("test_attribute_${algorithm.name}")
|
||||
logger.info("flowfile.test_attribute_${algorithm.name} = ${hashedAttribute}")
|
||||
|
||||
assert hashedAttribute == EXPECTED_HASH
|
||||
}
|
||||
}
|
||||
|
||||
static String translateEncodingToMapKey(String charsetName) {
|
||||
charsetName.toLowerCase().replaceAll(/[-\/]/, '_')
|
||||
}
|
||||
}
|
@ -1,289 +0,0 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License") you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.processors.standard
|
||||
|
||||
|
||||
import org.apache.nifi.security.util.crypto.HashAlgorithm
|
||||
import org.apache.nifi.security.util.crypto.HashService
|
||||
import org.apache.nifi.util.MockFlowFile
|
||||
import org.apache.nifi.util.TestRunner
|
||||
import org.apache.nifi.util.TestRunners
|
||||
import org.bouncycastle.jce.provider.BouncyCastleProvider
|
||||
import org.junit.After
|
||||
import org.junit.Before
|
||||
import org.junit.BeforeClass
|
||||
import org.junit.Test
|
||||
import org.junit.runner.RunWith
|
||||
import org.junit.runners.JUnit4
|
||||
import org.slf4j.Logger
|
||||
import org.slf4j.LoggerFactory
|
||||
|
||||
import java.nio.charset.StandardCharsets
|
||||
import java.security.Security
|
||||
|
||||
@RunWith(JUnit4.class)
|
||||
class CryptographicHashContentTest extends GroovyTestCase {
|
||||
private static final Logger logger = LoggerFactory.getLogger(CryptographicHashContentTest.class)
|
||||
|
||||
@BeforeClass
|
||||
static void setUpOnce() throws Exception {
|
||||
Security.addProvider(new BouncyCastleProvider())
|
||||
|
||||
logger.metaClass.methodMissing = { String name, args ->
|
||||
logger.info("[${name?.toUpperCase()}] ${(args as List).join(" ")}")
|
||||
}
|
||||
}
|
||||
|
||||
@Before
|
||||
void setUp() throws Exception {
|
||||
}
|
||||
|
||||
@After
|
||||
void tearDown() throws Exception {
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldCalculateHashOfPresentContent() {
|
||||
// Arrange
|
||||
def algorithms = HashAlgorithm.values()
|
||||
|
||||
// Generate some long content (90 KB)
|
||||
final String LONG_CONTENT = "apachenifi " * 8192
|
||||
|
||||
final TestRunner runner = TestRunners.newTestRunner(new CryptographicHashContent())
|
||||
|
||||
algorithms.each { HashAlgorithm algorithm ->
|
||||
final String EXPECTED_CONTENT_HASH = HashService.hashValueStreaming(algorithm, new ByteArrayInputStream(LONG_CONTENT.bytes))
|
||||
logger.info("Expected ${algorithm.name.padLeft(11)}: ${EXPECTED_CONTENT_HASH}")
|
||||
|
||||
// Reset the processor
|
||||
runner.clearProperties()
|
||||
runner.clearProvenanceEvents()
|
||||
runner.clearTransferState()
|
||||
|
||||
// Set the algorithm
|
||||
logger.info("Setting hash algorithm to ${algorithm.name}")
|
||||
runner.setProperty(CryptographicHashContent.HASH_ALGORITHM, algorithm.name)
|
||||
|
||||
// Insert the content in the mock flowfile
|
||||
runner.enqueue(LONG_CONTENT.getBytes(StandardCharsets.UTF_8),
|
||||
[size: LONG_CONTENT.length() as String])
|
||||
|
||||
// Act
|
||||
runner.run(1)
|
||||
|
||||
// Assert
|
||||
runner.assertTransferCount(CryptographicHashContent.REL_FAILURE, 0)
|
||||
runner.assertTransferCount(CryptographicHashContent.REL_SUCCESS, 1)
|
||||
|
||||
final List<MockFlowFile> successfulFlowfiles = runner.getFlowFilesForRelationship(CryptographicHashContent.REL_SUCCESS)
|
||||
|
||||
// Extract the generated attributes from the flowfile
|
||||
MockFlowFile flowFile = successfulFlowfiles.first()
|
||||
String hashAttribute = "content_${algorithm.name}"
|
||||
flowFile.assertAttributeExists(hashAttribute)
|
||||
|
||||
String hashedContent = flowFile.getAttribute(hashAttribute)
|
||||
logger.info("flowfile.${hashAttribute} = ${hashedContent}")
|
||||
|
||||
assert hashedContent == EXPECTED_CONTENT_HASH
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldCalculateHashOfEmptyContent() {
|
||||
// Arrange
|
||||
def algorithms = HashAlgorithm.values()
|
||||
|
||||
final String EMPTY_CONTENT = ""
|
||||
|
||||
final TestRunner runner = TestRunners.newTestRunner(new CryptographicHashContent())
|
||||
|
||||
algorithms.each { HashAlgorithm algorithm ->
|
||||
final String EXPECTED_CONTENT_HASH = HashService.hashValueStreaming(algorithm, new ByteArrayInputStream(EMPTY_CONTENT.bytes))
|
||||
logger.info("Expected ${algorithm.name.padLeft(11)}: ${EXPECTED_CONTENT_HASH}")
|
||||
|
||||
// Reset the processor
|
||||
runner.clearProperties()
|
||||
runner.clearProvenanceEvents()
|
||||
runner.clearTransferState()
|
||||
|
||||
// Set the algorithm
|
||||
logger.info("Setting hash algorithm to ${algorithm.name}")
|
||||
runner.setProperty(CryptographicHashContent.HASH_ALGORITHM, algorithm.name)
|
||||
|
||||
// Insert the content in the mock flowfile
|
||||
runner.enqueue(EMPTY_CONTENT.getBytes(StandardCharsets.UTF_8), [size: "0"])
|
||||
|
||||
// Act
|
||||
runner.run(1)
|
||||
|
||||
// Assert
|
||||
runner.assertTransferCount(CryptographicHashContent.REL_FAILURE, 0)
|
||||
runner.assertTransferCount(CryptographicHashContent.REL_SUCCESS, 1)
|
||||
|
||||
final List<MockFlowFile> successfulFlowfiles = runner.getFlowFilesForRelationship(CryptographicHashContent.REL_SUCCESS)
|
||||
|
||||
// Extract the generated attributes from the flowfile
|
||||
MockFlowFile flowFile = successfulFlowfiles.first()
|
||||
String hashAttribute = "content_${algorithm.name}"
|
||||
flowFile.assertAttributeExists(hashAttribute)
|
||||
|
||||
String hashedContent = flowFile.getAttribute(hashAttribute)
|
||||
logger.info("flowfile.${hashAttribute} = ${hashedContent}")
|
||||
|
||||
assert hashedContent == EXPECTED_CONTENT_HASH
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This test works because {@link MockFlowFile} uses the actual internal {@code data.size} for {@code getSize ( )}, while {@code StandardFlowFileRecord} uses a separate {@code size} field. May need to use {@code flowfile.getContentClaim ( ) .getLength ( )}.
|
||||
*/
|
||||
@Test
|
||||
void testShouldCalculateHashOfContentWithIncorrectSizeAttribute() {
|
||||
// Arrange
|
||||
def algorithms = HashAlgorithm.values()
|
||||
|
||||
final String NON_EMPTY_CONTENT = "apachenifi"
|
||||
|
||||
final TestRunner runner = TestRunners.newTestRunner(new CryptographicHashContent())
|
||||
|
||||
algorithms.each { HashAlgorithm algorithm ->
|
||||
final String EXPECTED_CONTENT_HASH = HashService.hashValueStreaming(algorithm, new ByteArrayInputStream(NON_EMPTY_CONTENT.bytes))
|
||||
logger.info("Expected ${algorithm.name.padLeft(11)}: ${EXPECTED_CONTENT_HASH}")
|
||||
|
||||
// Reset the processor
|
||||
runner.clearProperties()
|
||||
runner.clearProvenanceEvents()
|
||||
runner.clearTransferState()
|
||||
|
||||
// Set the algorithm
|
||||
logger.info("Setting hash algorithm to ${algorithm.name}")
|
||||
runner.setProperty(CryptographicHashContent.HASH_ALGORITHM, algorithm.name)
|
||||
|
||||
// Insert the content in the mock flowfile (with the wrong size attribute)
|
||||
runner.enqueue(NON_EMPTY_CONTENT.getBytes(StandardCharsets.UTF_8), [size: "0"])
|
||||
|
||||
// Act
|
||||
runner.run(1)
|
||||
|
||||
// Assert
|
||||
runner.assertTransferCount(CryptographicHashContent.REL_FAILURE, 0)
|
||||
runner.assertTransferCount(CryptographicHashContent.REL_SUCCESS, 1)
|
||||
|
||||
final List<MockFlowFile> successfulFlowfiles = runner.getFlowFilesForRelationship(CryptographicHashContent.REL_SUCCESS)
|
||||
|
||||
// Extract the generated attributes from the flowfile
|
||||
MockFlowFile flowFile = successfulFlowfiles.first()
|
||||
String hashAttribute = "content_${algorithm.name}"
|
||||
flowFile.assertAttributeExists(hashAttribute)
|
||||
|
||||
String hashedContent = flowFile.getAttribute(hashAttribute)
|
||||
logger.info("flowfile.${hashAttribute} = ${hashedContent}")
|
||||
|
||||
assert hashedContent == EXPECTED_CONTENT_HASH
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldOverwriteExistingAttribute() {
|
||||
// Arrange
|
||||
final String NON_EMPTY_CONTENT = "apachenifi"
|
||||
final String OLD_HASH_ATTRIBUTE_VALUE = "OLD VALUE"
|
||||
|
||||
HashAlgorithm algorithm = HashAlgorithm.SHA256
|
||||
|
||||
final TestRunner runner = TestRunners.newTestRunner(new CryptographicHashContent())
|
||||
|
||||
final String EXPECTED_CONTENT_HASH = HashService.hashValue(algorithm, NON_EMPTY_CONTENT)
|
||||
logger.info("Expected ${algorithm.name.padLeft(11)}: ${EXPECTED_CONTENT_HASH}")
|
||||
|
||||
// Set the algorithm
|
||||
logger.info("Setting hash algorithm to ${algorithm.name}")
|
||||
runner.setProperty(CryptographicHashContent.HASH_ALGORITHM, algorithm.name)
|
||||
|
||||
// Insert the content in the mock flowfile (with an existing attribute)
|
||||
def oldAttributes = [("content_${algorithm.name}".toString()): OLD_HASH_ATTRIBUTE_VALUE]
|
||||
runner.enqueue(NON_EMPTY_CONTENT.getBytes(StandardCharsets.UTF_8),
|
||||
oldAttributes)
|
||||
|
||||
// Act
|
||||
runner.run(1)
|
||||
|
||||
// Assert
|
||||
runner.assertTransferCount(CryptographicHashContent.REL_FAILURE, 0)
|
||||
runner.assertTransferCount(CryptographicHashContent.REL_SUCCESS, 1)
|
||||
|
||||
final List<MockFlowFile> successfulFlowfiles = runner.getFlowFilesForRelationship(CryptographicHashContent.REL_SUCCESS)
|
||||
|
||||
// Extract the generated attributes from the flowfile
|
||||
MockFlowFile flowFile = successfulFlowfiles.first()
|
||||
String hashAttribute = "content_${algorithm.name}"
|
||||
flowFile.assertAttributeExists(hashAttribute)
|
||||
|
||||
String hashedContent = flowFile.getAttribute(hashAttribute)
|
||||
logger.info("flowfile.${hashAttribute} = ${hashedContent}")
|
||||
|
||||
assert hashedContent != OLD_HASH_ATTRIBUTE_VALUE
|
||||
assert hashedContent == EXPECTED_CONTENT_HASH
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldRouteToFailureOnEmptyContent() {
|
||||
// Arrange
|
||||
def algorithms = HashAlgorithm.values()
|
||||
|
||||
final String EMPTY_CONTENT = ""
|
||||
|
||||
final TestRunner runner = TestRunners.newTestRunner(new CryptographicHashContent())
|
||||
|
||||
algorithms.each { HashAlgorithm algorithm ->
|
||||
final String EXPECTED_CONTENT_HASH = HashService.hashValueStreaming(algorithm, new ByteArrayInputStream(EMPTY_CONTENT.bytes))
|
||||
logger.info("Expected ${algorithm.name.padLeft(11)}: ${EXPECTED_CONTENT_HASH}")
|
||||
|
||||
// Reset the processor
|
||||
runner.clearProperties()
|
||||
runner.clearProvenanceEvents()
|
||||
runner.clearTransferState()
|
||||
|
||||
// Set the failure property
|
||||
logger.info("Setting fail when empty to true")
|
||||
runner.setProperty(CryptographicHashContent.FAIL_WHEN_EMPTY, "true")
|
||||
|
||||
// Set the algorithm
|
||||
logger.info("Setting hash algorithm to ${algorithm.name}")
|
||||
runner.setProperty(CryptographicHashContent.HASH_ALGORITHM, algorithm.name)
|
||||
|
||||
// Insert the content in the mock flowfile
|
||||
runner.enqueue(EMPTY_CONTENT.getBytes(StandardCharsets.UTF_8))
|
||||
|
||||
// Act
|
||||
runner.run(1)
|
||||
|
||||
// Assert
|
||||
runner.assertTransferCount(CryptographicHashContent.REL_FAILURE, 1)
|
||||
runner.assertTransferCount(CryptographicHashContent.REL_SUCCESS, 0)
|
||||
|
||||
final List<MockFlowFile> failedFlowfiles = runner.getFlowFilesForRelationship(CryptographicHashContent.REL_FAILURE)
|
||||
|
||||
// Extract the generated attributes from the flowfile
|
||||
MockFlowFile flowFile = failedFlowfiles.first()
|
||||
String hashAttribute = "content_${algorithm.name}"
|
||||
flowFile.assertAttributeNotExists(hashAttribute)
|
||||
}
|
||||
}
|
||||
}
|
@ -1,83 +0,0 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.processors.standard
|
||||
|
||||
import org.apache.nifi.syslog.parsers.SyslogParser
|
||||
import org.apache.nifi.util.TestRunner
|
||||
import org.apache.nifi.util.TestRunners
|
||||
import org.bouncycastle.util.encoders.Hex
|
||||
import org.junit.After
|
||||
import org.junit.Assert
|
||||
import org.junit.Before
|
||||
import org.junit.BeforeClass
|
||||
import org.junit.Test
|
||||
import org.junit.runner.RunWith
|
||||
import org.junit.runners.JUnit4
|
||||
import org.slf4j.Logger
|
||||
import org.slf4j.LoggerFactory
|
||||
|
||||
@RunWith(JUnit4.class)
|
||||
class ParseSyslogGroovyTest extends GroovyTestCase {
|
||||
private static final Logger logger = LoggerFactory.getLogger(ParseSyslogGroovyTest.class)
|
||||
|
||||
@BeforeClass
|
||||
static void setUpOnce() throws Exception {
|
||||
logger.metaClass.methodMissing = { String name, args ->
|
||||
logger.info("[${name?.toUpperCase()}] ${(args as List).join(" ")}")
|
||||
}
|
||||
}
|
||||
|
||||
@Before
|
||||
void setUp() throws Exception {
|
||||
}
|
||||
|
||||
@After
|
||||
void tearDown() throws Exception {
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldHandleZeroLengthUDP() throws Exception {
|
||||
// Arrange
|
||||
final ParseSyslog proc = new ParseSyslog()
|
||||
final TestRunner runner = TestRunners.newTestRunner(proc)
|
||||
runner.setProperty(ParseSyslog.CHARSET, ParseSyslog.CHARSET.defaultValue)
|
||||
|
||||
// Inject a SyslogParser which will always return null
|
||||
def nullEventParser = [parseEvent: { byte[] bytes, String sender ->
|
||||
logger.mock("Regardless of input bytes: [${Hex.toHexString(bytes)}] and sender: [${sender}], this parser will return null")
|
||||
return null
|
||||
}] as SyslogParser
|
||||
proc.parser = nullEventParser
|
||||
|
||||
final int numMessages = 10
|
||||
|
||||
// Act
|
||||
numMessages.times {
|
||||
runner.enqueue("Doesn't matter what is enqueued here")
|
||||
}
|
||||
runner.run(numMessages)
|
||||
|
||||
int numFailed = runner.getFlowFilesForRelationship(ParseSyslog.REL_FAILURE).size()
|
||||
int numSuccess = runner.getFlowFilesForRelationship(ParseSyslog.REL_SUCCESS).size()
|
||||
logger.info("Transferred " + numSuccess + " to SUCCESS and " + numFailed + " to FAILURE")
|
||||
|
||||
// Assert
|
||||
|
||||
// all messages should be transferred to invalid
|
||||
Assert.assertEquals("Did not process all the messages", numMessages, numFailed)
|
||||
}
|
||||
}
|
@ -1,85 +0,0 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.processors.standard
|
||||
|
||||
import org.apache.nifi.util.TestRunner
|
||||
import org.apache.nifi.util.TestRunners
|
||||
import org.junit.After
|
||||
import org.junit.Before
|
||||
import org.junit.BeforeClass
|
||||
import org.junit.Test
|
||||
import org.junit.runner.RunWith
|
||||
import org.junit.runners.JUnit4
|
||||
import org.slf4j.Logger
|
||||
import org.slf4j.LoggerFactory
|
||||
|
||||
import java.nio.file.Paths
|
||||
|
||||
|
||||
@RunWith(JUnit4.class)
|
||||
class SplitXmlTest extends GroovyTestCase {
|
||||
private static final Logger logger = LoggerFactory.getLogger(SplitXmlTest.class)
|
||||
|
||||
@BeforeClass
|
||||
static void setUpOnce() throws Exception {
|
||||
logger.metaClass.methodMissing = { String name, args ->
|
||||
logger.info("[${name?.toUpperCase()}] ${(args as List).join(" ")}")
|
||||
}
|
||||
}
|
||||
|
||||
@Before
|
||||
void setUp() throws Exception {
|
||||
|
||||
}
|
||||
|
||||
@After
|
||||
void tearDown() throws Exception {
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldHandleXXEInTemplate() {
|
||||
// Arrange
|
||||
final String XXE_TEMPLATE_FILEPATH = "src/test/resources/xxe_template.xml"
|
||||
final TestRunner runner = TestRunners.newTestRunner(new SplitXml())
|
||||
runner.setProperty(SplitXml.SPLIT_DEPTH, "3")
|
||||
runner.enqueue(Paths.get(XXE_TEMPLATE_FILEPATH))
|
||||
|
||||
// Act
|
||||
runner.run()
|
||||
logger.info("SplitXML processor ran")
|
||||
|
||||
// Assert
|
||||
runner.assertAllFlowFilesTransferred(SplitXml.REL_FAILURE)
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldHandleRemoteCallXXE() {
|
||||
// Arrange
|
||||
final String XXE_TEMPLATE_FILEPATH = "src/test/resources/xxe_from_report.xml"
|
||||
final TestRunner runner = TestRunners.newTestRunner(new SplitXml())
|
||||
runner.setProperty(SplitXml.SPLIT_DEPTH, "3")
|
||||
runner.enqueue(Paths.get(XXE_TEMPLATE_FILEPATH))
|
||||
|
||||
// Act
|
||||
runner.run()
|
||||
logger.info("SplitXML processor ran")
|
||||
|
||||
// Assert
|
||||
runner.assertAllFlowFilesTransferred(SplitXml.REL_FAILURE)
|
||||
}
|
||||
}
|
@ -1,151 +0,0 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.processors.standard
|
||||
|
||||
import org.apache.nifi.serialization.SimpleRecordSchema
|
||||
import org.apache.nifi.serialization.record.MapRecord
|
||||
import org.apache.nifi.serialization.record.MockRecordParser
|
||||
import org.apache.nifi.serialization.record.RecordField
|
||||
import org.apache.nifi.serialization.record.RecordFieldType
|
||||
import org.apache.nifi.serialization.record.RecordSchema
|
||||
import org.apache.nifi.util.TestRunner
|
||||
import org.apache.nifi.util.TestRunners
|
||||
import org.junit.Assert
|
||||
import org.junit.Before
|
||||
import org.junit.Test
|
||||
|
||||
class TestCalculateRecordStats {
|
||||
TestRunner runner
|
||||
MockRecordParser recordParser
|
||||
RecordSchema personSchema
|
||||
|
||||
@Before
|
||||
void setup() {
|
||||
runner = TestRunners.newTestRunner(CalculateRecordStats.class)
|
||||
recordParser = new MockRecordParser()
|
||||
runner.addControllerService("recordReader", recordParser)
|
||||
runner.setProperty(CalculateRecordStats.RECORD_READER, "recordReader")
|
||||
runner.enableControllerService(recordParser)
|
||||
runner.assertValid()
|
||||
|
||||
recordParser.addSchemaField("id", RecordFieldType.INT)
|
||||
List<RecordField> personFields = new ArrayList<>()
|
||||
RecordField nameField = new RecordField("name", RecordFieldType.STRING.getDataType())
|
||||
RecordField ageField = new RecordField("age", RecordFieldType.INT.getDataType())
|
||||
RecordField sportField = new RecordField("sport", RecordFieldType.STRING.getDataType())
|
||||
personFields.add(nameField)
|
||||
personFields.add(ageField)
|
||||
personFields.add(sportField)
|
||||
personSchema = new SimpleRecordSchema(personFields)
|
||||
recordParser.addSchemaField("person", RecordFieldType.RECORD)
|
||||
}
|
||||
|
||||
@Test
|
||||
void testNoNullOrEmptyRecordFields() {
|
||||
def sports = [ "Soccer", "Soccer", "Soccer", "Football", "Football", "Basketball" ]
|
||||
def expectedAttributes = [
|
||||
"recordStats.sport.Soccer": "3",
|
||||
"recordStats.sport.Football": "2",
|
||||
"recordStats.sport.Basketball": "1",
|
||||
"recordStats.sport": "6",
|
||||
"record.count": "6"
|
||||
]
|
||||
|
||||
commonTest([ "sport": "/person/sport"], sports, expectedAttributes)
|
||||
}
|
||||
|
||||
@Test
|
||||
void testWithNullFields() {
|
||||
def sports = [ "Soccer", null, null, "Football", null, "Basketball" ]
|
||||
def expectedAttributes = [
|
||||
"recordStats.sport.Soccer": "1",
|
||||
"recordStats.sport.Football": "1",
|
||||
"recordStats.sport.Basketball": "1",
|
||||
"recordStats.sport": "3",
|
||||
"record.count": "6"
|
||||
]
|
||||
|
||||
commonTest([ "sport": "/person/sport"], sports, expectedAttributes)
|
||||
}
|
||||
|
||||
@Test
|
||||
void testWithFilters() {
|
||||
def sports = [ "Soccer", "Soccer", "Soccer", "Football", "Football", "Basketball" ]
|
||||
def expectedAttributes = [
|
||||
"recordStats.sport.Soccer": "3",
|
||||
"recordStats.sport.Basketball": "1",
|
||||
"recordStats.sport": "4",
|
||||
"record.count": "6"
|
||||
]
|
||||
|
||||
def propz = [
|
||||
"sport": "/person/sport[. != 'Football']"
|
||||
]
|
||||
|
||||
commonTest(propz, sports, expectedAttributes)
|
||||
}
|
||||
|
||||
@Test
|
||||
void testWithSizeLimit() {
|
||||
runner.setProperty(CalculateRecordStats.LIMIT, "3")
|
||||
def sports = [ "Soccer", "Soccer", "Soccer", "Football", "Football",
|
||||
"Basketball", "Baseball", "Baseball", "Baseball", "Baseball",
|
||||
"Skiing", "Skiing", "Skiing", "Snowboarding"
|
||||
]
|
||||
def expectedAttributes = [
|
||||
"recordStats.sport.Skiing": "3",
|
||||
"recordStats.sport.Soccer": "3",
|
||||
"recordStats.sport.Baseball": "4",
|
||||
"recordStats.sport": String.valueOf(sports.size()),
|
||||
"record.count": String.valueOf(sports.size())
|
||||
]
|
||||
|
||||
def propz = [
|
||||
"sport": "/person/sport"
|
||||
]
|
||||
|
||||
commonTest(propz, sports, expectedAttributes)
|
||||
}
|
||||
|
||||
private void commonTest(Map procProperties, List sports, Map expectedAttributes) {
|
||||
int index = 1
|
||||
sports.each { sport ->
|
||||
recordParser.addRecord(index++, new MapRecord(personSchema, [
|
||||
"name" : "John Doe",
|
||||
"age" : 48,
|
||||
"sport": sport
|
||||
]))
|
||||
}
|
||||
|
||||
procProperties.each { kv ->
|
||||
runner.setProperty(kv.key, kv.value)
|
||||
}
|
||||
|
||||
runner.enqueue("")
|
||||
runner.run()
|
||||
runner.assertTransferCount(CalculateRecordStats.REL_FAILURE, 0)
|
||||
runner.assertTransferCount(CalculateRecordStats.REL_SUCCESS, 1)
|
||||
|
||||
def flowFiles = runner.getFlowFilesForRelationship(CalculateRecordStats.REL_SUCCESS)
|
||||
def ff = flowFiles[0]
|
||||
expectedAttributes.each { kv ->
|
||||
Assert.assertNotNull("Missing ${kv.key}", ff.getAttribute(kv.key))
|
||||
Assert.assertEquals(kv.value, ff.getAttribute(kv.key))
|
||||
}
|
||||
}
|
||||
}
|
@ -1,943 +0,0 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License") you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.processors.standard
|
||||
|
||||
import groovy.time.TimeCategory
|
||||
import groovy.time.TimeDuration
|
||||
import org.apache.commons.codec.binary.Hex
|
||||
import org.apache.nifi.components.ValidationResult
|
||||
import org.apache.nifi.security.util.EncryptionMethod
|
||||
import org.apache.nifi.security.util.KeyDerivationFunction
|
||||
import org.apache.nifi.security.util.crypto.Argon2CipherProvider
|
||||
import org.apache.nifi.security.util.crypto.Argon2SecureHasher
|
||||
import org.apache.nifi.security.util.crypto.CipherUtility
|
||||
import org.apache.nifi.security.util.crypto.KeyedEncryptor
|
||||
import org.apache.nifi.security.util.crypto.PasswordBasedEncryptor
|
||||
import org.apache.nifi.security.util.crypto.RandomIVPBECipherProvider
|
||||
import org.apache.nifi.util.MockFlowFile
|
||||
import org.apache.nifi.util.MockProcessContext
|
||||
import org.apache.nifi.util.TestRunner
|
||||
import org.apache.nifi.util.TestRunners
|
||||
import org.bouncycastle.jce.provider.BouncyCastleProvider
|
||||
import org.junit.After
|
||||
import org.junit.Assert
|
||||
import org.junit.Before
|
||||
import org.junit.BeforeClass
|
||||
import org.junit.Test
|
||||
import org.junit.runner.RunWith
|
||||
import org.junit.runners.JUnit4
|
||||
import org.slf4j.Logger
|
||||
import org.slf4j.LoggerFactory
|
||||
|
||||
import javax.crypto.Cipher
|
||||
import java.nio.charset.StandardCharsets
|
||||
import java.nio.file.Paths
|
||||
import java.security.Security
|
||||
import java.text.SimpleDateFormat
|
||||
import java.time.Instant
|
||||
import java.time.temporal.ChronoUnit
|
||||
|
||||
@RunWith(JUnit4.class)
|
||||
class TestEncryptContentGroovy {
|
||||
private static final Logger logger = LoggerFactory.getLogger(TestEncryptContentGroovy.class)
|
||||
|
||||
private static final String WEAK_CRYPTO_ALLOWED = EncryptContent.WEAK_CRYPTO_ALLOWED_NAME
|
||||
private static final String WEAK_CRYPTO_NOT_ALLOWED = EncryptContent.WEAK_CRYPTO_NOT_ALLOWED_NAME
|
||||
|
||||
private static final List<EncryptionMethod> SUPPORTED_KEYED_ENCRYPTION_METHODS = EncryptionMethod.values().findAll { it.isKeyedCipher() && it != EncryptionMethod.AES_CBC_NO_PADDING }
|
||||
|
||||
@BeforeClass
|
||||
static void setUpOnce() throws Exception {
|
||||
Security.addProvider(new BouncyCastleProvider())
|
||||
|
||||
logger.metaClass.methodMissing = { String name, args ->
|
||||
logger.info("[${name?.toUpperCase()}] ${(args as List).join(" ")}")
|
||||
}
|
||||
}
|
||||
|
||||
@Before
|
||||
void setUp() throws Exception {
|
||||
}
|
||||
|
||||
@After
|
||||
void tearDown() throws Exception {
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldValidateMaxKeySizeForAlgorithmsOnUnlimitedStrengthJVM() throws IOException {
|
||||
// Arrange
|
||||
final TestRunner runner = TestRunners.newTestRunner(EncryptContent.class)
|
||||
Collection<ValidationResult> results
|
||||
MockProcessContext pc
|
||||
|
||||
EncryptionMethod encryptionMethod = EncryptionMethod.AES_CBC
|
||||
|
||||
// Integer.MAX_VALUE or 128, so use 256 or 128
|
||||
final int MAX_KEY_LENGTH = [PasswordBasedEncryptor.getMaxAllowedKeyLength(encryptionMethod.algorithm), 256].min()
|
||||
final String TOO_LONG_KEY_HEX = "ab" * (MAX_KEY_LENGTH / 8 + 1)
|
||||
logger.info("Using key ${TOO_LONG_KEY_HEX} (${TOO_LONG_KEY_HEX.length() * 4} bits)")
|
||||
|
||||
runner.setProperty(EncryptContent.MODE, EncryptContent.ENCRYPT_MODE)
|
||||
runner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, encryptionMethod.name())
|
||||
runner.setProperty(EncryptContent.KEY_DERIVATION_FUNCTION, KeyDerivationFunction.NONE.name())
|
||||
runner.setProperty(EncryptContent.RAW_KEY_HEX, TOO_LONG_KEY_HEX)
|
||||
|
||||
runner.enqueue(new byte[0])
|
||||
pc = (MockProcessContext) runner.getProcessContext()
|
||||
|
||||
// Act
|
||||
results = pc.validate()
|
||||
|
||||
// Assert
|
||||
Assert.assertEquals(1, results.size())
|
||||
logger.expected(results)
|
||||
ValidationResult vr = results.first()
|
||||
|
||||
String expectedResult = "'raw-key-hex' is invalid because Key must be valid length [128, 192, 256]"
|
||||
String message = "'" + vr.toString() + "' contains '" + expectedResult + "'"
|
||||
Assert.assertTrue(message, vr.toString().contains(expectedResult))
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldValidateKeyFormatAndSizeForAlgorithms() throws IOException {
|
||||
// Arrange
|
||||
final TestRunner runner = TestRunners.newTestRunner(EncryptContent.class)
|
||||
Collection<ValidationResult> results
|
||||
MockProcessContext pc
|
||||
|
||||
EncryptionMethod encryptionMethod = EncryptionMethod.AES_CBC
|
||||
|
||||
final int INVALID_KEY_LENGTH = 120
|
||||
final String INVALID_KEY_HEX = "ab" * (INVALID_KEY_LENGTH / 8)
|
||||
logger.info("Using key ${INVALID_KEY_HEX} (${INVALID_KEY_HEX.length() * 4} bits)")
|
||||
|
||||
runner.setProperty(EncryptContent.MODE, EncryptContent.ENCRYPT_MODE)
|
||||
runner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, encryptionMethod.name())
|
||||
runner.setProperty(EncryptContent.KEY_DERIVATION_FUNCTION, KeyDerivationFunction.NONE.name())
|
||||
runner.setProperty(EncryptContent.RAW_KEY_HEX, INVALID_KEY_HEX)
|
||||
|
||||
runner.enqueue(new byte[0])
|
||||
pc = (MockProcessContext) runner.getProcessContext()
|
||||
|
||||
// Act
|
||||
results = pc.validate()
|
||||
|
||||
// Assert
|
||||
Assert.assertEquals(1, results.size())
|
||||
logger.expected(results)
|
||||
ValidationResult keyLengthInvalidVR = results.first()
|
||||
|
||||
String expectedResult = "'raw-key-hex' is invalid because Key must be valid length [128, 192, 256]"
|
||||
String message = "'" + keyLengthInvalidVR.toString() + "' contains '" + expectedResult + "'"
|
||||
Assert.assertTrue(message, keyLengthInvalidVR.toString().contains(expectedResult))
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldValidateKDFWhenKeyedCipherSelected() {
|
||||
// Arrange
|
||||
final TestRunner runner = TestRunners.newTestRunner(EncryptContent.class)
|
||||
Collection<ValidationResult> results
|
||||
MockProcessContext pc
|
||||
|
||||
final int VALID_KEY_LENGTH = 128
|
||||
final String VALID_KEY_HEX = "ab" * (VALID_KEY_LENGTH / 8)
|
||||
logger.info("Using key ${VALID_KEY_HEX} (${VALID_KEY_HEX.length() * 4} bits)")
|
||||
|
||||
runner.setProperty(EncryptContent.MODE, EncryptContent.ENCRYPT_MODE)
|
||||
|
||||
SUPPORTED_KEYED_ENCRYPTION_METHODS.each { EncryptionMethod encryptionMethod ->
|
||||
logger.info("Trying encryption method ${encryptionMethod.name()}")
|
||||
runner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, encryptionMethod.name())
|
||||
|
||||
// Scenario 1: Legacy KDF + keyed cipher -> validation error
|
||||
final def INVALID_KDFS = [KeyDerivationFunction.NIFI_LEGACY, KeyDerivationFunction.OPENSSL_EVP_BYTES_TO_KEY]
|
||||
INVALID_KDFS.each { KeyDerivationFunction invalidKDF ->
|
||||
logger.info("Trying KDF ${invalidKDF.name()}")
|
||||
|
||||
runner.setProperty(EncryptContent.KEY_DERIVATION_FUNCTION, invalidKDF.name())
|
||||
runner.setProperty(EncryptContent.RAW_KEY_HEX, VALID_KEY_HEX)
|
||||
runner.removeProperty(EncryptContent.PASSWORD)
|
||||
|
||||
runner.enqueue(new byte[0])
|
||||
pc = (MockProcessContext) runner.getProcessContext()
|
||||
|
||||
// Act
|
||||
results = pc.validate()
|
||||
|
||||
// Assert
|
||||
logger.expected(results)
|
||||
assert results.size() == 1
|
||||
ValidationResult keyLengthInvalidVR = results.first()
|
||||
|
||||
String expectedResult = "'key-derivation-function' is invalid because Key Derivation Function is required to be BCRYPT, SCRYPT, PBKDF2, ARGON2, NONE when using " +
|
||||
"algorithm ${encryptionMethod.algorithm}"
|
||||
String message = "'" + keyLengthInvalidVR.toString() + "' contains '" + expectedResult + "'"
|
||||
assert keyLengthInvalidVR.toString().contains(expectedResult)
|
||||
}
|
||||
|
||||
// Scenario 2: No KDF + keyed cipher + raw-key-hex -> valid
|
||||
def none = KeyDerivationFunction.NONE
|
||||
logger.info("Trying KDF ${none.name()}")
|
||||
|
||||
runner.setProperty(EncryptContent.KEY_DERIVATION_FUNCTION, none.name())
|
||||
runner.setProperty(EncryptContent.RAW_KEY_HEX, VALID_KEY_HEX)
|
||||
runner.removeProperty(EncryptContent.PASSWORD)
|
||||
|
||||
runner.enqueue(new byte[0])
|
||||
pc = (MockProcessContext) runner.getProcessContext()
|
||||
|
||||
// Act
|
||||
results = pc.validate()
|
||||
|
||||
// Assert
|
||||
assert results.isEmpty()
|
||||
|
||||
// Scenario 3: Strong KDF + keyed cipher + password -> valid
|
||||
final def VALID_KDFS = [KeyDerivationFunction.BCRYPT, KeyDerivationFunction.SCRYPT, KeyDerivationFunction.PBKDF2, KeyDerivationFunction.ARGON2]
|
||||
VALID_KDFS.each { KeyDerivationFunction validKDF ->
|
||||
logger.info("Trying KDF ${validKDF.name()}")
|
||||
|
||||
runner.setProperty(EncryptContent.KEY_DERIVATION_FUNCTION, validKDF.name())
|
||||
runner.setProperty(EncryptContent.PASSWORD, "thisIsABadPassword")
|
||||
runner.removeProperty(EncryptContent.RAW_KEY_HEX)
|
||||
|
||||
runner.enqueue(new byte[0])
|
||||
pc = (MockProcessContext) runner.getProcessContext()
|
||||
|
||||
// Act
|
||||
results = pc.validate()
|
||||
|
||||
// Assert
|
||||
assert results.isEmpty()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testKDFShouldDefaultToNone() {
|
||||
// Arrange
|
||||
final TestRunner runner = TestRunners.newTestRunner(EncryptContent.class)
|
||||
Collection<ValidationResult> results
|
||||
MockProcessContext pc
|
||||
|
||||
runner.enqueue(new byte[0])
|
||||
pc = (MockProcessContext) runner.getProcessContext()
|
||||
|
||||
// Act
|
||||
String defaultKDF = pc.getProperty("key-derivation-function").getValue()
|
||||
|
||||
// Assert
|
||||
assert defaultKDF == KeyDerivationFunction.NONE.name()
|
||||
}
|
||||
|
||||
@Test
|
||||
void testEMShouldDefaultToAES_GCM() {
|
||||
// Arrange
|
||||
final TestRunner runner = TestRunners.newTestRunner(EncryptContent.class)
|
||||
Collection<ValidationResult> results
|
||||
MockProcessContext pc
|
||||
|
||||
runner.enqueue(new byte[0])
|
||||
pc = (MockProcessContext) runner.getProcessContext()
|
||||
|
||||
// Act
|
||||
String defaultEM = pc.getProperty("Encryption Algorithm").getValue()
|
||||
|
||||
// Assert
|
||||
assert defaultEM == EncryptionMethod.AES_GCM.name()
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldValidateKeyMaterialSourceWhenKeyedCipherSelected() {
|
||||
// Arrange
|
||||
final TestRunner runner = TestRunners.newTestRunner(EncryptContent.class)
|
||||
Collection<ValidationResult> results
|
||||
MockProcessContext pc
|
||||
|
||||
logger.info("Testing keyed encryption methods: ${SUPPORTED_KEYED_ENCRYPTION_METHODS*.name()}")
|
||||
|
||||
final int VALID_KEY_LENGTH = 128
|
||||
final String VALID_KEY_HEX = "ab" * (VALID_KEY_LENGTH / 8)
|
||||
logger.info("Using key ${VALID_KEY_HEX} (${VALID_KEY_HEX.length() * 4} bits)")
|
||||
|
||||
final String VALID_PASSWORD = "thisIsABadPassword"
|
||||
logger.info("Using password ${VALID_PASSWORD} (${VALID_PASSWORD.length()} bytes)")
|
||||
|
||||
runner.setProperty(EncryptContent.MODE, EncryptContent.ENCRYPT_MODE)
|
||||
KeyDerivationFunction none = KeyDerivationFunction.NONE
|
||||
final def VALID_KDFS = KeyDerivationFunction.values().findAll { it.isStrongKDF() }
|
||||
|
||||
// Scenario 1 - RKH w/ KDF NONE & em in [CBC, CTR, GCM] (no password)
|
||||
SUPPORTED_KEYED_ENCRYPTION_METHODS.each { EncryptionMethod kem ->
|
||||
logger.info("Trying encryption method ${kem.name()} with KDF ${none.name()}")
|
||||
runner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, kem.name())
|
||||
runner.setProperty(EncryptContent.KEY_DERIVATION_FUNCTION, none.name())
|
||||
|
||||
logger.info("Setting raw key hex: ${VALID_KEY_HEX}")
|
||||
runner.setProperty(EncryptContent.RAW_KEY_HEX, VALID_KEY_HEX)
|
||||
runner.removeProperty(EncryptContent.PASSWORD)
|
||||
|
||||
runner.enqueue(new byte[0])
|
||||
pc = (MockProcessContext) runner.getProcessContext()
|
||||
|
||||
// Act
|
||||
results = pc.validate()
|
||||
|
||||
// Assert
|
||||
assert results.isEmpty()
|
||||
|
||||
// Scenario 2 - PW w/ KDF in [BCRYPT, SCRYPT, PBKDF2, ARGON2] & em in [CBC, CTR, GCM] (no RKH)
|
||||
VALID_KDFS.each { KeyDerivationFunction kdf ->
|
||||
logger.info("Trying encryption method ${kem.name()} with KDF ${kdf.name()}")
|
||||
runner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, kem.name())
|
||||
runner.setProperty(EncryptContent.KEY_DERIVATION_FUNCTION, kdf.name())
|
||||
|
||||
logger.info("Setting password: ${VALID_PASSWORD}")
|
||||
runner.removeProperty(EncryptContent.RAW_KEY_HEX)
|
||||
runner.setProperty(EncryptContent.PASSWORD, VALID_PASSWORD)
|
||||
|
||||
runner.enqueue(new byte[0])
|
||||
pc = (MockProcessContext) runner.getProcessContext()
|
||||
|
||||
// Act
|
||||
results = pc.validate()
|
||||
|
||||
// Assert
|
||||
assert results.isEmpty()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldValidateKDFWhenPBECipherSelected() {
|
||||
// Arrange
|
||||
final TestRunner runner = TestRunners.newTestRunner(EncryptContent.class)
|
||||
Collection<ValidationResult> results
|
||||
MockProcessContext pc
|
||||
final String PASSWORD = "short"
|
||||
|
||||
def encryptionMethods = EncryptionMethod.values().findAll { it.algorithm.startsWith("PBE") }
|
||||
|
||||
runner.setProperty(EncryptContent.MODE, EncryptContent.ENCRYPT_MODE)
|
||||
runner.setProperty(EncryptContent.PASSWORD, PASSWORD)
|
||||
runner.setProperty(EncryptContent.ALLOW_WEAK_CRYPTO, WEAK_CRYPTO_ALLOWED)
|
||||
|
||||
encryptionMethods.each { EncryptionMethod encryptionMethod ->
|
||||
logger.info("Trying encryption method ${encryptionMethod.name()}")
|
||||
runner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, encryptionMethod.name())
|
||||
|
||||
final def INVALID_KDFS = [KeyDerivationFunction.NONE, KeyDerivationFunction.BCRYPT, KeyDerivationFunction.SCRYPT, KeyDerivationFunction.PBKDF2, KeyDerivationFunction.ARGON2]
|
||||
INVALID_KDFS.each { KeyDerivationFunction invalidKDF ->
|
||||
logger.info("Trying KDF ${invalidKDF.name()}")
|
||||
|
||||
runner.setProperty(EncryptContent.KEY_DERIVATION_FUNCTION, invalidKDF.name())
|
||||
|
||||
runner.enqueue(new byte[0])
|
||||
pc = (MockProcessContext) runner.getProcessContext()
|
||||
|
||||
// Act
|
||||
results = pc.validate()
|
||||
|
||||
// Assert
|
||||
logger.expected(results)
|
||||
Assert.assertEquals(1, results.size())
|
||||
ValidationResult keyLengthInvalidVR = results.first()
|
||||
|
||||
String expectedResult = "'Key Derivation Function' is invalid because Key Derivation Function is required to be NIFI_LEGACY, OPENSSL_EVP_BYTES_TO_KEY when using " +
|
||||
"algorithm ${encryptionMethod.algorithm}"
|
||||
String message = "'" + keyLengthInvalidVR.toString() + "' contains '" + expectedResult + "'"
|
||||
Assert.assertTrue(message, keyLengthInvalidVR.toString().contains(expectedResult))
|
||||
}
|
||||
|
||||
final def VALID_KDFS = [KeyDerivationFunction.NIFI_LEGACY, KeyDerivationFunction.OPENSSL_EVP_BYTES_TO_KEY]
|
||||
VALID_KDFS.each { KeyDerivationFunction validKDF ->
|
||||
logger.info("Trying KDF ${validKDF.name()}")
|
||||
|
||||
runner.setProperty(EncryptContent.KEY_DERIVATION_FUNCTION, validKDF.name())
|
||||
|
||||
runner.enqueue(new byte[0])
|
||||
pc = (MockProcessContext) runner.getProcessContext()
|
||||
|
||||
// Act
|
||||
results = pc.validate()
|
||||
|
||||
// Assert
|
||||
Assert.assertEquals(0, results.size())
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testRoundTrip() throws IOException {
|
||||
final TestRunner testRunner = TestRunners.newTestRunner(new EncryptContent())
|
||||
final String RAW_KEY_HEX = "ab" * 16
|
||||
testRunner.setProperty(EncryptContent.RAW_KEY_HEX, RAW_KEY_HEX)
|
||||
testRunner.setProperty(EncryptContent.KEY_DERIVATION_FUNCTION, KeyDerivationFunction.NONE.name())
|
||||
|
||||
SUPPORTED_KEYED_ENCRYPTION_METHODS.each { EncryptionMethod encryptionMethod ->
|
||||
logger.info("Attempting {}", encryptionMethod.name())
|
||||
testRunner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, encryptionMethod.name())
|
||||
testRunner.setProperty(EncryptContent.MODE, EncryptContent.ENCRYPT_MODE)
|
||||
|
||||
testRunner.enqueue(Paths.get("src/test/resources/hello.txt"))
|
||||
testRunner.clearTransferState()
|
||||
testRunner.run()
|
||||
|
||||
testRunner.assertAllFlowFilesTransferred(EncryptContent.REL_SUCCESS, 1)
|
||||
|
||||
MockFlowFile flowFile = testRunner.getFlowFilesForRelationship(EncryptContent.REL_SUCCESS).get(0)
|
||||
testRunner.assertQueueEmpty()
|
||||
|
||||
testRunner.setProperty(EncryptContent.MODE, EncryptContent.DECRYPT_MODE)
|
||||
testRunner.enqueue(flowFile)
|
||||
testRunner.clearTransferState()
|
||||
testRunner.run()
|
||||
testRunner.assertAllFlowFilesTransferred(EncryptContent.REL_SUCCESS, 1)
|
||||
|
||||
logger.info("Successfully decrypted {}", encryptionMethod.name())
|
||||
|
||||
flowFile = testRunner.getFlowFilesForRelationship(EncryptContent.REL_SUCCESS).get(0)
|
||||
flowFile.assertContentEquals(new File("src/test/resources/hello.txt"))
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testDecryptAesCbcNoPadding() {
|
||||
final TestRunner testRunner = TestRunners.newTestRunner(new EncryptContent())
|
||||
final String RAW_KEY_HEX = "ab" * 16
|
||||
testRunner.setProperty(EncryptContent.RAW_KEY_HEX, RAW_KEY_HEX)
|
||||
testRunner.setProperty(EncryptContent.KEY_DERIVATION_FUNCTION, KeyDerivationFunction.NONE.name())
|
||||
testRunner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, EncryptionMethod.AES_CBC_NO_PADDING.name())
|
||||
testRunner.setProperty(EncryptContent.MODE, EncryptContent.DECRYPT_MODE)
|
||||
|
||||
final String content = "ExactBlockSizeRequiredForProcess"
|
||||
final byte[] bytes = content.getBytes(StandardCharsets.UTF_8)
|
||||
final ByteArrayInputStream inputStream = new ByteArrayInputStream(bytes)
|
||||
final ByteArrayOutputStream outputStream = new ByteArrayOutputStream()
|
||||
|
||||
final KeyedEncryptor encryptor = new KeyedEncryptor(EncryptionMethod.AES_CBC_NO_PADDING, Hex.decodeHex(RAW_KEY_HEX))
|
||||
encryptor.encryptionCallback.process(inputStream, outputStream)
|
||||
outputStream.close()
|
||||
|
||||
final byte[] encrypted = outputStream.toByteArray()
|
||||
testRunner.enqueue(encrypted)
|
||||
testRunner.run()
|
||||
|
||||
testRunner.assertAllFlowFilesTransferred(EncryptContent.REL_SUCCESS, 1)
|
||||
MockFlowFile flowFile = testRunner.getFlowFilesForRelationship(EncryptContent.REL_SUCCESS).get(0)
|
||||
flowFile.assertContentEquals(content)
|
||||
}
|
||||
|
||||
// TODO: Implement
|
||||
@Test
|
||||
void testArgon2EncryptionShouldWriteAttributesWithEncryptionMetadata() throws IOException {
|
||||
// Arrange
|
||||
final TestRunner testRunner = TestRunners.newTestRunner(new EncryptContent())
|
||||
KeyDerivationFunction kdf = KeyDerivationFunction.ARGON2
|
||||
EncryptionMethod encryptionMethod = EncryptionMethod.AES_CBC
|
||||
logger.info("Attempting encryption with {}", encryptionMethod.name())
|
||||
|
||||
testRunner.setProperty(EncryptContent.PASSWORD, "thisIsABadPassword")
|
||||
testRunner.setProperty(EncryptContent.KEY_DERIVATION_FUNCTION, kdf.name())
|
||||
testRunner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, encryptionMethod.name())
|
||||
testRunner.setProperty(EncryptContent.MODE, EncryptContent.ENCRYPT_MODE)
|
||||
|
||||
String PLAINTEXT = "This is a plaintext message. "
|
||||
|
||||
// Act
|
||||
testRunner.enqueue(PLAINTEXT)
|
||||
testRunner.clearTransferState()
|
||||
testRunner.run()
|
||||
|
||||
// Assert
|
||||
testRunner.assertAllFlowFilesTransferred(EncryptContent.REL_SUCCESS, 1)
|
||||
logger.info("Successfully encrypted with {}", encryptionMethod.name())
|
||||
|
||||
MockFlowFile flowFile = testRunner.getFlowFilesForRelationship(EncryptContent.REL_SUCCESS).get(0)
|
||||
testRunner.assertQueueEmpty()
|
||||
|
||||
printFlowFileAttributes(flowFile.getAttributes())
|
||||
|
||||
byte[] flowfileContentBytes = flowFile.getData()
|
||||
String flowfileContent = flowFile.getContent()
|
||||
|
||||
int ivDelimiterStart = CipherUtility.findSequence(flowfileContentBytes, RandomIVPBECipherProvider.IV_DELIMITER)
|
||||
logger.info("IV delimiter starts at ${ivDelimiterStart}")
|
||||
|
||||
final byte[] EXPECTED_KDF_SALT_BYTES = extractFullSaltFromCipherBytes(flowfileContentBytes)
|
||||
final String EXPECTED_KDF_SALT = new String(EXPECTED_KDF_SALT_BYTES)
|
||||
final String EXPECTED_SALT_HEX = extractRawSaltHexFromFullSalt(EXPECTED_KDF_SALT_BYTES, kdf)
|
||||
logger.info("Extracted expected raw salt (hex): ${EXPECTED_SALT_HEX}")
|
||||
|
||||
final String EXPECTED_IV_HEX = Hex.encodeHexString(flowfileContentBytes[(ivDelimiterStart - 16)..<ivDelimiterStart] as byte[])
|
||||
|
||||
printFlowFileAttributes(flowFile.getAttributes())
|
||||
|
||||
// Assert the timestamp attribute was written and is accurate
|
||||
def diff = calculateTimestampDifference(new Date(), flowFile.getAttribute("encryptcontent.timestamp"))
|
||||
assert diff.toMilliseconds() < 1_000
|
||||
assert flowFile.getAttribute("encryptcontent.algorithm") == encryptionMethod.name()
|
||||
assert flowFile.getAttribute("encryptcontent.kdf") == kdf.name()
|
||||
assert flowFile.getAttribute("encryptcontent.action") == "encrypted"
|
||||
assert flowFile.getAttribute("encryptcontent.salt") == EXPECTED_SALT_HEX
|
||||
assert flowFile.getAttribute("encryptcontent.salt_length") == "16"
|
||||
assert flowFile.getAttribute("encryptcontent.kdf_salt") == EXPECTED_KDF_SALT
|
||||
assert (29..54)*.toString().contains(flowFile.getAttribute("encryptcontent.kdf_salt_length"))
|
||||
assert flowFile.getAttribute("encryptcontent.iv") == EXPECTED_IV_HEX
|
||||
assert flowFile.getAttribute("encryptcontent.iv_length") == "16"
|
||||
assert flowFile.getAttribute("encryptcontent.plaintext_length") == PLAINTEXT.size() as String
|
||||
assert flowFile.getAttribute("encryptcontent.cipher_text_length") == flowfileContentBytes.size() as String
|
||||
}
|
||||
|
||||
static void printFlowFileAttributes(Map<String, String> attributes) {
|
||||
int maxLength = attributes.keySet()*.length().max()
|
||||
attributes.sort().each { attr, value ->
|
||||
logger.info("Attribute: ${attr.padRight(maxLength)}: ${value}")
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testKeyedEncryptionShouldWriteAttributesWithEncryptionMetadata() throws IOException {
|
||||
// Arrange
|
||||
final TestRunner testRunner = TestRunners.newTestRunner(new EncryptContent())
|
||||
KeyDerivationFunction kdf = KeyDerivationFunction.NONE
|
||||
EncryptionMethod encryptionMethod = EncryptionMethod.AES_CBC
|
||||
logger.info("Attempting encryption with {}", encryptionMethod.name())
|
||||
|
||||
testRunner.setProperty(EncryptContent.RAW_KEY_HEX, "0123456789ABCDEFFEDCBA9876543210")
|
||||
testRunner.setProperty(EncryptContent.KEY_DERIVATION_FUNCTION, kdf.name())
|
||||
testRunner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, encryptionMethod.name())
|
||||
testRunner.setProperty(EncryptContent.MODE, EncryptContent.ENCRYPT_MODE)
|
||||
|
||||
String PLAINTEXT = "This is a plaintext message. "
|
||||
|
||||
// Act
|
||||
testRunner.enqueue(PLAINTEXT)
|
||||
testRunner.clearTransferState()
|
||||
testRunner.run()
|
||||
|
||||
// Assert
|
||||
testRunner.assertAllFlowFilesTransferred(EncryptContent.REL_SUCCESS, 1)
|
||||
logger.info("Successfully encrypted with {}", encryptionMethod.name())
|
||||
|
||||
MockFlowFile flowFile = testRunner.getFlowFilesForRelationship(EncryptContent.REL_SUCCESS).get(0)
|
||||
testRunner.assertQueueEmpty()
|
||||
|
||||
printFlowFileAttributes(flowFile.getAttributes())
|
||||
|
||||
byte[] flowfileContentBytes = flowFile.getData()
|
||||
String flowfileContent = flowFile.getContent()
|
||||
logger.info("Cipher text (${flowfileContentBytes.length}): ${Hex.encodeHexString(flowfileContentBytes)}")
|
||||
|
||||
int ivDelimiterStart = CipherUtility.findSequence(flowfileContentBytes, RandomIVPBECipherProvider.IV_DELIMITER)
|
||||
logger.info("IV delimiter starts at ${ivDelimiterStart}")
|
||||
assert ivDelimiterStart == 16
|
||||
|
||||
def diff = calculateTimestampDifference(new Date(), flowFile.getAttribute("encryptcontent.timestamp"))
|
||||
logger.info("Timestamp difference: ${diff}")
|
||||
|
||||
// Assert the timestamp attribute was written and is accurate
|
||||
assert diff.toMilliseconds() < 1_000
|
||||
|
||||
final String EXPECTED_IV_HEX = Hex.encodeHexString(flowfileContentBytes[0..<ivDelimiterStart] as byte[])
|
||||
final int EXPECTED_CIPHER_TEXT_LENGTH = CipherUtility.calculateCipherTextLength(PLAINTEXT.size(), 0)
|
||||
|
||||
assert flowFile.getAttribute("encryptcontent.algorithm") == encryptionMethod.name()
|
||||
assert flowFile.getAttribute("encryptcontent.kdf") == kdf.name()
|
||||
assert flowFile.getAttribute("encryptcontent.action") == "encrypted"
|
||||
assert flowFile.getAttribute("encryptcontent.iv") == EXPECTED_IV_HEX
|
||||
assert flowFile.getAttribute("encryptcontent.iv_length") == "16"
|
||||
assert flowFile.getAttribute("encryptcontent.plaintext_length") == PLAINTEXT.size() as String
|
||||
assert flowFile.getAttribute("encryptcontent.cipher_text_length") == EXPECTED_CIPHER_TEXT_LENGTH as String
|
||||
}
|
||||
|
||||
@Test
|
||||
void testKeyedDecryptionShouldWriteAttributesWithEncryptionMetadata() throws IOException {
|
||||
// Arrange
|
||||
final TestRunner testRunner = TestRunners.newTestRunner(new EncryptContent())
|
||||
KeyDerivationFunction kdf = KeyDerivationFunction.NONE
|
||||
EncryptionMethod encryptionMethod = EncryptionMethod.AES_CBC
|
||||
logger.info("Attempting decryption with {}", encryptionMethod.name())
|
||||
|
||||
testRunner.setProperty(EncryptContent.RAW_KEY_HEX, "0123456789ABCDEFFEDCBA9876543210")
|
||||
testRunner.setProperty(EncryptContent.KEY_DERIVATION_FUNCTION, kdf.name())
|
||||
testRunner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, encryptionMethod.name())
|
||||
testRunner.setProperty(EncryptContent.MODE, EncryptContent.ENCRYPT_MODE)
|
||||
|
||||
String PLAINTEXT = "This is a plaintext message. "
|
||||
|
||||
testRunner.enqueue(PLAINTEXT)
|
||||
testRunner.clearTransferState()
|
||||
testRunner.run()
|
||||
|
||||
MockFlowFile encryptedFlowFile = testRunner.getFlowFilesForRelationship(EncryptContent.REL_SUCCESS).first()
|
||||
byte[] cipherText = encryptedFlowFile.getData()
|
||||
|
||||
int ivDelimiterStart = CipherUtility.findSequence(cipherText, RandomIVPBECipherProvider.IV_DELIMITER)
|
||||
logger.info("IV delimiter starts at ${ivDelimiterStart}")
|
||||
assert ivDelimiterStart == 16
|
||||
final String EXPECTED_IV_HEX = Hex.encodeHexString(cipherText[0..<ivDelimiterStart] as byte[])
|
||||
|
||||
testRunner.setProperty(EncryptContent.MODE, EncryptContent.DECRYPT_MODE)
|
||||
testRunner.clearTransferState()
|
||||
testRunner.enqueue(cipherText)
|
||||
|
||||
// Act
|
||||
testRunner.run()
|
||||
|
||||
// Assert
|
||||
testRunner.assertAllFlowFilesTransferred(EncryptContent.REL_SUCCESS, 1)
|
||||
logger.info("Successfully decrypted with {}", encryptionMethod.name())
|
||||
|
||||
MockFlowFile flowFile = testRunner.getFlowFilesForRelationship(EncryptContent.REL_SUCCESS).get(0)
|
||||
testRunner.assertQueueEmpty()
|
||||
|
||||
printFlowFileAttributes(flowFile.getAttributes())
|
||||
|
||||
byte[] flowfileContentBytes = flowFile.getData()
|
||||
String flowfileContent = flowFile.getContent()
|
||||
logger.info("Plaintext (${flowfileContentBytes.length}): ${Hex.encodeHexString(flowfileContentBytes)}")
|
||||
|
||||
def diff = calculateTimestampDifference(new Date(), flowFile.getAttribute("encryptcontent.timestamp"))
|
||||
logger.info("Timestamp difference: ${diff}")
|
||||
|
||||
// Assert the timestamp attribute was written and is accurate
|
||||
assert diff.toMilliseconds() < 1_000
|
||||
assert flowFile.getAttribute("encryptcontent.algorithm") == encryptionMethod.name()
|
||||
assert flowFile.getAttribute("encryptcontent.kdf") == kdf.name()
|
||||
assert flowFile.getAttribute("encryptcontent.action") == "decrypted"
|
||||
assert flowFile.getAttribute("encryptcontent.iv") == EXPECTED_IV_HEX
|
||||
assert flowFile.getAttribute("encryptcontent.iv_length") == "16"
|
||||
assert flowFile.getAttribute("encryptcontent.plaintext_length") == PLAINTEXT.size() as String
|
||||
assert flowFile.getAttribute("encryptcontent.cipher_text_length") == cipherText.length as String
|
||||
}
|
||||
|
||||
@Test
|
||||
void testDifferentCompatibleConfigurations() throws IOException {
|
||||
// Arrange
|
||||
final TestRunner testRunner = TestRunners.newTestRunner(new EncryptContent())
|
||||
KeyDerivationFunction argon2 = KeyDerivationFunction.ARGON2
|
||||
EncryptionMethod aesCbcEM = EncryptionMethod.AES_CBC
|
||||
logger.info("Attempting encryption with ${argon2} and ${aesCbcEM.name()}")
|
||||
int keyLength = CipherUtility.parseKeyLengthFromAlgorithm(aesCbcEM.algorithm)
|
||||
|
||||
final String PASSWORD = "thisIsABadPassword"
|
||||
testRunner.setProperty(EncryptContent.PASSWORD, PASSWORD)
|
||||
testRunner.setProperty(EncryptContent.KEY_DERIVATION_FUNCTION, argon2.name())
|
||||
testRunner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, aesCbcEM.name())
|
||||
testRunner.setProperty(EncryptContent.MODE, EncryptContent.ENCRYPT_MODE)
|
||||
|
||||
String PLAINTEXT = "This is a plaintext message. "
|
||||
|
||||
testRunner.enqueue(PLAINTEXT)
|
||||
testRunner.clearTransferState()
|
||||
testRunner.run()
|
||||
|
||||
MockFlowFile encryptedFlowFile = testRunner.getFlowFilesForRelationship(EncryptContent.REL_SUCCESS).first()
|
||||
byte[] fullCipherBytes = encryptedFlowFile.getData()
|
||||
printFlowFileAttributes(encryptedFlowFile.getAttributes())
|
||||
|
||||
// Extract the KDF salt from the encryption metadata in the flowfile attribute
|
||||
String argon2Salt = encryptedFlowFile.getAttribute("encryptcontent.kdf_salt")
|
||||
Argon2SecureHasher a2sh = new Argon2SecureHasher(keyLength / 8 as int)
|
||||
byte[] fullSaltBytes = argon2Salt.getBytes(StandardCharsets.UTF_8)
|
||||
byte[] rawSaltBytes = Hex.decodeHex(encryptedFlowFile.getAttribute("encryptcontent.salt"))
|
||||
byte[] keyBytes = a2sh.hashRaw(PASSWORD.getBytes(StandardCharsets.UTF_8), rawSaltBytes)
|
||||
String keyHex = Hex.encodeHexString(keyBytes)
|
||||
logger.sanity("Derived key bytes: ${keyHex}")
|
||||
|
||||
byte[] ivBytes = Hex.decodeHex(encryptedFlowFile.getAttribute("encryptcontent.iv"))
|
||||
logger.sanity("Extracted IV bytes: ${Hex.encodeHexString(ivBytes)}")
|
||||
|
||||
// Sanity check the encryption
|
||||
Argon2CipherProvider a2cp = new Argon2CipherProvider()
|
||||
Cipher sanityCipher = a2cp.getCipher(aesCbcEM, PASSWORD, fullSaltBytes, ivBytes, CipherUtility.parseKeyLengthFromAlgorithm(aesCbcEM.algorithm), false)
|
||||
byte[] cipherTextBytes = fullCipherBytes[-32..-1]
|
||||
byte[] recoveredBytes = sanityCipher.doFinal(cipherTextBytes)
|
||||
logger.sanity("Recovered text: ${new String(recoveredBytes, StandardCharsets.UTF_8)}")
|
||||
|
||||
// Act
|
||||
|
||||
// Configure decrypting processor with raw key
|
||||
KeyDerivationFunction kdf = KeyDerivationFunction.NONE
|
||||
EncryptionMethod encryptionMethod = EncryptionMethod.AES_CBC
|
||||
logger.info("Attempting decryption with {}", encryptionMethod.name())
|
||||
|
||||
testRunner.setProperty(EncryptContent.RAW_KEY_HEX, keyHex)
|
||||
testRunner.setProperty(EncryptContent.KEY_DERIVATION_FUNCTION, kdf.name())
|
||||
testRunner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, encryptionMethod.name())
|
||||
testRunner.setProperty(EncryptContent.MODE, EncryptContent.DECRYPT_MODE)
|
||||
testRunner.removeProperty(EncryptContent.PASSWORD)
|
||||
|
||||
testRunner.enqueue(fullCipherBytes)
|
||||
testRunner.clearTransferState()
|
||||
testRunner.run()
|
||||
|
||||
// Assert
|
||||
testRunner.assertAllFlowFilesTransferred(EncryptContent.REL_SUCCESS, 1)
|
||||
logger.info("Successfully decrypted with {}", encryptionMethod.name())
|
||||
|
||||
MockFlowFile decryptedFlowFile = testRunner.getFlowFilesForRelationship(EncryptContent.REL_SUCCESS).get(0)
|
||||
testRunner.assertQueueEmpty()
|
||||
|
||||
printFlowFileAttributes(decryptedFlowFile.getAttributes())
|
||||
|
||||
byte[] flowfileContentBytes = decryptedFlowFile.getData()
|
||||
logger.info("Plaintext (${flowfileContentBytes.length}): ${new String(flowfileContentBytes, StandardCharsets.UTF_8)}")
|
||||
|
||||
assert flowfileContentBytes == recoveredBytes
|
||||
}
|
||||
|
||||
static TimeDuration calculateTimestampDifference(Date date, String timestamp) {
|
||||
SimpleDateFormat formatter = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS Z")
|
||||
final long dateMillis = date.toInstant().toEpochMilli()
|
||||
logger.info("Provided timestamp ${formatter.format(date)} -> (ms): ${dateMillis}")
|
||||
Date parsedTimestamp = formatter.parse(timestamp)
|
||||
long parsedTimestampMillis = parsedTimestamp.toInstant().toEpochMilli()
|
||||
logger.info("Parsed timestamp ${timestamp} -> (ms): ${parsedTimestampMillis}")
|
||||
|
||||
TimeCategory.minus(date, parsedTimestamp)
|
||||
}
|
||||
|
||||
static byte[] extractFullSaltFromCipherBytes(byte[] cipherBytes) {
|
||||
int saltDelimiterStart = CipherUtility.findSequence(cipherBytes, RandomIVPBECipherProvider.SALT_DELIMITER)
|
||||
logger.info("Salt delimiter starts at ${saltDelimiterStart}")
|
||||
byte[] saltBytes = cipherBytes[0..<saltDelimiterStart]
|
||||
logger.info("Extracted full salt (${saltBytes.length}): ${new String(saltBytes, StandardCharsets.UTF_8)}")
|
||||
saltBytes
|
||||
}
|
||||
|
||||
static String extractRawSaltHexFromFullSalt(byte[] fullSaltBytes, KeyDerivationFunction kdf) {
|
||||
logger.info("Full salt (${fullSaltBytes.length}): ${Hex.encodeHexString(fullSaltBytes)}")
|
||||
// Salt will be in Base64 (or Radix64) for strong KDFs
|
||||
byte[] rawSaltBytes = CipherUtility.extractRawSalt(fullSaltBytes, kdf)
|
||||
logger.info("Raw salt (${rawSaltBytes.length}): ${Hex.encodeHexString(rawSaltBytes)}")
|
||||
String rawSaltHex = Hex.encodeHexString(rawSaltBytes)
|
||||
logger.info("Extracted expected raw salt (hex): ${rawSaltHex}")
|
||||
rawSaltHex
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldCompareDate() {
|
||||
// Arrange
|
||||
Date now = new Date()
|
||||
logger.info("Now: ${now} -- ${now.toInstant().toEpochMilli()}")
|
||||
|
||||
Instant fiveSecondsLater = now.toInstant().plus(5, ChronoUnit.SECONDS)
|
||||
Date fSLDate = Date.from(fiveSecondsLater)
|
||||
logger.info("FSL: ${fSLDate} -- ${fiveSecondsLater.toEpochMilli()}")
|
||||
|
||||
// Convert entirely to String & parse back
|
||||
Instant tenSecondsLater = fiveSecondsLater.plusMillis(5000)
|
||||
Date tSLDate = Date.from(tenSecondsLater)
|
||||
logger.info("TSL: ${tSLDate} -- ${tenSecondsLater.toEpochMilli()}")
|
||||
|
||||
// Java way ('y' is deterministic vs. 'Y' which is week-based and calendar & JVM dependent)
|
||||
SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS Z")
|
||||
String tslString = sdf.format(tSLDate)
|
||||
logger.info("TSL formatted: ${tslString}")
|
||||
|
||||
// Parse back to date
|
||||
Date parsedTSLDate = sdf.parse(tslString)
|
||||
logger.info("TSL parsed: ${parsedTSLDate} -- ${parsedTSLDate.toInstant().toEpochMilli()}")
|
||||
|
||||
// Act
|
||||
def fiveSecondDiff = TimeCategory.minus(fSLDate, now)
|
||||
logger.info(" FSL - now difference: ${fiveSecondDiff}")
|
||||
|
||||
def tenSecondDiff = TimeCategory.minus(tSLDate, now)
|
||||
logger.info(" TSL - now difference: ${tenSecondDiff}")
|
||||
|
||||
def parsedTenSecondDiff = TimeCategory.minus(parsedTSLDate, now)
|
||||
logger.info("PTSL - now difference: ${parsedTenSecondDiff}")
|
||||
|
||||
// Assert
|
||||
assert fiveSecondDiff.seconds == 5
|
||||
assert tenSecondDiff.seconds == 10
|
||||
assert parsedTenSecondDiff.seconds == 10
|
||||
|
||||
assert [fiveSecondDiff, tenSecondDiff, parsedTenSecondDiff].every { it.days == 0 }
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldCheckLengthOfPasswordWhenNotAllowed() throws IOException {
|
||||
// Arrange
|
||||
final TestRunner testRunner = TestRunners.newTestRunner(new EncryptContent())
|
||||
testRunner.setProperty(EncryptContent.KEY_DERIVATION_FUNCTION, KeyDerivationFunction.NIFI_LEGACY.name())
|
||||
|
||||
Collection<ValidationResult> results
|
||||
MockProcessContext pc
|
||||
|
||||
def encryptionMethods = EncryptionMethod.values().findAll { it.algorithm.startsWith("PBE") }
|
||||
|
||||
boolean limitedStrengthCrypto = false
|
||||
boolean allowWeakCrypto = false
|
||||
testRunner.setProperty(EncryptContent.ALLOW_WEAK_CRYPTO, WEAK_CRYPTO_NOT_ALLOWED)
|
||||
|
||||
// Use .find instead of .each to allow "breaks" using return false
|
||||
encryptionMethods.find { EncryptionMethod encryptionMethod ->
|
||||
// Determine the minimum of the algorithm-accepted length or the global safe minimum to ensure only one validation result
|
||||
def shortPasswordLength = [PasswordBasedEncryptor.getMinimumSafePasswordLength() - 1, CipherUtility.getMaximumPasswordLengthForAlgorithmOnLimitedStrengthCrypto(encryptionMethod) - 1].min()
|
||||
String shortPassword = "x" * shortPasswordLength
|
||||
if (encryptionMethod.isUnlimitedStrength() || encryptionMethod.isKeyedCipher()) {
|
||||
return false
|
||||
// cannot test unlimited strength in unit tests because it's not enabled by the JVM by default.
|
||||
}
|
||||
|
||||
testRunner.setProperty(EncryptContent.PASSWORD, shortPassword)
|
||||
logger.info("Attempting ${encryptionMethod.algorithm} with password of length ${shortPasswordLength}")
|
||||
logger.state("Limited strength crypto ${limitedStrengthCrypto} and allow weak crypto: ${allowWeakCrypto}")
|
||||
testRunner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, encryptionMethod.name())
|
||||
testRunner.setProperty(EncryptContent.MODE, EncryptContent.ENCRYPT_MODE)
|
||||
|
||||
testRunner.clearTransferState()
|
||||
testRunner.enqueue(new byte[0])
|
||||
pc = (MockProcessContext) testRunner.getProcessContext()
|
||||
|
||||
// Act
|
||||
results = pc.validate()
|
||||
|
||||
// Assert
|
||||
logger.expected(results)
|
||||
Assert.assertEquals(1, results.size())
|
||||
ValidationResult passwordLengthVR = results.first()
|
||||
|
||||
String expectedResult = "'Password' is invalid because Password length less than ${PasswordBasedEncryptor.getMinimumSafePasswordLength()} characters is potentially unsafe. " +
|
||||
"See Admin Guide."
|
||||
String message = "'" + passwordLengthVR.toString() + "' contains '" + expectedResult + "'"
|
||||
Assert.assertTrue(message, passwordLengthVR.toString().contains(expectedResult))
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldNotCheckLengthOfPasswordWhenAllowed() throws IOException {
|
||||
// Arrange
|
||||
final TestRunner testRunner = TestRunners.newTestRunner(new EncryptContent())
|
||||
testRunner.setProperty(EncryptContent.KEY_DERIVATION_FUNCTION, KeyDerivationFunction.NIFI_LEGACY.name())
|
||||
|
||||
Collection<ValidationResult> results
|
||||
MockProcessContext pc
|
||||
|
||||
def encryptionMethods = EncryptionMethod.values().findAll { it.algorithm.startsWith("PBE") }
|
||||
|
||||
boolean limitedStrengthCrypto = false
|
||||
boolean allowWeakCrypto = true
|
||||
testRunner.setProperty(EncryptContent.ALLOW_WEAK_CRYPTO, WEAK_CRYPTO_ALLOWED)
|
||||
|
||||
// Use .find instead of .each to allow "breaks" using return false
|
||||
encryptionMethods.find { EncryptionMethod encryptionMethod ->
|
||||
// Determine the minimum of the algorithm-accepted length or the global safe minimum to ensure only one validation result
|
||||
def shortPasswordLength = [PasswordBasedEncryptor.getMinimumSafePasswordLength() - 1, CipherUtility.getMaximumPasswordLengthForAlgorithmOnLimitedStrengthCrypto(encryptionMethod) - 1].min()
|
||||
String shortPassword = "x" * shortPasswordLength
|
||||
if (encryptionMethod.isUnlimitedStrength() || encryptionMethod.isKeyedCipher()) {
|
||||
return false
|
||||
// cannot test unlimited strength in unit tests because it's not enabled by the JVM by default.
|
||||
}
|
||||
|
||||
testRunner.setProperty(EncryptContent.PASSWORD, shortPassword)
|
||||
logger.info("Attempting ${encryptionMethod.algorithm} with password of length ${shortPasswordLength}")
|
||||
logger.state("Limited strength crypto ${limitedStrengthCrypto} and allow weak crypto: ${allowWeakCrypto}")
|
||||
testRunner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, encryptionMethod.name())
|
||||
testRunner.setProperty(EncryptContent.MODE, EncryptContent.ENCRYPT_MODE)
|
||||
|
||||
testRunner.clearTransferState()
|
||||
testRunner.enqueue(new byte[0])
|
||||
pc = (MockProcessContext) testRunner.getProcessContext()
|
||||
|
||||
// Act
|
||||
results = pc.validate()
|
||||
|
||||
// Assert
|
||||
Assert.assertEquals(results.toString(), 0, results.size())
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testPGPPasswordShouldSupportExpressionLanguage() throws IOException {
|
||||
// Arrange
|
||||
final TestRunner testRunner = TestRunners.newTestRunner(new EncryptContent())
|
||||
testRunner.setProperty(EncryptContent.MODE, EncryptContent.DECRYPT_MODE)
|
||||
testRunner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, EncryptionMethod.PGP.name())
|
||||
testRunner.setProperty(EncryptContent.PRIVATE_KEYRING, "src/test/resources/TestEncryptContent/secring.gpg")
|
||||
|
||||
Collection<ValidationResult> results
|
||||
MockProcessContext pc
|
||||
|
||||
// Verify this is the correct password
|
||||
final String passphraseWithoutEL = "thisIsABadPassword"
|
||||
testRunner.setProperty(EncryptContent.PRIVATE_KEYRING_PASSPHRASE, passphraseWithoutEL)
|
||||
|
||||
testRunner.clearTransferState()
|
||||
testRunner.enqueue(new byte[0])
|
||||
pc = (MockProcessContext) testRunner.getProcessContext()
|
||||
|
||||
results = pc.validate()
|
||||
Assert.assertEquals(results.toString(), 0, results.size())
|
||||
|
||||
final String passphraseWithEL = "\${literal('thisIsABadPassword')}"
|
||||
testRunner.setProperty(EncryptContent.PRIVATE_KEYRING_PASSPHRASE, passphraseWithEL)
|
||||
|
||||
testRunner.clearTransferState()
|
||||
testRunner.enqueue(new byte[0])
|
||||
|
||||
// Act
|
||||
results = pc.validate()
|
||||
|
||||
// Assert
|
||||
Assert.assertEquals(results.toString(), 0, results.size())
|
||||
}
|
||||
|
||||
@Test
|
||||
void testArgon2ShouldIncludeFullSalt() throws IOException {
|
||||
// Arrange
|
||||
final TestRunner testRunner = TestRunners.newTestRunner(new EncryptContent())
|
||||
testRunner.setProperty(EncryptContent.PASSWORD, "thisIsABadPassword")
|
||||
testRunner.setProperty(EncryptContent.KEY_DERIVATION_FUNCTION, KeyDerivationFunction.ARGON2.name())
|
||||
|
||||
EncryptionMethod encryptionMethod = EncryptionMethod.AES_CBC
|
||||
|
||||
logger.info("Attempting {}", encryptionMethod.name())
|
||||
testRunner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, encryptionMethod.name())
|
||||
testRunner.setProperty(EncryptContent.MODE, EncryptContent.ENCRYPT_MODE)
|
||||
|
||||
// Act
|
||||
testRunner.enqueue(Paths.get("src/test/resources/hello.txt"))
|
||||
testRunner.clearTransferState()
|
||||
testRunner.run()
|
||||
|
||||
// Assert
|
||||
testRunner.assertAllFlowFilesTransferred(EncryptContent.REL_SUCCESS, 1)
|
||||
|
||||
MockFlowFile flowFile = testRunner.getFlowFilesForRelationship(EncryptContent.REL_SUCCESS).get(0)
|
||||
testRunner.assertQueueEmpty()
|
||||
|
||||
def flowFileContent = flowFile.getContent()
|
||||
logger.info("Flowfile content (${flowFile.getData().length}): ${Hex.encodeHexString(flowFile.getData())}")
|
||||
|
||||
def fullSalt = flowFileContent.substring(0, flowFileContent.indexOf(new String(RandomIVPBECipherProvider.SALT_DELIMITER, StandardCharsets.UTF_8)))
|
||||
logger.info("Full salt (${fullSalt.size()}): ${fullSalt}")
|
||||
|
||||
boolean isValidFormattedSalt = Argon2CipherProvider.isArgon2FormattedSalt(fullSalt)
|
||||
logger.info("Salt is Argon2 format: ${isValidFormattedSalt}")
|
||||
assert isValidFormattedSalt
|
||||
|
||||
def FULL_SALT_LENGTH_RANGE = (49..57)
|
||||
boolean fullSaltIsValidLength = FULL_SALT_LENGTH_RANGE.contains(fullSalt.bytes.length)
|
||||
logger.info("Salt length (${fullSalt.length()}) in valid range (${FULL_SALT_LENGTH_RANGE})")
|
||||
assert fullSaltIsValidLength
|
||||
}
|
||||
}
|
@ -1,441 +0,0 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License") you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.processors.standard
|
||||
|
||||
import groovy.json.JsonSlurper
|
||||
import org.apache.nifi.util.TestRunners
|
||||
import org.junit.Assert
|
||||
import org.junit.Test
|
||||
import static groovy.json.JsonOutput.prettyPrint
|
||||
import static groovy.json.JsonOutput.toJson
|
||||
|
||||
class TestFlattenJson {
|
||||
@Test
|
||||
void testFlatten() {
|
||||
def testRunner = TestRunners.newTestRunner(FlattenJson.class)
|
||||
def json = prettyPrint(toJson([
|
||||
test: [
|
||||
msg: "Hello, world"
|
||||
],
|
||||
first: [
|
||||
second: [
|
||||
third: [
|
||||
"one", "two", "three", "four", "five"
|
||||
]
|
||||
]
|
||||
]
|
||||
]))
|
||||
baseTest(testRunner, json, 2) { parsed ->
|
||||
Assert.assertEquals("test.msg should exist, but doesn't", parsed["test.msg"], "Hello, world")
|
||||
Assert.assertEquals("Three level block doesn't exist.", parsed["first.second.third"], [
|
||||
"one", "two", "three", "four", "five"
|
||||
])
|
||||
}
|
||||
}
|
||||
|
||||
void baseTest(testRunner, String json, int keyCount, Closure c) {
|
||||
baseTest(testRunner, json, [:], keyCount, c);
|
||||
}
|
||||
|
||||
void baseTest(def testRunner, String json, Map attrs, int keyCount, Closure c) {
|
||||
testRunner.enqueue(json, attrs)
|
||||
testRunner.run(1, true)
|
||||
testRunner.assertTransferCount(FlattenJson.REL_FAILURE, 0)
|
||||
testRunner.assertTransferCount(FlattenJson.REL_SUCCESS, 1)
|
||||
|
||||
def flowFiles = testRunner.getFlowFilesForRelationship(FlattenJson.REL_SUCCESS)
|
||||
def content = testRunner.getContentAsByteArray(flowFiles[0])
|
||||
def asJson = new String(content)
|
||||
def slurper = new JsonSlurper()
|
||||
def parsed = slurper.parseText(asJson) as Map
|
||||
|
||||
Assert.assertEquals("Too many keys", keyCount, parsed.size())
|
||||
c.call(parsed)
|
||||
}
|
||||
|
||||
@Test
|
||||
void testFlattenRecordSet() {
|
||||
def testRunner = TestRunners.newTestRunner(FlattenJson.class)
|
||||
def json = prettyPrint(toJson([
|
||||
[
|
||||
first: [
|
||||
second: "Hello"
|
||||
]
|
||||
],
|
||||
[
|
||||
first: [
|
||||
second: "World"
|
||||
]
|
||||
]
|
||||
]))
|
||||
|
||||
def expected = ["Hello", "World"]
|
||||
baseTest(testRunner, json, 2) { parsed ->
|
||||
Assert.assertTrue("Not a list", parsed instanceof List)
|
||||
0.upto(parsed.size() - 1) {
|
||||
Assert.assertEquals("Missing values.", parsed[it]["first.second"], expected[it])
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testDifferentSeparator() {
|
||||
def testRunner = TestRunners.newTestRunner(FlattenJson.class)
|
||||
def json = prettyPrint(toJson([
|
||||
first: [
|
||||
second: [
|
||||
third: [
|
||||
"one", "two", "three", "four", "five"
|
||||
]
|
||||
]
|
||||
]
|
||||
]))
|
||||
testRunner.setProperty(FlattenJson.SEPARATOR, "_")
|
||||
baseTest(testRunner, json, 1) { parsed ->
|
||||
Assert.assertEquals("Separator not applied.", parsed["first_second_third"], [
|
||||
"one", "two", "three", "four", "five"
|
||||
])
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testExpressionLanguage() {
|
||||
def testRunner = TestRunners.newTestRunner(FlattenJson.class)
|
||||
def json = prettyPrint(toJson([
|
||||
first: [
|
||||
second: [
|
||||
third: [
|
||||
"one", "two", "three", "four", "five"
|
||||
]
|
||||
]
|
||||
]
|
||||
]))
|
||||
|
||||
testRunner.setValidateExpressionUsage(true);
|
||||
testRunner.setProperty(FlattenJson.SEPARATOR, '${separator.char}')
|
||||
baseTest(testRunner, json, ["separator.char": "_"], 1) { parsed ->
|
||||
Assert.assertEquals("Separator not applied.", parsed["first_second_third"], [
|
||||
"one", "two", "three", "four", "five"
|
||||
])
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testFlattenModeNormal() {
|
||||
def testRunner = TestRunners.newTestRunner(FlattenJson.class)
|
||||
def json = prettyPrint(toJson([
|
||||
first: [
|
||||
second: [
|
||||
third: [
|
||||
"one", "two", "three", "four", "five"
|
||||
]
|
||||
]
|
||||
]
|
||||
]))
|
||||
|
||||
testRunner.setProperty(FlattenJson.FLATTEN_MODE, FlattenJson.FLATTEN_MODE_NORMAL)
|
||||
baseTest(testRunner, json,5) { parsed ->
|
||||
Assert.assertEquals("Separator not applied.", "one", parsed["first.second.third[0]"])
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testFlattenModeKeepArrays() {
|
||||
def testRunner = TestRunners.newTestRunner(FlattenJson.class)
|
||||
def json = prettyPrint(toJson([
|
||||
first: [
|
||||
second: [
|
||||
[
|
||||
x: 1,
|
||||
y: 2,
|
||||
z: [3, 4, 5]
|
||||
],
|
||||
[ 6, 7, 8],
|
||||
[
|
||||
[9, 10],
|
||||
11,
|
||||
12
|
||||
]
|
||||
],
|
||||
"third" : [
|
||||
a: "b",
|
||||
c: "d",
|
||||
e: "f"
|
||||
]
|
||||
]
|
||||
]))
|
||||
|
||||
testRunner.setProperty(FlattenJson.FLATTEN_MODE, FlattenJson.FLATTEN_MODE_KEEP_ARRAYS)
|
||||
baseTest(testRunner, json,4) { parsed ->
|
||||
assert parsed["first.second"] instanceof List // [{x=1, y=2, z=[3, 4, 5]}, [6, 7, 8], [[9, 10], 11, 12]]
|
||||
assert parsed["first.second"][1] == [6, 7, 8]
|
||||
Assert.assertEquals("Separator not applied.", "b", parsed["first.third.a"])
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testFlattenModeKeepPrimitiveArrays() {
|
||||
def testRunner = TestRunners.newTestRunner(FlattenJson.class)
|
||||
def json = prettyPrint(toJson([
|
||||
first: [
|
||||
second: [
|
||||
[
|
||||
x: 1,
|
||||
y: 2,
|
||||
z: [3, 4, 5]
|
||||
],
|
||||
[ 6, 7, 8],
|
||||
[
|
||||
[9, 10],
|
||||
11,
|
||||
12
|
||||
]
|
||||
],
|
||||
"third" : [
|
||||
a: "b",
|
||||
c: "d",
|
||||
e: "f"
|
||||
]
|
||||
]
|
||||
]))
|
||||
|
||||
testRunner.setProperty(FlattenJson.FLATTEN_MODE, FlattenJson.FLATTEN_MODE_KEEP_PRIMITIVE_ARRAYS)
|
||||
baseTest(testRunner, json,10) { parsed ->
|
||||
Assert.assertEquals("Separator not applied.", 1, parsed["first.second[0].x"])
|
||||
Assert.assertEquals("Separator not applied.", [3, 4, 5], parsed["first.second[0].z"])
|
||||
Assert.assertEquals("Separator not applied.", [9, 10], parsed["first.second[2][0]"])
|
||||
Assert.assertEquals("Separator not applied.", 11, parsed["first.second[2][1]"])
|
||||
Assert.assertEquals("Separator not applied.", 12, parsed["first.second[2][2]"])
|
||||
Assert.assertEquals("Separator not applied.", "d", parsed["first.third.c"])
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testFlattenModeDotNotation() {
|
||||
def testRunner = TestRunners.newTestRunner(FlattenJson.class)
|
||||
def json = prettyPrint(toJson([
|
||||
first: [
|
||||
second: [
|
||||
third: [
|
||||
"one", "two", "three", "four", "five"
|
||||
]
|
||||
]
|
||||
]
|
||||
]))
|
||||
|
||||
testRunner.setProperty(FlattenJson.FLATTEN_MODE, FlattenJson.FLATTEN_MODE_DOT_NOTATION)
|
||||
baseTest(testRunner, json,5) { parsed ->
|
||||
Assert.assertEquals("Separator not applied.", "one", parsed["first.second.third.0"])
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testFlattenSlash() {
|
||||
def testRunner = TestRunners.newTestRunner(FlattenJson.class)
|
||||
def json = prettyPrint(toJson([
|
||||
first: [
|
||||
second: [
|
||||
third: [
|
||||
"http://localhost/value1", "http://localhost/value2"
|
||||
]
|
||||
]
|
||||
]
|
||||
]))
|
||||
|
||||
testRunner.setProperty(FlattenJson.FLATTEN_MODE, FlattenJson.FLATTEN_MODE_NORMAL)
|
||||
baseTest(testRunner, json,2) { parsed ->
|
||||
Assert.assertEquals("Separator not applied.", "http://localhost/value1", parsed["first.second.third[0]"])
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testEscapeForJson() {
|
||||
def testRunner = TestRunners.newTestRunner(FlattenJson.class)
|
||||
def json = prettyPrint(toJson([ name: "José"
|
||||
]))
|
||||
|
||||
testRunner.setProperty(FlattenJson.FLATTEN_MODE, FlattenJson.FLATTEN_MODE_NORMAL)
|
||||
baseTest(testRunner, json,1) { parsed ->
|
||||
Assert.assertEquals("Separator not applied.", "José", parsed["name"])
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testUnFlatten() {
|
||||
def testRunner = TestRunners.newTestRunner(FlattenJson.class)
|
||||
def json = prettyPrint(toJson([
|
||||
"test.msg": "Hello, world",
|
||||
"first.second.third": [ "one", "two", "three", "four", "five" ]
|
||||
]))
|
||||
|
||||
testRunner.setProperty(FlattenJson.RETURN_TYPE, FlattenJson.RETURN_TYPE_UNFLATTEN)
|
||||
baseTest(testRunner, json, 2) { parsed ->
|
||||
assert parsed.test instanceof Map
|
||||
assert parsed.test.msg == "Hello, world"
|
||||
assert parsed.first.second.third == [ "one", "two", "three", "four", "five" ]
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testUnFlattenWithDifferentSeparator() {
|
||||
def testRunner = TestRunners.newTestRunner(FlattenJson.class)
|
||||
def json = prettyPrint(toJson([
|
||||
"first_second_third": [ "one", "two", "three", "four", "five" ]
|
||||
]))
|
||||
|
||||
testRunner.setProperty(FlattenJson.SEPARATOR, "_")
|
||||
testRunner.setProperty(FlattenJson.RETURN_TYPE, FlattenJson.RETURN_TYPE_UNFLATTEN)
|
||||
baseTest(testRunner, json, 1) { parsed ->
|
||||
assert parsed.first instanceof Map
|
||||
assert parsed.first.second.third == [ "one", "two", "three", "four", "five" ]
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testUnFlattenForKeepArraysMode() {
|
||||
def testRunner = TestRunners.newTestRunner(FlattenJson.class)
|
||||
def json = prettyPrint(toJson([
|
||||
"a.b": 1,
|
||||
"a.c": [
|
||||
false,
|
||||
["i.j": [ false, true, "xy" ] ]
|
||||
]
|
||||
]))
|
||||
|
||||
testRunner.setProperty(FlattenJson.FLATTEN_MODE, FlattenJson.FLATTEN_MODE_KEEP_ARRAYS)
|
||||
testRunner.setProperty(FlattenJson.RETURN_TYPE, FlattenJson.RETURN_TYPE_UNFLATTEN)
|
||||
baseTest(testRunner, json, 1) { parsed ->
|
||||
assert parsed.a instanceof Map
|
||||
assert parsed.a.b == 1
|
||||
assert parsed.a.c[0] == false
|
||||
assert parsed.a.c[1].i instanceof Map
|
||||
assert parsed.a.c[1].i.j == [false, true, "xy"]
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testUnFlattenForKeepPrimitiveArraysMode() {
|
||||
def testRunner = TestRunners.newTestRunner(FlattenJson.class)
|
||||
def json = prettyPrint(toJson([
|
||||
"first.second[0].x": 1,
|
||||
"first.second[0].y": 2,
|
||||
"first.second[0].z": [3, 4, 5],
|
||||
"first.second[1]": [6, 7, 8],
|
||||
"first.second[2][0]": [9, 10],
|
||||
"first.second[2][1]": 11,
|
||||
"first.second[2][2]": 12,
|
||||
"first.third.a": "b",
|
||||
"first.third.c": "d",
|
||||
"first.third.e": "f"
|
||||
]))
|
||||
|
||||
testRunner.setProperty(FlattenJson.FLATTEN_MODE, FlattenJson.FLATTEN_MODE_KEEP_PRIMITIVE_ARRAYS)
|
||||
testRunner.setProperty(FlattenJson.RETURN_TYPE, FlattenJson.RETURN_TYPE_UNFLATTEN)
|
||||
baseTest(testRunner, json, 1) { parsed ->
|
||||
assert parsed.first instanceof Map
|
||||
assert parsed.first.second[0].x == 1
|
||||
assert parsed.first.second[2][0] == [9, 10]
|
||||
assert parsed.first.third.c == "d"
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testUnFlattenForDotNotationMode() {
|
||||
def testRunner = TestRunners.newTestRunner(FlattenJson.class)
|
||||
def json = prettyPrint(toJson([
|
||||
"first.second.third.0": ["one", "two", "three", "four", "five"]
|
||||
]))
|
||||
|
||||
testRunner.setProperty(FlattenJson.FLATTEN_MODE, FlattenJson.FLATTEN_MODE_DOT_NOTATION)
|
||||
testRunner.setProperty(FlattenJson.RETURN_TYPE, FlattenJson.RETURN_TYPE_UNFLATTEN)
|
||||
baseTest(testRunner, json,1) { parsed ->
|
||||
assert parsed.first instanceof Map
|
||||
assert parsed.first.second.third[0] == ["one", "two", "three", "four", "five"]
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testFlattenWithIgnoreReservedCharacters() {
|
||||
def testRunner = TestRunners.newTestRunner(FlattenJson.class)
|
||||
def json = prettyPrint(toJson([
|
||||
"first": [
|
||||
"second.third": "Hello",
|
||||
"fourth" : "World"
|
||||
]
|
||||
]))
|
||||
|
||||
testRunner.setProperty(FlattenJson.IGNORE_RESERVED_CHARACTERS, "true")
|
||||
|
||||
baseTest(testRunner, json, 2) { parsed ->
|
||||
Assert.assertEquals("Separator not applied.", parsed["first.second.third"], "Hello")
|
||||
Assert.assertEquals("Separator not applied.", parsed["first.fourth"], "World")
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testFlattenRecordSetWithIgnoreReservedCharacters() {
|
||||
def testRunner = TestRunners.newTestRunner(FlattenJson.class)
|
||||
def json = prettyPrint(toJson([
|
||||
[
|
||||
"first": [
|
||||
"second_third": "Hello"
|
||||
]
|
||||
],
|
||||
[
|
||||
"first": [
|
||||
"second_third": "World"
|
||||
]
|
||||
]
|
||||
]))
|
||||
testRunner.setProperty(FlattenJson.SEPARATOR, "_")
|
||||
testRunner.setProperty(FlattenJson.IGNORE_RESERVED_CHARACTERS, "true")
|
||||
|
||||
def expected = ["Hello", "World"]
|
||||
baseTest(testRunner, json, 2) { parsed ->
|
||||
Assert.assertTrue("Not a list", parsed instanceof List)
|
||||
0.upto(parsed.size() - 1) {
|
||||
Assert.assertEquals("Missing values.", parsed[it]["first_second_third"], expected[it])
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testFlattenModeNormalWithIgnoreReservedCharacters() {
|
||||
def testRunner = TestRunners.newTestRunner(FlattenJson.class)
|
||||
def json = prettyPrint(toJson([
|
||||
[
|
||||
"first": [
|
||||
"second_third": "Hello"
|
||||
]
|
||||
],
|
||||
[
|
||||
"first": [
|
||||
"second_third": "World"
|
||||
]
|
||||
]
|
||||
]))
|
||||
testRunner.setProperty(FlattenJson.SEPARATOR, "_")
|
||||
testRunner.setProperty(FlattenJson.IGNORE_RESERVED_CHARACTERS, "true")
|
||||
testRunner.setProperty(FlattenJson.FLATTEN_MODE, FlattenJson.FLATTEN_MODE_NORMAL)
|
||||
|
||||
baseTest(testRunner, json, 2) { parsed ->
|
||||
Assert.assertEquals("Separator not applied.", "Hello", parsed["[0]_first_second_third"])
|
||||
Assert.assertEquals("Separator not applied.", "World", parsed["[1]_first_second_third"])
|
||||
}
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
@ -0,0 +1,307 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License") you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.processors.standard;
|
||||
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.util.MockComponentLog;
|
||||
import org.apache.nifi.util.MockFlowFile;
|
||||
import org.apache.nifi.util.TestRunner;
|
||||
import org.apache.nifi.util.TestRunners;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.nio.file.Paths;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.mockito.Mockito.doReturn;
|
||||
import static org.mockito.Mockito.spy;
|
||||
|
||||
public class CountTextTest {
|
||||
private static final String TLC = "text.line.count";
|
||||
private static final String TLNEC = "text.line.nonempty.count";
|
||||
private static final String TWC = "text.word.count";
|
||||
private static final String TCC = "text.character.count";
|
||||
private TestRunner runner;
|
||||
|
||||
@BeforeEach
|
||||
void setupRunner() {
|
||||
runner = TestRunners.newTestRunner(CountText.class);
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
void testShouldCountAllMetrics() throws IOException {
|
||||
runner.setProperty(CountText.TEXT_LINE_COUNT_PD, "true");
|
||||
runner.setProperty(CountText.TEXT_LINE_NONEMPTY_COUNT_PD, "true");
|
||||
runner.setProperty(CountText.TEXT_WORD_COUNT_PD, "true");
|
||||
runner.setProperty(CountText.TEXT_CHARACTER_COUNT_PD, "true");
|
||||
|
||||
final Path inputPath = Paths.get("src/test/resources/TestCountText/jabberwocky.txt");
|
||||
|
||||
final Map<String, String> expectedValues = new HashMap<>();
|
||||
expectedValues.put(TLC, "34");
|
||||
expectedValues.put(TLNEC, "28");
|
||||
expectedValues.put(TWC, "166");
|
||||
expectedValues.put(TCC, "900");
|
||||
|
||||
runner.enqueue(Files.readAllBytes(inputPath));
|
||||
|
||||
runner.run();
|
||||
|
||||
runner.assertAllFlowFilesTransferred(CountText.REL_SUCCESS, 1);
|
||||
MockFlowFile flowFile = runner.getFlowFilesForRelationship(CountText.REL_SUCCESS).get(0);
|
||||
for (final Map.Entry<String, String> entry: expectedValues.entrySet()) {
|
||||
final String attribute = entry.getKey();
|
||||
final String expectedValue = entry.getValue();
|
||||
flowFile.assertAttributeEquals(attribute, expectedValue);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldCountEachMetric() throws IOException {
|
||||
final Path inputPath = Paths.get("src/test/resources/TestCountText/jabberwocky.txt");
|
||||
|
||||
final Map<String, String> expectedValues = new HashMap<>();
|
||||
expectedValues.put(TLC, "34");
|
||||
expectedValues.put(TLNEC, "28");
|
||||
expectedValues.put(TWC, "166");
|
||||
expectedValues.put(TCC, "900");
|
||||
|
||||
final Map<PropertyDescriptor, String> linesOnly = Collections.singletonMap(CountText.TEXT_LINE_COUNT_PD, "true");
|
||||
final Map<PropertyDescriptor, String> linesNonEmptyOnly = Collections.singletonMap(CountText.TEXT_LINE_NONEMPTY_COUNT_PD, "true");
|
||||
final Map<PropertyDescriptor, String> wordsOnly = Collections.singletonMap(CountText.TEXT_WORD_COUNT_PD, "true");
|
||||
final Map<PropertyDescriptor, String> charactersOnly = Collections.singletonMap(CountText.TEXT_CHARACTER_COUNT_PD, "true");
|
||||
|
||||
final List<Map<PropertyDescriptor, String>> scenarios = Arrays.asList(linesOnly, linesNonEmptyOnly, wordsOnly, charactersOnly);
|
||||
|
||||
for (final Map<PropertyDescriptor, String> map: scenarios) {
|
||||
// Reset the processor properties
|
||||
runner.setProperty(CountText.TEXT_LINE_COUNT_PD, "false");
|
||||
runner.setProperty(CountText.TEXT_LINE_NONEMPTY_COUNT_PD, "false");
|
||||
runner.setProperty(CountText.TEXT_WORD_COUNT_PD, "false");
|
||||
runner.setProperty(CountText.TEXT_CHARACTER_COUNT_PD, "false");
|
||||
|
||||
// Apply the scenario-specific properties
|
||||
for (final Map.Entry<PropertyDescriptor, String> entry: map.entrySet()) {
|
||||
runner.setProperty(entry.getKey(), entry.getValue());
|
||||
}
|
||||
|
||||
runner.clearProvenanceEvents();
|
||||
runner.clearTransferState();
|
||||
runner.enqueue(Files.readAllBytes(inputPath));
|
||||
|
||||
runner.run();
|
||||
|
||||
runner.assertAllFlowFilesTransferred(CountText.REL_SUCCESS, 1);
|
||||
MockFlowFile flowFile = runner.getFlowFilesForRelationship(CountText.REL_SUCCESS).get(0);
|
||||
for (final Map.Entry<String, String> entry: expectedValues.entrySet()) {
|
||||
final String attribute = entry.getKey();
|
||||
final String expectedValue = entry.getValue();
|
||||
|
||||
if (flowFile.getAttributes().containsKey(attribute)) {
|
||||
flowFile.assertAttributeEquals(attribute, expectedValue);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldCountWordsSplitOnSymbol() throws IOException {
|
||||
final Path inputPath = Paths.get("src/test/resources/TestCountText/jabberwocky.txt");
|
||||
|
||||
final String EXPECTED_WORD_COUNT = "167";
|
||||
|
||||
// Reset the processor properties
|
||||
runner.setProperty(CountText.TEXT_LINE_COUNT_PD, "false");
|
||||
runner.setProperty(CountText.TEXT_LINE_NONEMPTY_COUNT_PD, "false");
|
||||
runner.setProperty(CountText.TEXT_WORD_COUNT_PD, "true");
|
||||
runner.setProperty(CountText.TEXT_CHARACTER_COUNT_PD, "false");
|
||||
runner.setProperty(CountText.SPLIT_WORDS_ON_SYMBOLS_PD, "true");
|
||||
|
||||
runner.clearProvenanceEvents();
|
||||
runner.clearTransferState();
|
||||
runner.enqueue(Files.readAllBytes(inputPath));
|
||||
|
||||
runner.run();
|
||||
|
||||
runner.assertAllFlowFilesTransferred(CountText.REL_SUCCESS, 1);
|
||||
MockFlowFile flowFile = runner.getFlowFilesForRelationship(CountText.REL_SUCCESS).get(0);
|
||||
flowFile.assertAttributeEquals(CountText.TEXT_WORD_COUNT, EXPECTED_WORD_COUNT);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldCountIndependentlyPerFlowFile() throws IOException {
|
||||
final Path inputPath = Paths.get("src/test/resources/TestCountText/jabberwocky.txt");
|
||||
|
||||
final Map<String, String> expectedValues = new HashMap<>();
|
||||
expectedValues.put(TLC, "34");
|
||||
expectedValues.put(TLNEC, "28");
|
||||
expectedValues.put(TWC, "166");
|
||||
expectedValues.put(TCC, "900");
|
||||
|
||||
// Reset the processor properties
|
||||
runner.setProperty(CountText.TEXT_LINE_COUNT_PD, "true");
|
||||
runner.setProperty(CountText.TEXT_LINE_NONEMPTY_COUNT_PD, "true");
|
||||
runner.setProperty(CountText.TEXT_WORD_COUNT_PD, "true");
|
||||
runner.setProperty(CountText.TEXT_CHARACTER_COUNT_PD, "true");
|
||||
|
||||
for (int i = 0; i < 2; i++) {
|
||||
runner.clearProvenanceEvents();
|
||||
runner.clearTransferState();
|
||||
runner.enqueue(Files.readAllBytes(inputPath));
|
||||
|
||||
runner.run();
|
||||
|
||||
runner.assertAllFlowFilesTransferred(CountText.REL_SUCCESS, 1);
|
||||
MockFlowFile flowFile = runner.getFlowFilesForRelationship(CountText.REL_SUCCESS).get(0);
|
||||
for (final Map.Entry<String, String> entry: expectedValues.entrySet()) {
|
||||
final String attribute = entry.getKey();
|
||||
final String expectedValue = entry.getValue();
|
||||
|
||||
flowFile.assertAttributeEquals(attribute, expectedValue);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldTrackSessionCountersAcrossMultipleFlowfiles() throws IOException, NoSuchFieldException, IllegalAccessException {
|
||||
final Path inputPath = Paths.get("src/test/resources/TestCountText/jabberwocky.txt");
|
||||
|
||||
final Map<String, String> expectedValues = new HashMap<>();
|
||||
expectedValues.put(TLC, "34");
|
||||
expectedValues.put(TLNEC, "28");
|
||||
expectedValues.put(TWC, "166");
|
||||
expectedValues.put(TCC, "900");
|
||||
|
||||
// Reset the processor properties
|
||||
runner.setProperty(CountText.TEXT_LINE_COUNT_PD, "true");
|
||||
runner.setProperty(CountText.TEXT_LINE_NONEMPTY_COUNT_PD, "true");
|
||||
runner.setProperty(CountText.TEXT_WORD_COUNT_PD, "true");
|
||||
runner.setProperty(CountText.TEXT_CHARACTER_COUNT_PD, "true");
|
||||
|
||||
final int n = 2;
|
||||
for (int i = 0; i < n; i++) {
|
||||
runner.clearTransferState();
|
||||
runner.enqueue(Files.readAllBytes(inputPath));
|
||||
|
||||
runner.run();
|
||||
|
||||
runner.assertAllFlowFilesTransferred(CountText.REL_SUCCESS, 1);
|
||||
MockFlowFile flowFile = runner.getFlowFilesForRelationship(CountText.REL_SUCCESS).get(0);
|
||||
for (final Map.Entry<String, String> entry: expectedValues.entrySet()) {
|
||||
final String attribute = entry.getKey();
|
||||
final String expectedValue = entry.getValue();
|
||||
|
||||
flowFile.assertAttributeEquals(attribute, expectedValue);
|
||||
}
|
||||
}
|
||||
|
||||
assertEquals(Long.valueOf(expectedValues.get(TLC)) * n, runner.getCounterValue("Lines Counted"));
|
||||
assertEquals(Long.valueOf(expectedValues.get(TLNEC)) * n, runner.getCounterValue("Lines (non-empty) Counted"));
|
||||
assertEquals(Long.valueOf(expectedValues.get(TWC)) * n, runner.getCounterValue("Words Counted"));
|
||||
assertEquals(Long.valueOf(expectedValues.get(TCC)) * n, runner.getCounterValue("Characters Counted"));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldHandleInternalError() {
|
||||
CountText ct = new CountText() {
|
||||
@Override
|
||||
int countWordsInLine(String line, boolean splitWordsOnSymbols) throws IOException {
|
||||
throw new IOException("Expected exception");
|
||||
}
|
||||
};
|
||||
|
||||
final TestRunner runner = TestRunners.newTestRunner(ct);
|
||||
final String INPUT_TEXT = "This flowfile should throw an error";
|
||||
|
||||
// Reset the processor properties
|
||||
runner.setProperty(CountText.TEXT_LINE_COUNT_PD, "true");
|
||||
runner.setProperty(CountText.TEXT_LINE_NONEMPTY_COUNT_PD, "true");
|
||||
runner.setProperty(CountText.TEXT_WORD_COUNT_PD, "true");
|
||||
runner.setProperty(CountText.TEXT_CHARACTER_COUNT_PD, "true");
|
||||
runner.setProperty(CountText.CHARACTER_ENCODING_PD, StandardCharsets.US_ASCII.displayName());
|
||||
|
||||
runner.enqueue(INPUT_TEXT.getBytes());
|
||||
|
||||
// Need initialize = true to run #onScheduled()
|
||||
runner.run(1, true, true);
|
||||
|
||||
runner.assertAllFlowFilesTransferred(CountText.REL_FAILURE, 1);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldIgnoreWhitespaceWordsWhenCounting() {
|
||||
final String INPUT_TEXT = "a b c";
|
||||
|
||||
final String EXPECTED_WORD_COUNT = "3";
|
||||
|
||||
// Reset the processor properties
|
||||
runner.setProperty(CountText.TEXT_LINE_COUNT_PD, "false");
|
||||
runner.setProperty(CountText.TEXT_LINE_NONEMPTY_COUNT_PD, "false");
|
||||
runner.setProperty(CountText.TEXT_WORD_COUNT_PD, "true");
|
||||
runner.setProperty(CountText.TEXT_CHARACTER_COUNT_PD, "false");
|
||||
runner.setProperty(CountText.SPLIT_WORDS_ON_SYMBOLS_PD, "true");
|
||||
|
||||
runner.clearProvenanceEvents();
|
||||
runner.clearTransferState();
|
||||
runner.enqueue(INPUT_TEXT.getBytes());
|
||||
|
||||
runner.run();
|
||||
|
||||
runner.assertAllFlowFilesTransferred(CountText.REL_SUCCESS, 1);
|
||||
MockFlowFile flowFile = runner.getFlowFilesForRelationship(CountText.REL_SUCCESS).get(0);
|
||||
flowFile.assertAttributeEquals(CountText.TEXT_WORD_COUNT, EXPECTED_WORD_COUNT);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldIgnoreWhitespaceWordsWhenCountingDebugMode() {
|
||||
final MockComponentLog componentLogger = spy(new MockComponentLog("processorId", new CountText()));
|
||||
doReturn(true).when(componentLogger).isDebugEnabled();
|
||||
final TestRunner runner = TestRunners.newTestRunner(CountText.class, componentLogger);
|
||||
final String INPUT_TEXT = "a b c";
|
||||
|
||||
final String EXPECTED_WORD_COUNT = "3";
|
||||
|
||||
// Reset the processor properties
|
||||
runner.setProperty(CountText.TEXT_LINE_COUNT_PD, "false");
|
||||
runner.setProperty(CountText.TEXT_LINE_NONEMPTY_COUNT_PD, "false");
|
||||
runner.setProperty(CountText.TEXT_WORD_COUNT_PD, "true");
|
||||
runner.setProperty(CountText.TEXT_CHARACTER_COUNT_PD, "false");
|
||||
runner.setProperty(CountText.SPLIT_WORDS_ON_SYMBOLS_PD, "true");
|
||||
|
||||
runner.clearProvenanceEvents();
|
||||
runner.clearTransferState();
|
||||
runner.enqueue(INPUT_TEXT.getBytes());
|
||||
|
||||
runner.run();
|
||||
|
||||
runner.assertAllFlowFilesTransferred(CountText.REL_SUCCESS, 1);
|
||||
MockFlowFile flowFile = runner.getFlowFilesForRelationship(CountText.REL_SUCCESS).get(0);
|
||||
|
||||
flowFile.assertAttributeEquals(CountText.TEXT_WORD_COUNT, EXPECTED_WORD_COUNT);
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,323 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License") you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.processors.standard;
|
||||
|
||||
import org.apache.nifi.security.util.crypto.HashAlgorithm;
|
||||
import org.apache.nifi.security.util.crypto.HashService;
|
||||
import org.apache.nifi.util.MockFlowFile;
|
||||
import org.apache.nifi.util.TestRunner;
|
||||
import org.apache.nifi.util.TestRunners;
|
||||
import org.bouncycastle.jce.provider.BouncyCastleProvider;
|
||||
import org.junit.jupiter.api.BeforeAll;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.nio.charset.Charset;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.security.Security;
|
||||
import java.time.ZonedDateTime;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
public class CryptographicHashAttributeTest {
|
||||
private TestRunner runner;
|
||||
|
||||
@BeforeAll
|
||||
static void setUpOnce() {
|
||||
Security.addProvider(new BouncyCastleProvider());
|
||||
}
|
||||
|
||||
@BeforeEach
|
||||
void setupRunner() {
|
||||
runner = TestRunners.newTestRunner(new CryptographicHashAttribute());
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldCalculateHashOfPresentAttribute() {
|
||||
// Create attributes for username and date
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("username", "alopresto");
|
||||
attributes.put("date", ZonedDateTime.now().format(DateTimeFormatter.ofPattern("YYYY-MM-dd HH:mm:ss.SSS Z")));
|
||||
|
||||
final Set<String> attributeKeys = attributes.keySet();
|
||||
|
||||
for (final HashAlgorithm algorithm : HashAlgorithm.values()) {
|
||||
final String expectedUsernameHash = HashService.hashValue(algorithm, attributes.get("username"));
|
||||
final String expectedDateHash = HashService.hashValue(algorithm, attributes.get("date"));
|
||||
|
||||
// Reset the processor
|
||||
runner.clearProperties();
|
||||
runner.clearProvenanceEvents();
|
||||
runner.clearTransferState();
|
||||
|
||||
// Set the algorithm
|
||||
runner.setProperty(CryptographicHashAttribute.HASH_ALGORITHM, algorithm.getName());
|
||||
|
||||
// Add the desired dynamic properties
|
||||
for (final String attr: attributeKeys) {
|
||||
runner.setProperty(attr, String.format("%s_%s", attr, algorithm.getName()));
|
||||
}
|
||||
|
||||
// Insert the attributes in the mock flowfile
|
||||
runner.enqueue(new byte[0], attributes);
|
||||
|
||||
runner.run(1);
|
||||
|
||||
runner.assertTransferCount(CryptographicHashAttribute.REL_FAILURE, 0);
|
||||
runner.assertTransferCount(CryptographicHashAttribute.REL_SUCCESS, 1);
|
||||
|
||||
final List<MockFlowFile> successfulFlowfiles = runner.getFlowFilesForRelationship(CryptographicHashAttribute.REL_SUCCESS);
|
||||
|
||||
// Extract the generated attributes from the flowfile
|
||||
MockFlowFile flowFile = successfulFlowfiles.get(0);
|
||||
|
||||
flowFile.assertAttributeEquals(String.format("username_%s", algorithm.getName()), expectedUsernameHash);
|
||||
flowFile.assertAttributeEquals(String.format("date_%s", algorithm.getName()), expectedDateHash);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldCalculateHashOfMissingAttribute() {
|
||||
// Create attributes for username (empty string) and date (null)
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("username", "");
|
||||
attributes.put("date", null);
|
||||
|
||||
final Set<String> attributeKeys = attributes.keySet();
|
||||
|
||||
for (final HashAlgorithm algorithm: HashAlgorithm.values()) {
|
||||
final String expectedUsernameHash = HashService.hashValue(algorithm, attributes.get("username"));
|
||||
final String expectedDateHash = null;
|
||||
|
||||
// Reset the processor
|
||||
runner.clearProperties();
|
||||
runner.clearProvenanceEvents();
|
||||
runner.clearTransferState();
|
||||
|
||||
// Set the algorithm
|
||||
runner.setProperty(CryptographicHashAttribute.HASH_ALGORITHM, algorithm.getName());
|
||||
|
||||
// Add the desired dynamic properties
|
||||
for (final String attr: attributeKeys) {
|
||||
runner.setProperty(attr, String.format("%s_%s", attr, algorithm.getName()));
|
||||
}
|
||||
|
||||
// Insert the attributes in the mock flowfile
|
||||
runner.enqueue(new byte[0], attributes);
|
||||
|
||||
runner.run(1);
|
||||
|
||||
runner.assertTransferCount(CryptographicHashAttribute.REL_FAILURE, 0);
|
||||
runner.assertTransferCount(CryptographicHashAttribute.REL_SUCCESS, 1);
|
||||
|
||||
final List<MockFlowFile> successfulFlowfiles = runner.getFlowFilesForRelationship(CryptographicHashAttribute.REL_SUCCESS);
|
||||
|
||||
// Extract the generated attributes from the flowfile
|
||||
MockFlowFile flowFile = successfulFlowfiles.get(0);
|
||||
|
||||
flowFile.assertAttributeEquals(String.format("username_%s", algorithm.getName()), expectedUsernameHash);
|
||||
flowFile.assertAttributeEquals(String.format("date_%s", algorithm.getName()), expectedDateHash);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldRouteToFailureOnProhibitedMissingAttribute() {
|
||||
// Create attributes for username (empty string) and date (null)
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("username", "");
|
||||
attributes.put("date", null);
|
||||
|
||||
final Set<String> attributeKeys = attributes.keySet();
|
||||
|
||||
for (final HashAlgorithm algorithm: HashAlgorithm.values()) {
|
||||
// Reset the processor
|
||||
runner.clearProperties();
|
||||
runner.clearProvenanceEvents();
|
||||
runner.clearTransferState();
|
||||
|
||||
// Set the algorithm
|
||||
runner.setProperty(CryptographicHashAttribute.HASH_ALGORITHM, algorithm.getName());
|
||||
|
||||
// Set to fail if there are missing attributes
|
||||
runner.setProperty(CryptographicHashAttribute.PARTIAL_ATTR_ROUTE_POLICY, CryptographicHashAttribute.PartialAttributePolicy.PROHIBIT.name());
|
||||
|
||||
// Add the desired dynamic properties
|
||||
for (final String attr: attributeKeys) {
|
||||
runner.setProperty(attr, String.format("%s_%s", attr, algorithm.getName()));
|
||||
}
|
||||
|
||||
// Insert the attributes in the mock flowfile
|
||||
runner.enqueue(new byte[0], attributes);
|
||||
|
||||
runner.run(1);
|
||||
|
||||
runner.assertTransferCount(CryptographicHashAttribute.REL_FAILURE, 1);
|
||||
runner.assertTransferCount(CryptographicHashAttribute.REL_SUCCESS, 0);
|
||||
|
||||
final List<MockFlowFile> failedFlowFiles = runner.getFlowFilesForRelationship(CryptographicHashAttribute.REL_FAILURE);
|
||||
|
||||
// Extract the generated attributes from the flowfile
|
||||
MockFlowFile flowFile = failedFlowFiles.get(0);
|
||||
for (final String missingAttribute: attributeKeys) {
|
||||
flowFile.assertAttributeNotExists(String.format("%s_%s", missingAttribute, algorithm.getName()));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldRouteToFailureOnEmptyAttributes() {
|
||||
// Create attributes for username (empty string) and date (null)
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("username", "");
|
||||
attributes.put("date", null);
|
||||
|
||||
final Set<String> attributeKeys = attributes.keySet();
|
||||
|
||||
for (final HashAlgorithm algorithm: HashAlgorithm.values()) {
|
||||
// Reset the processor
|
||||
runner.clearProperties();
|
||||
runner.clearProvenanceEvents();
|
||||
runner.clearTransferState();
|
||||
|
||||
// Set the algorithm
|
||||
runner.setProperty(CryptographicHashAttribute.HASH_ALGORITHM, algorithm.getName());
|
||||
|
||||
// Set to fail if all attributes are missing
|
||||
runner.setProperty(CryptographicHashAttribute.FAIL_WHEN_EMPTY, "true");
|
||||
|
||||
// Insert the attributes in the mock flowfile
|
||||
runner.enqueue(new byte[0], attributes);
|
||||
|
||||
runner.run(1);
|
||||
|
||||
runner.assertTransferCount(CryptographicHashAttribute.REL_FAILURE, 1);
|
||||
runner.assertTransferCount(CryptographicHashAttribute.REL_SUCCESS, 0);
|
||||
|
||||
final List<MockFlowFile> failedFlowFiles = runner.getFlowFilesForRelationship(CryptographicHashAttribute.REL_FAILURE);
|
||||
|
||||
// Extract the generated attributes from the flowfile
|
||||
MockFlowFile flowFile = failedFlowFiles.get(0);
|
||||
for (final String missingAttribute: attributeKeys) {
|
||||
flowFile.assertAttributeNotExists(String.format("%s_%s", missingAttribute, algorithm.getName()));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldRouteToSuccessOnAllowPartial() {
|
||||
// Create attributes for username (empty string) and date (null)
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("username", "");
|
||||
|
||||
final Set<String> attributeKeys = attributes.keySet();
|
||||
|
||||
for (final HashAlgorithm algorithm: HashAlgorithm.values()) {
|
||||
// Reset the processor
|
||||
runner.clearProperties();
|
||||
runner.clearProvenanceEvents();
|
||||
runner.clearTransferState();
|
||||
|
||||
// Set the algorithm
|
||||
runner.setProperty(CryptographicHashAttribute.HASH_ALGORITHM, algorithm.getName());
|
||||
|
||||
// Set to fail if there are missing attributes
|
||||
runner.setProperty(CryptographicHashAttribute.PARTIAL_ATTR_ROUTE_POLICY, CryptographicHashAttribute.PartialAttributePolicy.ALLOW.name());
|
||||
|
||||
// Add the desired dynamic properties
|
||||
for (final String attr: attributeKeys) {
|
||||
runner.setProperty(attr, String.format("%s_%s", attr, algorithm.getName()));
|
||||
}
|
||||
|
||||
// Insert the attributes in the mock flowfile
|
||||
runner.enqueue(new byte[0], attributes);
|
||||
|
||||
runner.run(1);
|
||||
|
||||
runner.assertTransferCount(CryptographicHashAttribute.REL_FAILURE, 0);
|
||||
runner.assertTransferCount(CryptographicHashAttribute.REL_SUCCESS, 1);
|
||||
|
||||
final List<MockFlowFile> successfulFlowFiles = runner.getFlowFilesForRelationship(CryptographicHashAttribute.REL_SUCCESS);
|
||||
|
||||
// Extract the generated attributes from the flowfile
|
||||
MockFlowFile flowFile = successfulFlowFiles.get(0);
|
||||
for (final String attribute: attributeKeys) {
|
||||
flowFile.assertAttributeExists(String.format("%s_%s", attribute, algorithm.getName()));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldCalculateHashWithVariousCharacterEncodings() {
|
||||
// Create attributes
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("test_attribute", "apachenifi");
|
||||
final Set<String> attributeKeys = attributes.keySet();
|
||||
|
||||
final HashAlgorithm algorithm = HashAlgorithm.MD5;
|
||||
|
||||
final List<Charset> charsets = Arrays.asList(StandardCharsets.UTF_8, StandardCharsets.UTF_16, StandardCharsets.UTF_16LE, StandardCharsets.UTF_16BE);
|
||||
|
||||
final Map<String, String> EXPECTED_MD5_HASHES = new HashMap<>();
|
||||
EXPECTED_MD5_HASHES.put(StandardCharsets.UTF_8.name(), "a968b5ec1d52449963dcc517789baaaf");
|
||||
EXPECTED_MD5_HASHES.put(StandardCharsets.UTF_16.name(), "b8413d18f7e64042bb0322a1cd61eba2");
|
||||
EXPECTED_MD5_HASHES.put(StandardCharsets.UTF_16BE.name(), "b8413d18f7e64042bb0322a1cd61eba2");
|
||||
EXPECTED_MD5_HASHES.put(StandardCharsets.UTF_16LE.name(), "91c3b67f9f8ae77156f21f271cc09121");
|
||||
|
||||
for (final Charset charset: charsets) {
|
||||
// Calculate the expected hash value given the character set
|
||||
final String EXPECTED_HASH = HashService.hashValue(algorithm, attributes.get("test_attribute"), charset);
|
||||
|
||||
// Sanity check
|
||||
assertEquals(EXPECTED_HASH, EXPECTED_MD5_HASHES.get(charset.name()));
|
||||
|
||||
// Reset the processor
|
||||
runner.clearProperties();
|
||||
runner.clearProvenanceEvents();
|
||||
runner.clearTransferState();
|
||||
|
||||
// Set the properties
|
||||
runner.setProperty(CryptographicHashAttribute.HASH_ALGORITHM, algorithm.getName());
|
||||
runner.setProperty(CryptographicHashAttribute.CHARACTER_SET, charset.name());
|
||||
|
||||
// Add the desired dynamic properties
|
||||
for (final String attr: attributeKeys) {
|
||||
runner.setProperty(attr, String.format("%s_%s", attr, algorithm.getName()));
|
||||
}
|
||||
|
||||
// Insert the attributes in the mock flowfile
|
||||
runner.enqueue(new byte[0], attributes);
|
||||
|
||||
runner.run(1);
|
||||
|
||||
runner.assertTransferCount(CryptographicHashAttribute.REL_FAILURE, 0);
|
||||
runner.assertTransferCount(CryptographicHashAttribute.REL_SUCCESS, 1);
|
||||
|
||||
final List<MockFlowFile> successfulFlowfiles = runner.getFlowFilesForRelationship(CryptographicHashAttribute.REL_SUCCESS);
|
||||
|
||||
// Extract the generated attributes from the flowfile
|
||||
MockFlowFile flowFile = successfulFlowfiles.get(0);
|
||||
|
||||
flowFile.assertAttributeEquals(String.format("test_attribute_%s", algorithm.getName()), EXPECTED_HASH);
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,232 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License") you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.processors.standard;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.security.util.crypto.HashAlgorithm;
|
||||
import org.apache.nifi.security.util.crypto.HashService;
|
||||
import org.apache.nifi.util.MockFlowFile;
|
||||
import org.apache.nifi.util.TestRunner;
|
||||
import org.apache.nifi.util.TestRunners;
|
||||
import org.bouncycastle.jce.provider.BouncyCastleProvider;
|
||||
import org.junit.jupiter.api.BeforeAll;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.security.Security;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotEquals;
|
||||
|
||||
public class CryptographicHashContentTest {
|
||||
private TestRunner runner;
|
||||
|
||||
@BeforeAll
|
||||
static void setUpOnce() {
|
||||
Security.addProvider(new BouncyCastleProvider());
|
||||
}
|
||||
|
||||
@BeforeEach
|
||||
void setupRunner() {
|
||||
runner = TestRunners.newTestRunner(new CryptographicHashContent());
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldCalculateHashOfPresentContent() throws IOException {
|
||||
// Generate some long content (90 KB)
|
||||
final String longContent = StringUtils.repeat("apachenifi ", 8192);
|
||||
|
||||
for (final HashAlgorithm algorithm : HashAlgorithm.values()) {
|
||||
final String expectedContentHash = HashService.hashValueStreaming(algorithm, new ByteArrayInputStream(longContent.getBytes()));
|
||||
|
||||
// Reset the processor
|
||||
runner.clearProperties();
|
||||
runner.clearProvenanceEvents();
|
||||
runner.clearTransferState();
|
||||
|
||||
// Set the algorithm
|
||||
runner.setProperty(CryptographicHashContent.HASH_ALGORITHM, algorithm.getName());
|
||||
|
||||
// Insert the content in the mock flowfile
|
||||
runner.enqueue(longContent.getBytes(StandardCharsets.UTF_8),
|
||||
Collections.singletonMap("size", String.valueOf(longContent.length())));
|
||||
|
||||
runner.run(1);
|
||||
|
||||
runner.assertTransferCount(CryptographicHashContent.REL_FAILURE, 0);
|
||||
runner.assertTransferCount(CryptographicHashContent.REL_SUCCESS, 1);
|
||||
|
||||
final List<MockFlowFile> successfulFlowfiles = runner.getFlowFilesForRelationship(CryptographicHashContent.REL_SUCCESS);
|
||||
|
||||
// Extract the generated attributes from the flowfile
|
||||
MockFlowFile flowFile = successfulFlowfiles.get(0);
|
||||
String hashAttribute = String.format("content_%s", algorithm.getName());
|
||||
flowFile.assertAttributeExists(hashAttribute);
|
||||
flowFile.assertAttributeEquals(hashAttribute, expectedContentHash);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldCalculateHashOfEmptyContent() throws IOException {
|
||||
final String emptyContent = "";
|
||||
|
||||
for (final HashAlgorithm algorithm : HashAlgorithm.values()) {
|
||||
final String expectedContentHash = HashService.hashValueStreaming(algorithm, new ByteArrayInputStream(emptyContent.getBytes()));
|
||||
|
||||
// Reset the processor
|
||||
runner.clearProperties();
|
||||
runner.clearProvenanceEvents();
|
||||
runner.clearTransferState();
|
||||
|
||||
// Set the algorithm
|
||||
runner.setProperty(CryptographicHashContent.HASH_ALGORITHM, algorithm.getName());
|
||||
|
||||
// Insert the content in the mock flowfile
|
||||
runner.enqueue(emptyContent.getBytes(StandardCharsets.UTF_8), Collections.singletonMap("size", "0"));
|
||||
|
||||
runner.run(1);
|
||||
|
||||
runner.assertTransferCount(CryptographicHashContent.REL_FAILURE, 0);
|
||||
runner.assertTransferCount(CryptographicHashContent.REL_SUCCESS, 1);
|
||||
|
||||
final List<MockFlowFile> successfulFlowfiles = runner.getFlowFilesForRelationship(CryptographicHashContent.REL_SUCCESS);
|
||||
|
||||
// Extract the generated attributes from the flowfile
|
||||
MockFlowFile flowFile = successfulFlowfiles.get(0);
|
||||
String hashAttribute = String.format("content_%s", algorithm.getName());
|
||||
flowFile.assertAttributeExists(hashAttribute);
|
||||
|
||||
String hashedContent = flowFile.getAttribute(hashAttribute);
|
||||
|
||||
assertEquals(expectedContentHash, hashedContent);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This test works because {@link MockFlowFile} uses the actual internal {@code data.size} for {@code getSize ( )},
|
||||
* while {@code StandardFlowFileRecord} uses a separate {@code size} field. May need to use {@code flowfile.getContentClaim ( ) .getLength ( )}.
|
||||
*/
|
||||
@Test
|
||||
void testShouldCalculateHashOfContentWithIncorrectSizeAttribute() throws IOException {
|
||||
final String nonEmptyContent = "apachenifi";
|
||||
|
||||
final TestRunner runner = TestRunners.newTestRunner(new CryptographicHashContent());
|
||||
|
||||
for (final HashAlgorithm algorithm : HashAlgorithm.values()) {
|
||||
final String expectedContentHash = HashService.hashValueStreaming(algorithm, new ByteArrayInputStream(nonEmptyContent.getBytes()));
|
||||
|
||||
// Reset the processor
|
||||
runner.clearProperties();
|
||||
runner.clearProvenanceEvents();
|
||||
runner.clearTransferState();
|
||||
|
||||
// Set the algorithm
|
||||
runner.setProperty(CryptographicHashContent.HASH_ALGORITHM, algorithm.getName());
|
||||
|
||||
// Insert the content in the mock flowfile (with the wrong size attribute)
|
||||
runner.enqueue(nonEmptyContent.getBytes(StandardCharsets.UTF_8), Collections.singletonMap("size", "0"));
|
||||
|
||||
runner.run(1);
|
||||
|
||||
runner.assertTransferCount(CryptographicHashContent.REL_FAILURE, 0);
|
||||
runner.assertTransferCount(CryptographicHashContent.REL_SUCCESS, 1);
|
||||
|
||||
final List<MockFlowFile> successfulFlowfiles = runner.getFlowFilesForRelationship(CryptographicHashContent.REL_SUCCESS);
|
||||
|
||||
// Extract the generated attributes from the flowfile
|
||||
MockFlowFile flowFile = successfulFlowfiles.get(0);
|
||||
String hashAttribute = String.format("content_%s", algorithm.getName());
|
||||
flowFile.assertAttributeExists(hashAttribute);
|
||||
flowFile.assertAttributeEquals(hashAttribute, expectedContentHash);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldOverwriteExistingAttribute() {
|
||||
final String nonEmptyContent = "apachenifi";
|
||||
final String oldHashAttributeValue = "OLD VALUE";
|
||||
|
||||
HashAlgorithm algorithm = HashAlgorithm.SHA256;
|
||||
|
||||
final String expectedContentHash = HashService.hashValue(algorithm, nonEmptyContent);
|
||||
|
||||
// Set the algorithm
|
||||
runner.setProperty(CryptographicHashContent.HASH_ALGORITHM, algorithm.getName());
|
||||
|
||||
// Insert the content in the mock flowfile (with an existing attribute)
|
||||
final Map<String, String> oldAttributes = Collections.singletonMap(String.format("content_%s", algorithm.getName()),
|
||||
oldHashAttributeValue);
|
||||
runner.enqueue(nonEmptyContent.getBytes(StandardCharsets.UTF_8),
|
||||
oldAttributes);
|
||||
|
||||
runner.run(1);
|
||||
|
||||
runner.assertTransferCount(CryptographicHashContent.REL_FAILURE, 0);
|
||||
runner.assertTransferCount(CryptographicHashContent.REL_SUCCESS, 1);
|
||||
|
||||
final List<MockFlowFile> successfulFlowfiles = runner.getFlowFilesForRelationship(CryptographicHashContent.REL_SUCCESS);
|
||||
|
||||
// Extract the generated attributes from the flowfile
|
||||
MockFlowFile flowFile = successfulFlowfiles.get(0);
|
||||
String hashAttribute = String.format("content_%s", algorithm.getName());
|
||||
flowFile.assertAttributeExists(hashAttribute);
|
||||
|
||||
String hashedContent = flowFile.getAttribute(hashAttribute);
|
||||
|
||||
assertNotEquals(oldHashAttributeValue, hashedContent);
|
||||
assertEquals(expectedContentHash, hashedContent);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldRouteToFailureOnEmptyContent() {
|
||||
final String emptyContent = "";
|
||||
|
||||
for (final HashAlgorithm algorithm : HashAlgorithm.values()) {
|
||||
// Reset the processor
|
||||
runner.clearProperties();
|
||||
runner.clearProvenanceEvents();
|
||||
runner.clearTransferState();
|
||||
|
||||
// Set the failure property
|
||||
runner.setProperty(CryptographicHashContent.FAIL_WHEN_EMPTY, "true");
|
||||
|
||||
// Set the algorithm
|
||||
runner.setProperty(CryptographicHashContent.HASH_ALGORITHM, algorithm.getName());
|
||||
|
||||
// Insert the content in the mock flowfile
|
||||
runner.enqueue(emptyContent.getBytes(StandardCharsets.UTF_8));
|
||||
|
||||
runner.run(1);
|
||||
|
||||
runner.assertTransferCount(CryptographicHashContent.REL_FAILURE, 1);
|
||||
runner.assertTransferCount(CryptographicHashContent.REL_SUCCESS, 0);
|
||||
|
||||
final List<MockFlowFile> failedFlowfiles = runner.getFlowFilesForRelationship(CryptographicHashContent.REL_FAILURE);
|
||||
|
||||
// Extract the generated attributes from the flowfile
|
||||
MockFlowFile flowFile = failedFlowfiles.get(0);
|
||||
String hashAttribute = String.format("content_%s", algorithm.getName());
|
||||
flowFile.assertAttributeNotExists(hashAttribute);
|
||||
}
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
@ -0,0 +1,55 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.processors.standard;
|
||||
|
||||
import org.apache.nifi.util.TestRunner;
|
||||
import org.apache.nifi.util.TestRunners;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.file.Paths;
|
||||
|
||||
public class SplitXmlTest {
|
||||
private TestRunner runner;
|
||||
|
||||
@BeforeEach
|
||||
void setupRunner() {
|
||||
runner = TestRunners.newTestRunner(new SplitXml());
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldHandleXXEInTemplate() throws IOException {
|
||||
final String xxeTemplateFilepath = "src/test/resources/xxe_template.xml";
|
||||
assertExternalEntitiesFailure(xxeTemplateFilepath);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldHandleRemoteCallXXE() throws IOException {
|
||||
final String xxeTemplateFilepath = "src/test/resources/xxe_from_report.xml";
|
||||
assertExternalEntitiesFailure(xxeTemplateFilepath);
|
||||
}
|
||||
|
||||
private void assertExternalEntitiesFailure(final String filePath) throws IOException {
|
||||
runner.setProperty(SplitXml.SPLIT_DEPTH, "3");
|
||||
runner.enqueue(Paths.get(filePath));
|
||||
|
||||
runner.run();
|
||||
|
||||
runner.assertAllFlowFilesTransferred(SplitXml.REL_FAILURE);
|
||||
}
|
||||
}
|
@ -0,0 +1,156 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.processors.standard;
|
||||
|
||||
import org.apache.nifi.reporting.InitializationException;
|
||||
import org.apache.nifi.serialization.SimpleRecordSchema;
|
||||
|
||||
import org.apache.nifi.serialization.record.MapRecord;
|
||||
import org.apache.nifi.serialization.record.MockRecordParser;
|
||||
import org.apache.nifi.serialization.record.RecordField;
|
||||
import org.apache.nifi.serialization.record.RecordFieldType;
|
||||
import org.apache.nifi.serialization.record.RecordSchema;
|
||||
import org.apache.nifi.util.MockFlowFile;
|
||||
import org.apache.nifi.util.TestRunner;
|
||||
import org.apache.nifi.util.TestRunners;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotNull;
|
||||
|
||||
public class TestCalculateRecordStats {
|
||||
TestRunner runner;
|
||||
MockRecordParser recordParser;
|
||||
RecordSchema personSchema;
|
||||
|
||||
@BeforeEach
|
||||
void setup() throws InitializationException {
|
||||
runner = TestRunners.newTestRunner(CalculateRecordStats.class);
|
||||
recordParser = new MockRecordParser();
|
||||
runner.addControllerService("recordReader", recordParser);
|
||||
runner.setProperty(CalculateRecordStats.RECORD_READER, "recordReader");
|
||||
runner.enableControllerService(recordParser);
|
||||
runner.assertValid();
|
||||
|
||||
recordParser.addSchemaField("id", RecordFieldType.INT);
|
||||
List<RecordField> personFields = new ArrayList<>();
|
||||
RecordField nameField = new RecordField("name", RecordFieldType.STRING.getDataType());
|
||||
RecordField ageField = new RecordField("age", RecordFieldType.INT.getDataType());
|
||||
RecordField sportField = new RecordField("sport", RecordFieldType.STRING.getDataType());
|
||||
personFields.add(nameField);
|
||||
personFields.add(ageField);
|
||||
personFields.add(sportField);
|
||||
personSchema = new SimpleRecordSchema(personFields);
|
||||
recordParser.addSchemaField("person", RecordFieldType.RECORD);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testNoNullOrEmptyRecordFields() {
|
||||
final List<String> sports = Arrays.asList("Soccer", "Soccer", "Soccer", "Football", "Football", "Basketball");
|
||||
final Map<String, String> expectedAttributes = new HashMap<>();
|
||||
expectedAttributes.put("recordStats.sport.Soccer", "3");
|
||||
expectedAttributes.put("recordStats.sport.Football", "2");
|
||||
expectedAttributes.put("recordStats.sport.Basketball", "1");
|
||||
expectedAttributes.put("recordStats.sport", "6");
|
||||
expectedAttributes.put("record.count", "6");
|
||||
|
||||
commonTest(Collections.singletonMap("sport", "/person/sport"), sports, expectedAttributes);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testWithNullFields() {
|
||||
final List<String> sports = Arrays.asList("Soccer", null, null, "Football", null, "Basketball");
|
||||
final Map<String, String> expectedAttributes = new HashMap<>();
|
||||
expectedAttributes.put("recordStats.sport.Soccer", "1");
|
||||
expectedAttributes.put("recordStats.sport.Football", "1");
|
||||
expectedAttributes.put("recordStats.sport.Basketball", "1");
|
||||
expectedAttributes.put("recordStats.sport", "3");
|
||||
expectedAttributes.put("record.count", "6");
|
||||
|
||||
commonTest(Collections.singletonMap("sport", "/person/sport"), sports, expectedAttributes);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testWithFilters() {
|
||||
final List<String> sports = Arrays.asList("Soccer", "Soccer", "Soccer", "Football", "Football", "Basketball");
|
||||
final Map<String, String> expectedAttributes = new HashMap<>();
|
||||
expectedAttributes.put("recordStats.sport.Soccer", "3");
|
||||
expectedAttributes.put("recordStats.sport.Basketball", "1");
|
||||
expectedAttributes.put("recordStats.sport", "4");
|
||||
expectedAttributes.put("record.count", "6");
|
||||
|
||||
final Map<String, String> propz = Collections.singletonMap("sport", "/person/sport[. != 'Football']");
|
||||
|
||||
commonTest(propz, sports, expectedAttributes);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testWithSizeLimit() {
|
||||
runner.setProperty(CalculateRecordStats.LIMIT, "3");
|
||||
final List<String> sports = Arrays.asList("Soccer", "Soccer", "Soccer", "Football", "Football",
|
||||
"Basketball", "Baseball", "Baseball", "Baseball", "Baseball",
|
||||
"Skiing", "Skiing", "Skiing", "Snowboarding");
|
||||
final Map<String, String> expectedAttributes = new HashMap<>();
|
||||
expectedAttributes.put("recordStats.sport.Skiing", "3");
|
||||
expectedAttributes.put("recordStats.sport.Soccer", "3");
|
||||
expectedAttributes.put("recordStats.sport.Baseball", "4");
|
||||
expectedAttributes.put("recordStats.sport", String.valueOf(sports.size()));
|
||||
expectedAttributes.put("record.count", String.valueOf(sports.size()));
|
||||
|
||||
final Map<String, String> propz = Collections.singletonMap("sport", "/person/sport");
|
||||
|
||||
commonTest(propz, sports, expectedAttributes);
|
||||
}
|
||||
|
||||
private void commonTest(Map<String, String> procProperties, List<String> sports, Map<String, String> expectedAttributes) {
|
||||
int index = 1;
|
||||
for (final String sport : sports) {
|
||||
final Map<String, Object> newRecord = new HashMap<>();
|
||||
newRecord.put("name", "John Doe");
|
||||
newRecord.put("age", 48);
|
||||
newRecord.put("sport", sport);
|
||||
recordParser.addRecord(index++, new MapRecord(personSchema, newRecord));
|
||||
}
|
||||
|
||||
for (final Map.Entry<String, String> property : procProperties.entrySet()) {
|
||||
runner.setProperty(property.getKey(), property.getValue());
|
||||
}
|
||||
|
||||
runner.enqueue("");
|
||||
runner.run();
|
||||
runner.assertTransferCount(CalculateRecordStats.REL_FAILURE, 0);
|
||||
runner.assertTransferCount(CalculateRecordStats.REL_SUCCESS, 1);
|
||||
|
||||
final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(CalculateRecordStats.REL_SUCCESS);
|
||||
final MockFlowFile ff = flowFiles.get(0);
|
||||
for (final Map.Entry<String, String> expectedAttribute : expectedAttributes.entrySet()) {
|
||||
final String key = expectedAttribute.getKey();
|
||||
final String value = expectedAttribute.getValue();
|
||||
assertNotNull(ff.getAttribute(key), String.format("Missing %s", key));
|
||||
assertEquals(value, ff.getAttribute(key));
|
||||
}
|
||||
}
|
||||
}
|
@ -16,14 +16,23 @@
|
||||
*/
|
||||
package org.apache.nifi.processors.standard;
|
||||
|
||||
import groovy.time.TimeCategory;
|
||||
import groovy.time.TimeDuration;
|
||||
import org.apache.commons.codec.DecoderException;
|
||||
import org.apache.commons.codec.binary.Hex;
|
||||
import org.apache.nifi.components.AllowableValue;
|
||||
import org.apache.nifi.components.ValidationResult;
|
||||
import org.apache.nifi.security.util.EncryptionMethod;
|
||||
import org.apache.nifi.security.util.KeyDerivationFunction;
|
||||
import org.apache.nifi.security.util.crypto.Argon2CipherProvider;
|
||||
import org.apache.nifi.security.util.crypto.Argon2SecureHasher;
|
||||
import org.apache.nifi.security.util.crypto.CipherUtility;
|
||||
import org.apache.nifi.security.util.crypto.KeyedEncryptor;
|
||||
import org.apache.nifi.security.util.crypto.PasswordBasedEncryptor;
|
||||
import org.apache.nifi.security.util.crypto.RandomIVPBECipherProvider;
|
||||
import org.apache.nifi.util.MockFlowFile;
|
||||
import org.apache.nifi.util.MockProcessContext;
|
||||
import org.apache.nifi.util.StringUtils;
|
||||
import org.apache.nifi.util.TestRunner;
|
||||
import org.apache.nifi.util.TestRunners;
|
||||
import org.bouncycastle.bcpg.BCPGInputStream;
|
||||
@ -31,10 +40,10 @@ import org.bouncycastle.bcpg.SymmetricKeyEncSessionPacket;
|
||||
import org.bouncycastle.jce.provider.BouncyCastleProvider;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import javax.crypto.Cipher;
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
@ -42,34 +51,42 @@ import java.lang.reflect.Method;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.nio.file.Paths;
|
||||
import java.security.Security;
|
||||
import java.text.ParseException;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Date;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.bouncycastle.openpgp.PGPUtil.getDecoderStream;
|
||||
import static org.junit.jupiter.api.Assertions.assertArrayEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.junit.jupiter.api.Assertions.fail;
|
||||
|
||||
public class TestEncryptContent {
|
||||
|
||||
private static final Logger logger = LoggerFactory.getLogger(TestEncryptContent.class);
|
||||
|
||||
private static AllowableValue[] getPGPCipherList() {
|
||||
try{
|
||||
Method method = EncryptContent.class.getDeclaredMethod("buildPGPSymmetricCipherAllowableValues");
|
||||
method.setAccessible(true);
|
||||
return ((AllowableValue[]) method.invoke(null));
|
||||
} catch (Exception e){
|
||||
logger.error("Cannot access buildPGPSymmetricCipherAllowableValues", e);
|
||||
fail("Cannot access buildPGPSymmetricCipherAllowableValues");
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
private static final List<EncryptionMethod> SUPPORTED_KEYED_ENCRYPTION_METHODS = Arrays
|
||||
.stream(EncryptionMethod.values())
|
||||
.filter(method -> method.isKeyedCipher() && method != EncryptionMethod.AES_CBC_NO_PADDING)
|
||||
.collect(Collectors.toList());
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() {
|
||||
Security.addProvider(new BouncyCastleProvider());
|
||||
@ -93,7 +110,6 @@ public class TestEncryptContent {
|
||||
continue;
|
||||
}
|
||||
|
||||
logger.info("Attempting {}", encryptionMethod.name());
|
||||
testRunner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, encryptionMethod.name());
|
||||
testRunner.setProperty(EncryptContent.MODE, EncryptContent.ENCRYPT_MODE);
|
||||
|
||||
@ -112,13 +128,43 @@ public class TestEncryptContent {
|
||||
testRunner.run();
|
||||
testRunner.assertAllFlowFilesTransferred(EncryptContent.REL_SUCCESS, 1);
|
||||
|
||||
logger.info("Successfully decrypted {}", encryptionMethod.name());
|
||||
flowFile = testRunner.getFlowFilesForRelationship(EncryptContent.REL_SUCCESS).get(0);
|
||||
flowFile.assertContentEquals(new File("src/test/resources/hello.txt"));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testKeyedCiphersRoundTrip() throws IOException {
|
||||
final TestRunner testRunner = TestRunners.newTestRunner(new EncryptContent());
|
||||
final String RAW_KEY_HEX = StringUtils.repeat("ab", 16);
|
||||
testRunner.setProperty(EncryptContent.RAW_KEY_HEX, RAW_KEY_HEX);
|
||||
testRunner.setProperty(EncryptContent.KEY_DERIVATION_FUNCTION, KeyDerivationFunction.NONE.name());
|
||||
|
||||
for (final EncryptionMethod encryptionMethod : SUPPORTED_KEYED_ENCRYPTION_METHODS) {
|
||||
testRunner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, encryptionMethod.name());
|
||||
testRunner.setProperty(EncryptContent.MODE, EncryptContent.ENCRYPT_MODE);
|
||||
|
||||
testRunner.enqueue(Paths.get("src/test/resources/hello.txt"));
|
||||
testRunner.clearTransferState();
|
||||
testRunner.run();
|
||||
|
||||
testRunner.assertAllFlowFilesTransferred(EncryptContent.REL_SUCCESS, 1);
|
||||
|
||||
MockFlowFile flowFile = testRunner.getFlowFilesForRelationship(EncryptContent.REL_SUCCESS).get(0);
|
||||
testRunner.assertQueueEmpty();
|
||||
|
||||
testRunner.setProperty(EncryptContent.MODE, EncryptContent.DECRYPT_MODE);
|
||||
testRunner.enqueue(flowFile);
|
||||
testRunner.clearTransferState();
|
||||
testRunner.run();
|
||||
testRunner.assertAllFlowFilesTransferred(EncryptContent.REL_SUCCESS, 1);
|
||||
|
||||
flowFile = testRunner.getFlowFilesForRelationship(EncryptContent.REL_SUCCESS).get(0);
|
||||
flowFile.assertContentEquals(new File("src/test/resources/hello.txt"));
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testPGPCiphersRoundTrip() {
|
||||
final TestRunner testRunner = TestRunners.newTestRunner(new EncryptContent());
|
||||
@ -197,27 +243,22 @@ public class TestEncryptContent {
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testShouldDetermineMaxKeySizeForAlgorithms() throws IOException {
|
||||
// Arrange
|
||||
public void testShouldDetermineMaxKeySizeForAlgorithms() {
|
||||
final String AES_ALGORITHM = EncryptionMethod.MD5_256AES.getAlgorithm();
|
||||
final String DES_ALGORITHM = EncryptionMethod.MD5_DES.getAlgorithm();
|
||||
|
||||
final int AES_MAX_LENGTH = Integer.MAX_VALUE;
|
||||
final int DES_MAX_LENGTH = Integer.MAX_VALUE;
|
||||
|
||||
// Act
|
||||
int determinedAESMaxLength = PasswordBasedEncryptor.getMaxAllowedKeyLength(AES_ALGORITHM);
|
||||
int determinedTDESMaxLength = PasswordBasedEncryptor.getMaxAllowedKeyLength(DES_ALGORITHM);
|
||||
|
||||
// Assert
|
||||
assert determinedAESMaxLength == AES_MAX_LENGTH;
|
||||
assert determinedTDESMaxLength == DES_MAX_LENGTH;
|
||||
assertEquals(AES_MAX_LENGTH, determinedAESMaxLength);
|
||||
assertEquals(DES_MAX_LENGTH, determinedTDESMaxLength);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testShouldDecryptOpenSSLRawSalted() throws IOException {
|
||||
// Arrange
|
||||
|
||||
final TestRunner testRunner = TestRunners.newTestRunner(new EncryptContent());
|
||||
|
||||
final String password = "thisIsABadPassword";
|
||||
@ -229,27 +270,20 @@ public class TestEncryptContent {
|
||||
testRunner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, method.name());
|
||||
testRunner.setProperty(EncryptContent.MODE, EncryptContent.DECRYPT_MODE);
|
||||
|
||||
// Act
|
||||
testRunner.enqueue(Paths.get("src/test/resources/TestEncryptContent/salted_raw.enc"));
|
||||
testRunner.clearTransferState();
|
||||
testRunner.run();
|
||||
|
||||
// Assert
|
||||
testRunner.assertAllFlowFilesTransferred(EncryptContent.REL_SUCCESS, 1);
|
||||
testRunner.assertQueueEmpty();
|
||||
|
||||
MockFlowFile flowFile = testRunner.getFlowFilesForRelationship(EncryptContent.REL_SUCCESS).get(0);
|
||||
logger.info("Decrypted contents (hex): {}", Hex.encodeHexString(flowFile.toByteArray()));
|
||||
logger.info("Decrypted contents: {}", new String(flowFile.toByteArray(), StandardCharsets.UTF_8));
|
||||
|
||||
// Assert
|
||||
flowFile.assertContentEquals(new File("src/test/resources/TestEncryptContent/plain.txt"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testShouldDecryptOpenSSLRawUnsalted() throws IOException {
|
||||
// Arrange
|
||||
|
||||
final TestRunner testRunner = TestRunners.newTestRunner(new EncryptContent());
|
||||
|
||||
final String password = "thisIsABadPassword";
|
||||
@ -261,33 +295,18 @@ public class TestEncryptContent {
|
||||
testRunner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, method.name());
|
||||
testRunner.setProperty(EncryptContent.MODE, EncryptContent.DECRYPT_MODE);
|
||||
|
||||
// Act
|
||||
testRunner.enqueue(Paths.get("src/test/resources/TestEncryptContent/unsalted_raw.enc"));
|
||||
testRunner.clearTransferState();
|
||||
testRunner.run();
|
||||
|
||||
// Assert
|
||||
testRunner.assertAllFlowFilesTransferred(EncryptContent.REL_SUCCESS, 1);
|
||||
testRunner.assertQueueEmpty();
|
||||
|
||||
MockFlowFile flowFile = testRunner.getFlowFilesForRelationship(EncryptContent.REL_SUCCESS).get(0);
|
||||
logger.info("Decrypted contents (hex): {}", Hex.encodeHexString(flowFile.toByteArray()));
|
||||
logger.info("Decrypted contents: {}", new String(flowFile.toByteArray(), StandardCharsets.UTF_8));
|
||||
|
||||
// Assert
|
||||
flowFile.assertContentEquals(new File("src/test/resources/TestEncryptContent/plain.txt"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDecryptShouldDefaultToNone() throws IOException {
|
||||
// Arrange
|
||||
final TestRunner testRunner = TestRunners.newTestRunner(new EncryptContent());
|
||||
|
||||
// Assert
|
||||
assertEquals(testRunner.getProcessor().getPropertyDescriptor(EncryptContent.KEY_DERIVATION_FUNCTION
|
||||
.getName()).getDefaultValue(), KeyDerivationFunction.NONE.name(), "Decrypt should default to None");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDecryptSmallerThanSaltSize() {
|
||||
final TestRunner runner = TestRunners.newTestRunner(EncryptContent.class);
|
||||
@ -447,10 +466,6 @@ public class TestEncryptContent {
|
||||
pc = (MockProcessContext) runner.getProcessContext();
|
||||
results = pc.validate();
|
||||
|
||||
for (ValidationResult vr : results) {
|
||||
logger.info(vr.toString());
|
||||
}
|
||||
|
||||
// The default validation error is:
|
||||
// Raw key hex cannot be empty
|
||||
final String RAW_KEY_ERROR = "'raw-key-hex' is invalid because Raw Key (hexadecimal) is " +
|
||||
@ -552,4 +567,588 @@ public class TestEncryptContent {
|
||||
runner.removeProperty(EncryptContent.PGP_SYMMETRIC_ENCRYPTION_CIPHER);
|
||||
runner.assertValid();
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldValidateMaxKeySizeForAlgorithmsOnUnlimitedStrengthJVM() {
|
||||
final TestRunner runner = TestRunners.newTestRunner(EncryptContent.class);
|
||||
Collection<ValidationResult> results;
|
||||
MockProcessContext pc;
|
||||
|
||||
EncryptionMethod encryptionMethod = EncryptionMethod.AES_CBC;
|
||||
|
||||
// Integer.MAX_VALUE or 128, so use 256 or 128
|
||||
final int MAX_KEY_LENGTH = Math.min(PasswordBasedEncryptor.getMaxAllowedKeyLength(encryptionMethod.getAlgorithm()), 256);
|
||||
final String TOO_LONG_KEY_HEX = StringUtils.repeat("ab", (MAX_KEY_LENGTH / 8 + 1));
|
||||
|
||||
runner.setProperty(EncryptContent.MODE, EncryptContent.ENCRYPT_MODE);
|
||||
runner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, encryptionMethod.name());
|
||||
runner.setProperty(EncryptContent.KEY_DERIVATION_FUNCTION, KeyDerivationFunction.NONE.name());
|
||||
runner.setProperty(EncryptContent.RAW_KEY_HEX, TOO_LONG_KEY_HEX);
|
||||
|
||||
runner.enqueue(new byte[0]);
|
||||
pc = (MockProcessContext) runner.getProcessContext();
|
||||
|
||||
results = pc.validate();
|
||||
|
||||
assertEquals(1, results.size());
|
||||
ValidationResult vr = results.iterator().next();
|
||||
|
||||
String expectedResult = "'raw-key-hex' is invalid because Key must be valid length [128, 192, 256]";
|
||||
String message = "'" + vr.toString() + "' contains '" + expectedResult + "'";
|
||||
assertTrue(vr.toString().contains(expectedResult), message);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldValidateKeyFormatAndSizeForAlgorithms() {
|
||||
final TestRunner runner = TestRunners.newTestRunner(EncryptContent.class);
|
||||
Collection<ValidationResult> results;
|
||||
MockProcessContext pc;
|
||||
|
||||
EncryptionMethod encryptionMethod = EncryptionMethod.AES_CBC;
|
||||
|
||||
final int INVALID_KEY_LENGTH = 120;
|
||||
final String INVALID_KEY_HEX = StringUtils.repeat("ab", (INVALID_KEY_LENGTH / 8));
|
||||
|
||||
runner.setProperty(EncryptContent.MODE, EncryptContent.ENCRYPT_MODE);
|
||||
runner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, encryptionMethod.name());
|
||||
runner.setProperty(EncryptContent.KEY_DERIVATION_FUNCTION, KeyDerivationFunction.NONE.name());
|
||||
runner.setProperty(EncryptContent.RAW_KEY_HEX, INVALID_KEY_HEX);
|
||||
|
||||
runner.enqueue(new byte[0]);
|
||||
pc = (MockProcessContext) runner.getProcessContext();
|
||||
|
||||
results = pc.validate();
|
||||
|
||||
assertEquals(1, results.size());
|
||||
ValidationResult keyLengthInvalidVR = results.iterator().next();
|
||||
|
||||
String expectedResult = "'raw-key-hex' is invalid because Key must be valid length [128, 192, 256]";
|
||||
String message = "'" + keyLengthInvalidVR.toString() + "' contains '" + expectedResult + "'";
|
||||
assertTrue(keyLengthInvalidVR.toString().contains(expectedResult), message);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldValidateKDFWhenKeyedCipherSelected() {
|
||||
final TestRunner runner = TestRunners.newTestRunner(EncryptContent.class);
|
||||
Collection<ValidationResult> results;
|
||||
MockProcessContext pc;
|
||||
|
||||
final int VALID_KEY_LENGTH = 128;
|
||||
final String VALID_KEY_HEX = StringUtils.repeat("ab", (VALID_KEY_LENGTH / 8));
|
||||
|
||||
runner.setProperty(EncryptContent.MODE, EncryptContent.ENCRYPT_MODE);
|
||||
|
||||
for (final EncryptionMethod encryptionMethod : SUPPORTED_KEYED_ENCRYPTION_METHODS) {
|
||||
runner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, encryptionMethod.name());
|
||||
|
||||
// Scenario 1: Legacy KDF + keyed cipher -> validation error
|
||||
final List<KeyDerivationFunction> invalidKDFs = Arrays.asList(KeyDerivationFunction.NIFI_LEGACY, KeyDerivationFunction.OPENSSL_EVP_BYTES_TO_KEY);
|
||||
for (final KeyDerivationFunction invalidKDF : invalidKDFs) {
|
||||
runner.setProperty(EncryptContent.KEY_DERIVATION_FUNCTION, invalidKDF.name());
|
||||
runner.setProperty(EncryptContent.RAW_KEY_HEX, VALID_KEY_HEX);
|
||||
runner.removeProperty(EncryptContent.PASSWORD);
|
||||
|
||||
runner.enqueue(new byte[0]);
|
||||
pc = (MockProcessContext) runner.getProcessContext();
|
||||
|
||||
results = pc.validate();
|
||||
|
||||
assertEquals(1, results.size());
|
||||
ValidationResult keyLengthInvalidVR = results.iterator().next();
|
||||
|
||||
String expectedResult = String.format("'key-derivation-function' is invalid because Key Derivation Function is required to be BCRYPT, SCRYPT, PBKDF2, ARGON2, NONE when using " +
|
||||
"algorithm %s", encryptionMethod.getAlgorithm());
|
||||
String message = "'" + keyLengthInvalidVR.toString() + "' contains '" + expectedResult + "'";
|
||||
assertTrue(keyLengthInvalidVR.toString().contains(expectedResult), message);
|
||||
}
|
||||
|
||||
// Scenario 2: No KDF + keyed cipher + raw-key-hex -> valid
|
||||
|
||||
runner.setProperty(EncryptContent.KEY_DERIVATION_FUNCTION, KeyDerivationFunction.NONE.name());
|
||||
runner.setProperty(EncryptContent.RAW_KEY_HEX, VALID_KEY_HEX);
|
||||
runner.removeProperty(EncryptContent.PASSWORD);
|
||||
|
||||
runner.enqueue(new byte[0]);
|
||||
pc = (MockProcessContext) runner.getProcessContext();
|
||||
|
||||
results = pc.validate();
|
||||
|
||||
assertTrue(results.isEmpty());
|
||||
|
||||
// Scenario 3: Strong KDF + keyed cipher + password -> valid
|
||||
final List<KeyDerivationFunction> validKDFs = Arrays.asList(KeyDerivationFunction.BCRYPT,
|
||||
KeyDerivationFunction.SCRYPT,
|
||||
KeyDerivationFunction.PBKDF2,
|
||||
KeyDerivationFunction.ARGON2);
|
||||
for (final KeyDerivationFunction validKDF : validKDFs) {
|
||||
runner.setProperty(EncryptContent.KEY_DERIVATION_FUNCTION, validKDF.name());
|
||||
runner.setProperty(EncryptContent.PASSWORD, "thisIsABadPassword");
|
||||
runner.removeProperty(EncryptContent.RAW_KEY_HEX);
|
||||
|
||||
runner.enqueue(new byte[0]);
|
||||
pc = (MockProcessContext) runner.getProcessContext();
|
||||
|
||||
results = pc.validate();
|
||||
|
||||
assertTrue(results.isEmpty());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldValidateKeyMaterialSourceWhenKeyedCipherSelected() {
|
||||
final TestRunner runner = TestRunners.newTestRunner(EncryptContent.class);
|
||||
Collection<ValidationResult> results;
|
||||
MockProcessContext pc;
|
||||
|
||||
final int VALID_KEY_LENGTH = 128;
|
||||
final String VALID_KEY_HEX = StringUtils.repeat("ab", (VALID_KEY_LENGTH / 8));
|
||||
|
||||
final String VALID_PASSWORD = "thisIsABadPassword";
|
||||
|
||||
runner.setProperty(EncryptContent.MODE, EncryptContent.ENCRYPT_MODE);
|
||||
KeyDerivationFunction none = KeyDerivationFunction.NONE;
|
||||
|
||||
// Scenario 1 - RKH w/ KDF NONE & em in [CBC, CTR, GCM] (no password)
|
||||
for (final EncryptionMethod kem : SUPPORTED_KEYED_ENCRYPTION_METHODS) {
|
||||
runner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, kem.name());
|
||||
runner.setProperty(EncryptContent.KEY_DERIVATION_FUNCTION, none.name());
|
||||
|
||||
runner.setProperty(EncryptContent.RAW_KEY_HEX, VALID_KEY_HEX);
|
||||
runner.removeProperty(EncryptContent.PASSWORD);
|
||||
|
||||
runner.enqueue(new byte[0]);
|
||||
pc = (MockProcessContext) runner.getProcessContext();
|
||||
|
||||
results = pc.validate();
|
||||
|
||||
assertTrue(results.isEmpty());
|
||||
|
||||
// Scenario 2 - PW w/ KDF in [BCRYPT, SCRYPT, PBKDF2, ARGON2] & em in [CBC, CTR, GCM] (no RKH)
|
||||
final List<KeyDerivationFunction> validKDFs = Arrays
|
||||
.stream(KeyDerivationFunction.values())
|
||||
.filter(it -> it.isStrongKDF())
|
||||
.collect(Collectors.toList());
|
||||
for (final KeyDerivationFunction kdf : validKDFs) {
|
||||
runner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, kem.name());
|
||||
runner.setProperty(EncryptContent.KEY_DERIVATION_FUNCTION, kdf.name());
|
||||
|
||||
runner.removeProperty(EncryptContent.RAW_KEY_HEX);
|
||||
runner.setProperty(EncryptContent.PASSWORD, VALID_PASSWORD);
|
||||
|
||||
runner.enqueue(new byte[0]);
|
||||
pc = (MockProcessContext) runner.getProcessContext();
|
||||
|
||||
results = pc.validate();
|
||||
|
||||
assertTrue(results.isEmpty());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldValidateKDFWhenPBECipherSelected() {
|
||||
final TestRunner runner = TestRunners.newTestRunner(EncryptContent.class);
|
||||
Collection<ValidationResult> results;
|
||||
MockProcessContext pc;
|
||||
final String PASSWORD = "short";
|
||||
|
||||
final List<EncryptionMethod> encryptionMethods = Arrays
|
||||
.stream(EncryptionMethod.values())
|
||||
.filter(it -> it.getAlgorithm().startsWith("PBE"))
|
||||
.collect(Collectors.toList());
|
||||
|
||||
runner.setProperty(EncryptContent.MODE, EncryptContent.ENCRYPT_MODE);
|
||||
runner.setProperty(EncryptContent.PASSWORD, PASSWORD);
|
||||
runner.setProperty(EncryptContent.ALLOW_WEAK_CRYPTO, "allowed");
|
||||
|
||||
for (final EncryptionMethod encryptionMethod : encryptionMethods) {
|
||||
runner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, encryptionMethod.name());
|
||||
|
||||
final List<KeyDerivationFunction> invalidKDFs = Arrays.asList(
|
||||
KeyDerivationFunction.NONE,
|
||||
KeyDerivationFunction.BCRYPT,
|
||||
KeyDerivationFunction.SCRYPT,
|
||||
KeyDerivationFunction.PBKDF2,
|
||||
KeyDerivationFunction.ARGON2
|
||||
);
|
||||
for (final KeyDerivationFunction invalidKDF : invalidKDFs) {
|
||||
runner.setProperty(EncryptContent.KEY_DERIVATION_FUNCTION, invalidKDF.name());
|
||||
|
||||
runner.enqueue(new byte[0]);
|
||||
pc = (MockProcessContext) runner.getProcessContext();
|
||||
|
||||
results = pc.validate();
|
||||
|
||||
assertEquals(1, results.size());
|
||||
ValidationResult keyLengthInvalidVR = results.iterator().next();
|
||||
|
||||
String expectedResult = String.format("'Key Derivation Function' is invalid because Key Derivation Function is required to be NIFI_LEGACY, OPENSSL_EVP_BYTES_TO_KEY when using " +
|
||||
"algorithm %s", encryptionMethod.getAlgorithm());
|
||||
String message = "'" + keyLengthInvalidVR.toString() + "' contains '" + expectedResult + "'";
|
||||
assertTrue(keyLengthInvalidVR.toString().contains(expectedResult), message);
|
||||
}
|
||||
|
||||
final List<KeyDerivationFunction> validKDFs = Arrays.asList(KeyDerivationFunction.NIFI_LEGACY, KeyDerivationFunction.OPENSSL_EVP_BYTES_TO_KEY);
|
||||
for (final KeyDerivationFunction validKDF : validKDFs) {
|
||||
runner.setProperty(EncryptContent.KEY_DERIVATION_FUNCTION, validKDF.name());
|
||||
|
||||
runner.enqueue(new byte[0]);
|
||||
pc = (MockProcessContext) runner.getProcessContext();
|
||||
|
||||
results = pc.validate();
|
||||
|
||||
assertEquals(0, results.size());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testDecryptAesCbcNoPadding() throws DecoderException, IOException {
|
||||
final TestRunner testRunner = TestRunners.newTestRunner(new EncryptContent());
|
||||
final String RAW_KEY_HEX = StringUtils.repeat("ab", 16);
|
||||
testRunner.setProperty(EncryptContent.RAW_KEY_HEX, RAW_KEY_HEX);
|
||||
testRunner.setProperty(EncryptContent.KEY_DERIVATION_FUNCTION, KeyDerivationFunction.NONE.name());
|
||||
testRunner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, EncryptionMethod.AES_CBC_NO_PADDING.name());
|
||||
testRunner.setProperty(EncryptContent.MODE, EncryptContent.DECRYPT_MODE);
|
||||
|
||||
final String content = "ExactBlockSizeRequiredForProcess";
|
||||
final byte[] bytes = content.getBytes(StandardCharsets.UTF_8);
|
||||
final ByteArrayInputStream inputStream = new ByteArrayInputStream(bytes);
|
||||
final ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
|
||||
|
||||
final KeyedEncryptor encryptor = new KeyedEncryptor(EncryptionMethod.AES_CBC_NO_PADDING, Hex.decodeHex(RAW_KEY_HEX));
|
||||
encryptor.getEncryptionCallback().process(inputStream, outputStream);
|
||||
outputStream.close();
|
||||
|
||||
final byte[] encrypted = outputStream.toByteArray();
|
||||
testRunner.enqueue(encrypted);
|
||||
testRunner.run();
|
||||
|
||||
testRunner.assertAllFlowFilesTransferred(EncryptContent.REL_SUCCESS, 1);
|
||||
MockFlowFile flowFile = testRunner.getFlowFilesForRelationship(EncryptContent.REL_SUCCESS).get(0);
|
||||
flowFile.assertContentEquals(content);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testArgon2EncryptionShouldWriteAttributesWithEncryptionMetadata() throws ParseException {
|
||||
final TestRunner testRunner = TestRunners.newTestRunner(new EncryptContent());
|
||||
KeyDerivationFunction kdf = KeyDerivationFunction.ARGON2;
|
||||
EncryptionMethod encryptionMethod = EncryptionMethod.AES_CBC;
|
||||
|
||||
testRunner.setProperty(EncryptContent.PASSWORD, "thisIsABadPassword");
|
||||
testRunner.setProperty(EncryptContent.KEY_DERIVATION_FUNCTION, kdf.name());
|
||||
testRunner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, encryptionMethod.name());
|
||||
testRunner.setProperty(EncryptContent.MODE, EncryptContent.ENCRYPT_MODE);
|
||||
|
||||
String PLAINTEXT = "This is a plaintext message. ";
|
||||
|
||||
testRunner.enqueue(PLAINTEXT);
|
||||
testRunner.clearTransferState();
|
||||
testRunner.run();
|
||||
|
||||
testRunner.assertAllFlowFilesTransferred(EncryptContent.REL_SUCCESS, 1);
|
||||
|
||||
MockFlowFile flowFile = testRunner.getFlowFilesForRelationship(EncryptContent.REL_SUCCESS).get(0);
|
||||
testRunner.assertQueueEmpty();
|
||||
|
||||
byte[] flowfileContentBytes = flowFile.getData();
|
||||
|
||||
int ivDelimiterStart = CipherUtility.findSequence(flowfileContentBytes, RandomIVPBECipherProvider.IV_DELIMITER);
|
||||
|
||||
final byte[] EXPECTED_KDF_SALT_BYTES = extractFullSaltFromCipherBytes(flowfileContentBytes);
|
||||
final String EXPECTED_KDF_SALT = new String(EXPECTED_KDF_SALT_BYTES);
|
||||
final String EXPECTED_SALT_HEX = extractRawSaltHexFromFullSalt(EXPECTED_KDF_SALT_BYTES, kdf);
|
||||
|
||||
final String EXPECTED_IV_HEX = Hex.encodeHexString(Arrays.copyOfRange(flowfileContentBytes, ivDelimiterStart - 16, ivDelimiterStart));
|
||||
|
||||
// Assert the timestamp attribute was written and is accurate
|
||||
final TimeDuration diff = calculateTimestampDifference(new Date(), flowFile.getAttribute("encryptcontent.timestamp"));
|
||||
assertTrue(diff.toMilliseconds() < 1_000);
|
||||
assertEquals(encryptionMethod.name(), flowFile.getAttribute("encryptcontent.algorithm"));
|
||||
assertEquals(kdf.name(), flowFile.getAttribute("encryptcontent.kdf"));
|
||||
assertEquals("encrypted", flowFile.getAttribute("encryptcontent.action"));
|
||||
assertEquals(EXPECTED_SALT_HEX, flowFile.getAttribute("encryptcontent.salt"));
|
||||
assertEquals("16", flowFile.getAttribute("encryptcontent.salt_length"));
|
||||
assertEquals(EXPECTED_KDF_SALT, flowFile.getAttribute("encryptcontent.kdf_salt"));
|
||||
final int kdfSaltLength = Integer.valueOf(flowFile.getAttribute("encryptcontent.kdf_salt_length"));
|
||||
assertTrue(kdfSaltLength >= 29 && kdfSaltLength <= 54);
|
||||
assertEquals(EXPECTED_IV_HEX, flowFile.getAttribute("encryptcontent.iv"));
|
||||
assertEquals("16", flowFile.getAttribute("encryptcontent.iv_length"));
|
||||
assertEquals(String.valueOf(PLAINTEXT.length()), flowFile.getAttribute("encryptcontent.plaintext_length"));
|
||||
assertEquals(String.valueOf(flowfileContentBytes.length), flowFile.getAttribute("encryptcontent.cipher_text_length"));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testKeyedEncryptionShouldWriteAttributesWithEncryptionMetadata() throws ParseException {
|
||||
// Arrange
|
||||
final TestRunner testRunner = TestRunners.newTestRunner(new EncryptContent());
|
||||
KeyDerivationFunction kdf = KeyDerivationFunction.NONE;
|
||||
EncryptionMethod encryptionMethod = EncryptionMethod.AES_CBC;
|
||||
|
||||
testRunner.setProperty(EncryptContent.RAW_KEY_HEX, "0123456789ABCDEFFEDCBA9876543210");
|
||||
testRunner.setProperty(EncryptContent.KEY_DERIVATION_FUNCTION, kdf.name());
|
||||
testRunner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, encryptionMethod.name());
|
||||
testRunner.setProperty(EncryptContent.MODE, EncryptContent.ENCRYPT_MODE);
|
||||
|
||||
String PLAINTEXT = "This is a plaintext message. ";
|
||||
|
||||
testRunner.enqueue(PLAINTEXT);
|
||||
testRunner.clearTransferState();
|
||||
testRunner.run();
|
||||
|
||||
testRunner.assertAllFlowFilesTransferred(EncryptContent.REL_SUCCESS, 1);
|
||||
|
||||
MockFlowFile flowFile = testRunner.getFlowFilesForRelationship(EncryptContent.REL_SUCCESS).get(0);
|
||||
testRunner.assertQueueEmpty();
|
||||
|
||||
byte[] flowfileContentBytes = flowFile.getData();
|
||||
|
||||
int ivDelimiterStart = CipherUtility.findSequence(flowfileContentBytes, RandomIVPBECipherProvider.IV_DELIMITER);
|
||||
assertEquals(16, ivDelimiterStart);
|
||||
|
||||
final TimeDuration diff = calculateTimestampDifference(new Date(), flowFile.getAttribute("encryptcontent.timestamp"));
|
||||
|
||||
// Assert the timestamp attribute was written and is accurate
|
||||
assertTrue(diff.toMilliseconds() < 1_000);
|
||||
|
||||
final String EXPECTED_IV_HEX = Hex.encodeHexString(Arrays.copyOfRange(flowfileContentBytes, 0, ivDelimiterStart));
|
||||
final int EXPECTED_CIPHER_TEXT_LENGTH = CipherUtility.calculateCipherTextLength(PLAINTEXT.length(), 0);
|
||||
assertEquals(encryptionMethod.name(), flowFile.getAttribute("encryptcontent.algorithm"));
|
||||
assertEquals(kdf.name(), flowFile.getAttribute("encryptcontent.kdf"));
|
||||
assertEquals("encrypted", flowFile.getAttribute("encryptcontent.action"));
|
||||
assertEquals(EXPECTED_IV_HEX, flowFile.getAttribute("encryptcontent.iv"));
|
||||
assertEquals("16", flowFile.getAttribute("encryptcontent.iv_length"));
|
||||
assertEquals(String.valueOf(PLAINTEXT.length()), flowFile.getAttribute("encryptcontent.plaintext_length"));
|
||||
assertEquals(String.valueOf(EXPECTED_CIPHER_TEXT_LENGTH), flowFile.getAttribute("encryptcontent.cipher_text_length"));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testDifferentCompatibleConfigurations() throws Exception {
|
||||
final TestRunner testRunner = TestRunners.newTestRunner(new EncryptContent());
|
||||
KeyDerivationFunction argon2 = KeyDerivationFunction.ARGON2;
|
||||
EncryptionMethod aesCbcEM = EncryptionMethod.AES_CBC;
|
||||
int keyLength = CipherUtility.parseKeyLengthFromAlgorithm(aesCbcEM.getAlgorithm());
|
||||
|
||||
final String PASSWORD = "thisIsABadPassword";
|
||||
testRunner.setProperty(EncryptContent.PASSWORD, PASSWORD);
|
||||
testRunner.setProperty(EncryptContent.KEY_DERIVATION_FUNCTION, argon2.name());
|
||||
testRunner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, aesCbcEM.name());
|
||||
testRunner.setProperty(EncryptContent.MODE, EncryptContent.ENCRYPT_MODE);
|
||||
|
||||
String PLAINTEXT = "This is a plaintext message. ";
|
||||
|
||||
testRunner.enqueue(PLAINTEXT);
|
||||
testRunner.clearTransferState();
|
||||
testRunner.run();
|
||||
|
||||
MockFlowFile encryptedFlowFile = testRunner.getFlowFilesForRelationship(EncryptContent.REL_SUCCESS).get(0);
|
||||
byte[] fullCipherBytes = encryptedFlowFile.getData();
|
||||
|
||||
// Extract the KDF salt from the encryption metadata in the flowfile attribute
|
||||
String argon2Salt = encryptedFlowFile.getAttribute("encryptcontent.kdf_salt");
|
||||
Argon2SecureHasher a2sh = new Argon2SecureHasher(keyLength / 8);
|
||||
byte[] fullSaltBytes = argon2Salt.getBytes(StandardCharsets.UTF_8);
|
||||
byte[] rawSaltBytes = Hex.decodeHex(encryptedFlowFile.getAttribute("encryptcontent.salt"));
|
||||
byte[] keyBytes = a2sh.hashRaw(PASSWORD.getBytes(StandardCharsets.UTF_8), rawSaltBytes);
|
||||
String keyHex = Hex.encodeHexString(keyBytes);
|
||||
|
||||
byte[] ivBytes = Hex.decodeHex(encryptedFlowFile.getAttribute("encryptcontent.iv"));
|
||||
|
||||
// Sanity check the encryption
|
||||
Argon2CipherProvider a2cp = new Argon2CipherProvider();
|
||||
Cipher sanityCipher = a2cp.getCipher(aesCbcEM, PASSWORD, fullSaltBytes, ivBytes, CipherUtility.parseKeyLengthFromAlgorithm(aesCbcEM.getAlgorithm()), false);
|
||||
byte[] cipherTextBytes = Arrays.copyOfRange(fullCipherBytes, fullCipherBytes.length - 32, fullCipherBytes.length);
|
||||
byte[] recoveredBytes = sanityCipher.doFinal(cipherTextBytes);
|
||||
|
||||
// Configure decrypting processor with raw key
|
||||
KeyDerivationFunction kdf = KeyDerivationFunction.NONE;
|
||||
EncryptionMethod encryptionMethod = EncryptionMethod.AES_CBC;
|
||||
|
||||
testRunner.setProperty(EncryptContent.RAW_KEY_HEX, keyHex);
|
||||
testRunner.setProperty(EncryptContent.KEY_DERIVATION_FUNCTION, kdf.name());
|
||||
testRunner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, encryptionMethod.name());
|
||||
testRunner.setProperty(EncryptContent.MODE, EncryptContent.DECRYPT_MODE);
|
||||
testRunner.removeProperty(EncryptContent.PASSWORD);
|
||||
|
||||
testRunner.enqueue(fullCipherBytes);
|
||||
testRunner.clearTransferState();
|
||||
testRunner.run();
|
||||
|
||||
testRunner.assertAllFlowFilesTransferred(EncryptContent.REL_SUCCESS, 1);
|
||||
|
||||
MockFlowFile decryptedFlowFile = testRunner.getFlowFilesForRelationship(EncryptContent.REL_SUCCESS).get(0);
|
||||
testRunner.assertQueueEmpty();
|
||||
|
||||
byte[] flowfileContentBytes = decryptedFlowFile.getData();
|
||||
|
||||
assertArrayEquals(recoveredBytes, flowfileContentBytes);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldCheckLengthOfPasswordWhenNotAllowed() {
|
||||
final TestRunner testRunner = TestRunners.newTestRunner(new EncryptContent());
|
||||
testRunner.setProperty(EncryptContent.KEY_DERIVATION_FUNCTION, KeyDerivationFunction.NIFI_LEGACY.name());
|
||||
|
||||
Collection<ValidationResult> results;
|
||||
MockProcessContext pc;
|
||||
|
||||
final List<EncryptionMethod> encryptionMethods = Arrays
|
||||
.stream(EncryptionMethod.values())
|
||||
.filter(it -> it.getAlgorithm().startsWith("PBE"))
|
||||
.collect(Collectors.toList());
|
||||
|
||||
testRunner.setProperty(EncryptContent.ALLOW_WEAK_CRYPTO, "not-allowed");
|
||||
|
||||
// Use .find instead of .each to allow "breaks" using return false
|
||||
for (final EncryptionMethod encryptionMethod : encryptionMethods) {
|
||||
// Determine the minimum of the algorithm-accepted length or the global safe minimum to ensure only one validation result
|
||||
final int shortPasswordLength = Math.min(PasswordBasedEncryptor.getMinimumSafePasswordLength() - 1,
|
||||
CipherUtility.getMaximumPasswordLengthForAlgorithmOnLimitedStrengthCrypto(encryptionMethod) - 1);
|
||||
String shortPassword = StringUtils.repeat("x", shortPasswordLength);
|
||||
if (encryptionMethod.isUnlimitedStrength() || encryptionMethod.isKeyedCipher()) {
|
||||
continue;
|
||||
// cannot test unlimited strength in unit tests because it's not enabled by the JVM by default.
|
||||
}
|
||||
|
||||
testRunner.setProperty(EncryptContent.PASSWORD, shortPassword);
|
||||
testRunner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, encryptionMethod.name());
|
||||
testRunner.setProperty(EncryptContent.MODE, EncryptContent.ENCRYPT_MODE);
|
||||
|
||||
testRunner.clearTransferState();
|
||||
testRunner.enqueue(new byte[0]);
|
||||
pc = (MockProcessContext) testRunner.getProcessContext();
|
||||
|
||||
results = pc.validate();
|
||||
|
||||
assertEquals(1, results.size());
|
||||
ValidationResult passwordLengthVR = results.iterator().next();
|
||||
|
||||
String expectedResult = String.format("'Password' is invalid because Password length less than %s characters is potentially unsafe. " +
|
||||
"See Admin Guide.", PasswordBasedEncryptor.getMinimumSafePasswordLength());
|
||||
String message = "'" + passwordLengthVR.toString() + "' contains '" + expectedResult + "'";
|
||||
assertTrue(passwordLengthVR.toString().contains(expectedResult), message);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldNotCheckLengthOfPasswordWhenAllowed() {
|
||||
final TestRunner testRunner = TestRunners.newTestRunner(new EncryptContent());
|
||||
testRunner.setProperty(EncryptContent.KEY_DERIVATION_FUNCTION, KeyDerivationFunction.NIFI_LEGACY.name());
|
||||
|
||||
Collection<ValidationResult> results;
|
||||
MockProcessContext pc;
|
||||
|
||||
final List<EncryptionMethod> encryptionMethods = Arrays
|
||||
.stream(EncryptionMethod.values())
|
||||
.filter(it -> it.getAlgorithm().startsWith("PBE"))
|
||||
.collect(Collectors.toList());
|
||||
|
||||
testRunner.setProperty(EncryptContent.ALLOW_WEAK_CRYPTO, "allowed");
|
||||
|
||||
for (final EncryptionMethod encryptionMethod : encryptionMethods) {
|
||||
// Determine the minimum of the algorithm-accepted length or the global safe minimum to ensure only one validation result
|
||||
final int shortPasswordLength = Math.min(PasswordBasedEncryptor.getMinimumSafePasswordLength() - 1,
|
||||
CipherUtility.getMaximumPasswordLengthForAlgorithmOnLimitedStrengthCrypto(encryptionMethod) - 1);
|
||||
String shortPassword = StringUtils.repeat("x", shortPasswordLength);
|
||||
if (encryptionMethod.isUnlimitedStrength() || encryptionMethod.isKeyedCipher()) {
|
||||
continue;
|
||||
// cannot test unlimited strength in unit tests because it's not enabled by the JVM by default.
|
||||
}
|
||||
|
||||
testRunner.setProperty(EncryptContent.PASSWORD, shortPassword);
|
||||
testRunner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, encryptionMethod.name());
|
||||
testRunner.setProperty(EncryptContent.MODE, EncryptContent.ENCRYPT_MODE);
|
||||
|
||||
testRunner.clearTransferState();
|
||||
testRunner.enqueue(new byte[0]);
|
||||
pc = (MockProcessContext) testRunner.getProcessContext();
|
||||
|
||||
results = pc.validate();
|
||||
|
||||
assertEquals(0, results.size(), results.toString());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testPGPPasswordShouldSupportExpressionLanguage() {
|
||||
final TestRunner testRunner = TestRunners.newTestRunner(new EncryptContent());
|
||||
testRunner.setProperty(EncryptContent.MODE, EncryptContent.DECRYPT_MODE);
|
||||
testRunner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, EncryptionMethod.PGP.name());
|
||||
testRunner.setProperty(EncryptContent.PRIVATE_KEYRING, "src/test/resources/TestEncryptContent/secring.gpg");
|
||||
|
||||
Collection<ValidationResult> results;
|
||||
MockProcessContext pc;
|
||||
|
||||
// Verify this is the correct password
|
||||
final String passphraseWithoutEL = "thisIsABadPassword";
|
||||
testRunner.setProperty(EncryptContent.PRIVATE_KEYRING_PASSPHRASE, passphraseWithoutEL);
|
||||
|
||||
testRunner.clearTransferState();
|
||||
testRunner.enqueue(new byte[0]);
|
||||
pc = (MockProcessContext) testRunner.getProcessContext();
|
||||
|
||||
results = pc.validate();
|
||||
assertEquals(0, results.size(), results.toString());
|
||||
|
||||
final String passphraseWithEL = "${literal('thisIsABadPassword')}";
|
||||
testRunner.setProperty(EncryptContent.PRIVATE_KEYRING_PASSPHRASE, passphraseWithEL);
|
||||
|
||||
testRunner.clearTransferState();
|
||||
testRunner.enqueue(new byte[0]);
|
||||
|
||||
results = pc.validate();
|
||||
|
||||
assertEquals(0, results.size(), results.toString());
|
||||
}
|
||||
|
||||
@Test
|
||||
void testArgon2ShouldIncludeFullSalt() throws IOException {
|
||||
final TestRunner testRunner = TestRunners.newTestRunner(new EncryptContent());
|
||||
testRunner.setProperty(EncryptContent.PASSWORD, "thisIsABadPassword");
|
||||
testRunner.setProperty(EncryptContent.KEY_DERIVATION_FUNCTION, KeyDerivationFunction.ARGON2.name());
|
||||
|
||||
EncryptionMethod encryptionMethod = EncryptionMethod.AES_CBC;
|
||||
|
||||
testRunner.setProperty(EncryptContent.ENCRYPTION_ALGORITHM, encryptionMethod.name());
|
||||
testRunner.setProperty(EncryptContent.MODE, EncryptContent.ENCRYPT_MODE);
|
||||
|
||||
testRunner.enqueue(Paths.get("src/test/resources/hello.txt"));
|
||||
testRunner.clearTransferState();
|
||||
testRunner.run();
|
||||
|
||||
testRunner.assertAllFlowFilesTransferred(EncryptContent.REL_SUCCESS, 1);
|
||||
|
||||
MockFlowFile flowFile = testRunner.getFlowFilesForRelationship(EncryptContent.REL_SUCCESS).get(0);
|
||||
testRunner.assertQueueEmpty();
|
||||
|
||||
final String flowFileContent = flowFile.getContent();
|
||||
|
||||
final String fullSalt = flowFileContent.substring(0, flowFileContent.indexOf(new String(RandomIVPBECipherProvider.SALT_DELIMITER, StandardCharsets.UTF_8)));
|
||||
|
||||
boolean isValidFormattedSalt = Argon2CipherProvider.isArgon2FormattedSalt(fullSalt);
|
||||
assertTrue(isValidFormattedSalt);
|
||||
|
||||
boolean fullSaltIsValidLength = fullSalt.getBytes().length >= 49 && fullSalt.getBytes().length <= 57;
|
||||
assertTrue(fullSaltIsValidLength);
|
||||
}
|
||||
|
||||
private static byte[] extractFullSaltFromCipherBytes(byte[] cipherBytes) {
|
||||
int saltDelimiterStart = CipherUtility.findSequence(cipherBytes, RandomIVPBECipherProvider.SALT_DELIMITER);
|
||||
return Arrays.copyOfRange(cipherBytes, 0, saltDelimiterStart);
|
||||
}
|
||||
|
||||
private static String extractRawSaltHexFromFullSalt(byte[] fullSaltBytes, KeyDerivationFunction kdf) {
|
||||
// Salt will be in Base64 (or Radix64) for strong KDFs
|
||||
byte[] rawSaltBytes = CipherUtility.extractRawSalt(fullSaltBytes, kdf);
|
||||
String rawSaltHex = Hex.encodeHexString(rawSaltBytes);
|
||||
return rawSaltHex;
|
||||
}
|
||||
|
||||
private static TimeDuration calculateTimestampDifference(Date date, String timestamp) throws ParseException {
|
||||
SimpleDateFormat formatter = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS Z");
|
||||
Date parsedTimestamp = formatter.parse(timestamp);
|
||||
|
||||
return TimeCategory.minus(date, parsedTimestamp);
|
||||
}
|
||||
}
|
||||
|
@ -0,0 +1,510 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License") you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.processors.standard;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import org.apache.nifi.util.MockFlowFile;
|
||||
import org.apache.nifi.util.TestRunner;
|
||||
import org.apache.nifi.util.TestRunners;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertInstanceOf;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
public class TestFlattenJson {
|
||||
private static final ObjectMapper mapper = new ObjectMapper();
|
||||
|
||||
private TestRunner testRunner;
|
||||
|
||||
@BeforeEach
|
||||
void setupRunner() {
|
||||
testRunner = TestRunners.newTestRunner(FlattenJson.class);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testFlatten() throws JsonProcessingException {
|
||||
final String json = "{\n" +
|
||||
" \"test\": {\n" +
|
||||
" \"msg\": \"Hello, world\"\n" +
|
||||
" },\n" +
|
||||
" \"first\": {\n" +
|
||||
" \"second\": {\n" +
|
||||
" \"third\": [\n" +
|
||||
" \"one\",\n" +
|
||||
" \"two\",\n" +
|
||||
" \"three\",\n" +
|
||||
" \"four\",\n" +
|
||||
" \"five\"\n" +
|
||||
" ]\n" +
|
||||
" }\n" +
|
||||
" }\n" +
|
||||
"}";
|
||||
final Map parsed = (Map) baseTest(testRunner, json, 2);
|
||||
assertEquals(parsed.get("test.msg"), "Hello, world", "test.msg should exist, but doesn't");
|
||||
assertEquals(parsed.get("first.second.third"),
|
||||
Arrays.asList("one", "two", "three", "four", "five"),
|
||||
"Three level block doesn't exist.");
|
||||
}
|
||||
|
||||
@Test
|
||||
void testFlattenRecordSet() throws JsonProcessingException {
|
||||
final String json = "[\n" +
|
||||
" {\n" +
|
||||
" \"first\": {\n" +
|
||||
" \"second\": \"Hello\"\n" +
|
||||
" }\n" +
|
||||
" },\n" +
|
||||
" {\n" +
|
||||
" \"first\": {\n" +
|
||||
" \"second\": \"World\"\n" +
|
||||
" }\n" +
|
||||
" }\n" +
|
||||
"]";
|
||||
|
||||
final List<String> expected = Arrays.asList("Hello", "World");
|
||||
final List parsed = (List) baseTest(testRunner, json, 2);
|
||||
assertTrue(parsed instanceof List, "Not a list");
|
||||
for (int i = 0; i < parsed.size(); i++) {
|
||||
final Map map = (Map) parsed.get(i);
|
||||
assertEquals(map.get("first.second"), expected.get(i), "Missing values.");
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testDifferentSeparator() throws JsonProcessingException {
|
||||
final String json = "{\n" +
|
||||
" \"first\": {\n" +
|
||||
" \"second\": {\n" +
|
||||
" \"third\": [\n" +
|
||||
" \"one\",\n" +
|
||||
" \"two\",\n" +
|
||||
" \"three\",\n" +
|
||||
" \"four\",\n" +
|
||||
" \"five\"\n" +
|
||||
" ]\n" +
|
||||
" }\n" +
|
||||
" }\n" +
|
||||
"}";
|
||||
testRunner.setProperty(FlattenJson.SEPARATOR, "_");
|
||||
final Map parsed = (Map) baseTest(testRunner, json, 1);
|
||||
|
||||
assertEquals(parsed.get("first_second_third"),
|
||||
Arrays.asList("one", "two", "three", "four", "five"),
|
||||
"Separator not applied.");
|
||||
}
|
||||
|
||||
@Test
|
||||
void testExpressionLanguage() throws JsonProcessingException {
|
||||
final String json = "{\n" +
|
||||
" \"first\": {\n" +
|
||||
" \"second\": {\n" +
|
||||
" \"third\": [\n" +
|
||||
" \"one\",\n" +
|
||||
" \"two\",\n" +
|
||||
" \"three\",\n" +
|
||||
" \"four\",\n" +
|
||||
" \"five\"\n" +
|
||||
" ]\n" +
|
||||
" }\n" +
|
||||
" }\n" +
|
||||
"}";
|
||||
|
||||
testRunner.setValidateExpressionUsage(true);
|
||||
testRunner.setProperty(FlattenJson.SEPARATOR, "${separator.char}");
|
||||
final Map parsed = (Map) baseTest(testRunner, json, Collections.singletonMap("separator.char", "_"), 1);
|
||||
assertEquals(parsed.get("first_second_third"),
|
||||
Arrays.asList("one", "two", "three", "four", "five"),
|
||||
"Separator not applied.");
|
||||
}
|
||||
|
||||
@Test
|
||||
void testFlattenModeNormal() throws JsonProcessingException {
|
||||
final String json = "{\n" +
|
||||
" \"first\": {\n" +
|
||||
" \"second\": {\n" +
|
||||
" \"third\": [\n" +
|
||||
" \"one\",\n" +
|
||||
" \"two\",\n" +
|
||||
" \"three\",\n" +
|
||||
" \"four\",\n" +
|
||||
" \"five\"\n" +
|
||||
" ]\n" +
|
||||
" }\n" +
|
||||
" }\n" +
|
||||
"}";
|
||||
|
||||
testRunner.setProperty(FlattenJson.FLATTEN_MODE, FlattenJson.FLATTEN_MODE_NORMAL);
|
||||
final Map parsed = (Map) baseTest(testRunner, json,5);
|
||||
assertEquals("one", parsed.get("first.second.third[0]"), "Separator not applied.");
|
||||
}
|
||||
|
||||
@Test
|
||||
void testFlattenModeKeepArrays() throws JsonProcessingException {
|
||||
final String json = "{\n" +
|
||||
" \"first\": {\n" +
|
||||
" \"second\": [\n" +
|
||||
" {\n" +
|
||||
" \"x\": 1,\n" +
|
||||
" \"y\": 2,\n" +
|
||||
" \"z\": [\n" +
|
||||
" 3,\n" +
|
||||
" 4,\n" +
|
||||
" 5\n" +
|
||||
" ]\n" +
|
||||
" },\n" +
|
||||
" [\n" +
|
||||
" 6,\n" +
|
||||
" 7,\n" +
|
||||
" 8\n" +
|
||||
" ],\n" +
|
||||
" [\n" +
|
||||
" [\n" +
|
||||
" 9,\n" +
|
||||
" 10\n" +
|
||||
" ],\n" +
|
||||
" 11,\n" +
|
||||
" 12\n" +
|
||||
" ]\n" +
|
||||
" ],\n" +
|
||||
" \"third\": {\n" +
|
||||
" \"a\": \"b\",\n" +
|
||||
" \"c\": \"d\",\n" +
|
||||
" \"e\": \"f\"\n" +
|
||||
" }\n" +
|
||||
" }\n" +
|
||||
"}";
|
||||
|
||||
testRunner.setProperty(FlattenJson.FLATTEN_MODE, FlattenJson.FLATTEN_MODE_KEEP_ARRAYS);
|
||||
final Map parsed = (Map) baseTest(testRunner, json, 4);
|
||||
assertInstanceOf(List.class, parsed.get("first.second"));
|
||||
assertEquals(Arrays.asList(6, 7, 8), ((List) parsed.get("first.second")).get(1));
|
||||
assertEquals("b", parsed.get("first.third.a"), "Separator not applied.");
|
||||
}
|
||||
|
||||
@Test
|
||||
void testFlattenModeKeepPrimitiveArrays() throws JsonProcessingException {
|
||||
final String json = "{\n" +
|
||||
" \"first\": {\n" +
|
||||
" \"second\": [\n" +
|
||||
" {\n" +
|
||||
" \"x\": 1,\n" +
|
||||
" \"y\": 2,\n" +
|
||||
" \"z\": [\n" +
|
||||
" 3,\n" +
|
||||
" 4,\n" +
|
||||
" 5\n" +
|
||||
" ]\n" +
|
||||
" },\n" +
|
||||
" [\n" +
|
||||
" 6,\n" +
|
||||
" 7,\n" +
|
||||
" 8\n" +
|
||||
" ],\n" +
|
||||
" [\n" +
|
||||
" [\n" +
|
||||
" 9,\n" +
|
||||
" 10\n" +
|
||||
" ],\n" +
|
||||
" 11,\n" +
|
||||
" 12\n" +
|
||||
" ]\n" +
|
||||
" ],\n" +
|
||||
" \"third\": {\n" +
|
||||
" \"a\": \"b\",\n" +
|
||||
" \"c\": \"d\",\n" +
|
||||
" \"e\": \"f\"\n" +
|
||||
" }\n" +
|
||||
" }\n" +
|
||||
"}";
|
||||
|
||||
testRunner.setProperty(FlattenJson.FLATTEN_MODE, FlattenJson.FLATTEN_MODE_KEEP_PRIMITIVE_ARRAYS);
|
||||
final Map parsed = (Map) baseTest(testRunner, json, 10);
|
||||
assertEquals(1, parsed.get("first.second[0].x"), "Separator not applied.");
|
||||
assertEquals(Arrays.asList(3, 4, 5), parsed.get("first.second[0].z"), "Separator not applied.");
|
||||
assertEquals(Arrays.asList(9, 10), parsed.get("first.second[2][0]"), "Separator not applied.");
|
||||
assertEquals(11, parsed.get("first.second[2][1]"), "Separator not applied.");
|
||||
assertEquals(12, parsed.get("first.second[2][2]"), "Separator not applied.");
|
||||
assertEquals("d", parsed.get("first.third.c"), "Separator not applied.");
|
||||
}
|
||||
|
||||
@Test
|
||||
void testFlattenModeDotNotation() throws JsonProcessingException {
|
||||
final String json = "{\n" +
|
||||
" \"first\": {\n" +
|
||||
" \"second\": {\n" +
|
||||
" \"third\": [\n" +
|
||||
" \"one\",\n" +
|
||||
" \"two\",\n" +
|
||||
" \"three\",\n" +
|
||||
" \"four\",\n" +
|
||||
" \"five\"\n" +
|
||||
" ]\n" +
|
||||
" }\n" +
|
||||
" }\n" +
|
||||
"}";
|
||||
|
||||
testRunner.setProperty(FlattenJson.FLATTEN_MODE, FlattenJson.FLATTEN_MODE_DOT_NOTATION);
|
||||
final Map parsed = (Map) baseTest(testRunner, json, 5);
|
||||
assertEquals("one", parsed.get("first.second.third.0"), "Separator not applied.");
|
||||
}
|
||||
|
||||
@Test
|
||||
void testFlattenSlash() throws JsonProcessingException {
|
||||
final String json = "{\n" +
|
||||
" \"first\": {\n" +
|
||||
" \"second\": {\n" +
|
||||
" \"third\": [\n" +
|
||||
" \"http://localhost/value1\",\n" +
|
||||
" \"http://localhost/value2\"\n" +
|
||||
" ]\n" +
|
||||
" }\n" +
|
||||
" }\n" +
|
||||
"}";
|
||||
|
||||
testRunner.setProperty(FlattenJson.FLATTEN_MODE, FlattenJson.FLATTEN_MODE_NORMAL);
|
||||
final Map parsed = (Map) baseTest(testRunner, json, 2);
|
||||
assertEquals("http://localhost/value1", parsed.get("first.second.third[0]"), "Separator not applied.");
|
||||
}
|
||||
|
||||
@Test
|
||||
void testEscapeForJson() throws JsonProcessingException {
|
||||
final String json = "{\n" +
|
||||
" \"name\": \"Jos\\u00e9\"\n" +
|
||||
"}";
|
||||
|
||||
testRunner.setProperty(FlattenJson.FLATTEN_MODE, FlattenJson.FLATTEN_MODE_NORMAL);
|
||||
final Map parsed = (Map) baseTest(testRunner, json, 1);
|
||||
assertEquals("José", parsed.get("name"), "Separator not applied.");
|
||||
}
|
||||
|
||||
@Test
|
||||
void testUnFlatten() throws JsonProcessingException {
|
||||
final String json = "{\n" +
|
||||
" \"test.msg\": \"Hello, world\",\n" +
|
||||
" \"first.second.third\": [\n" +
|
||||
" \"one\",\n" +
|
||||
" \"two\",\n" +
|
||||
" \"three\",\n" +
|
||||
" \"four\",\n" +
|
||||
" \"five\"\n" +
|
||||
" ]\n" +
|
||||
"}";
|
||||
|
||||
testRunner.setProperty(FlattenJson.RETURN_TYPE, FlattenJson.RETURN_TYPE_UNFLATTEN);
|
||||
final Map parsed = (Map) baseTest(testRunner, json, 2);
|
||||
assertEquals("Hello, world", ((Map) parsed.get("test")).get("msg"));
|
||||
assertEquals(Arrays.asList("one", "two", "three", "four", "five"),
|
||||
((Map) ((Map) parsed.get("first")).get("second")).get("third"));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testUnFlattenWithDifferentSeparator() throws JsonProcessingException {
|
||||
final String json = "{\n" +
|
||||
" \"first_second_third\": [\n" +
|
||||
" \"one\",\n" +
|
||||
" \"two\",\n" +
|
||||
" \"three\",\n" +
|
||||
" \"four\",\n" +
|
||||
" \"five\"\n" +
|
||||
" ]\n" +
|
||||
"}";
|
||||
|
||||
testRunner.setProperty(FlattenJson.SEPARATOR, "_");
|
||||
testRunner.setProperty(FlattenJson.RETURN_TYPE, FlattenJson.RETURN_TYPE_UNFLATTEN);
|
||||
final Map parsed = (Map) baseTest(testRunner, json, 1);
|
||||
assertEquals(Arrays.asList("one", "two", "three", "four", "five"),
|
||||
((Map) ((Map) parsed.get("first")).get("second")).get("third"));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testUnFlattenForKeepArraysMode() throws JsonProcessingException {
|
||||
final String json = "{\n" +
|
||||
" \"a.b\": 1,\n" +
|
||||
" \"a.c\": [\n" +
|
||||
" false,\n" +
|
||||
" {\n" +
|
||||
" \"i.j\": [\n" +
|
||||
" false,\n" +
|
||||
" true,\n" +
|
||||
" \"xy\"\n" +
|
||||
" ]\n" +
|
||||
" }\n" +
|
||||
" ]\n" +
|
||||
"}";
|
||||
|
||||
testRunner.setProperty(FlattenJson.FLATTEN_MODE, FlattenJson.FLATTEN_MODE_KEEP_ARRAYS);
|
||||
testRunner.setProperty(FlattenJson.RETURN_TYPE, FlattenJson.RETURN_TYPE_UNFLATTEN);
|
||||
final Map parsed = (Map) baseTest(testRunner, json, 1);
|
||||
assertEquals(1, ((Map) parsed.get("a")).get("b"));
|
||||
assertEquals(false, ((List) ((Map) parsed.get("a")).get("c")).get(0));
|
||||
assertEquals(Arrays.asList(false, true, "xy"),
|
||||
((Map) ((Map) ((List) ((Map) parsed.get("a")).get("c")).get(1)).get("i")).get("j"));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testUnFlattenForKeepPrimitiveArraysMode() throws JsonProcessingException {
|
||||
final String json = "{\n" +
|
||||
" \"first.second[0].x\": 1,\n" +
|
||||
" \"first.second[0].y\": 2,\n" +
|
||||
" \"first.second[0].z\": [\n" +
|
||||
" 3,\n" +
|
||||
" 4,\n" +
|
||||
" 5\n" +
|
||||
" ],\n" +
|
||||
" \"first.second[1]\": [\n" +
|
||||
" 6,\n" +
|
||||
" 7,\n" +
|
||||
" 8\n" +
|
||||
" ],\n" +
|
||||
" \"first.second[2][0]\": [\n" +
|
||||
" 9,\n" +
|
||||
" 10\n" +
|
||||
" ],\n" +
|
||||
" \"first.second[2][1]\": 11,\n" +
|
||||
" \"first.second[2][2]\": 12,\n" +
|
||||
" \"first.third.a\": \"b\",\n" +
|
||||
" \"first.third.c\": \"d\",\n" +
|
||||
" \"first.third.e\": \"f\"\n" +
|
||||
"}";
|
||||
|
||||
testRunner.setProperty(FlattenJson.FLATTEN_MODE, FlattenJson.FLATTEN_MODE_KEEP_PRIMITIVE_ARRAYS);
|
||||
testRunner.setProperty(FlattenJson.RETURN_TYPE, FlattenJson.RETURN_TYPE_UNFLATTEN);
|
||||
final Map parsed = (Map) baseTest(testRunner, json, 1);
|
||||
assertEquals(1, ((Map) ((List) ((Map) parsed.get("first")).get("second")).get(0)).get("x"));
|
||||
assertEquals(Arrays.asList(9, 10), ((List) ((List) ((Map) parsed.get("first")).get("second")).get(2)).get(0));
|
||||
assertEquals("d", ((Map) ((Map) parsed.get("first")).get("third")).get("c"));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testUnFlattenForDotNotationMode() throws JsonProcessingException {
|
||||
final String json = "{\n" +
|
||||
" \"first.second.third.0\": [\n" +
|
||||
" \"one\",\n" +
|
||||
" \"two\",\n" +
|
||||
" \"three\",\n" +
|
||||
" \"four\",\n" +
|
||||
" \"five\"\n" +
|
||||
" ]\n" +
|
||||
"}";
|
||||
|
||||
testRunner.setProperty(FlattenJson.FLATTEN_MODE, FlattenJson.FLATTEN_MODE_DOT_NOTATION);
|
||||
testRunner.setProperty(FlattenJson.RETURN_TYPE, FlattenJson.RETURN_TYPE_UNFLATTEN);
|
||||
|
||||
final Map parsed = (Map) baseTest(testRunner, json, 1);
|
||||
assertEquals(Arrays.asList("one", "two", "three", "four", "five"),
|
||||
((List) ((Map) ((Map) parsed.get("first")).get("second")).get("third")).get(0));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testFlattenWithIgnoreReservedCharacters() throws JsonProcessingException {
|
||||
final String json = "{\n" +
|
||||
" \"first\": {\n" +
|
||||
" \"second.third\": \"Hello\",\n" +
|
||||
" \"fourth\": \"World\"\n" +
|
||||
" }\n" +
|
||||
"}";
|
||||
|
||||
testRunner.setProperty(FlattenJson.IGNORE_RESERVED_CHARACTERS, "true");
|
||||
|
||||
final Map parsed = (Map) baseTest(testRunner, json, 2);
|
||||
assertEquals("Hello", parsed.get("first.second.third"), "Separator not applied.");
|
||||
assertEquals("World", parsed.get("first.fourth"), "Separator not applied.");
|
||||
}
|
||||
|
||||
@Test
|
||||
void testFlattenRecordSetWithIgnoreReservedCharacters() throws JsonProcessingException {
|
||||
final String json = "[\n" +
|
||||
" {\n" +
|
||||
" \"first\": {\n" +
|
||||
" \"second_third\": \"Hello\"\n" +
|
||||
" }\n" +
|
||||
" },\n" +
|
||||
" {\n" +
|
||||
" \"first\": {\n" +
|
||||
" \"second_third\": \"World\"\n" +
|
||||
" }\n" +
|
||||
" }\n" +
|
||||
"]";
|
||||
testRunner.setProperty(FlattenJson.SEPARATOR, "_");
|
||||
testRunner.setProperty(FlattenJson.IGNORE_RESERVED_CHARACTERS, "true");
|
||||
|
||||
final List<String> expected = Arrays.asList("Hello", "World");
|
||||
|
||||
final List parsed = (List) baseTest(testRunner, json, 2);
|
||||
for (int i = 0; i < parsed.size(); i++) {
|
||||
assertEquals(expected.get(i), ((Map) parsed.get(i)).get("first_second_third"), "Missing values.");
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testFlattenModeNormalWithIgnoreReservedCharacters() throws JsonProcessingException {
|
||||
final String json = "[\n" +
|
||||
" {\n" +
|
||||
" \"first\": {\n" +
|
||||
" \"second_third\": \"Hello\"\n" +
|
||||
" }\n" +
|
||||
" },\n" +
|
||||
" {\n" +
|
||||
" \"first\": {\n" +
|
||||
" \"second_third\": \"World\"\n" +
|
||||
" }\n" +
|
||||
" }\n" +
|
||||
"]";
|
||||
testRunner.setProperty(FlattenJson.SEPARATOR, "_");
|
||||
testRunner.setProperty(FlattenJson.IGNORE_RESERVED_CHARACTERS, "true");
|
||||
testRunner.setProperty(FlattenJson.FLATTEN_MODE, FlattenJson.FLATTEN_MODE_NORMAL);
|
||||
|
||||
final Map parsed = (Map) baseTest(testRunner, json, 2);
|
||||
assertEquals("Hello", parsed.get("[0]_first_second_third"), "Separator not applied.");
|
||||
assertEquals("World", parsed.get("[1]_first_second_third"), "Separator not applied.");
|
||||
}
|
||||
|
||||
private Object baseTest(TestRunner testRunner, String json, int keyCount) throws JsonProcessingException {
|
||||
return baseTest(testRunner, json, Collections.emptyMap(), keyCount);
|
||||
}
|
||||
|
||||
private Object baseTest(TestRunner testRunner, String json, Map attrs, int keyCount) throws JsonProcessingException {
|
||||
testRunner.enqueue(json, attrs);
|
||||
testRunner.run(1, true);
|
||||
testRunner.assertTransferCount(FlattenJson.REL_FAILURE, 0);
|
||||
testRunner.assertTransferCount(FlattenJson.REL_SUCCESS, 1);
|
||||
|
||||
final List<MockFlowFile> flowFiles = testRunner.getFlowFilesForRelationship(FlattenJson.REL_SUCCESS);
|
||||
final byte[] content = testRunner.getContentAsByteArray(flowFiles.get(0));
|
||||
final String asJson = new String(content);
|
||||
if (asJson.startsWith("[")) {
|
||||
final List parsed;
|
||||
parsed = mapper.readValue(asJson, List.class);
|
||||
assertEquals(keyCount, parsed.size(), "Too many keys");
|
||||
return parsed;
|
||||
} else {
|
||||
final Map parsed;
|
||||
parsed = mapper.readValue(asJson, Map.class);
|
||||
assertEquals(keyCount, parsed.size(), "Too many keys");
|
||||
return parsed;
|
||||
}
|
||||
}
|
||||
}
|
Loading…
x
Reference in New Issue
Block a user