NIFI-14051 Removed unnecessary generic type declarations (#9562)

Signed-off-by: David Handermann <exceptionfactory@apache.org>
This commit is contained in:
dan-s1 2024-12-06 17:56:02 -05:00 committed by GitHub
parent 3c9ecd876d
commit ba9273f8fa
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
134 changed files with 1114 additions and 1657 deletions

View File

@ -815,7 +815,7 @@ public class ExpressionCompiler {
toStringEvaluator(argEvaluators.get(0), "first argument to contains")), "contains");
}
case IN: {
List<Evaluator<String>> list = new ArrayList<Evaluator<String>>();
List<Evaluator<String>> list = new ArrayList<>();
for (int i = 0; i < argEvaluators.size(); i++) {
list.add(toStringEvaluator(argEvaluators.get(i), i + "th argument to in"));
}

View File

@ -304,7 +304,7 @@ public class HL7Query {
if (value instanceof List) {
possibleValues = (List<Object>) value;
} else if (value instanceof Collection) {
possibleValues = new ArrayList<Object>((Collection<Object>) value);
possibleValues = new ArrayList<>((Collection<Object>) value);
} else {
possibleValues = new ArrayList<>(1);
possibleValues.add(value);

View File

@ -210,7 +210,7 @@ public class JmxJvmMetrics implements JvmMetrics {
@Override
public Map<Thread.State, Double> threadStatePercentages() {
int totalThreadCount = (Integer) getMetric(THREADS_COUNT);
final Map<Thread.State, Double> threadStatePercentages = new HashMap<Thread.State, Double>();
final Map<Thread.State, Double> threadStatePercentages = new HashMap<>();
for (Thread.State state : Thread.State.values()) {
threadStatePercentages.put(state, (Integer) getMetric(REGISTRY_METRICSET_THREADS + "." + state.name().toLowerCase() + ".count") / (double) totalThreadCount);
}

View File

@ -243,17 +243,14 @@ public class NiFiPropertiesTest {
@Test
public void testShouldHaveReasonableMaxContentLengthValues() {
// Arrange with default values:
NiFiProperties properties = NiFiProperties.createBasicNiFiProperties(null, new HashMap<String, String>() {{
}});
NiFiProperties properties = NiFiProperties.createBasicNiFiProperties(null, new HashMap<>());
// Assert defaults match expectations:
assertNull(properties.getWebMaxContentSize());
// Re-arrange with specific values:
final String size = "size value";
properties = NiFiProperties.createBasicNiFiProperties(null, new HashMap<String, String>() {{
put(NiFiProperties.WEB_MAX_CONTENT_SIZE, size);
}});
properties = NiFiProperties.createBasicNiFiProperties(null, Map.of(NiFiProperties.WEB_MAX_CONTENT_SIZE, size));
// Assert specific values are used:
assertEquals(properties.getWebMaxContentSize(), size);
@ -261,15 +258,14 @@ public class NiFiPropertiesTest {
@Test
public void testIsZooKeeperTlsConfigurationPresent() {
NiFiProperties properties = NiFiProperties.createBasicNiFiProperties(null, new HashMap<String, String>() {{
put(NiFiProperties.ZOOKEEPER_CLIENT_SECURE, "true");
put(NiFiProperties.ZOOKEEPER_SECURITY_KEYSTORE, "/a/keystore/filepath/keystore.jks");
put(NiFiProperties.ZOOKEEPER_SECURITY_KEYSTORE_PASSWD, "password");
put(NiFiProperties.ZOOKEEPER_SECURITY_KEYSTORE_TYPE, "JKS");
put(NiFiProperties.ZOOKEEPER_SECURITY_TRUSTSTORE, "/a/truststore/filepath/truststore.jks");
put(NiFiProperties.ZOOKEEPER_SECURITY_TRUSTSTORE_PASSWD, "password");
put(NiFiProperties.ZOOKEEPER_SECURITY_TRUSTSTORE_TYPE, "JKS");
}});
NiFiProperties properties = NiFiProperties.createBasicNiFiProperties(null, Map.of(
NiFiProperties.ZOOKEEPER_CLIENT_SECURE, "true",
NiFiProperties.ZOOKEEPER_SECURITY_KEYSTORE, "/a/keystore/filepath/keystore.jks",
NiFiProperties.ZOOKEEPER_SECURITY_KEYSTORE_PASSWD, "password",
NiFiProperties.ZOOKEEPER_SECURITY_KEYSTORE_TYPE, "JKS",
NiFiProperties.ZOOKEEPER_SECURITY_TRUSTSTORE, "/a/truststore/filepath/truststore.jks",
NiFiProperties.ZOOKEEPER_SECURITY_TRUSTSTORE_PASSWD, "password",
NiFiProperties.ZOOKEEPER_SECURITY_TRUSTSTORE_TYPE, "JKS"));
assertTrue(properties.isZooKeeperClientSecure());
assertTrue(properties.isZooKeeperTlsConfigurationPresent());
@ -277,13 +273,12 @@ public class NiFiPropertiesTest {
@Test
public void testSomeZooKeeperTlsConfigurationIsMissing() {
NiFiProperties properties = NiFiProperties.createBasicNiFiProperties(null, new HashMap<String, String>() {{
put(NiFiProperties.ZOOKEEPER_CLIENT_SECURE, "true");
put(NiFiProperties.ZOOKEEPER_SECURITY_KEYSTORE_PASSWD, "password");
put(NiFiProperties.ZOOKEEPER_SECURITY_KEYSTORE_TYPE, "JKS");
put(NiFiProperties.ZOOKEEPER_SECURITY_TRUSTSTORE, "/a/truststore/filepath/truststore.jks");
put(NiFiProperties.ZOOKEEPER_SECURITY_TRUSTSTORE_TYPE, "JKS");
}});
NiFiProperties properties = NiFiProperties.createBasicNiFiProperties(null, Map.of(
NiFiProperties.ZOOKEEPER_CLIENT_SECURE, "true",
NiFiProperties.ZOOKEEPER_SECURITY_KEYSTORE_PASSWD, "password",
NiFiProperties.ZOOKEEPER_SECURITY_KEYSTORE_TYPE, "JKS",
NiFiProperties.ZOOKEEPER_SECURITY_TRUSTSTORE, "/a/truststore/filepath/truststore.jks",
NiFiProperties.ZOOKEEPER_SECURITY_TRUSTSTORE_TYPE, "JKS"));
assertTrue(properties.isZooKeeperClientSecure());
assertFalse(properties.isZooKeeperTlsConfigurationPresent());
@ -291,15 +286,14 @@ public class NiFiPropertiesTest {
@Test
public void testZooKeeperTlsPasswordsBlank() {
NiFiProperties properties = NiFiProperties.createBasicNiFiProperties(null, new HashMap<String, String>() {{
put(NiFiProperties.ZOOKEEPER_CLIENT_SECURE, "true");
put(NiFiProperties.ZOOKEEPER_SECURITY_KEYSTORE, "/a/keystore/filepath/keystore.jks");
put(NiFiProperties.ZOOKEEPER_SECURITY_KEYSTORE_PASSWD, "");
put(NiFiProperties.ZOOKEEPER_SECURITY_KEYSTORE_TYPE, "JKS");
put(NiFiProperties.ZOOKEEPER_SECURITY_TRUSTSTORE, "/a/truststore/filepath/truststore.jks");
put(NiFiProperties.ZOOKEEPER_SECURITY_TRUSTSTORE_PASSWD, "");
put(NiFiProperties.ZOOKEEPER_SECURITY_TRUSTSTORE_TYPE, "JKS");
}});
NiFiProperties properties = NiFiProperties.createBasicNiFiProperties(null, Map.of(
NiFiProperties.ZOOKEEPER_CLIENT_SECURE, "true",
NiFiProperties.ZOOKEEPER_SECURITY_KEYSTORE, "/a/keystore/filepath/keystore.jks",
NiFiProperties.ZOOKEEPER_SECURITY_KEYSTORE_PASSWD, "",
NiFiProperties.ZOOKEEPER_SECURITY_KEYSTORE_TYPE, "JKS",
NiFiProperties.ZOOKEEPER_SECURITY_TRUSTSTORE, "/a/truststore/filepath/truststore.jks",
NiFiProperties.ZOOKEEPER_SECURITY_TRUSTSTORE_PASSWD, "",
NiFiProperties.ZOOKEEPER_SECURITY_TRUSTSTORE_TYPE, "JKS"));
assertTrue(properties.isZooKeeperClientSecure());
assertTrue(properties.isZooKeeperTlsConfigurationPresent());
@ -307,46 +301,42 @@ public class NiFiPropertiesTest {
@Test
public void testKeystorePasswordIsMissing() {
NiFiProperties properties = NiFiProperties.createBasicNiFiProperties(null, new HashMap<String, String>() {{
put(NiFiProperties.SECURITY_KEYSTORE, "/a/keystore/filepath/keystore.jks");
put(NiFiProperties.SECURITY_KEYSTORE_TYPE, "JKS");
put(NiFiProperties.SECURITY_TRUSTSTORE, "/a/truststore/filepath/truststore.jks");
put(NiFiProperties.SECURITY_TRUSTSTORE_PASSWD, "");
put(NiFiProperties.SECURITY_TRUSTSTORE_TYPE, "JKS");
}});
NiFiProperties properties = NiFiProperties.createBasicNiFiProperties(null, Map.of(
NiFiProperties.SECURITY_KEYSTORE, "/a/keystore/filepath/keystore.jks",
NiFiProperties.SECURITY_KEYSTORE_TYPE, "JKS",
NiFiProperties.SECURITY_TRUSTSTORE, "/a/truststore/filepath/truststore.jks",
NiFiProperties.SECURITY_TRUSTSTORE_PASSWD, "",
NiFiProperties.SECURITY_TRUSTSTORE_TYPE, "JKS"));
assertFalse(properties.isTlsConfigurationPresent());
}
@Test
public void testTlsConfigurationIsPresentWithEmptyPasswords() {
NiFiProperties properties = NiFiProperties.createBasicNiFiProperties(null, new HashMap<String, String>() {{
put(NiFiProperties.SECURITY_KEYSTORE, "/a/keystore/filepath/keystore.jks");
put(NiFiProperties.SECURITY_KEYSTORE_PASSWD, "");
put(NiFiProperties.SECURITY_KEYSTORE_TYPE, "JKS");
put(NiFiProperties.SECURITY_TRUSTSTORE, "/a/truststore/filepath/truststore.jks");
put(NiFiProperties.SECURITY_TRUSTSTORE_PASSWD, "");
put(NiFiProperties.SECURITY_TRUSTSTORE_TYPE, "JKS");
}});
NiFiProperties properties = NiFiProperties.createBasicNiFiProperties(null, Map.of(
NiFiProperties.SECURITY_KEYSTORE, "/a/keystore/filepath/keystore.jks",
NiFiProperties.SECURITY_KEYSTORE_PASSWD, "",
NiFiProperties.SECURITY_KEYSTORE_TYPE, "JKS",
NiFiProperties.SECURITY_TRUSTSTORE, "/a/truststore/filepath/truststore.jks",
NiFiProperties.SECURITY_TRUSTSTORE_PASSWD, "",
NiFiProperties.SECURITY_TRUSTSTORE_TYPE, "JKS"));
assertTrue(properties.isTlsConfigurationPresent());
}
@Test
public void testTlsConfigurationIsNotPresentWithPropertiesMissing() {
NiFiProperties properties = NiFiProperties.createBasicNiFiProperties(null, new HashMap<String, String>() {{
put(NiFiProperties.SECURITY_KEYSTORE_PASSWD, "password");
put(NiFiProperties.SECURITY_KEYSTORE_TYPE, "JKS");
put(NiFiProperties.SECURITY_TRUSTSTORE, "/a/truststore/filepath/truststore.jks");
}});
NiFiProperties properties = NiFiProperties.createBasicNiFiProperties(null, Map.of(
NiFiProperties.SECURITY_KEYSTORE_PASSWD, "password",
NiFiProperties.SECURITY_KEYSTORE_TYPE, "JKS",
NiFiProperties.SECURITY_TRUSTSTORE, "/a/truststore/filepath/truststore.jks"));
assertFalse(properties.isTlsConfigurationPresent());
}
@Test
public void testTlsConfigurationIsNotPresentWithNoProperties() {
NiFiProperties properties = NiFiProperties.createBasicNiFiProperties(null, new HashMap<String, String>() {{
}});
NiFiProperties properties = NiFiProperties.createBasicNiFiProperties(null, new HashMap<>());
assertFalse(properties.isTlsConfigurationPresent());
}

View File

@ -225,7 +225,7 @@ public enum RecordFieldType {
MAP("map", null, new MapDataType(null));
private static final Map<String, RecordFieldType> SIMPLE_NAME_MAP = new HashMap<String, RecordFieldType>();
private static final Map<String, RecordFieldType> SIMPLE_NAME_MAP = new HashMap<>();
static {
for (RecordFieldType value : values()) {

View File

@ -48,12 +48,12 @@ import java.util.UUID;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.DoubleAdder;
import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import java.util.stream.Stream;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertInstanceOf;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertThrows;
@ -251,18 +251,14 @@ public class TestDataTypeUtils {
int[] intArray = {3, 2, 1};
Map<String, Object> inputMap = new HashMap<String, Object>() {{
put("field1", "hello");
put("field2", 1);
put("field3", intArray);
}};
Map<String, Object> inputMap = Map.of("field1", "hello", "field2", 1, "field3", intArray);
resultMap = DataTypeUtils.convertRecordMapToJavaMap(inputMap, RecordFieldType.STRING.getDataType());
assertNotNull(resultMap);
assertFalse(resultMap.isEmpty());
assertEquals("hello", resultMap.get("field1"));
assertEquals(1, resultMap.get("field2"));
assertTrue(resultMap.get("field3") instanceof int[]);
assertInstanceOf(int[].class, resultMap.get("field3"));
assertNull(resultMap.get("field4"));
}
@ -273,7 +269,7 @@ public class TestDataTypeUtils {
String uuidString = generated.toString();
Object result = DataTypeUtils.convertType(uuidString, RecordFieldType.UUID.getDataType(), "uuid_test");
assertTrue(result instanceof UUID);
assertInstanceOf(UUID.class, result);
assertEquals(generated, result);
}
@ -283,7 +279,7 @@ public class TestDataTypeUtils {
String uuid = generated.toString();
Object result = DataTypeUtils.convertType(generated, RecordFieldType.STRING.getDataType(), "uuid_test");
assertTrue(result instanceof String);
assertInstanceOf(String.class, result);
assertEquals(uuid, result);
}
@ -296,7 +292,7 @@ public class TestDataTypeUtils {
byte[] expected = buffer.array();
Object result = DataTypeUtils.convertType(expected, RecordFieldType.UUID.getDataType(), "uuid_test");
assertTrue(result instanceof UUID);
assertInstanceOf(UUID.class, result);
assertEquals(generated, result);
}
@ -309,7 +305,7 @@ public class TestDataTypeUtils {
byte[] expected = buffer.array();
Object result = DataTypeUtils.convertType(expected, RecordFieldType.ARRAY.getDataType(), "uuid_test");
assertTrue(result instanceof Byte[]);
assertInstanceOf(Byte[].class, result);
assertEquals( 16, ((Byte[]) result).length);
Byte[] bytes = (Byte[]) result;
for (int x = 0; x < bytes.length; x++) {
@ -326,7 +322,7 @@ public class TestDataTypeUtils {
Object[] resultArray = DataTypeUtils.convertRecordArrayToJavaArray(stringArray, RecordFieldType.STRING.getDataType());
assertNotNull(resultArray);
for (Object o : resultArray) {
assertTrue(o instanceof String);
assertInstanceOf(String.class, o);
}
}
@ -351,10 +347,10 @@ public class TestDataTypeUtils {
Object[] recordArray = {inputRecord1, inputRecord2};
Object resultObj = DataTypeUtils.convertRecordFieldtoObject(recordArray, RecordFieldType.ARRAY.getArrayDataType(RecordFieldType.RECORD.getRecordDataType(schema)));
assertNotNull(resultObj);
assertTrue(resultObj instanceof Object[]);
assertInstanceOf(Object[].class, resultObj);
Object[] resultArray = (Object[]) resultObj;
for (Object o : resultArray) {
assertTrue(o instanceof Map);
assertInstanceOf(Map.class, o);
}
}
@ -367,17 +363,17 @@ public class TestDataTypeUtils {
}}, "");
final Object obj = DataTypeUtils.convertRecordFieldtoObject(record, RecordFieldType.RECORD.getDataType());
assertTrue(obj instanceof Map);
assertInstanceOf(Map.class, obj);
final Map<String, Object> map = (Map<String, Object>) obj;
assertEquals("John", map.get("firstName"));
assertEquals(30, map.get("age"));
assertTrue(map.get("addresses") instanceof Object[]);
assertInstanceOf(Object[].class, map.get("addresses"));
final Object[] objArray = (Object[]) map.get("addresses");
assertEquals(2, objArray.length);
assertEquals("some string", objArray[0]);
assertTrue(objArray[1] instanceof Map);
assertInstanceOf(Map.class, objArray[1]);
final Map<String, Object> addressMap = (Map<String, Object>) objArray[1];
assertEquals("123 Fake Street", addressMap.get("address_1"));
}
@ -433,53 +429,53 @@ public class TestDataTypeUtils {
final Record inputRecord = new MapRecord(schema, values);
Object o = DataTypeUtils.convertRecordFieldtoObject(inputRecord, RecordFieldType.RECORD.getRecordDataType(schema));
assertTrue(o instanceof Map);
assertInstanceOf(Map.class, o);
final Map<String, Object> outputMap = (Map<String, Object>) o;
assertEquals("hello", outputMap.get("defaultOfHello"));
assertEquals("world", outputMap.get("noDefault"));
o = outputMap.get("intField");
assertEquals(5, o);
o = outputMap.get("intArray");
assertTrue(o instanceof Integer[]);
assertInstanceOf(Integer[].class, o);
final Integer[] intArray = (Integer[]) o;
assertEquals(3, intArray.length);
assertEquals((Integer) 3, intArray[0]);
o = outputMap.get("objArray");
assertTrue(o instanceof Object[]);
assertInstanceOf(Object[].class, o);
final Object[] objArray = (Object[]) o;
assertEquals(4, objArray.length);
assertEquals(3, objArray[0]);
assertEquals("2", objArray[1]);
o = outputMap.get("choiceArray");
assertTrue(o instanceof Object[]);
assertInstanceOf(Object[].class, o);
final Object[] choiceArray = (Object[]) o;
assertEquals(2, choiceArray.length);
assertEquals("foo", choiceArray[0]);
assertTrue(choiceArray[1] instanceof Object[]);
assertInstanceOf(Object[].class, choiceArray[1]);
final Object[] strArray = (Object[]) choiceArray[1];
assertEquals(2, strArray.length);
assertEquals("bar", strArray[0]);
assertEquals("baz", strArray[1]);
o = outputMap.get("complex");
assertTrue(o instanceof Map);
assertInstanceOf(Map.class, o);
final Map<String, Object> nestedOutputMap = (Map<String, Object>) o;
o = nestedOutputMap.get("complex1");
assertTrue(o instanceof Map);
assertInstanceOf(Map.class, o);
final Map<String, Object> complex1 = (Map<String, Object>) o;
o = complex1.get("a");
assertTrue(o instanceof Integer[]);
assertInstanceOf(Integer[].class, o);
assertEquals((Integer) 2, ((Integer[]) o)[1]);
o = complex1.get("b");
assertTrue(o instanceof Integer[]);
assertInstanceOf(Integer[].class, o);
assertEquals((Integer) 3, ((Integer[]) o)[2]);
o = nestedOutputMap.get("complex2");
assertTrue(o instanceof Map);
assertInstanceOf(Map.class, o);
final Map<String, Object> complex2 = (Map<String, Object>) o;
o = complex2.get("a");
assertTrue(o instanceof String[]);
assertInstanceOf(String[].class, o);
assertEquals("hello", ((String[]) o)[0]);
o = complex2.get("b");
assertTrue(o instanceof String[]);
assertInstanceOf(String[].class, o);
assertEquals("4", ((String[]) o)[1]);
}
@ -498,7 +494,7 @@ public class TestDataTypeUtils {
@Test
public void testStringToBytes() {
Object bytes = DataTypeUtils.convertType("Hello", RecordFieldType.ARRAY.getArrayDataType(RecordFieldType.BYTE.getDataType()), null, StandardCharsets.UTF_8);
assertTrue(bytes instanceof Byte[]);
assertInstanceOf(Byte[].class, bytes);
assertNotNull(bytes);
Byte[] b = (Byte[]) bytes;
assertEquals((long) 72, (long) b[0], "Conversion from String to byte[] failed"); // H
@ -512,7 +508,7 @@ public class TestDataTypeUtils {
public void testBytesToString() {
Object s = DataTypeUtils.convertType("Hello".getBytes(StandardCharsets.UTF_16), RecordFieldType.STRING.getDataType(), null, StandardCharsets.UTF_16);
assertNotNull(s);
assertTrue(s instanceof String);
assertInstanceOf(String.class, s);
assertEquals("Hello", s, "Conversion from byte[] to String failed");
}
@ -520,7 +516,7 @@ public class TestDataTypeUtils {
public void testBytesToBytes() {
Object b = DataTypeUtils.convertType("Hello".getBytes(StandardCharsets.UTF_16), RecordFieldType.ARRAY.getArrayDataType(RecordFieldType.BYTE.getDataType()), null, StandardCharsets.UTF_16);
assertNotNull(b);
assertTrue(b instanceof Byte[]);
assertInstanceOf(Byte[].class, b);
assertEquals((Object) "Hello".getBytes(StandardCharsets.UTF_16)[0], ((Byte[]) b)[0], "Conversion from byte[] to String failed at char 0");
}
@ -827,7 +823,7 @@ public class TestDataTypeUtils {
testChooseDataTypeAlsoReverseTypes(value, dataTypes, expected);
}
private <E> void testChooseDataTypeAlsoReverseTypes(Object value, List<DataType> dataTypes, DataType expected) {
private void testChooseDataTypeAlsoReverseTypes(Object value, List<DataType> dataTypes, DataType expected) {
testChooseDataType(dataTypes, value, expected);
Collections.reverse(dataTypes);
testChooseDataType(dataTypes, value, expected);
@ -881,12 +877,12 @@ public class TestDataTypeUtils {
@Test
public void testFindMostSuitableTypeWithByte() {
testFindMostSuitableType(Byte.valueOf((byte) 123), RecordFieldType.BYTE.getDataType());
testFindMostSuitableType((byte) 123, RecordFieldType.BYTE.getDataType());
}
@Test
public void testFindMostSuitableTypeWithShort() {
testFindMostSuitableType(Short.valueOf((short) 123), RecordFieldType.SHORT.getDataType());
testFindMostSuitableType((short) 123, RecordFieldType.SHORT.getDataType());
}
@Test
@ -955,7 +951,7 @@ public class TestDataTypeUtils {
}
private void testFindMostSuitableType(Object value, DataType expected, DataType... filtered) {
List<DataType> filteredOutDataTypes = Arrays.stream(filtered).collect(Collectors.toList());
List<DataType> filteredOutDataTypes = Arrays.stream(filtered).toList();
// GIVEN
List<DataType> unexpectedTypes = Arrays.stream(RecordFieldType.values())
@ -972,7 +968,7 @@ public class TestDataTypeUtils {
})
.filter(dataType -> !dataType.equals(expected))
.filter(dataType -> !filteredOutDataTypes.contains(dataType))
.collect(Collectors.toList());
.toList();
IntStream.rangeClosed(0, unexpectedTypes.size()).forEach(insertIndex -> {
List<DataType> allTypes = new LinkedList<>(unexpectedTypes);
@ -1186,7 +1182,7 @@ public class TestDataTypeUtils {
@Test
public void testConvertTypeStringToDateDefaultTimeZoneFormat() {
final Object converted = DataTypeUtils.convertType(ISO_8601_YEAR_MONTH_DAY, RecordFieldType.DATE.getDataType(), DATE_FIELD);
assertTrue(converted instanceof java.sql.Date, "Converted value is not java.sql.Date");
assertInstanceOf(Date.class, converted, "Converted value is not java.sql.Date");
assertEquals(ISO_8601_YEAR_MONTH_DAY, converted.toString());
}
@ -1198,7 +1194,7 @@ public class TestDataTypeUtils {
final Object converted = DataTypeUtils.convertType(
CUSTOM_MONTH_DAY_YEAR, RecordFieldType.DATE.getDataType(), Optional.of(CUSTOM_MONTH_DAY_YEAR_PATTERN), Optional.empty(), Optional.empty(), "date"
);
assertTrue(converted instanceof java.sql.Date, "Converted value is not java.sql.Date");
assertInstanceOf(Date.class, converted, "Converted value is not java.sql.Date");
assertEquals(ISO_8601_YEAR_MONTH_DAY, converted.toString());
}

View File

@ -31,7 +31,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
public class LeakyBucketStreamThrottler implements StreamThrottler {
private final int maxBytesPerSecond;
private final BlockingQueue<Request> requestQueue = new LinkedBlockingQueue<Request>();
private final BlockingQueue<Request> requestQueue = new LinkedBlockingQueue<>();
private final ScheduledExecutorService executorService;
private final AtomicBoolean shutdown = new AtomicBoolean(false);
@ -304,7 +304,7 @@ public class LeakyBucketStreamThrottler implements StreamThrottler {
this.out = out;
this.in = in;
this.maxBytesToCopy = maxBytesToCopy;
this.responseQueue = new LinkedBlockingQueue<Response>(1);
this.responseQueue = new LinkedBlockingQueue<>(1);
}
public BlockingQueue<Response> getResponseQueue() {

View File

@ -52,7 +52,7 @@ public class SearchState<T> {
}
void addResult(final SearchTerm matchingTerm) {
final List<Long> indexes = (resultMap.containsKey(matchingTerm)) ? resultMap.get(matchingTerm) : new ArrayList<Long>(5);
final List<Long> indexes = (resultMap.containsKey(matchingTerm)) ? resultMap.get(matchingTerm) : new ArrayList<>(5);
indexes.add(bytesRead);
resultMap.put(matchingTerm, indexes);
}

View File

@ -91,7 +91,7 @@ public class TestLeakyBucketThrottler {
try (final LeakyBucketStreamThrottler throttler = new LeakyBucketStreamThrottler(1024 * 1024);
final ByteArrayOutputStream baos = new ByteArrayOutputStream()) {
// create 3 threads, each sending ~2 MB
final List<Thread> threads = new ArrayList<Thread>();
final List<Thread> threads = new ArrayList<>();
for (int i = 0; i < 3; i++) {
final Thread t = new WriterThread(i, throttler, baos);
threads.add(t);

View File

@ -81,7 +81,7 @@ public class TestRingBuffer {
}
final AtomicInteger countHolder = new AtomicInteger(0);
ringBuffer.forEach(new ForEachEvaluator<Integer>() {
ringBuffer.forEach(new ForEachEvaluator<>() {
int counter = 0;
@Override
@ -110,7 +110,7 @@ public class TestRingBuffer {
ringBuffer.add(v);
}
ringBuffer.forEach(new ForEachEvaluator<Integer>() {
ringBuffer.forEach(new ForEachEvaluator<>() {
int counter = 0;
@Override
@ -139,7 +139,7 @@ public class TestRingBuffer {
}
final AtomicInteger countHolder = new AtomicInteger(0);
ringBuffer.forEach(new ForEachEvaluator<Integer>() {
ringBuffer.forEach(new ForEachEvaluator<>() {
int counter = 0;
@Override
@ -171,7 +171,7 @@ public class TestRingBuffer {
ringBuffer.add(v);
}
ringBuffer.forEach(new ForEachEvaluator<Integer>() {
ringBuffer.forEach(new ForEachEvaluator<>() {
int counter = 0;
@Override

View File

@ -224,7 +224,7 @@ public class TestLengthDelimitedJournal {
final Collection<DummyRecord> records = Collections.singleton(thirdRecord);
for (int i = 0; i < 10; i++) {
assertThrows(IOException.class, () -> journal.update(records, lookup));
assertThrows(IOException.class, () -> journal.fsync());
assertThrows(IOException.class, journal::fsync);
}
}
}
@ -251,7 +251,7 @@ public class TestLengthDelimitedJournal {
final Collection<DummyRecord> records = Collections.singleton(thirdRecord);
for (int i = 0; i < 10; i++) {
assertThrows(IOException.class, () -> journal.update(records, lookup));
assertThrows(IOException.class, () -> journal.fsync());
assertThrows(IOException.class, journal::fsync);
}
}
}
@ -289,7 +289,7 @@ public class TestLengthDelimitedJournal {
@Test
public void testMultipleThreadsCreatingOverflowDirectory() throws IOException, InterruptedException {
final LengthDelimitedJournal<DummyRecord> journal = new LengthDelimitedJournal<DummyRecord>(journalFile, serdeFactory, streamPool, 3820L, 100) {
final LengthDelimitedJournal<DummyRecord> journal = new LengthDelimitedJournal<>(journalFile, serdeFactory, streamPool, 3820L, 100) {
@Override
protected void createOverflowDirectory(final Path path) throws IOException {
// Create the overflow directory.
@ -475,7 +475,7 @@ public class TestLengthDelimitedJournal {
};
final Supplier<ByteArrayDataOutputStream> badosSupplier = new Supplier<ByteArrayDataOutputStream>() {
final Supplier<ByteArrayDataOutputStream> badosSupplier = new Supplier<>() {
private final AtomicInteger count = new AtomicInteger(0);
@Override
@ -497,15 +497,15 @@ public class TestLengthDelimitedJournal {
final Thread[] threads = new Thread[2];
final LengthDelimitedJournal<DummyRecord> journal = new LengthDelimitedJournal<DummyRecord>(journalFile, serdeFactory, corruptingStreamPool, 0L) {
final LengthDelimitedJournal<DummyRecord> journal = new LengthDelimitedJournal<>(journalFile, serdeFactory, corruptingStreamPool, 0L) {
private final AtomicInteger count = new AtomicInteger(0);
@Override
protected void poison(final Throwable t) {
protected void poison(final Throwable t) {
if (count.getAndIncrement() == 0) { // it is only important that we sleep the first time. If we sleep every time, it just slows the test down.
try {
Thread.sleep(3000L);
} catch (InterruptedException e) {
} catch (InterruptedException ignore) {
}
}
@ -524,7 +524,7 @@ public class TestLengthDelimitedJournal {
final Thread t1 = new Thread(() -> {
try {
journal.update(Collections.singleton(firstRecord), key -> null);
} catch (final IOException ioe) {
} catch (final IOException ignore) {
}
});
@ -532,7 +532,7 @@ public class TestLengthDelimitedJournal {
final Thread t2 = new Thread(() -> {
try {
journal.update(Collections.singleton(secondRecord), key -> firstRecord);
} catch (final IOException ioe) {
} catch (final IOException ignore) {
}
});

View File

@ -86,7 +86,7 @@ class TestChannel implements Channel {
if (this.routingKeyToQueueMappings != null) {
for (List<String> queues : routingKeyToQueueMappings.values()) {
for (String queue : queues) {
this.enqueuedMessages.put(queue, new ArrayBlockingQueue<GetResponse>(100));
this.enqueuedMessages.put(queue, new ArrayBlockingQueue<>(100));
}
}
}

View File

@ -108,7 +108,7 @@ public abstract class GenericAsanaObjectFetcher<T extends Resource> extends Abst
});
return new FilterIterator<>(
new Iterator<AsanaObject>() {
new Iterator<>() {
Iterator<AsanaObject> it = currentObjects.iterator();
Set<String> unseenIds = new HashSet<>(lastFingerprints.keySet()); // copy all previously seen ids.

View File

@ -36,7 +36,7 @@ public class StandardRecordModelIteratorProvider implements RecordModelIteratorP
@SuppressWarnings("unchecked")
public Iterator<BerType> iterator(InputStream inputStream, ComponentLog logger, Class<? extends BerType> rootClass, String recordField, Field seqOfField) {
if (StringUtils.isEmpty(recordField)) {
return new Iterator<BerType>() {
return new Iterator<>() {
@Override
public boolean hasNext() {
boolean hasNext;

View File

@ -43,7 +43,6 @@ import org.junit.jupiter.api.Test;
import java.math.BigInteger;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.function.Function;
@ -63,9 +62,8 @@ public class TestJASN1RecordReaderWithComplexTypes implements JASN1ReadRecordTes
SequenceOfIntegerWrapper berValue = new SequenceOfIntegerWrapper();
berValue.setValue(value);
Map<String, Object> expectedValues = new HashMap<String, Object>() {{
put("value", new BigInteger[]{BigInteger.valueOf(1234), BigInteger.valueOf(567)});
}};
Map<String, Object> expectedValues =
Map.of("value", new BigInteger[]{BigInteger.valueOf(1234), BigInteger.valueOf(567)});
RecordSchema expectedSchema = new SimpleRecordSchema(Arrays.asList(
new RecordField("value", RecordFieldType.ARRAY.getArrayDataType(RecordFieldType.BIGINT.getDataType())))
@ -85,13 +83,12 @@ public class TestJASN1RecordReaderWithComplexTypes implements JASN1ReadRecordTes
basicTypes.setUtf8Str(new BerUTF8String("Some UTF-8 String. こんにちは世界。"));
basicTypes.setBitStr(new BerBitString(new boolean[] {true, false, true, true}));
Map<String, Object> expectedValues = new HashMap<String, Object>() {{
put("b", true);
put("i", BigInteger.valueOf(789));
put("octStr", "0102030405");
put("utf8Str", "Some UTF-8 String. こんにちは世界。");
put("bitStr", "1011");
}};
Map<String, Object> expectedValues =
Map.of("b", true,
"i", BigInteger.valueOf(789),
"octStr", "0102030405",
"utf8Str", "Some UTF-8 String. こんにちは世界。",
"bitStr", "1011");
RecordSchema expectedSchema = new SimpleRecordSchema(Arrays.asList(
new RecordField("b", RecordFieldType.BOOLEAN.getDataType()),
@ -189,57 +186,37 @@ public class TestJASN1RecordReaderWithComplexTypes implements JASN1ReadRecordTes
return expectedSchema;
};
Function<Record, Map<String, Object>> expectedValuesProvider = __ -> new HashMap<String, Object>() {{
put(
Function<Record, Map<String, Object>> expectedValuesProvider = __ -> Map.of(
"child",
new MapRecord(expectedChildSchema, new HashMap<String, Object>() {{
put("b", true);
put("i", BigInteger.valueOf(789));
put("octStr", "0102030405");
}})
);
put(
new MapRecord(expectedChildSchema, Map.of("b", true,
"i", BigInteger.valueOf(789),
"octStr", "0102030405")),
"children",
new MapRecord[]{
new MapRecord(expectedChildSchema, new HashMap<String, Object>() {{
put("b", true);
put("i", BigInteger.valueOf(0));
put("octStr", "000000");
}}),
new MapRecord(expectedChildSchema, new HashMap<String, Object>() {{
put("b", false);
put("i", BigInteger.valueOf(1));
put("octStr", "010101");
}}),
new MapRecord(expectedChildSchema, new HashMap<String, Object>() {{
put("b", true);
put("i", BigInteger.valueOf(2));
put("octStr", "020202");
}})
}
);
put(
new MapRecord(expectedChildSchema, Map.of("b", true,
"i", BigInteger.valueOf(0),
"octStr", "000000")),
new MapRecord(expectedChildSchema, Map.of("b", false,
"i", BigInteger.valueOf(1),
"octStr", "010101")),
new MapRecord(expectedChildSchema, Map.of("b", true,
"i", BigInteger.valueOf(2),
"octStr", "020202"))
},
"unordered",
new MapRecord[]{
new MapRecord(expectedChildSchema, new HashMap<String, Object>() {{
put("b", true);
put("i", BigInteger.valueOf(0));
put("octStr", "000000");
}}),
new MapRecord(expectedChildSchema, new HashMap<String, Object>() {{
put("b", false);
put("i", BigInteger.valueOf(1));
put("octStr", "010101");
}})
}
);
put(
new MapRecord(expectedChildSchema, Map.of("b", true,
"i", BigInteger.valueOf(0),
"octStr", "000000")),
new MapRecord(expectedChildSchema, Map.of("b", false,
"i", BigInteger.valueOf(1),
"octStr", "010101"))
},
"numbers",
new BigInteger[]{
BigInteger.valueOf(0), BigInteger.valueOf(1), BigInteger.valueOf(2), BigInteger.valueOf(3),
}
);
}};
testReadRecord(dataFile, composite, expectedValuesProvider, expectedSchemaProvider);
}
@ -288,24 +265,16 @@ public class TestJASN1RecordReaderWithComplexTypes implements JASN1ReadRecordTes
expectedSchema.setSchemaName("Recursive");
expectedSchema.setSchemaNamespace("org.apache.nifi.jasn1.example");
Map<String, Object> expectedValues = new HashMap<String, Object>() {{
put("name", "name");
put("children", new MapRecord[]{
new MapRecord(expectedSchema, new HashMap<String, Object>() {{
put("name", "childName1");
put("children", new MapRecord[]{
new MapRecord(expectedSchema, new HashMap<String, Object>() {{
put("name", "grandChildName11");
put("children", new MapRecord[0]);
}})
});
}}),
new MapRecord(expectedSchema, new HashMap<String, Object>() {{
put("name", "childName2");
put("children", new MapRecord[0]);
}}),
Map<String, Object> expectedValues = Map.of("name", "name",
"children", new MapRecord[]{
new MapRecord(expectedSchema, Map.of("name", "childName1",
"children", new MapRecord[]{
new MapRecord(expectedSchema, Map.of("name", "grandChildName11",
"children", new MapRecord[0]))
})),
new MapRecord(expectedSchema, Map.of("name", "childName2",
"children", new MapRecord[0])),
});
}};
testReadRecord(dataFile, recursive, expectedValues, expectedSchema);
}
@ -323,10 +292,8 @@ public class TestJASN1RecordReaderWithComplexTypes implements JASN1ReadRecordTes
new RecordField("str", RecordFieldType.STRING.getDataType())
));
Map<String, Object> expectedValues = new HashMap<String, Object>() {{
put("i", BigInteger.valueOf(53286L));
put("str", "Some UTF-8 String. こんにちは世界。");
}};
Map<String, Object> expectedValues = Map.of("i", BigInteger.valueOf(53286L),
"str", "Some UTF-8 String. こんにちは世界。");
testReadRecord(dataFile, berValue, expectedValues, expectedSchema);
}

View File

@ -51,7 +51,6 @@ import java.time.LocalDate;
import java.time.LocalDateTime;
import java.time.LocalTime;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Map;
import static org.junit.jupiter.api.Assertions.assertTrue;
@ -68,9 +67,7 @@ public class TestJASN1RecordReaderWithSimpleTypes implements JASN1ReadRecordTest
BooleanWrapper berValue = new BooleanWrapper();
berValue.setValue(new BerBoolean(true));
Map<String, Object> expectedValues = new HashMap<String, Object>() {{
put("value", true);
}};
Map<String, Object> expectedValues = Map.of("value", true);
RecordSchema expectedSchema = new SimpleRecordSchema(Arrays.asList(
new RecordField("value", RecordFieldType.BOOLEAN.getDataType()))
@ -86,9 +83,7 @@ public class TestJASN1RecordReaderWithSimpleTypes implements JASN1ReadRecordTest
IntegerWrapper berValue = new IntegerWrapper();
berValue.setValue(new BerInteger(4321234));
Map<String, Object> expectedValues = new HashMap<String, Object>() {{
put("value", BigInteger.valueOf(4321234));
}};
Map<String, Object> expectedValues = Map.of("value", BigInteger.valueOf(4321234));
RecordSchema expectedSchema = new SimpleRecordSchema(Arrays.asList(
new RecordField("value", RecordFieldType.BIGINT.getDataType()))
@ -104,9 +99,7 @@ public class TestJASN1RecordReaderWithSimpleTypes implements JASN1ReadRecordTest
BitStringWrapper berValue = new BitStringWrapper();
berValue.setValue(new BerBitString(new boolean[]{false, true, false, false, true, true, true, true, false, true, false, false}));
Map<String, Object> expectedValues = new HashMap<String, Object>() {{
put("value", "010011110100");
}};
Map<String, Object> expectedValues = Map.of("value", "010011110100");
RecordSchema expectedSchema = new SimpleRecordSchema(Arrays.asList(
new RecordField("value", RecordFieldType.STRING.getDataType()))
@ -122,9 +115,8 @@ public class TestJASN1RecordReaderWithSimpleTypes implements JASN1ReadRecordTest
OctetStringWrapper berValue = new OctetStringWrapper();
berValue.setValue(new BerOctetString("0123456789ABCDEFGHIJKLMNopqrstuvwxyz".getBytes()));
Map<String, Object> expectedValues = new HashMap<String, Object>() {{
put("value", octetStringExpectedValueConverter("0123456789ABCDEFGHIJKLMNopqrstuvwxyz".getBytes()));
}};
Map<String, Object> expectedValues =
Map.of("value", octetStringExpectedValueConverter("0123456789ABCDEFGHIJKLMNopqrstuvwxyz".getBytes()));
RecordSchema expectedSchema = new SimpleRecordSchema(Arrays.asList(
new RecordField("value", RecordFieldType.STRING.getDataType()))
@ -140,9 +132,7 @@ public class TestJASN1RecordReaderWithSimpleTypes implements JASN1ReadRecordTest
UTF8StringWrapper berValue = new UTF8StringWrapper();
berValue.setValue(new BerUTF8String("Some UTF-8 String. こんにちは世界。"));
Map<String, Object> expectedValues = new HashMap<String, Object>() {{
put("value", "Some UTF-8 String. こんにちは世界。");
}};
Map<String, Object> expectedValues = Map.of("value", "Some UTF-8 String. こんにちは世界。");
RecordSchema expectedSchema = new SimpleRecordSchema(Arrays.asList(
new RecordField("value", RecordFieldType.STRING.getDataType()))
@ -158,9 +148,7 @@ public class TestJASN1RecordReaderWithSimpleTypes implements JASN1ReadRecordTest
BMPStringWrapper berValue = new BMPStringWrapper();
berValue.setValue(new BerBMPString("Some UTF-8 String. こんにちは世界。".getBytes(StandardCharsets.UTF_8)));
Map<String, Object> expectedValues = new HashMap<String, Object>() {{
put("value", "Some UTF-8 String. こんにちは世界。");
}};
Map<String, Object> expectedValues = Map.of("value", "Some UTF-8 String. こんにちは世界。");
RecordSchema expectedSchema = new SimpleRecordSchema(Arrays.asList(
new RecordField("value", RecordFieldType.STRING.getDataType()))
@ -176,9 +164,7 @@ public class TestJASN1RecordReaderWithSimpleTypes implements JASN1ReadRecordTest
DateWrapper berValue = new DateWrapper();
berValue.setValue(new BerDate("2019-10-16"));
Map<String, Object> expectedValues = new HashMap<String, Object>() {{
put("value", LocalDate.parse("2019-10-16"));
}};
Map<String, Object> expectedValues = Map.of("value", LocalDate.parse("2019-10-16"));
RecordSchema expectedSchema = new SimpleRecordSchema(Arrays.asList(
new RecordField("value", RecordFieldType.DATE.getDataType()))
@ -209,9 +195,7 @@ public class TestJASN1RecordReaderWithSimpleTypes implements JASN1ReadRecordTest
TimeOfDayWrapper berValue = new TimeOfDayWrapper();
berValue.setValue(new BerTimeOfDay("16:13:12"));
Map<String, Object> expectedValues = new HashMap<String, Object>() {{
put("value", LocalTime.parse("16:13:12"));
}};
Map<String, Object> expectedValues = Map.of("value", LocalTime.parse("16:13:12"));
RecordSchema expectedSchema = new SimpleRecordSchema(Arrays.asList(
new RecordField("value", RecordFieldType.TIME.getDataType()))
@ -242,9 +226,7 @@ public class TestJASN1RecordReaderWithSimpleTypes implements JASN1ReadRecordTest
DateTimeWrapper berValue = new DateTimeWrapper();
berValue.setValue(new BerDateTime("2019-10-16T16:18:20"));
Map<String, Object> expectedValues = new HashMap<String, Object>() {{
put("value", LocalDateTime.parse("2019-10-16T16:18:20"));
}};
Map<String, Object> expectedValues = Map.of("value", LocalDateTime.parse("2019-10-16T16:18:20"));
RecordSchema expectedSchema = new SimpleRecordSchema(Arrays.asList(
new RecordField("value", RecordFieldType.TIMESTAMP.getDataType("yyyy-MM-ddTHH:mm:ss")))
@ -275,9 +257,7 @@ public class TestJASN1RecordReaderWithSimpleTypes implements JASN1ReadRecordTest
RealWrapper berValue = new RealWrapper();
berValue.setValue(new BerReal(176.34D));
Map<String, Object> expectedValues = new HashMap<String, Object>() {{
put("value", 176.34D);
}};
Map<String, Object> expectedValues = Map.of("value", 176.34D);
RecordSchema expectedSchema = new SimpleRecordSchema(Arrays.asList(
new RecordField("value", RecordFieldType.DOUBLE.getDataType()))
@ -293,9 +273,7 @@ public class TestJASN1RecordReaderWithSimpleTypes implements JASN1ReadRecordTest
EnumeratedWrapper berValue = new EnumeratedWrapper();
berValue.setValue(new BerEnum(0));
Map<String, Object> expectedValues = new HashMap<String, Object>() {{
put("value", 0);
}};
Map<String, Object> expectedValues = Map.of("value", 0);
RecordSchema expectedSchema = new SimpleRecordSchema(Arrays.asList(
new RecordField("value", RecordFieldType.INT.getDataType()))

View File

@ -172,7 +172,7 @@ public class ExtractAvroMetadata extends AbstractProcessor {
@Override
public void process(InputStream rawIn) throws IOException {
try (final InputStream in = new BufferedInputStream(rawIn);
final DataFileStream<GenericRecord> reader = new DataFileStream<>(in, new GenericDatumReader<GenericRecord>())) {
final DataFileStream<GenericRecord> reader = new DataFileStream<>(in, new GenericDatumReader<>())) {
final Schema schema = reader.getSchema();
if (schema == null) {

View File

@ -270,7 +270,7 @@ public class SplitAvro extends AbstractProcessor {
@Override
public void process(InputStream rawIn) throws IOException {
try (final InputStream in = new BufferedInputStream(rawIn);
final DataFileStream<GenericRecord> reader = new DataFileStream<>(in, new GenericDatumReader<GenericRecord>())) {
final DataFileStream<GenericRecord> reader = new DataFileStream<>(in, new GenericDatumReader<>())) {
final AtomicReference<String> codec = new AtomicReference<>(reader.getMetaString(DataFileConstants.CODEC));
if (codec.get() == null) {
@ -278,7 +278,7 @@ public class SplitAvro extends AbstractProcessor {
}
// while records are left, start a new split by spawning a FlowFile
final AtomicReference<Boolean> hasNextHolder = new AtomicReference<Boolean>(reader.hasNext());
final AtomicReference<Boolean> hasNextHolder = new AtomicReference<>(reader.hasNext());
while (hasNextHolder.get()) {
FlowFile childFlowFile = session.create(originalFlowFile);
childFlowFile = session.write(childFlowFile, new OutputStreamCallback() {
@ -347,7 +347,7 @@ public class SplitAvro extends AbstractProcessor {
@Override
public void init(final DataFileStream<GenericRecord> reader, final String codec, final OutputStream out) throws IOException {
writer = new DataFileWriter<>(new GenericDatumWriter<GenericRecord>());
writer = new DataFileWriter<>(new GenericDatumWriter<>());
if (transferMetadata) {
for (String metaKey : reader.getMetaKeys()) {

View File

@ -79,7 +79,7 @@ public class TestSplitAvro {
userList.add(user);
}
try (final DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<>(new GenericDatumWriter<GenericRecord>(schema))) {
try (final DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<>(new GenericDatumWriter<>(schema))) {
dataFileWriter.setMeta(META_KEY1, META_VALUE1);
dataFileWriter.setMeta(META_KEY2, META_VALUE2);
dataFileWriter.setMeta(META_KEY3, META_VALUE3.getBytes("UTF-8"));
@ -112,7 +112,7 @@ public class TestSplitAvro {
final TestRunner runner = TestRunners.newTestRunner(new SplitAvro());
final String filename = "users.avro";
runner.enqueue(users.toByteArray(), new HashMap<String, String>() {{
runner.enqueue(users.toByteArray(), new HashMap<>() {{
put(CoreAttributes.FILENAME.key(), filename);
}});
runner.run();
@ -187,7 +187,7 @@ public class TestSplitAvro {
for (final MockFlowFile flowFile : flowFiles) {
try (final ByteArrayInputStream in = new ByteArrayInputStream(flowFile.toByteArray());
final DataFileStream<GenericRecord> reader = new DataFileStream<>(in, new GenericDatumReader<GenericRecord>())) {
final DataFileStream<GenericRecord> reader = new DataFileStream<>(in, new GenericDatumReader<>())) {
assertFalse(reader.getMetaKeys().contains(META_KEY1));
assertFalse(reader.getMetaKeys().contains(META_KEY2));
assertFalse(reader.getMetaKeys().contains(META_KEY3));
@ -303,7 +303,7 @@ public class TestSplitAvro {
private void checkDataFileSplitSize(List<MockFlowFile> flowFiles, int expectedRecordsPerSplit, boolean checkMetadata) throws IOException {
for (final MockFlowFile flowFile : flowFiles) {
try (final ByteArrayInputStream in = new ByteArrayInputStream(flowFile.toByteArray());
final DataFileStream<GenericRecord> reader = new DataFileStream<>(in, new GenericDatumReader<GenericRecord>())) {
final DataFileStream<GenericRecord> reader = new DataFileStream<>(in, new GenericDatumReader<>())) {
int count = 0;
GenericRecord record = null;
@ -328,7 +328,7 @@ public class TestSplitAvro {
int count = 0;
for (final MockFlowFile flowFile : flowFiles) {
try (final ByteArrayInputStream in = new ByteArrayInputStream(flowFile.toByteArray());
final DataFileStream<GenericRecord> reader = new DataFileStream<>(in, new GenericDatumReader<GenericRecord>())) {
final DataFileStream<GenericRecord> reader = new DataFileStream<>(in, new GenericDatumReader<>())) {
GenericRecord record = null;
while (reader.hasNext()) {

View File

@ -346,7 +346,7 @@ public class AWSCredentialsProviderControllerService extends AbstractControllerS
@Override
protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
final CredentialsStrategy selectedStrategy = selectPrimaryStrategy(validationContext);
final ArrayList<ValidationResult> validationFailureResults = new ArrayList<ValidationResult>();
final ArrayList<ValidationResult> validationFailureResults = new ArrayList<>();
for (CredentialsStrategy strategy : strategies) {
final Collection<ValidationResult> strategyValidationFailures = strategy.validate(validationContext,

View File

@ -695,12 +695,12 @@ public class ListS3 extends AbstractS3Processor implements VerifiableProcessor {
.stream()
.filter(s3VersionSummary -> s3VersionSummary.getLastModified().getTime() >= minTimestampToList
&& includeObjectInListing(s3VersionSummary, currentTime))
.map(s3VersionSummary -> new ListableEntityWrapper<S3VersionSummary>(
s3VersionSummary,
S3VersionSummary::getKey,
summary -> summary.getKey() + "_" + summary.getVersionId(),
summary -> summary.getLastModified().getTime(),
S3VersionSummary::getSize
.map(s3VersionSummary -> new ListableEntityWrapper<>(
s3VersionSummary,
S3VersionSummary::getKey,
summary -> summary.getKey() + "_" + summary.getVersionId(),
summary -> summary.getLastModified().getTime(),
S3VersionSummary::getSize
))
.collect(Collectors.toList());
}, null);

View File

@ -46,7 +46,6 @@ import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.function.Consumer;
@ -73,14 +72,13 @@ public class StandardS3EncryptionService extends AbstractControllerService imple
private static final AllowableValue CSE_KMS = new AllowableValue(STRATEGY_NAME_CSE_KMS, "Client-side KMS", "Use client-side, KMS key to perform encryption.");
private static final AllowableValue CSE_C = new AllowableValue(STRATEGY_NAME_CSE_C, "Client-side Customer Key", "Use client-side, customer-supplied key to perform encryption.");
public static final Map<String, AllowableValue> ENCRYPTION_STRATEGY_ALLOWABLE_VALUES = new HashMap<String, AllowableValue>() {{
put(STRATEGY_NAME_NONE, NONE);
put(STRATEGY_NAME_SSE_S3, SSE_S3);
put(STRATEGY_NAME_SSE_KMS, SSE_KMS);
put(STRATEGY_NAME_SSE_C, SSE_C);
put(STRATEGY_NAME_CSE_KMS, CSE_KMS);
put(STRATEGY_NAME_CSE_C, CSE_C);
}};
public static final Map<String, AllowableValue> ENCRYPTION_STRATEGY_ALLOWABLE_VALUES =
Map.of(STRATEGY_NAME_NONE, NONE,
STRATEGY_NAME_SSE_S3, SSE_S3,
STRATEGY_NAME_SSE_KMS, SSE_KMS,
STRATEGY_NAME_SSE_C, SSE_C,
STRATEGY_NAME_CSE_KMS, CSE_KMS,
STRATEGY_NAME_CSE_C, CSE_C);
public static final PropertyDescriptor ENCRYPTION_STRATEGY = new PropertyDescriptor.Builder()
.name("encryption-strategy")

View File

@ -142,7 +142,7 @@ public class PutAzureCosmosDBRecordTest extends MockTestBase {
final RecordSchema personSchema = new SimpleRecordSchema(personFields);
recordReader.addSchemaField("person", RecordFieldType.RECORD);
recordReader.addRecord("1", "A", new MapRecord(personSchema, new HashMap<String, Object>() {
recordReader.addRecord("1", "A", new MapRecord(personSchema, new HashMap<>() {
private static final long serialVersionUID = -3185956498135742190L;
{
put("name", "John Doe");
@ -150,7 +150,7 @@ public class PutAzureCosmosDBRecordTest extends MockTestBase {
put("sport", "Soccer");
}
}));
recordReader.addRecord("2", "B", new MapRecord(personSchema, new HashMap<String, Object>() {
recordReader.addRecord("2", "B", new MapRecord(personSchema, new HashMap<>() {
private static final long serialVersionUID = 1L;
{
put("name", "Jane Doe");
@ -158,7 +158,7 @@ public class PutAzureCosmosDBRecordTest extends MockTestBase {
put("sport", "Tennis");
}
}));
recordReader.addRecord("3", "A", new MapRecord(personSchema, new HashMap<String, Object>() {
recordReader.addRecord("3", "A", new MapRecord(personSchema, new HashMap<>() {
private static final long serialVersionUID = -1329194249439570573L;
{
put("name", "Sally Doe");
@ -166,7 +166,7 @@ public class PutAzureCosmosDBRecordTest extends MockTestBase {
put("sport", "Curling");
}
}));
recordReader.addRecord("4", "C", new MapRecord(personSchema, new HashMap<String, Object>() {
recordReader.addRecord("4", "C", new MapRecord(personSchema, new HashMap<>() {
private static final long serialVersionUID = -1329194249439570574L;
{
put("name", "Jimmy Doe");

View File

@ -64,7 +64,7 @@ public class TestAzureLogAnalyticsProvenanceReportingTask {
final Map<String, Object> config = Collections.emptyMap();
final JsonBuilderFactory factory = Json.createBuilderFactory(config);
final JsonObjectBuilder builder = factory.createObjectBuilder();
Map<String, String> values = new LinkedHashMap<String, String>();
Map<String, String> values = new LinkedHashMap<>();
values.put("TestKeyString1", "StringValue1");
values.put("TestKeyString2", "StringValue2");
AzureLogAnalyticsProvenanceReportingTask.addField(builder, factory, "TestKeyString", values, true);
@ -80,7 +80,7 @@ public class TestAzureLogAnalyticsProvenanceReportingTask {
final Map<String, Object> config = Collections.emptyMap();
final JsonBuilderFactory factory = Json.createBuilderFactory(config);
final JsonObjectBuilder builder = factory.createObjectBuilder();
Collection<String> values = new ArrayList<String>();
Collection<String> values = new ArrayList<>();
values.add("TestValueString1");
values.add("TestValueString2");
AzureLogAnalyticsProvenanceReportingTask.addField(builder, factory, "TestKeyString", values, true);

View File

@ -17,7 +17,6 @@
package org.apache.nifi.elasticsearch;
import java.util.HashSet;
import java.util.Set;
public class ElasticsearchException extends RuntimeException {
@ -25,12 +24,8 @@ public class ElasticsearchException extends RuntimeException {
* These are names of common Elasticsearch exceptions where it is safe to assume
* that it's OK to retry the operation instead of just sending it to an error relationship.
*/
public static final Set<String> ELASTIC_ERROR_NAMES = new HashSet<String>() {{
add("NoNodeAvailableException");
add("ElasticsearchTimeoutException");
add("ReceiveTimeoutTransportException");
add("NodeClosedException");
}};
public static final Set<String> ELASTIC_ERROR_NAMES = Set.of("NoNodeAvailableException",
"ElasticsearchTimeoutException", "ReceiveTimeoutTransportException", "NodeClosedException");
protected boolean elastic;

View File

@ -196,13 +196,8 @@ public class ElasticSearchLookupService extends JsonInferenceSchemaRegistryServi
@SuppressWarnings("unchecked")
private Record getById(final String _id, final Map<String, String> context) throws IOException, LookupFailureException, SchemaNotFoundException {
final Map<String, Object> query = new HashMap<String, Object>() {{
put("query", new HashMap<String, Object>() {{
put("match", new HashMap<String, String>() {{
put("_id", _id);
}});
}});
}};
final Map<String, Object> query = Map.of(
"query", Map.of("match", Map.of("_id", _id)));
final String json = mapper.writeValueAsString(query);
@ -229,37 +224,27 @@ public class ElasticSearchLookupService extends JsonInferenceSchemaRegistryServi
Map<String, Object> getNested(final String key, final Object value) {
final String path = key.substring(0, key.lastIndexOf("."));
return new HashMap<String, Object>() {{
put("path", path);
put("query", new HashMap<String, Object>() {{
put("match", new HashMap<String, Object>() {{
put(key, value);
}});
}});
}};
return Map.of("path", path, "query", Map.of("match", Map.of(key, value)));
}
private Map<String, Object> buildQuery(final Map<String, Object> coordinates) {
final Map<String, Object> query = new HashMap<String, Object>() {{
final Map<String, Object> query = new HashMap<>() {{
put("bool", new HashMap<String, Object>() {{
put("must", coordinates.entrySet().stream()
.map(e -> new HashMap<String, Object>() {{
if (e.getKey().contains(".")) {
put("nested", getNested(e.getKey(), e.getValue()));
} else {
put("match", new HashMap<String, Object>() {{
put(e.getKey(), e.getValue());
}});
}
}}).collect(Collectors.toList())
.map(e -> new HashMap<String, Object>() {{
if (e.getKey().contains(".")) {
put("nested", getNested(e.getKey(), e.getValue()));
} else {
put("match", new HashMap<String, Object>() {{
put(e.getKey(), e.getValue());
}});
}
}}).collect(Collectors.toList())
);
}});
}};
return new HashMap<String, Object>() {{
put("size", 1);
put("query", query);
}};
return Map.of("size", 1, "query", query);
}
@SuppressWarnings("unchecked")

View File

@ -260,16 +260,15 @@ public abstract class AbstractJsonQueryElasticsearch<Q extends JsonQueryParamete
final ProcessSession session, final FlowFile aggFlowFile,
final Map<String, String> attributes) {
FlowFile ff = session.write(aggFlowFile, out -> out.write(json.getBytes()));
ff = session.putAllAttributes(ff, new HashMap<String, String>() {{
if (name != null) {
put("aggregation.name", name);
}
if (number != null) {
put("aggregation.number", number.toString());
}
}});
Map<String, String> latestAttributes = new HashMap<>(attributes);
if (name != null) {
latestAttributes.put("aggregation.name", name);
}
if (number != null) {
latestAttributes.put("aggregation.number", number.toString());
}
return session.putAllAttributes(ff, attributes);
return session.putAllAttributes(ff, latestAttributes);
}
private void handleAggregations(final Map<String, Object> aggregations, final ProcessSession session,

View File

@ -279,7 +279,7 @@ public class ConsumeElasticsearch extends SearchElasticsearch {
filters.addAll(mapper.convertValue(additionalFilters, new TypeReference<List<Map<String, Object>>>() {
}));
} else {
filters.add(mapper.convertValue(additionalFilters, new TypeReference<Map<String, Object>>() {
filters.add(mapper.convertValue(additionalFilters, new TypeReference<>() {
}));
}
}

View File

@ -311,9 +311,11 @@ public interface ElasticsearchRestProcessor extends Processor, VerifiableProcess
final List<Map<String, Object>> sortList;
final JsonNode sort = mapper.readTree(context.getProperty(SORT).evaluateAttributeExpressions(attributes).getValue());
if (sort.isArray()) {
sortList = mapper.convertValue(sort, new TypeReference<List<Map<String, Object>>>() { });
sortList = mapper.convertValue(sort, new TypeReference<>() {
});
} else {
sortList = Collections.singletonList(mapper.convertValue(sort, new TypeReference<Map<String, Object>>() { }));
sortList = Collections.singletonList(mapper.convertValue(sort, new TypeReference<>() {
}));
}
query.put("sort", new ArrayList<>(sortList));
}

View File

@ -586,7 +586,8 @@ public abstract class AbstractJsonQueryElasticsearchTest<P extends AbstractJsonQ
}
if (QueryDefinitionType.BUILD_QUERY.getValue().equals(runner.getProcessContext().getProperty(ElasticsearchRestProcessor.QUERY_DEFINITION_STYLE).getValue())) {
final Map<String, Object> queryMap = TEST_MAPPER.readValue(query, new TypeReference<Map<String, Object>>() { });
final Map<String, Object> queryMap = TEST_MAPPER.readValue(query, new TypeReference<>() {
});
if (queryMap.containsKey("query")) {
if (runner.getProcessor() instanceof ConsumeElasticsearch) {
runner.setProperty(ConsumeElasticsearch.RANGE_FIELD, RANGE_FIELD_NAME);

View File

@ -38,7 +38,7 @@ public class BinProcessingResult {
public BinProcessingResult(boolean isCommitted) {
this.setCommitted(isCommitted);
this.setAttributes(new HashMap<String, String>());
this.setAttributes(new HashMap<>());
}
public BinProcessingResult(boolean isCommitted, Map<String, String> attributes) {

View File

@ -174,7 +174,7 @@ public abstract class AbstractListenEventBatchingProcessor<E extends Event> exte
// if we don't have a batch for this key then create a new one
if (batch == null) {
batch = new FlowFileEventBatch(session.create(), new ArrayList<E>());
batch = new FlowFileEventBatch(session.create(), new ArrayList<>());
batches.put(batchKey, batch);
}

View File

@ -72,7 +72,7 @@ public abstract class GetFileTransfer extends AbstractProcessor {
private final AtomicLong lastPollTime = new AtomicLong(-1L);
private final Lock listingLock = new ReentrantLock();
private final AtomicReference<BlockingQueue<FileInfo>> fileQueueRef = new AtomicReference<>();
private final Set<FileInfo> processing = Collections.synchronizedSet(new HashSet<FileInfo>());
private final Set<FileInfo> processing = Collections.synchronizedSet(new HashSet<>());
// Used when transferring filenames from the File Queue to the processing queue; multiple threads can do this
// simultaneously using the sharableTransferLock; however, in order to check if either has a given file, the
@ -288,7 +288,7 @@ public abstract class GetFileTransfer extends AbstractProcessor {
BlockingQueue<FileInfo> queue = fileQueueRef.get();
if (queue == null) {
final boolean useNaturalOrdering = context.getProperty(FileTransfer.USE_NATURAL_ORDERING).asBoolean();
queue = useNaturalOrdering ? new PriorityBlockingQueue<FileInfo>(25000) : new LinkedBlockingQueue<FileInfo>(25000);
queue = useNaturalOrdering ? new PriorityBlockingQueue<>(25000) : new LinkedBlockingQueue<>(25000);
fileQueueRef.set(queue);
}

View File

@ -66,7 +66,7 @@ public enum CompressionType {
}
public static AllowableValue[] allowableValues() {
List<AllowableValue> values = new ArrayList<AllowableValue>();
List<AllowableValue> values = new ArrayList<>();
for (CompressionType type : CompressionType.values()) {
values.add(new AllowableValue(type.name(), type.name(), type.getDescription()));
}

View File

@ -471,7 +471,7 @@ public abstract class AbstractListProcessor<T extends ListableEntity> extends Ab
final TreeMap<Long, List<T>> orderedEntries = new TreeMap<>();
for (final T entity : entityList) {
List<T> entitiesForTimestamp = orderedEntries.computeIfAbsent(entity.getTimestamp(), k -> new ArrayList<T>());
List<T> entitiesForTimestamp = orderedEntries.computeIfAbsent(entity.getTimestamp(), k -> new ArrayList<>());
entitiesForTimestamp.add(entity);
}
@ -678,11 +678,7 @@ public abstract class AbstractListProcessor<T extends ListableEntity> extends Ab
final boolean newEntry = minTimestampToListMillis == null || entityTimestampMillis >= minTimestampToListMillis && entityTimestampMillis >= lastProcessedLatestEntryTimestampMillis;
if (newEntry) {
List<T> entitiesForTimestamp = orderedEntries.get(entity.getTimestamp());
if (entitiesForTimestamp == null) {
entitiesForTimestamp = new ArrayList<T>();
orderedEntries.put(entity.getTimestamp(), entitiesForTimestamp);
}
List<T> entitiesForTimestamp = orderedEntries.computeIfAbsent(entity.getTimestamp(), k -> new ArrayList<>());
entitiesForTimestamp.add(entity);
}
}

View File

@ -791,9 +791,7 @@ public class TestAvroTypeUtil {
Map<String, Object> obj = new HashMap<>();
List<Map<String, Object>> list = new ArrayList<>();
for (int x = 0; x < 10; x++) {
list.add(new HashMap<String, Object>() {{
put("Message", UUID.randomUUID().toString());
}});
list.add(Map.of("Message", UUID.randomUUID().toString()));
}
obj.put("List", list);
@ -863,15 +861,8 @@ public class TestAvroTypeUtil {
@Test
public void testConvertAvroMap() {
// GIVEN
Map<?, ?> expected = new HashMap<String, Object>() {{
put(
"nullableMapField",
new HashMap<String, Object>() {{
put("key1", "value1");
put("key2", "value2");
}}
);
}};
Map<?, ?> expected = Map.of(
"nullableMapField", Map.of("key1", "value1", "key2", "value2"));
Schema nullableMapFieldAvroSchema = Schema.createUnion(
Schema.create(Type.NULL),
@ -886,11 +877,7 @@ public class TestAvroTypeUtil {
)
);
Map<?, ?> value = new HashMap<Utf8, Object>() {{
put(new Utf8("key1"), "value1");
put(new Utf8("key2"), "value2");
}};
Map<?, ?> value = Map.of(new Utf8("key1"), "value1", new Utf8("key2"), "value2");
Record avroRecord = new GenericRecordBuilder(avroRecordSchema)
.set("nullableMapField", value)
.build();

View File

@ -151,7 +151,7 @@ public abstract class AbstractBigQueryProcessor extends AbstractGCPProcessor<Big
@Override
protected final Collection<ValidationResult> customValidate(ValidationContext validationContext) {
final Collection<ValidationResult> results = new ArrayList<ValidationResult>(super.customValidate(validationContext));
final Collection<ValidationResult> results = new ArrayList<>(super.customValidate(validationContext));
ProxyConfiguration.validateProxySpec(validationContext, results, ProxyAwareTransportFactory.PROXY_SPECS);
final boolean projectId = validationContext.getProperty(PROJECT_ID).isSet();

View File

@ -45,7 +45,7 @@ import java.util.Map;
*/
public class CredentialsFactory {
private final List<CredentialsStrategy> strategies = new ArrayList<CredentialsStrategy>();
private final List<CredentialsStrategy> strategies = new ArrayList<>();
public CredentialsFactory() {
// Primary Credential Strategies
@ -78,7 +78,7 @@ public class CredentialsFactory {
*/
public Collection<ValidationResult> validate(final ValidationContext validationContext) {
final CredentialsStrategy selectedStrategy = selectPrimaryStrategy(validationContext);
final ArrayList<ValidationResult> validationFailureResults = new ArrayList<ValidationResult>();
final ArrayList<ValidationResult> validationFailureResults = new ArrayList<>();
for (CredentialsStrategy strategy : strategies) {
final Collection<ValidationResult> strategyValidationFailures = strategy.validate(validationContext,

View File

@ -54,7 +54,7 @@ public abstract class AbstractBooleanCredentialsStrategy extends AbstractCredent
Boolean useStrategy = validationContext.getProperty(strategyProperty).asBoolean();
if (!thisIsSelectedStrategy && (useStrategy == null ? false : useStrategy)) {
String failureFormat = "property %1$s cannot be used with %2$s";
Collection<ValidationResult> validationFailureResults = new ArrayList<ValidationResult>();
Collection<ValidationResult> validationFailureResults = new ArrayList<>();
String message = String.format(failureFormat, strategyProperty.getDisplayName(),
primaryStrategy.getName());
validationFailureResults.add(new ValidationResult.Builder()

View File

@ -69,11 +69,9 @@ public class FetchGoogleDriveIT extends AbstractGoogleDriveIT<FetchGoogleDrive>
inputFlowFileAttributes.put(GoogleDriveAttributes.FILENAME, "missing_filename");
Set<Map<String, String>> expectedFailureAttributes = new HashSet<>(singletonList(
new HashMap<String, String>() {{
put(GoogleDriveAttributes.ID, "missing");
put(GoogleDriveAttributes.FILENAME, "missing_filename");
put(GoogleDriveAttributes.ERROR_CODE, "404");
}}
Map.of(GoogleDriveAttributes.ID, "missing",
GoogleDriveAttributes.FILENAME, "missing_filename",
GoogleDriveAttributes.ERROR_CODE, "404")
));
testRunner.enqueue("unimportant_data", inputFlowFileAttributes);

View File

@ -88,13 +88,7 @@ public class ExecuteGraphQueryRecordTest {
setupGraphClient(false);
List<Map<String, Object>> test = new ArrayList<>();
Map<String, Object> tempMap = new HashMap<>();
tempMap.put("M", new ArrayList<Integer>() {
{
add(1);
add(2);
add(3);
}
});
tempMap.put("M", List.of(1, 2, 3));
test.add(tempMap);
byte[] json = JsonOutput.toJson(test).getBytes();

View File

@ -28,14 +28,8 @@ public class MockCypherClientService extends AbstractControllerService implement
@Override
public Map<String, String> executeQuery(String query, Map<String, Object> parameters, GraphQueryResultCallback handler) {
handler.process(new HashMap<String, Object>() {{
put("name", "John Smith");
put("age", 40);
}}, true);
handler.process(new HashMap<String, Object>() {{
put("name", "John Smith");
put("age", 40);
}}, false);
handler.process(Map.of("name", "John Smith", "age", 40), true);
handler.process(Map.of("name", "John Smith", "age", 40), false);
Map<String, String> resultAttributes = new HashMap<>();
resultAttributes.put(NODES_CREATED, String.valueOf(1));

View File

@ -24,7 +24,6 @@ import org.apache.nifi.util.TestRunners;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -63,9 +62,7 @@ public class TestExecuteGraphQuery {
@Test
public void testExecuteFromParameterWithEL() throws Exception {
runner.setProperty(AbstractGraphExecutor.QUERY, "${query}");
runner.enqueue("test-data", new HashMap<String, String>() {{
put("query", "MATCH (p:person) RETURN p");
}});
runner.enqueue("test-data", Map.of("query", "MATCH (p:person) RETURN p"));
testExecute(1, 0, 1);
}

View File

@ -94,7 +94,7 @@ public class InMemoryGraphClient extends AbstractControllerService implements Gr
Map.Entry<String, Object> tempResult = (Map.Entry<String, Object>) resultSet.next();
Map<String, Object> tempRetObject = new HashMap<>();
tempRetObject.put(tempResult.getKey(), tempResult.getValue());
SimpleEntry returnObject = new SimpleEntry<String, Object>(tempResult.getKey(), tempRetObject);
SimpleEntry<String, Object> returnObject = new SimpleEntry<>(tempResult.getKey(), tempRetObject);
Map<String, Object> resultReturnMap = new HashMap<>();
resultReturnMap.put(innerResultSet.getKey(), returnObject);
if (getLogger().isDebugEnabled()) {

View File

@ -172,7 +172,7 @@ public class TestGetHDFSFileInfo {
runner.setProperty(GetHDFSFileInfo.IGNORE_DOTTED_DIRS, "true");
runner.setProperty(GetHDFSFileInfo.IGNORE_DOTTED_FILES, "true");
runner.setProperty(GetHDFSFileInfo.DESTINATION, GetHDFSFileInfo.DESTINATION_CONTENT);
runner.enqueue("foo", new HashMap<String, String>());
runner.enqueue("foo", new HashMap<>());
runner.run();

View File

@ -242,7 +242,7 @@ public class ExtractHL7Attributes extends AbstractProcessor {
private static Map<String, Segment> getAllSegments(final Group group) throws HL7Exception {
final Map<String, Segment> segments = new TreeMap<>();
addSegments(group, segments, new HashMap<String, Integer>());
addSegments(group, segments, new HashMap<>());
return Collections.unmodifiableMap(segments);
}

View File

@ -133,7 +133,7 @@ public class JMSConnectionFactoryHandler extends CachedJMSConnectionFactoryHandl
} else {
String[] brokerList = brokerValue.split(",");
if (connectionFactoryValue.startsWith("com.ibm.mq.jms")) {
List<String> ibmConList = new ArrayList<String>();
List<String> ibmConList = new ArrayList<>();
for (String broker : brokerList) {
String[] hostPort = broker.split(":");
if (hostPort.length == 2) {

View File

@ -376,7 +376,7 @@ public class ConsumeJMSIT {
ActiveMQConnectionFactory cf = new ActiveMQConnectionFactory("validateNIFI7034://127.0.0.1:" + port);
final String destinationName = "nifi7034";
final AtomicReference<TcpTransport> tcpTransport = new AtomicReference<TcpTransport>();
final AtomicReference<TcpTransport> tcpTransport = new AtomicReference<>();
TcpTransportFactory.registerTransportFactory("validateNIFI7034", new TcpTransportFactory() {
@Override
protected TcpTransport createTcpTransport(WireFormat wf, SocketFactory socketFactory, URI location, URI localLocation) throws UnknownHostException, IOException {

View File

@ -178,8 +178,10 @@ public class JMSPublisherConsumerIT {
ObjectMapper objectMapper = new ObjectMapper();
try {
Map<String, Object> actual = objectMapper.readValue(response.getMessageBody(), new TypeReference<Map<String, Object>>() { });
Map<String, Object> expected = objectMapper.readValue(expectedJson.getBytes(), new TypeReference<Map<String, Object>>() { });
Map<String, Object> actual = objectMapper.readValue(response.getMessageBody(), new TypeReference<>() {
});
Map<String, Object> expected = objectMapper.readValue(expectedJson.getBytes(), new TypeReference<>() {
});
assertEquals(expected, actual);
} catch (IOException e) {

View File

@ -53,7 +53,6 @@ import javax.net.SocketFactory;
import java.io.IOException;
import java.lang.reflect.Proxy;
import java.net.URI;
import java.net.UnknownHostException;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -392,10 +391,10 @@ public class PublishJMSIT {
ActiveMQConnectionFactory cf = new ActiveMQConnectionFactory("validateNIFI7034://127.0.0.1:" + port);
final String destinationName = "nifi7034";
final AtomicReference<TcpTransport> tcpTransport = new AtomicReference<TcpTransport>();
final AtomicReference<TcpTransport> tcpTransport = new AtomicReference<>();
TcpTransportFactory.registerTransportFactory("validateNIFI7034", new TcpTransportFactory() {
@Override
protected TcpTransport createTcpTransport(WireFormat wf, SocketFactory socketFactory, URI location, URI localLocation) throws UnknownHostException, IOException {
protected TcpTransport createTcpTransport(WireFormat wf, SocketFactory socketFactory, URI location, URI localLocation) throws IOException {
TcpTransport transport = super.createTcpTransport(wf, socketFactory, location, localLocation);
tcpTransport.set(transport);
return transport;

View File

@ -363,34 +363,14 @@ public class TestJoltTransformRecord {
RecordField xRecord = new RecordField("x", RecordFieldType.ARRAY.getArrayDataType(RecordFieldType.RECORD.getRecordDataType(xSchema)));
parser.addSchemaField(xRecord);
final Record record1 = new MapRecord(xSchema, new HashMap<String, Object>() {{
put("a", 1);
put("b", 2);
put("c", 3);
}});
final Record record2 = new MapRecord(xSchema, new HashMap<String, Object>() {{
put("a", 11);
put("b", 21);
put("c", 31);
}});
final Record record3 = new MapRecord(xSchema, new HashMap<String, Object>() {{
put("a", 21);
put("b", 2);
put("c", 3);
}});
final Record record1 = new MapRecord(xSchema, Map.of("a", 1, "b", 2, "c", 3));
final Record record2 = new MapRecord(xSchema, Map.of("a", 11, "b", 21, "c", 31));
final Record record3 = new MapRecord(xSchema, Map.of("a", 21, "b", 2, "c", 3));
final Object[] recordArray1 = new Object[]{record1, record2, record3};
parser.addRecord((Object) recordArray1);
final Record record4 = new MapRecord(xSchema, new HashMap<String, Object>() {{
put("a", 100);
put("b", 200);
put("c", 300);
}});
final Record record5 = new MapRecord(xSchema, new HashMap<String, Object>() {{
put("a", 101);
put("b", 201);
put("c", 301);
}});
final Record record4 = new MapRecord(xSchema, Map.of("a", 100, "b", 200, "c", 300));
final Record record5 = new MapRecord(xSchema, Map.of("a", 101, "b", 201, "c", 301));
final Object[] recordArray2 = new Object[]{record4, record5};
parser.addRecord((Object) recordArray2);
@ -759,22 +739,12 @@ public class TestJoltTransformRecord {
for (int i = 0; i < numRecords; i++) {
final int index = i;
final Record primaryRecord = new MapRecord(primarySchema, new HashMap<String, Object>() {{
put("value", (10 * index) + 3);
}});
final Record seriesRecord = new MapRecord(seriesSchema, new HashMap<String, Object>() {{
put("value", new Integer[]{(10 * index) + 5, (10 * index) + 4});
}});
final Record qualityRecord = new MapRecord(qualitySchema, new HashMap<String, Object>() {{
put("value", 3);
}});
final Record primaryRecord = new MapRecord(primarySchema, Map.of("value", (10 * index) + 3));
final Record seriesRecord = new MapRecord(seriesSchema, Map.of("value", new Integer[]{(10 * index) + 5, (10 * index) + 4}));
final Record qualityRecord = new MapRecord(qualitySchema, Map.of("value", 3));
Record ratingRecord = new MapRecord(ratingSchema, new HashMap<String, Object>() {{
put("primary", primaryRecord);
put("series", seriesRecord);
put("quality", qualityRecord);
}});
Record ratingRecord = new MapRecord(ratingSchema, Map.of("primary", primaryRecord,
"series", seriesRecord, "quality", qualityRecord));
parser.addRecord(ratingRecord);
}

View File

@ -54,7 +54,7 @@ public class TransformJSONResource extends AbstractStandardResource {
if (evaluateAttributes) {
PreparedQuery preparedQuery = Query.prepare(specificationDTO.getSpecification());
Map<String, String> attributes = specificationDTO.getExpressionLanguageAttributes() == null ? Collections.unmodifiableMap(new HashMap<String, String>())
Map<String, String> attributes = specificationDTO.getExpressionLanguageAttributes() == null ? Collections.unmodifiableMap(new HashMap<>())
: specificationDTO.getExpressionLanguageAttributes();
specification = preparedQuery.evaluateExpressions(new StandardEvaluationContext(attributes), null);
} else {

View File

@ -49,7 +49,6 @@ import org.bson.types.ObjectId;
import java.io.OutputStream;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
@ -161,9 +160,7 @@ public class GetMongoRecord extends AbstractMongoQueryProcessor {
try {
final Map<String, String> attributes = getAttributes(context, input, query, mongoCollection);
try (OutputStream out = session.write(output)) {
Map<String, String> attrs = inputPtr != null ? inputPtr.getAttributes() : new HashMap<String, String>() {{
put("schema.name", schemaName);
}};
Map<String, String> attrs = inputPtr != null ? inputPtr.getAttributes() : Map.of("schema.name", schemaName);
RecordSchema schema = writerFactory.getSchema(attrs, null);
RecordSetWriter writer = writerFactory.createWriter(getLogger(), schema, out, attrs);
long count = 0L;

View File

@ -553,9 +553,7 @@ public class GetMongoIT extends AbstractMongoIT {
//Test a bad flowfile attribute
runner.setIncomingConnection(true);
runner.setProperty(GetMongo.QUERY, "${badfromff}");
runner.enqueue("<<?>>", new HashMap<String, String>() {{
put("badfromff", "{\"prop\":}");
}});
runner.enqueue("<<?>>", Map.of("badfromff", "{\"prop\":}"));
runner.run();
runner.assertTransferCount(GetMongo.REL_FAILURE, 1);
runner.assertTransferCount(GetMongo.REL_SUCCESS, 0);
@ -566,9 +564,7 @@ public class GetMongoIT extends AbstractMongoIT {
//Test for regression on a good query from a flowfile attribute
runner.setIncomingConnection(true);
runner.setProperty(GetMongo.QUERY, "${badfromff}");
runner.enqueue("<<?>>", new HashMap<String, String>() {{
put("badfromff", "{}");
}});
runner.enqueue("<<?>>", Map.of("badfromff", "{}"));
runner.run();
runner.assertTransferCount(GetMongo.REL_FAILURE, 0);
runner.assertTransferCount(GetMongo.REL_SUCCESS, 3);

View File

@ -160,26 +160,17 @@ public class PutMongoRecordIT extends MongoWriteTestBase {
personFields.add(sportField);
final RecordSchema personSchema = new SimpleRecordSchema(personFields);
recordReader.addSchemaField("person", RecordFieldType.RECORD);
recordReader.addRecord(1, new MapRecord(personSchema, new HashMap<String, Object>() {{
put("name", "John Doe");
put("age", 48);
put("sport", "Soccer");
}}));
recordReader.addRecord(2, new MapRecord(personSchema, new HashMap<String, Object>() {{
put("name", "Jane Doe");
put("age", 47);
put("sport", "Tennis");
}}));
recordReader.addRecord(3, new MapRecord(personSchema, new HashMap<String, Object>() {{
put("name", "Sally Doe");
put("age", 47);
put("sport", "Curling");
}}));
recordReader.addRecord(4, new MapRecord(personSchema, new HashMap<String, Object>() {{
put("name", "Jimmy Doe");
put("age", 14);
put("sport", null);
}}));
recordReader.addRecord(1, new MapRecord(personSchema, Map.of("name", "John Doe",
"age", 48, "sport", "Soccer")));
recordReader.addRecord(2, new MapRecord(personSchema, Map.of("name", "Jane Doe", "age",
47, "sport", "Tennis")));
recordReader.addRecord(3, new MapRecord(personSchema, Map.of("name", "Sally Doe",
"age", 47, "sport", "Curling")));
Map<String, Object> mapWithNullValue = new HashMap<>();
mapWithNullValue.put("name", "Jimmy Doe");
mapWithNullValue.put("age", 14);
mapWithNullValue.put("sport", null);
recordReader.addRecord(4, new MapRecord(personSchema, mapWithNullValue));
runner.enqueue("");
runner.run();
@ -230,35 +221,16 @@ public class PutMongoRecordIT extends MongoWriteTestBase {
new RecordField("age", RecordFieldType.INT.getDataType())
));
List<List<Object[]>> inputs = Arrays.asList(
Arrays.asList(
new Object[]{1, new MapRecord(personSchema, new HashMap<String, Object>() {{
put("name", "name1");
put("age", 21);
}})},
new Object[]{2, new MapRecord(personSchema, new HashMap<String, Object>() {{
put("name", "name2");
put("age", 22);
}})}
)
List<List<Object[]>> inputs = List.of(
List.of(
new Object[]{1, new MapRecord(personSchema, Map.of("name", "name1", "age", 21))},
new Object[]{2, new MapRecord(personSchema, Map.of("name", "name2", "age", 22))}
)
);
Set<Map<String, Object>> expected = new HashSet<>(Arrays.asList(
new HashMap<String, Object>() {{
put("id", 1);
put("person", new Document(new HashMap<String, Object>() {{
put("name", "name1");
put("age", 21);
}}));
}},
new HashMap<String, Object>() {{
put("id", 2);
put("person", new Document(new HashMap<String, Object>() {{
put("name", "name2");
put("age", 22);
}}));
}}
));
Set<Map<String, Object>> expected = Set.of(
Map.of("id", 1, "person", new Document(Map.of("name", "name1", "age", 21))),
Map.of("id", 2, "person", new Document(Map.of("name", "name2", "age", 22))));
testUpsertSuccess(runner, inputs, expected);
}
@ -279,43 +251,18 @@ public class PutMongoRecordIT extends MongoWriteTestBase {
List<List<Object[]>> inputs = Arrays.asList(
Arrays.asList(
new Object[]{1, new MapRecord(personSchema, new HashMap<String, Object>() {{
put("name", "updating_name1");
put("age", "age1".length());
}})},
new Object[]{2, new MapRecord(personSchema, new HashMap<String, Object>() {{
put("name", "name2");
put("age", "updating_age2".length());
}})}
new Object[]{1, new MapRecord(personSchema, Map.of("name", "updating_name1", "age", "age1".length()))},
new Object[]{2, new MapRecord(personSchema, Map.of("name", "name2", "age", "updating_age2".length()))}
),
Arrays.asList(
new Object[]{1, new MapRecord(personSchema, new HashMap<String, Object>() {{
put("name", "updated_name1");
put("age", "age1".length());
}})},
new Object[]{2, new MapRecord(personSchema, new HashMap<String, Object>() {{
put("name", "name2");
put("age", "updated_age2".length());
}})}
new Object[]{1, new MapRecord(personSchema, Map.of("name", "updated_name1", "age", "age1".length()))},
new Object[]{2, new MapRecord(personSchema, Map.of("name", "name2", "age", "updated_age2".length()))}
)
);
Set<Map<String, Object>> expected = new HashSet<>(Arrays.asList(
new HashMap<String, Object>() {{
put("id", 1);
put("person", new Document(new HashMap<String, Object>() {{
put("name", "updated_name1");
put("age", "age1".length());
}}));
}},
new HashMap<String, Object>() {{
put("id", 2);
put("person", new Document(new HashMap<String, Object>() {{
put("name", "name2");
put("age", "updated_age2".length());
}}));
}}
));
Map.of("id", 1, "person", new Document(Map.of("name", "updated_name1", "age", "age1".length()))),
Map.of("id", 2, "person", new Document(Map.of("name", "name2", "age", "updated_age2".length())))));
testUpsertSuccess(runner, inputs, expected);
}
@ -336,39 +283,17 @@ public class PutMongoRecordIT extends MongoWriteTestBase {
List<List<Object[]>> inputs = Arrays.asList(
Collections.singletonList(
new Object[]{1, new MapRecord(personSchema, new HashMap<String, Object>() {{
put("name", "updating_name1");
put("age", "updating_age1".length());
}})}
new Object[]{1, new MapRecord(personSchema, Map.of("name", "updating_name1", "age", "updating_age1".length()))}
),
Arrays.asList(
new Object[]{1, new MapRecord(personSchema, new HashMap<String, Object>() {{
put("name", "updated_name1");
put("age", "updated_age1".length());
}})},
new Object[]{2, new MapRecord(personSchema, new HashMap<String, Object>() {{
put("name", "inserted_name2");
put("age", "inserted_age2".length());
}})}
new Object[]{1, new MapRecord(personSchema, Map.of("name", "updated_name1", "age", "updated_age1".length()))},
new Object[]{2, new MapRecord(personSchema, Map.of("name", "inserted_name2", "age", "inserted_age2".length()))}
)
);
Set<Map<String, Object>> expected = new HashSet<>(Arrays.asList(
new HashMap<String, Object>() {{
put("id", 1);
put("person", new Document(new HashMap<String, Object>() {{
put("name", "updated_name1");
put("age", "updated_age1".length());
}}));
}},
new HashMap<String, Object>() {{
put("id", 2);
put("person", new Document(new HashMap<String, Object>() {{
put("name", "inserted_name2");
put("age", "inserted_age2".length());
}}));
}}
));
Set<Map<String, Object>> expected = Set.of(
Map.of("id", 1, "person", new Document(Map.of("name", "updated_name1", "age", "updated_age1".length()))),
Map.of("id", 2, "person", new Document(Map.of("name", "inserted_name2", "age", "inserted_age2".length()))));
testUpsertSuccess(runner, inputs, expected);
}
@ -387,13 +312,10 @@ public class PutMongoRecordIT extends MongoWriteTestBase {
new RecordField("age", RecordFieldType.INT.getDataType())
));
List<List<Object[]>> inputs = Arrays.asList(
Collections.singletonList(
new Object[]{1, new MapRecord(personSchema, new HashMap<String, Object>() {{
put("name", "unimportant");
put("age", "unimportant".length());
}})}
)
List<List<Object[]>> inputs = List.of(
Collections.singletonList(
new Object[]{1, new MapRecord(personSchema, Map.of("name", "unimportant", "age", "unimportant".length()))}
)
);
testUpsertFailure(runner, inputs);
@ -429,35 +351,18 @@ public class PutMongoRecordIT extends MongoWriteTestBase {
recordReader.addSchemaField("team", RecordFieldType.STRING);
recordReader.addSchemaField("color", RecordFieldType.STRING);
List<List<Object[]>> inputs = Arrays.asList(
Arrays.asList(
new Object[]{"A", "yellow"},
new Object[]{"B", "red"}
)
List<List<Object[]>> inputs = List.of(
Arrays.asList(
new Object[]{"A", "yellow"},
new Object[]{"B", "red"}
)
);
Set<Map<String, Object>> expected = new HashSet<>(Arrays.asList(
new HashMap<String, Object>() {{
put("name", "Joe");
put("team", "A");
put("color", "yellow");
}},
new HashMap<String, Object>() {{
put("name", "Jane");
put("team", "A");
put("color", "yellow");
}},
new HashMap<String, Object>() {{
put("name", "Jeff");
put("team", "B");
put("color", "red");
}},
new HashMap<String, Object>() {{
put("name", "Janet");
put("team", "B");
put("color", "red");
}}
));
Set<Map<String, Object>> expected = Set.of(
Map.of("name", "Joe", "team", "A", "color", "yellow"),
Map.of("name", "Jane", "team", "A", "color", "yellow"),
Map.of("name", "Jeff", "team", "B", "color", "red"),
Map.of("name", "Janet", "team", "B", "color", "red"));
testUpsertSuccess(updateRunner, inputs, expected);
}
@ -492,43 +397,24 @@ public class PutMongoRecordIT extends MongoWriteTestBase {
recordReader.addSchemaField("team", RecordFieldType.STRING);
recordReader.addSchemaField("color", RecordFieldType.STRING);
List<List<Object[]>> inputs = Arrays.asList(
Arrays.asList(
new Object[]{"A", "yellow"},
new Object[]{"B", "red"}
)
List<List<Object[]>> inputs = List.of(
Arrays.asList(
new Object[]{"A", "yellow"},
new Object[]{"B", "red"}
)
);
Set<Map<String, Object>> expected = new HashSet<>(Arrays.asList(
new HashMap<String, Object>() {{
put("name", "Joe");
put("team", "A");
put("color", "yellow");
}},
new HashMap<String, Object>() {{
put("name", "Jane");
put("team", "A");
put("color", "yellow");
}},
new HashMap<String, Object>() {{
put("name", "Jeff");
put("team", "B");
put("color", "red");
}},
new HashMap<String, Object>() {{
put("name", "Janet");
put("team", "B");
put("color", "red");
}}
));
Set<Map<String, Object>> expected = Set.of(
Map.of("name", "Joe", "team", "A", "color", "yellow"),
Map.of("name", "Jane", "team", "A", "color", "yellow"),
Map.of("name", "Jeff", "team", "B", "color", "red"),
Map.of("name", "Janet", "team", "B", "color", "red"));
inputs.forEach(input -> {
input.forEach(recordReader::addRecord);
MockFlowFile flowFile = new MockFlowFile(1);
flowFile.putAttributes(new HashMap<String, String>() {{
put(AbstractMongoProcessor.ATTRIBUTE_MONGODB_UPDATE_MODE, "many");
}});
flowFile.putAttributes(Map.of(AbstractMongoProcessor.ATTRIBUTE_MONGODB_UPDATE_MODE, "many"));
updateRunner.enqueue(flowFile);
updateRunner.run();
});
@ -557,11 +443,11 @@ public class PutMongoRecordIT extends MongoWriteTestBase {
recordReader.addSchemaField("team", RecordFieldType.STRING);
recordReader.addSchemaField("color", RecordFieldType.STRING);
List<List<Object[]>> inputs = Arrays.asList(
Arrays.asList(
new Object[]{"A", "yellow"},
new Object[]{"B", "red"}
)
List<List<Object[]>> inputs = List.of(
Arrays.asList(
new Object[]{"A", "yellow"},
new Object[]{"B", "red"}
)
);
testUpsertFailure(runner, inputs);
@ -581,14 +467,9 @@ public class PutMongoRecordIT extends MongoWriteTestBase {
new RecordField("age", RecordFieldType.INT.getDataType())
));
List<List<Object[]>> inputs = Arrays.asList(
Collections.singletonList(
new Object[]{1, new MapRecord(personSchema, new HashMap<String, Object>() {{
put("name", "unimportant");
put("age", "unimportant".length());
}})}
)
);
List<List<Object[]>> inputs = List.of(
Collections.singletonList(
new Object[]{1, new MapRecord(personSchema, Map.of("name", "unimportant", "age", "unimportant".length()))}));
testUpsertFailure(runner, inputs);
}

View File

@ -134,9 +134,7 @@ public class FetchGridFSIT extends GridFSITTestBase {
runner.clearTransferState();
id = writeTestFile(fileName, content, BUCKET, new HashMap<String, Object>() {{
put("lookupKey", "xyz");
}});
id = writeTestFile(fileName, content, BUCKET, Map.of("lookupKey", "xyz"));
assertNotNull(id);
String query = "{ \"metadata\": { \"lookupKey\": \"xyz\" }}";

View File

@ -81,12 +81,10 @@ public class PutGridFSIT extends GridFSITTestBase {
runner.run();
runner.assertAllFlowFilesTransferred(PutGridFS.REL_SUCCESS);
attrs = new HashMap<String, String>() {{
put("created_by", "john.smith");
put("created_for", "jane.doe");
put("restrictions", "PHI&PII");
put("department", "Accounting");
}};
attrs = Map.of("created_by", "john.smith",
"created_for", "jane.doe",
"restrictions", "PHI&PII",
"department", "Accounting");
assertTrue(fileExists(fileName, BUCKET), "File does not exist");
assertTrue(fileHasProperties(fileName, BUCKET, attrs), "File is missing PARENT_PROPERTIES");

View File

@ -27,7 +27,6 @@ import org.apache.nifi.serialization.record.RecordSchema;
import org.apache.nifi.serialization.record.SchemaIdentifier;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
@ -42,8 +41,6 @@ public class StubSchemaRegistry extends AbstractControllerService implements Sch
@Override
public Set<SchemaField> getSuppliedSchemaFields() {
return new HashSet<SchemaField>() {{
add(SchemaField.SCHEMA_NAME);
}};
return Set.of(SchemaField.SCHEMA_NAME);
}
}

View File

@ -23,9 +23,9 @@ import java.io.IOException;
import java.io.InputStream;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import org.apache.nifi.annotation.behavior.InputRequirement;
@ -203,12 +203,8 @@ public class CalculateParquetOffsets extends AbstractProcessor {
results.add(
session.putAllAttributes(
outputFlowFile,
new HashMap<String, String>() {
{
put(ParquetAttribute.RECORD_OFFSET, Long.toString(recordOffset + addedOffset));
put(ParquetAttribute.RECORD_COUNT, Long.toString(Math.min(partitionSize, recordCount - addedOffset)));
}
}
Map.of(ParquetAttribute.RECORD_OFFSET, Long.toString(recordOffset + addedOffset),
ParquetAttribute.RECORD_COUNT, Long.toString(Math.min(partitionSize, recordCount - addedOffset)))
)
);
}

View File

@ -22,9 +22,9 @@ import static java.util.Collections.singletonList;
import java.io.IOException;
import java.io.InputStream;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.SideEffectFree;
@ -154,14 +154,9 @@ public class CalculateParquetRowGroupOffsets extends AbstractProcessor {
results.add(
session.putAllAttributes(
outputFlowFile,
new HashMap<String, String>() {
{
put(ParquetAttribute.FILE_RANGE_START_OFFSET, String.valueOf(currentBlockStartOffset));
put(ParquetAttribute.FILE_RANGE_END_OFFSET, String.valueOf(currentBlockEndOffset));
put(ParquetAttribute.RECORD_COUNT, String.valueOf(currentBlock.getRowCount()));
}
}
)
Map.of(ParquetAttribute.FILE_RANGE_START_OFFSET, String.valueOf(currentBlockStartOffset),
ParquetAttribute.FILE_RANGE_END_OFFSET, String.valueOf(currentBlockEndOffset),
ParquetAttribute.RECORD_COUNT, String.valueOf(currentBlock.getRowCount())))
);
}

View File

@ -23,7 +23,6 @@ import java.io.IOException;
import java.io.InputStream;
import java.nio.charset.StandardCharsets;
import java.util.Collection;
import java.util.HashMap;
import java.util.Map;
import java.util.stream.IntStream;
import org.apache.avro.Schema;
@ -47,13 +46,7 @@ public class ParquetTestUtils {
}
public static Map<String, Object> createUser(int i) {
return new HashMap<String, Object>() {
{
put("name", "Bob" + i);
put("favorite_number", i);
put("favorite_color", "blue" + i);
}
};
return Map.of("name", "Bob" + i, "favorite_number", i, "favorite_color", "blue" + i);
}
private static File createUsersParquetFile(Collection<Map<String, Object>> users) throws IOException {

View File

@ -83,12 +83,8 @@ public class TestParquetReader {
final int numUsers = 1000025; // intentionally so large, to test input with many record groups
final int expectedRecords = 2;
final File parquetFile = ParquetTestUtils.createUsersParquetFile(numUsers);
final List<Record> results = getRecords(parquetFile, new HashMap<String, String>() {
{
put(ParquetAttribute.RECORD_OFFSET, "1000020");
put(ParquetAttribute.RECORD_COUNT, "2");
}
});
final List<Record> results = getRecords(parquetFile, Map.of(ParquetAttribute.RECORD_OFFSET, "1000020",
ParquetAttribute.RECORD_COUNT, "2"));
assertEquals(expectedRecords, results.size());
IntStream.range(0, expectedRecords)
@ -102,13 +98,9 @@ public class TestParquetReader {
final File parquetFile = ParquetTestUtils.createUsersParquetFile(numUsers);
final List<Record> results = getRecords(
parquetFile,
new HashMap<String, String>() {
{
put(ParquetAttribute.RECORD_OFFSET, "321");
put(ParquetAttribute.FILE_RANGE_START_OFFSET, "16543");
put(ParquetAttribute.FILE_RANGE_END_OFFSET, "24784");
}
}
Map.of(ParquetAttribute.RECORD_OFFSET, "321",
ParquetAttribute.FILE_RANGE_START_OFFSET, "16543",
ParquetAttribute.FILE_RANGE_END_OFFSET, "24784")
);
assertEquals(expectedRecords, results.size());
@ -124,14 +116,10 @@ public class TestParquetReader {
final File parquetFile = ParquetTestUtils.createUsersParquetFile(numUsers);
final List<Record> results = getRecords(
parquetFile,
new HashMap<String, String>() {
{
put(ParquetAttribute.RECORD_OFFSET, "321");
put(ParquetAttribute.RECORD_COUNT, "2");
put(ParquetAttribute.FILE_RANGE_START_OFFSET, "16543");
put(ParquetAttribute.FILE_RANGE_END_OFFSET, "24784");
}
}
Map.of(ParquetAttribute.RECORD_OFFSET, "321",
ParquetAttribute.RECORD_COUNT, "2",
ParquetAttribute.FILE_RANGE_START_OFFSET, "16543",
ParquetAttribute.FILE_RANGE_END_OFFSET, "24784")
);
assertEquals(expectedRecords, results.size());
@ -174,12 +162,8 @@ public class TestParquetReader {
runner.addControllerService("reader", parquetReader);
runner.enableControllerService(parquetReader);
runner.enqueue(Paths.get(PARQUET_PATH), new HashMap<String, String>() {
{
put(ParquetAttribute.RECORD_OFFSET, "6");
put(ParquetAttribute.RECORD_COUNT, "2");
}
});
runner.enqueue(Paths.get(PARQUET_PATH), Map.of(ParquetAttribute.RECORD_OFFSET, "6",
ParquetAttribute.RECORD_COUNT, "2"));
runner.setProperty(TestParquetProcessor.READER, "reader");

View File

@ -42,12 +42,7 @@ public class CalculateParquetOffsetsTest {
private static final Path PARQUET_PATH = Paths.get("src/test/resources/TestParquetReader.parquet");
private static final Path NOT_PARQUET_PATH = Paths.get("src/test/resources/core-site.xml");
private static final Map<String, String> PRESERVED_ATTRIBUTES = new HashMap<String, String>() {
{
put("foo", "bar");
put("example", "value");
}
};
private static final Map<String, String> PRESERVED_ATTRIBUTES = Map.of("foo", "bar", "example", "value");
private TestRunner runner;
@ -65,11 +60,11 @@ public class CalculateParquetOffsetsTest {
final List<MockFlowFile> results = runner.getFlowFilesForRelationship(REL_SUCCESS);
results.get(0).assertAttributeEquals(ParquetAttribute.RECORD_COUNT, "10");
results.get(0).assertAttributeEquals(ParquetAttribute.RECORD_OFFSET, "0");
results.get(0).assertContentEquals(PARQUET_PATH);
results.getFirst().assertAttributeEquals(ParquetAttribute.RECORD_COUNT, "10");
results.getFirst().assertAttributeEquals(ParquetAttribute.RECORD_OFFSET, "0");
results.getFirst().assertContentEquals(PARQUET_PATH);
PRESERVED_ATTRIBUTES.forEach(results.get(0)::assertAttributeEquals);
PRESERVED_ATTRIBUTES.forEach(results.getFirst()::assertAttributeEquals);
}
@Test
@ -99,9 +94,9 @@ public class CalculateParquetOffsetsTest {
final List<MockFlowFile> results = runner.getFlowFilesForRelationship(REL_SUCCESS);
results.get(0).assertAttributeEquals(ParquetAttribute.RECORD_COUNT, "5");
results.get(0).assertAttributeEquals(ParquetAttribute.RECORD_OFFSET, "0");
results.get(0).assertContentEquals(PARQUET_PATH);
results.getFirst().assertAttributeEquals(ParquetAttribute.RECORD_COUNT, "5");
results.getFirst().assertAttributeEquals(ParquetAttribute.RECORD_OFFSET, "0");
results.getFirst().assertContentEquals(PARQUET_PATH);
results.get(1).assertAttributeEquals(ParquetAttribute.RECORD_COUNT, "5");
results.get(1).assertAttributeEquals(ParquetAttribute.RECORD_OFFSET, "5");
@ -119,9 +114,9 @@ public class CalculateParquetOffsetsTest {
final List<MockFlowFile> results = runner.getFlowFilesForRelationship(REL_SUCCESS);
results.get(0).assertAttributeEquals(ParquetAttribute.RECORD_COUNT, "8");
results.get(0).assertAttributeEquals(ParquetAttribute.RECORD_OFFSET, "0");
results.get(0).assertContentEquals(PARQUET_PATH);
results.getFirst().assertAttributeEquals(ParquetAttribute.RECORD_COUNT, "8");
results.getFirst().assertAttributeEquals(ParquetAttribute.RECORD_OFFSET, "0");
results.getFirst().assertContentEquals(PARQUET_PATH);
results.get(1).assertAttributeEquals(ParquetAttribute.RECORD_COUNT, "2");
results.get(1).assertAttributeEquals(ParquetAttribute.RECORD_OFFSET, "8");
@ -133,7 +128,7 @@ public class CalculateParquetOffsetsTest {
@Test
public void testSubPartitioningWithCountAndOffset() throws Exception {
runner.setProperty(PROP_RECORDS_PER_SPLIT, "3");
runner.enqueue(PARQUET_PATH, createAttributes(new HashMap<String, String>() {
runner.enqueue(PARQUET_PATH, createAttributes(new HashMap<>() {
{
put(ParquetAttribute.RECORD_COUNT, "7");
put(ParquetAttribute.RECORD_OFFSET, "2");
@ -144,9 +139,9 @@ public class CalculateParquetOffsetsTest {
final List<MockFlowFile> results = runner.getFlowFilesForRelationship(REL_SUCCESS);
results.get(0).assertAttributeEquals(ParquetAttribute.RECORD_COUNT, "3");
results.get(0).assertAttributeEquals(ParquetAttribute.RECORD_OFFSET, "2");
results.get(0).assertContentEquals(PARQUET_PATH);
results.getFirst().assertAttributeEquals(ParquetAttribute.RECORD_COUNT, "3");
results.getFirst().assertAttributeEquals(ParquetAttribute.RECORD_OFFSET, "2");
results.getFirst().assertContentEquals(PARQUET_PATH);
results.get(1).assertAttributeEquals(ParquetAttribute.RECORD_COUNT, "3");
results.get(1).assertAttributeEquals(ParquetAttribute.RECORD_OFFSET, "5");
@ -168,9 +163,9 @@ public class CalculateParquetOffsetsTest {
final List<MockFlowFile> results = runner.getFlowFilesForRelationship(REL_SUCCESS);
results.get(0).assertAttributeEquals(ParquetAttribute.RECORD_COUNT, "2");
results.get(0).assertAttributeEquals(ParquetAttribute.RECORD_OFFSET, "0");
results.get(0).assertContentEquals(PARQUET_PATH);
results.getFirst().assertAttributeEquals(ParquetAttribute.RECORD_COUNT, "2");
results.getFirst().assertAttributeEquals(ParquetAttribute.RECORD_OFFSET, "0");
results.getFirst().assertContentEquals(PARQUET_PATH);
results.get(1).assertAttributeEquals(ParquetAttribute.RECORD_COUNT, "1");
results.get(1).assertAttributeEquals(ParquetAttribute.RECORD_OFFSET, "2");
@ -188,9 +183,9 @@ public class CalculateParquetOffsetsTest {
final List<MockFlowFile> results = runner.getFlowFilesForRelationship(REL_SUCCESS);
results.get(0).assertAttributeEquals(ParquetAttribute.RECORD_COUNT, "5");
results.get(0).assertAttributeEquals(ParquetAttribute.RECORD_OFFSET, "3");
results.get(0).assertContentEquals(PARQUET_PATH);
results.getFirst().assertAttributeEquals(ParquetAttribute.RECORD_COUNT, "5");
results.getFirst().assertAttributeEquals(ParquetAttribute.RECORD_OFFSET, "3");
results.getFirst().assertContentEquals(PARQUET_PATH);
results.get(1).assertAttributeEquals(ParquetAttribute.RECORD_COUNT, "2");
results.get(1).assertAttributeEquals(ParquetAttribute.RECORD_OFFSET, "8");
@ -209,9 +204,9 @@ public class CalculateParquetOffsetsTest {
final List<MockFlowFile> results = runner.getFlowFilesForRelationship(REL_SUCCESS);
results.get(0).assertAttributeEquals(ParquetAttribute.RECORD_COUNT, "8");
results.get(0).assertAttributeEquals(ParquetAttribute.RECORD_OFFSET, "0");
results.get(0).assertContentEquals("");
results.getFirst().assertAttributeEquals(ParquetAttribute.RECORD_COUNT, "8");
results.getFirst().assertAttributeEquals(ParquetAttribute.RECORD_OFFSET, "0");
results.getFirst().assertContentEquals("");
results.get(1).assertAttributeEquals(ParquetAttribute.RECORD_COUNT, "2");
results.get(1).assertAttributeEquals(ParquetAttribute.RECORD_OFFSET, "8");
@ -247,9 +242,9 @@ public class CalculateParquetOffsetsTest {
final List<MockFlowFile> results = runner.getFlowFilesForRelationship(REL_SUCCESS);
results.get(0).assertAttributeEquals(ParquetAttribute.RECORD_COUNT, "3");
results.get(0).assertAttributeEquals(ParquetAttribute.RECORD_OFFSET, "0");
results.get(0).assertContentEquals("");
results.getFirst().assertAttributeEquals(ParquetAttribute.RECORD_COUNT, "3");
results.getFirst().assertAttributeEquals(ParquetAttribute.RECORD_OFFSET, "0");
results.getFirst().assertContentEquals("");
results.get(1).assertAttributeEquals(ParquetAttribute.RECORD_COUNT, "1");
results.get(1).assertAttributeEquals(ParquetAttribute.RECORD_OFFSET, "3");
@ -261,7 +256,7 @@ public class CalculateParquetOffsetsTest {
@Test
public void testEmptyInputWithOffsetAndCountAttributes() {
runner.setProperty(PROP_RECORDS_PER_SPLIT, "3");
runner.enqueue("", createAttributes(new HashMap<String, String>() {
runner.enqueue("", createAttributes(new HashMap<>() {
{
put(ParquetAttribute.RECORD_OFFSET, "2");
put(ParquetAttribute.RECORD_COUNT, "4");
@ -272,9 +267,9 @@ public class CalculateParquetOffsetsTest {
final List<MockFlowFile> results = runner.getFlowFilesForRelationship(REL_SUCCESS);
results.get(0).assertAttributeEquals(ParquetAttribute.RECORD_COUNT, "3");
results.get(0).assertAttributeEquals(ParquetAttribute.RECORD_OFFSET, "2");
results.get(0).assertContentEquals("");
results.getFirst().assertAttributeEquals(ParquetAttribute.RECORD_COUNT, "3");
results.getFirst().assertAttributeEquals(ParquetAttribute.RECORD_OFFSET, "2");
results.getFirst().assertContentEquals("");
results.get(1).assertAttributeEquals(ParquetAttribute.RECORD_COUNT, "1");
results.get(1).assertAttributeEquals(ParquetAttribute.RECORD_OFFSET, "5");
@ -310,9 +305,9 @@ public class CalculateParquetOffsetsTest {
final List<MockFlowFile> results = runner.getFlowFilesForRelationship(REL_SUCCESS);
results.get(0).assertAttributeEquals(ParquetAttribute.RECORD_COUNT, "3");
results.get(0).assertAttributeEquals(ParquetAttribute.RECORD_OFFSET, "0");
results.get(0).assertContentEquals(NOT_PARQUET_PATH);
results.getFirst().assertAttributeEquals(ParquetAttribute.RECORD_COUNT, "3");
results.getFirst().assertAttributeEquals(ParquetAttribute.RECORD_OFFSET, "0");
results.getFirst().assertContentEquals(NOT_PARQUET_PATH);
results.get(1).assertAttributeEquals(ParquetAttribute.RECORD_COUNT, "1");
results.get(1).assertAttributeEquals(ParquetAttribute.RECORD_OFFSET, "3");
@ -324,7 +319,7 @@ public class CalculateParquetOffsetsTest {
@Test
public void testUnrecognizedInputWithOffsetAndCountAttributes() throws IOException {
runner.setProperty(PROP_RECORDS_PER_SPLIT, "3");
runner.enqueue(NOT_PARQUET_PATH, createAttributes(new HashMap<String, String>() {
runner.enqueue(NOT_PARQUET_PATH, createAttributes(new HashMap<>() {
{
put(ParquetAttribute.RECORD_OFFSET, "2");
put(ParquetAttribute.RECORD_COUNT, "4");
@ -335,9 +330,9 @@ public class CalculateParquetOffsetsTest {
final List<MockFlowFile> results = runner.getFlowFilesForRelationship(REL_SUCCESS);
results.get(0).assertAttributeEquals(ParquetAttribute.RECORD_COUNT, "3");
results.get(0).assertAttributeEquals(ParquetAttribute.RECORD_OFFSET, "2");
results.get(0).assertContentEquals(NOT_PARQUET_PATH);
results.getFirst().assertAttributeEquals(ParquetAttribute.RECORD_COUNT, "3");
results.getFirst().assertAttributeEquals(ParquetAttribute.RECORD_OFFSET, "2");
results.getFirst().assertContentEquals(NOT_PARQUET_PATH);
results.get(1).assertAttributeEquals(ParquetAttribute.RECORD_COUNT, "1");
results.get(1).assertAttributeEquals(ParquetAttribute.RECORD_OFFSET, "5");
@ -347,7 +342,7 @@ public class CalculateParquetOffsetsTest {
}
private HashMap<String, String> createAttributes(Map<String, String> additionalAttributes) {
return new HashMap<String, String>(PRESERVED_ATTRIBUTES) {{
return new HashMap<>(PRESERVED_ATTRIBUTES) {{
putAll(additionalAttributes);
}};
}

View File

@ -26,7 +26,6 @@ import java.io.File;
import java.io.IOException;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.nifi.parquet.ParquetTestUtils;
@ -44,12 +43,7 @@ public class CalculateParquetRowGroupOffsetsTest {
private static final Path NOT_PARQUET_PATH = Paths.get("src/test/resources/core-site.xml");
private static final Map<String, String> PRESERVED_ATTRIBUTES = new HashMap<String, String>() {
{
put("foo", "bar");
put("example", "value");
}
};
private static final Map<String, String> PRESERVED_ATTRIBUTES = Map.of("foo", "bar", "example", "value");
private TestRunner runner;
@ -67,12 +61,12 @@ public class CalculateParquetRowGroupOffsetsTest {
final List<MockFlowFile> results = runner.getFlowFilesForRelationship(REL_SUCCESS);
results.get(0).assertAttributeEquals(ParquetAttribute.RECORD_COUNT, "10");
results.get(0).assertAttributeEquals(ParquetAttribute.FILE_RANGE_START_OFFSET, "4");
results.get(0).assertAttributeEquals(ParquetAttribute.FILE_RANGE_END_OFFSET, "298");
results.get(0).assertContentEquals(parquetFile.toPath());
results.getFirst().assertAttributeEquals(ParquetAttribute.RECORD_COUNT, "10");
results.getFirst().assertAttributeEquals(ParquetAttribute.FILE_RANGE_START_OFFSET, "4");
results.getFirst().assertAttributeEquals(ParquetAttribute.FILE_RANGE_END_OFFSET, "298");
results.getFirst().assertContentEquals(parquetFile.toPath());
PRESERVED_ATTRIBUTES.forEach(results.get(0)::assertAttributeEquals);
PRESERVED_ATTRIBUTES.forEach(results.getFirst()::assertAttributeEquals);
}
@Test
@ -84,10 +78,10 @@ public class CalculateParquetRowGroupOffsetsTest {
final List<MockFlowFile> results = runner.getFlowFilesForRelationship(REL_SUCCESS);
results.get(0).assertAttributeEquals(ParquetAttribute.RECORD_COUNT, "337");
results.get(0).assertAttributeEquals(ParquetAttribute.FILE_RANGE_START_OFFSET, "4");
results.get(0).assertAttributeEquals(ParquetAttribute.FILE_RANGE_END_OFFSET, "8301");
results.get(0).assertContentEquals(parquetFile.toPath());
results.getFirst().assertAttributeEquals(ParquetAttribute.RECORD_COUNT, "337");
results.getFirst().assertAttributeEquals(ParquetAttribute.FILE_RANGE_START_OFFSET, "4");
results.getFirst().assertAttributeEquals(ParquetAttribute.FILE_RANGE_END_OFFSET, "8301");
results.getFirst().assertContentEquals(parquetFile.toPath());
results.get(1).assertAttributeEquals(ParquetAttribute.RECORD_COUNT, "326");
results.get(1).assertAttributeEquals(ParquetAttribute.FILE_RANGE_START_OFFSET, "8301");
@ -117,10 +111,10 @@ public class CalculateParquetRowGroupOffsetsTest {
final List<MockFlowFile> results = runner.getFlowFilesForRelationship(REL_SUCCESS);
results.get(0).assertAttributeEquals(ParquetAttribute.RECORD_COUNT, "337");
results.get(0).assertAttributeEquals(ParquetAttribute.FILE_RANGE_START_OFFSET, "4");
results.get(0).assertAttributeEquals(ParquetAttribute.FILE_RANGE_END_OFFSET, "8301");
results.get(0).assertContentEquals("");
results.getFirst().assertAttributeEquals(ParquetAttribute.RECORD_COUNT, "337");
results.getFirst().assertAttributeEquals(ParquetAttribute.FILE_RANGE_START_OFFSET, "4");
results.getFirst().assertAttributeEquals(ParquetAttribute.FILE_RANGE_END_OFFSET, "8301");
results.getFirst().assertContentEquals("");
results.get(1).assertAttributeEquals(ParquetAttribute.RECORD_COUNT, "163");
results.get(1).assertAttributeEquals(ParquetAttribute.FILE_RANGE_START_OFFSET, "8301");

View File

@ -22,8 +22,8 @@ import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import java.nio.charset.StandardCharsets;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static org.junit.jupiter.api.Assertions.assertEquals;
@ -115,10 +115,7 @@ public class TestExecuteClojure extends BaseScriptTest {
runner.setProperty("myProp", "${myAttr}");
runner.assertValid();
runner.enqueue(TEST_CSV_DATA.getBytes(StandardCharsets.UTF_8),
new HashMap<String, String>(1) {{
put("myAttr", "testValue");
}});
runner.enqueue(TEST_CSV_DATA.getBytes(StandardCharsets.UTF_8), Map.of("myAttr", "testValue"));
runner.run();
runner.assertAllFlowFilesTransferred(ExecuteScript.REL_SUCCESS, 1);

View File

@ -23,8 +23,8 @@ import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import java.nio.charset.StandardCharsets;
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.assertThrows;
@ -149,10 +149,7 @@ public class TestExecuteGroovy extends BaseScriptTest {
runner.setProperty("myProp", "${myAttr}");
runner.assertValid();
runner.enqueue(TEST_CSV_DATA.getBytes(StandardCharsets.UTF_8),
new HashMap<String, String>(1) {{
put("myAttr", "testValue");
}});
runner.enqueue(TEST_CSV_DATA.getBytes(StandardCharsets.UTF_8), Map.of("myAttr", "testValue"));
runner.run();
runner.assertAllFlowFilesTransferred(ExecuteScript.REL_SUCCESS, 1);

View File

@ -122,7 +122,7 @@ public class SiteToSiteStatusReportingTask extends AbstractSiteToSiteReportingTa
componentNameFilter = Pattern.compile(context.getProperty(COMPONENT_NAME_FILTER_REGEX).evaluateAttributeExpressions().getValue());
// initialize the map
processGroupIDToPath = new HashMap<String, String>();
processGroupIDToPath = new HashMap<>();
final ProcessGroupStatus procGroupStatus = context.getEventAccess().getControllerStatus();
final String rootGroupName = procGroupStatus == null ? null : procGroupStatus.getName();

View File

@ -112,7 +112,7 @@ public class TestSiteToSiteStatusReportingTask {
task.onTrigger(context);
assertEquals(16, task.dataSent.size());
final String msg = new String(task.dataSent.get(0), StandardCharsets.UTF_8);
final String msg = new String(task.dataSent.getFirst(), StandardCharsets.UTF_8);
JsonReader jsonReader = Json.createReader(new ByteArrayInputStream(msg.getBytes()));
JsonObject firstElement = jsonReader.readArray().getJsonObject(0);
JsonString componentId = firstElement.getJsonString("componentId");
@ -136,7 +136,7 @@ public class TestSiteToSiteStatusReportingTask {
task.onTrigger(context);
assertEquals(1, task.dataSent.size()); // Only root pg and 3 child pgs
final String msg = new String(task.dataSent.get(0), StandardCharsets.UTF_8);
final String msg = new String(task.dataSent.getFirst(), StandardCharsets.UTF_8);
JsonReader jsonReader = Json.createReader(new ByteArrayInputStream(msg.getBytes()));
JsonString componentId = jsonReader.readArray().getJsonObject(0).getJsonString("componentId");
assertEquals(pgStatus.getId(), componentId.getString());
@ -154,7 +154,7 @@ public class TestSiteToSiteStatusReportingTask {
MockSiteToSiteStatusReportingTask task = initTask(properties, pgStatus);
task.onTrigger(context);
final String msg = new String(task.dataSent.get(0), StandardCharsets.UTF_8);
final String msg = new String(task.dataSent.getFirst(), StandardCharsets.UTF_8);
JsonReader jsonReader = Json.createReader(new ByteArrayInputStream(msg.getBytes()));
JsonObject object = jsonReader.readArray().getJsonObject(0);
JsonString backpressure = object.getJsonString("isBackPressureEnabled");
@ -181,7 +181,7 @@ public class TestSiteToSiteStatusReportingTask {
MockSiteToSiteStatusReportingTask task = initTask(properties, pgStatus);
task.onTrigger(context);
final String msg = new String(task.dataSent.get(0), StandardCharsets.UTF_8);
final String msg = new String(task.dataSent.getFirst(), StandardCharsets.UTF_8);
JsonReader jsonReader = Json.createReader(new ByteArrayInputStream(msg.getBytes()));
JsonObject object = jsonReader.readArray().getJsonObject(0);
JsonValue destination = object.get("destinationName");
@ -202,7 +202,7 @@ public class TestSiteToSiteStatusReportingTask {
task.onTrigger(context);
assertEquals(3, task.dataSent.size()); // 3 processors for each of 4 groups
final String msg = new String(task.dataSent.get(0), StandardCharsets.UTF_8);
final String msg = new String(task.dataSent.getFirst(), StandardCharsets.UTF_8);
JsonReader jsonReader = Json.createReader(new ByteArrayInputStream(msg.getBytes()));
JsonString componentId = jsonReader.readArray().getJsonObject(0).getJsonString("componentId");
assertEquals("root.1.processor.1", componentId.getString());
@ -221,7 +221,7 @@ public class TestSiteToSiteStatusReportingTask {
task.onTrigger(context);
assertEquals(10, task.dataSent.size()); // 3 + (3 * 3) + (3 * 3 * 3) = 39, or 10 batches of 4
final String msg = new String(task.dataSent.get(0), StandardCharsets.UTF_8);
final String msg = new String(task.dataSent.getFirst(), StandardCharsets.UTF_8);
JsonReader jsonReader = Json.createReader(new ByteArrayInputStream(msg.getBytes()));
JsonString componentId = jsonReader.readArray().getJsonObject(0).getJsonString("componentId");
assertEquals("root.1.1.processor.1", componentId.getString());
@ -240,7 +240,7 @@ public class TestSiteToSiteStatusReportingTask {
MockSiteToSiteStatusReportingTask task = initTask(properties, pgStatus);
task.onTrigger(context);
final String msg = new String(task.dataSent.get(0), StandardCharsets.UTF_8);
final String msg = new String(task.dataSent.getFirst(), StandardCharsets.UTF_8);
JsonReader jsonReader = Json.createReader(new ByteArrayInputStream(msg.getBytes()));
JsonObject object = jsonReader.readArray().getJsonObject(0);
JsonString runStatus = object.getJsonString("runStatus");
@ -264,7 +264,7 @@ public class TestSiteToSiteStatusReportingTask {
MockSiteToSiteStatusReportingTask task = initTask(properties, pgStatus);
task.onTrigger(context);
final String msg = new String(task.dataSent.get(0), StandardCharsets.UTF_8);
final String msg = new String(task.dataSent.getFirst(), StandardCharsets.UTF_8);
JsonReader jsonReader = Json.createReader(new ByteArrayInputStream(msg.getBytes()));
JsonObject object = jsonReader.readArray().getJsonObject(0);
JsonValue activeThreadCount = object.get("activeThreadCount");
@ -284,7 +284,7 @@ public class TestSiteToSiteStatusReportingTask {
task.onTrigger(context);
assertEquals(3, task.dataSent.size());
final String msg = new String(task.dataSent.get(0), StandardCharsets.UTF_8);
final String msg = new String(task.dataSent.getFirst(), StandardCharsets.UTF_8);
JsonReader jsonReader = Json.createReader(new ByteArrayInputStream(msg.getBytes()));
JsonObject firstElement = jsonReader.readArray().getJsonObject(0);
JsonNumber activeThreadCount = firstElement.getJsonNumber("activeThreadCount");
@ -307,7 +307,7 @@ public class TestSiteToSiteStatusReportingTask {
task.onTrigger(context);
assertEquals(3, task.dataSent.size());
final String msg = new String(task.dataSent.get(0), StandardCharsets.UTF_8);
final String msg = new String(task.dataSent.getFirst(), StandardCharsets.UTF_8);
JsonReader jsonReader = Json.createReader(new ByteArrayInputStream(msg.getBytes()));
JsonObject firstElement = jsonReader.readArray().getJsonObject(0);
JsonValue targetURI = firstElement.get("targetURI");
@ -326,7 +326,7 @@ public class TestSiteToSiteStatusReportingTask {
MockSiteToSiteStatusReportingTask task = initTask(properties, pgStatus);
task.onTrigger(context);
final String msg = new String(task.dataSent.get(0), StandardCharsets.UTF_8);
final String msg = new String(task.dataSent.getFirst(), StandardCharsets.UTF_8);
JsonReader jsonReader = Json.createReader(new ByteArrayInputStream(msg.getBytes()));
JsonObject object = jsonReader.readArray().getJsonObject(0);
JsonString parentName = object.getJsonString("parentName");
@ -357,7 +357,7 @@ public class TestSiteToSiteStatusReportingTask {
MockSiteToSiteStatusReportingTask task = initTask(properties, pgStatus);
task.onTrigger(context);
final String msg = new String(task.dataSent.get(0), StandardCharsets.UTF_8);
final String msg = new String(task.dataSent.getFirst(), StandardCharsets.UTF_8);
JsonReader jsonReader = Json.createReader(new ByteArrayInputStream(msg.getBytes()));
JsonObject object = jsonReader.readArray().getJsonObject(0);
JsonValue type = object.get("processorType");
@ -401,7 +401,7 @@ public class TestSiteToSiteStatusReportingTask {
ProcessGroupStatus pgStatus = new ProcessGroupStatus();
pgStatus.setId(id);
pgStatus.setName(namePrefix + "-" + UUID.randomUUID().toString());
pgStatus.setName(namePrefix + "-" + UUID.randomUUID());
pgStatus.setInputPortStatus(ipStatus);
pgStatus.setOutputPortStatus(opStatus);
pgStatus.setProcessGroupStatus(childPgStatus);
@ -432,7 +432,7 @@ public class TestSiteToSiteStatusReportingTask {
public static PortStatus generatePortStatus(String id, String namePrefix) {
PortStatus pStatus = new PortStatus();
pStatus.setId(id);
pStatus.setName(namePrefix + "-" + UUID.randomUUID().toString());
pStatus.setName(namePrefix + "-" + UUID.randomUUID());
pStatus.setActiveThreadCount(null);
pStatus.setBytesReceived(1l);
pStatus.setBytesSent(2l);
@ -451,7 +451,7 @@ public class TestSiteToSiteStatusReportingTask {
public static ProcessorStatus generateProcessorStatus(String id, String namePrefix) {
ProcessorStatus pStatus = new ProcessorStatus();
pStatus.setId(id);
pStatus.setName(namePrefix + "-" + UUID.randomUUID().toString());
pStatus.setName(namePrefix + "-" + UUID.randomUUID());
pStatus.setActiveThreadCount(0);
pStatus.setAverageLineageDuration(1l);
pStatus.setBytesRead(2l);
@ -470,10 +470,7 @@ public class TestSiteToSiteStatusReportingTask {
pStatus.setType(null);
pStatus.setTerminatedThreadCount(1);
pStatus.setRunStatus(RunStatus.Running);
pStatus.setCounters(new HashMap<String, Long>() {{
put("counter1", 10L);
put("counter2", 5L);
}});
pStatus.setCounters(Map.of("counter1", 10L, "counter2", 5L));
return pStatus;
}
@ -481,7 +478,7 @@ public class TestSiteToSiteStatusReportingTask {
public static RemoteProcessGroupStatus generateRemoteProcessGroupStatus(String id, String namePrefix) {
RemoteProcessGroupStatus rpgStatus = new RemoteProcessGroupStatus();
rpgStatus.setId(id);
rpgStatus.setName(namePrefix + "-" + UUID.randomUUID().toString());
rpgStatus.setName(namePrefix + "-" + UUID.randomUUID());
rpgStatus.setActiveRemotePortCount(0);
rpgStatus.setActiveThreadCount(1);
rpgStatus.setAverageLineageDuration(2l);
@ -499,7 +496,7 @@ public class TestSiteToSiteStatusReportingTask {
public static ConnectionStatus generateConnectionStatus(String id, String namePrefix) {
ConnectionStatus cStatus = new ConnectionStatus();
cStatus.setId(id);
cStatus.setName(namePrefix + "-" + UUID.randomUUID().toString());
cStatus.setName(namePrefix + "-" + UUID.randomUUID());
cStatus.setBackPressureDataSizeThreshold("1 KB"); // sets backPressureBytesThreshold too
cStatus.setBackPressureObjectThreshold(1l);
cStatus.setInputBytes(2l);

View File

@ -583,7 +583,7 @@ public class TestConsumeSlack {
@Override
public Map<String, String> fetchChannelIds() {
final Map<String, String> nameIdMapping = new HashMap<String, String>();
final Map<String, String> nameIdMapping = new HashMap<>();
nameIdMapping.put("#cname1", "cid1");
nameIdMapping.put("#cname2", "cid2");
return nameIdMapping;

View File

@ -241,7 +241,7 @@ public class GetSmbFile extends AbstractProcessor {
@Override
protected void init(final ProcessorInitializationContext context) {
final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
final List<PropertyDescriptor> descriptors = new ArrayList<>();
descriptors.add(HOSTNAME);
descriptors.add(SHARE);
descriptors.add(DIRECTORY);
@ -262,7 +262,7 @@ public class GetSmbFile extends AbstractProcessor {
descriptors.add(TIMEOUT);
this.descriptors = Collections.unmodifiableList(descriptors);
final Set<Relationship> relationships = new HashSet<Relationship>();
final Set<Relationship> relationships = new HashSet<>();
relationships.add(REL_SUCCESS);
this.relationships = Collections.unmodifiableSet(relationships);
}

View File

@ -59,7 +59,6 @@ import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.EnumSet;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.TimeUnit;
@ -181,28 +180,24 @@ public class PutSmbFile extends AbstractProcessor {
@Override
protected void init(final ProcessorInitializationContext context) {
final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
descriptors.add(HOSTNAME);
descriptors.add(SHARE);
descriptors.add(DIRECTORY);
descriptors.add(DOMAIN);
descriptors.add(USERNAME);
descriptors.add(PASSWORD);
descriptors.add(CREATE_DIRS);
descriptors.add(SHARE_ACCESS);
descriptors.add(CONFLICT_RESOLUTION);
descriptors.add(BATCH_SIZE);
descriptors.add(RENAME_SUFFIX);
descriptors.add(SMB_DIALECT);
descriptors.add(USE_ENCRYPTION);
descriptors.add(ENABLE_DFS);
descriptors.add(TIMEOUT);
this.descriptors = Collections.unmodifiableList(descriptors);
this.descriptors = List.of(
HOSTNAME,
SHARE,
DIRECTORY,
DOMAIN,
USERNAME,
PASSWORD,
CREATE_DIRS,
SHARE_ACCESS,
CONFLICT_RESOLUTION,
BATCH_SIZE,
RENAME_SUFFIX,
SMB_DIALECT,
USE_ENCRYPTION,
ENABLE_DFS,
TIMEOUT);
final Set<Relationship> relationships = new HashSet<Relationship>();
relationships.add(REL_SUCCESS);
relationships.add(REL_FAILURE);
this.relationships = Collections.unmodifiableSet(relationships);
this.relationships = Set.of(REL_SUCCESS, REL_FAILURE);
}
@Override

View File

@ -41,7 +41,7 @@ public class UsmJsonParser {
}
static List<UsmUser> parse(final String json) throws JsonProcessingException {
return MAPPER.readValue(json, new TypeReference<List<UsmUser>>() {
return MAPPER.readValue(json, new TypeReference<>() {
});
}
}

View File

@ -111,22 +111,12 @@ public class TestDatabaseParameterProvider {
}
private void runColumnStrategiesTest(final Map<PropertyDescriptor, String> properties) throws SQLException {
final List<Map<String, String>> rows = Arrays.asList(
new HashMap<String, String>() { {
put("group", "Kafka"); put("name", "brokers"); put("value", "my-brokers"); put("unrelated_column", "unrelated_value");
} },
new HashMap<String, String>() { {
put("group", "Kafka"); put("name", "topic"); put("value", "my-topic"); put("unrelated_column", "unrelated_value");
} },
new HashMap<String, String>() { {
put("group", "Kafka"); put("name", "password"); put("value", "my-password"); put("unrelated_column", "unrelated_value");
} },
new HashMap<String, String>() { {
put("group", "S3"); put("name", "bucket"); put("value", "my-bucket"); put("unrelated_column", "unrelated_value");
} },
new HashMap<String, String>() { {
put("group", "S3"); put("name", "s3-password"); put("value", "my-s3-password"); put("unrelated_column", "unrelated_value");
} }
final List<Map<String, String>> rows = List.of(
Map.of("group", "Kafka", "name", "brokers", "value", "my-brokers", "unrelated_column", "unrelated_value"),
Map.of("group", "Kafka", "name", "topic", "value", "my-topic", "unrelated_column", "unrelated_value"),
Map.of("group", "Kafka", "name", "password", "value", "my-password", "unrelated_column", "unrelated_value"),
Map.of("group", "S3", "name", "bucket", "value", "my-bucket", "unrelated_column", "unrelated_value"),
Map.of("group", "S3", "name", "s3-password", "value", "my-s3-password", "unrelated_column", "unrelated_value")
);
mockTableResults(new MockTable(TABLE_NAME, rows));
@ -158,36 +148,18 @@ public class TestDatabaseParameterProvider {
}
private void runNonColumnStrategyTest(final Map<PropertyDescriptor, String> properties) throws SQLException {
final List<Map<String, String>> kafkaRows = Arrays.asList(
new HashMap<String, String>() { {
put("name", "nifi_brokers"); put("value", "my-brokers");
} },
new HashMap<String, String>() { {
put("name", "nifi_topic"); put("value", "my-topic");
} },
new HashMap<String, String>() { {
put("name", "unrelated_field"); put("value", "my-value");
} },
new HashMap<String, String>() { {
put("name", "kafka_password"); put("value", "my-password");
} },
new HashMap<String, String>() { {
put("name", "nifi_password"); put("value", "my-nifi-password");
} }
final List<Map<String, String>> kafkaRows = List.of(
Map.of("name", "nifi_brokers", "value", "my-brokers"),
Map.of("name", "nifi_topic", "value", "my-topic"),
Map.of("name", "unrelated_field", "value", "my-value"),
Map.of("name", "kafka_password", "value", "my-password"),
Map.of("name", "nifi_password", "value", "my-nifi-password")
);
final List<Map<String, String>> s3Rows = Arrays.asList(
new HashMap<String, String>() { {
put("name", "nifi_s3_bucket"); put("value", "my-bucket");
} },
new HashMap<String, String>() { {
put("name", "s3_password"); put("value", "my-password");
} },
new HashMap<String, String>() { {
put("name", "nifi_other_field"); put("value", "my-field");
} },
new HashMap<String, String>() { {
put("name", "other_password"); put("value", "my-password");
} }
final List<Map<String, String>> s3Rows = List.of(
Map.of("name", "nifi_s3_bucket", "value", "my-bucket"),
Map.of("name", "s3_password", "value", "my-password"),
Map.of("name", "nifi_other_field", "value", "my-field"),
Map.of("name", "other_password", "value", "my-password")
);
mockTableResults(new MockTable("KAFKA", kafkaRows), new MockTable("S3", s3Rows));
@ -216,22 +188,28 @@ public class TestDatabaseParameterProvider {
@Test
public void testNullNameColumn() throws SQLException {
mockTableResults(new MockTable(TABLE_NAME,
Arrays.asList(new HashMap<String, String>() { { put("name", null); } })));
final Map<String, String> mapWithNullValue = new HashMap<>();
mapWithNullValue.put("name", null);
mockTableResults(new MockTable(TABLE_NAME, List.of(mapWithNullValue)));
runTestWithExpectedFailure(columnBasedProperties);
}
@Test
public void testNullGroupNameColumn() throws SQLException {
mockTableResults(new MockTable(TABLE_NAME,
Arrays.asList(new HashMap<String, String>() { { put("name", "param"); put("value", "value"); put("group", null); } })));
final Map<String, String> mapWithNullGroupNameColumn = new HashMap<>();
mapWithNullGroupNameColumn.put("name", "param");
mapWithNullGroupNameColumn.put("value", "value");
mapWithNullGroupNameColumn.put("group", null);
mockTableResults(new MockTable(TABLE_NAME, List.of(mapWithNullGroupNameColumn)));
runTestWithExpectedFailure(columnBasedProperties);
}
@Test
public void testNullValueColumn() throws SQLException {
mockTableResults(new MockTable(TABLE_NAME,
Arrays.asList(new HashMap<String, String>() { { put("name", "param"); put("value", null); } })));
final Map<String, String> mapWithNullValueColumn = new HashMap<>();
mapWithNullValueColumn.put("name", "param");
mapWithNullValueColumn.put("value", null);
mockTableResults(new MockTable(TABLE_NAME, List.of(mapWithNullValueColumn)));
runTestWithExpectedFailure(columnBasedProperties);
}

View File

@ -226,7 +226,7 @@ public class GenerateFlowFile extends AbstractProcessor {
}
Map<PropertyDescriptor, String> processorProperties = context.getProperties();
Map<String, String> generatedAttributes = new HashMap<String, String>();
Map<String, String> generatedAttributes = new HashMap<>();
for (final Map.Entry<PropertyDescriptor, String> entry : processorProperties.entrySet()) {
PropertyDescriptor property = entry.getKey();
if (property.isDynamic() && property.isExpressionLanguageSupported()) {

View File

@ -122,7 +122,7 @@ public class GetFileResource extends AbstractProcessor {
@Override
public void onTrigger(final ProcessContext context, final ProcessSession session) {
final Map<PropertyDescriptor, String> processorProperties = context.getProperties();
final Map<String, String> generatedAttributes = new HashMap<String, String>();
final Map<String, String> generatedAttributes = new HashMap<>();
for (final Map.Entry<PropertyDescriptor, String> entry : processorProperties.entrySet()) {
final PropertyDescriptor property = entry.getKey();
if (property.isDynamic()) {

View File

@ -528,7 +528,7 @@ public class ListFile extends AbstractListProcessor<FileInfo> {
int maxDepth = recurse ? Integer.MAX_VALUE : 1;
final BiPredicate<Path, BasicFileAttributes> matcher = new BiPredicate<Path, BasicFileAttributes>() {
final BiPredicate<Path, BasicFileAttributes> matcher = new BiPredicate<>() {
private long lastTimestamp = System.currentTimeMillis();
@Override
@ -579,7 +579,7 @@ public class ListFile extends AbstractListProcessor<FileInfo> {
final long start = System.currentTimeMillis();
final List<FileInfo> result = new LinkedList<>();
Files.walkFileTree(basePath, Set.of(FileVisitOption.FOLLOW_LINKS), maxDepth, new FileVisitor<Path>() {
Files.walkFileTree(basePath, Set.of(FileVisitOption.FOLLOW_LINKS), maxDepth, new FileVisitor<>() {
@Override
public FileVisitResult preVisitDirectory(final Path dir, final BasicFileAttributes attributes) {
if (Files.isReadable(dir)) {
@ -841,7 +841,7 @@ public class ListFile extends AbstractListProcessor<FileInfo> {
this.logger = logger;
this.maxDiskOperationMillis = maxDiskOperationMillis;
directoryToTimingInfo = new LinkedHashMap<Tuple<String, String>, TimingInfo>() {
directoryToTimingInfo = new LinkedHashMap<>() {
@Override
protected boolean removeEldestEntry(final Map.Entry<Tuple<String, String>, TimingInfo> eldest) {
return size() > maxEntries;

View File

@ -285,7 +285,7 @@ public class ListenTCP extends AbstractProcessor {
private EventBatcher<ByteArrayMessage> getEventBatcher() {
if (eventBatcher == null) {
eventBatcher = new EventBatcher<ByteArrayMessage>(getLogger(), events, errorEvents) {
eventBatcher = new EventBatcher<>(getLogger(), events, errorEvents) {
@Override
protected String getBatchKey(ByteArrayMessage event) {
return event.getSender();

View File

@ -454,14 +454,14 @@ public class ValidateCsv extends AbstractProcessor {
final boolean isWholeFFValidation = context.getProperty(VALIDATION_STRATEGY).getValue().equals(VALIDATE_WHOLE_FLOWFILE.getValue());
final boolean includeAllViolations = context.getProperty(INCLUDE_ALL_VIOLATIONS).asBoolean();
final AtomicReference<Boolean> valid = new AtomicReference<Boolean>(true);
final AtomicReference<Boolean> isFirstLineValid = new AtomicReference<Boolean>(true);
final AtomicReference<Boolean> isFirstLineInvalid = new AtomicReference<Boolean>(true);
final AtomicReference<Integer> okCount = new AtomicReference<Integer>(0);
final AtomicReference<Integer> totalCount = new AtomicReference<Integer>(0);
final AtomicReference<FlowFile> invalidFF = new AtomicReference<FlowFile>(null);
final AtomicReference<FlowFile> validFF = new AtomicReference<FlowFile>(null);
final AtomicReference<String> validationError = new AtomicReference<String>(null);
final AtomicReference<Boolean> valid = new AtomicReference<>(true);
final AtomicReference<Boolean> isFirstLineValid = new AtomicReference<>(true);
final AtomicReference<Boolean> isFirstLineInvalid = new AtomicReference<>(true);
final AtomicReference<Integer> okCount = new AtomicReference<>(0);
final AtomicReference<Integer> totalCount = new AtomicReference<>(0);
final AtomicReference<FlowFile> invalidFF = new AtomicReference<>(null);
final AtomicReference<FlowFile> validFF = new AtomicReference<>(null);
final AtomicReference<String> validationError = new AtomicReference<>(null);
if (!isWholeFFValidation) {
invalidFF.set(session.create(flowFile));

View File

@ -35,7 +35,7 @@ public class DefaultAvroSqlWriter implements SqlWriter {
private final AvroConversionOptions options;
private final Map<String, String> attributesToAdd = new HashMap<String, String>() {{
private final Map<String, String> attributesToAdd = new HashMap<>() {{
put(CoreAttributes.MIME_TYPE.key(), JdbcCommon.MIME_TYPE_AVRO_BINARY);
}};

View File

@ -23,17 +23,17 @@ import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.junit.jupiter.api.Test;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.Arrays;
import java.util.LinkedHashMap;
import java.util.Map;
import java.util.HashMap;
import java.util.Set;
import java.util.HashSet;
import java.util.List;
import java.util.regex.Pattern;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertTrue;
@ -45,7 +45,7 @@ public class TestAttributesToCSV {
private static final String OUTPUT_SEPARATOR = ",";
private static final String OUTPUT_MIME_TYPE = "text/csv";
private static final String SPLIT_REGEX = OUTPUT_SEPARATOR + "(?=(?:[^\"]*\"[^\"]*\")*[^\"]*$)";
private static final String newline = System.getProperty("line.separator");
private static final String newline = System.lineSeparator();
@Test
public void testAttrListNoCoreNullOffNewAttrToAttribute() {
@ -59,13 +59,13 @@ public class TestAttributesToCSV {
testRunner.enqueue(new byte[0]);
testRunner.run();
testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS).get(0)
testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS).getFirst()
.assertAttributeExists("CSVData");
testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1);
testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0);
testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS)
.get(0).assertAttributeEquals("CSVData", "");
.getFirst().assertAttributeEquals("CSVData", "");
}
@Test
@ -76,19 +76,19 @@ public class TestAttributesToCSV {
testRunner.setProperty(AttributesToCSV.INCLUDE_CORE_ATTRIBUTES, "false");
testRunner.setProperty(AttributesToCSV.NULL_VALUE_FOR_EMPTY_STRING, "false");
//use only one attribute, which does not exists, as the list of attributes to convert to csv
//use only one attribute, which does not exist, as the list of attributes to convert to csv
final String NON_PRESENT_ATTRIBUTE_KEY = "beach-type";
testRunner.setProperty(AttributesToCSV.ATTRIBUTES_LIST, NON_PRESENT_ATTRIBUTE_KEY);
testRunner.enqueue(new byte[0]);
testRunner.run();
testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS).get(0)
testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS).getFirst()
.assertAttributeExists("CSVData");
testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1);
testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0);
testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS)
.get(0).assertAttributeEquals("CSVData", "");
.getFirst().assertAttributeEquals("CSVData", "");
}
@Test
@ -103,13 +103,13 @@ public class TestAttributesToCSV {
testRunner.enqueue(new byte[0]);
testRunner.run();
testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS).get(0)
testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS).getFirst()
.assertAttributeExists("CSVData");
testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1);
testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0);
testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS)
.get(0).assertAttributeEquals("CSVData", ",");
.getFirst().assertAttributeEquals("CSVData", ",");
}
@Test
@ -124,13 +124,13 @@ public class TestAttributesToCSV {
testRunner.enqueue(new byte[0]);
testRunner.run();
testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS).get(0)
testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS).getFirst()
.assertAttributeExists("CSVData");
testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1);
testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0);
testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS)
.get(0).assertAttributeEquals("CSVData", "null,null");
.getFirst().assertAttributeEquals("CSVData", "null,null");
}
@Test
@ -143,13 +143,13 @@ public class TestAttributesToCSV {
testRunner.enqueue(new byte[0]);
testRunner.run();
testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS).get(0)
testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS).getFirst()
.assertAttributeExists("CSVData");
testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1);
testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0);
testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS)
.get(0).assertAttributeEquals("CSVData", "");
.getFirst().assertAttributeEquals("CSVData", "");
}
@Test
@ -161,28 +161,25 @@ public class TestAttributesToCSV {
testRunner.enqueue(new byte[0]);
testRunner.run();
testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS).get(0)
testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS).getFirst()
.assertAttributeExists("CSVData");
testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1);
testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0);
testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS)
.get(0).assertAttributeEquals("CSVData", "");
.getFirst().assertAttributeEquals("CSVData", "");
}
@Test
public void testNoAttrListCoreNullOffToContent() throws IOException {
public void testNoAttrListCoreNullOffToContent() {
final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToCSV());
testRunner.setProperty(AttributesToCSV.DESTINATION, OUTPUT_OVERWRITE_CONTENT);
testRunner.setProperty(AttributesToCSV.INCLUDE_CORE_ATTRIBUTES, "true");
testRunner.setProperty(AttributesToCSV.NULL_VALUE_FOR_EMPTY_STRING, "false");
final Map<String, String> attrs = new HashMap<String, String>() {{
put("beach-name", "Malibu Beach");
put("beach-location", "California, US");
put("beach-endorsement", "This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim");
}};
final Map<String, String> attrs = Map.of("beach-name", "Malibu Beach", "beach-location", "California, US",
"beach-endorsement", "This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim");
testRunner.enqueue(new byte[0], attrs);
testRunner.run();
@ -192,13 +189,13 @@ public class TestAttributesToCSV {
testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0);
testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1);
MockFlowFile flowFile = flowFilesForRelationship.get(0);
MockFlowFile flowFile = flowFilesForRelationship.getFirst();
assertEquals(OUTPUT_MIME_TYPE, flowFile.getAttribute(CoreAttributes.MIME_TYPE.key()));
final byte[] contentData = testRunner.getContentAsByteArray(flowFile);
final String contentDataString = new String(contentData, "UTF-8");
final String contentDataString = new String(contentData, StandardCharsets.UTF_8);
Set<String> contentValues = new HashSet<>(getStrings(contentDataString));
@ -218,11 +215,8 @@ public class TestAttributesToCSV {
testRunner.setProperty(AttributesToCSV.INCLUDE_CORE_ATTRIBUTES, "true");
testRunner.setProperty(AttributesToCSV.NULL_VALUE_FOR_EMPTY_STRING, "false");
Map<String, String> attrs = new HashMap<String, String>() {{
put("beach-name", "Malibu Beach");
put("beach-location", "California, US");
put("beach-endorsement", "This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim");
}};
Map<String, String> attrs = Map.of("beach-name", "Malibu Beach", "beach-location", "California, US",
"beach-endorsement", "This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim");
testRunner.enqueue(new byte[0], attrs);
testRunner.run();
@ -232,7 +226,7 @@ public class TestAttributesToCSV {
testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0);
testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1);
MockFlowFile flowFile = flowFilesForRelationship.get(0);
MockFlowFile flowFile = flowFilesForRelationship.getFirst();
assertNull(flowFile.getAttribute(CoreAttributes.MIME_TYPE.key()));
@ -252,17 +246,14 @@ public class TestAttributesToCSV {
}
@Test
public void testNoAttrListNoCoreNullOffToContent() throws IOException {
public void testNoAttrListNoCoreNullOffToContent() {
final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToCSV());
testRunner.setProperty(AttributesToCSV.DESTINATION, OUTPUT_OVERWRITE_CONTENT);
testRunner.setProperty(AttributesToCSV.INCLUDE_CORE_ATTRIBUTES, "false");
testRunner.setProperty(AttributesToCSV.NULL_VALUE_FOR_EMPTY_STRING, "false");
Map<String, String> attrs = new HashMap<String, String>() {{
put("beach-name", "Malibu Beach");
put("beach-location", "California, US");
put("beach-endorsement", "This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim");
}};
Map<String, String> attrs = Map.of("beach-name", "Malibu Beach", "beach-location", "California, US",
"beach-endorsement", "This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim");
testRunner.enqueue(new byte[0], attrs);
testRunner.run();
@ -272,13 +263,13 @@ public class TestAttributesToCSV {
testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0);
testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1);
MockFlowFile flowFile = flowFilesForRelationship.get(0);
MockFlowFile flowFile = flowFilesForRelationship.getFirst();
assertEquals(OUTPUT_MIME_TYPE, flowFile.getAttribute(CoreAttributes.MIME_TYPE.key()));
final byte[] contentData = testRunner.getContentAsByteArray(flowFile);
final String contentDataString = new String(contentData, "UTF-8");
final String contentDataString = new String(contentData, StandardCharsets.UTF_8);
Set<String> contentValues = new HashSet<>(getStrings(contentDataString));
assertEquals(3, contentValues.size());
@ -298,12 +289,9 @@ public class TestAttributesToCSV {
testRunner.setProperty(AttributesToCSV.ATTRIBUTES_LIST, "beach-name,beach-location,beach-endorsement");
testRunner.setProperty(AttributesToCSV.NULL_VALUE_FOR_EMPTY_STRING, "false");
Map<String, String> attrs = new HashMap<String, String>() {{
put("beach-name", "Malibu Beach");
put("beach-location", "California, US");
put("beach-endorsement", "This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim");
put("attribute-should-be-eliminated", "This should not be in CSVAttribute!");
}};
Map<String, String> attrs = Map.of("beach-name", "Malibu Beach", "beach-location", "California, US",
"beach-endorsement", "This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim",
"attribute-should-be-eliminated", "This should not be in CSVAttribute!");
testRunner.enqueue(new byte[0], attrs);
testRunner.run();
@ -313,7 +301,7 @@ public class TestAttributesToCSV {
testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0);
testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1);
MockFlowFile flowFile = flowFilesForRelationship.get(0);
MockFlowFile flowFile = flowFilesForRelationship.getFirst();
assertNull(flowFile.getAttribute(CoreAttributes.MIME_TYPE.key()));
@ -337,12 +325,10 @@ public class TestAttributesToCSV {
testRunner.setProperty(AttributesToCSV.ATTRIBUTES_LIST, "beach-name,beach-location,beach-endorsement");
testRunner.setProperty(AttributesToCSV.NULL_VALUE_FOR_EMPTY_STRING, "false");
Map<String, String> attrs = new HashMap<String, String>() {{
put("beach-name", "Malibu Beach");
put("beach-location", "California, US");
put("beach-endorsement", "This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim");
put("attribute-should-be-eliminated", "This should not be in CSVData!");
}};
Map<String, String> attrs = Map.of("beach-name", "Malibu Beach",
"beach-location", "California, US",
"beach-endorsement", "This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim",
"attribute-should-be-eliminated", "This should not be in CSVData!");
testRunner.enqueue(new byte[0], attrs);
testRunner.run();
@ -352,7 +338,7 @@ public class TestAttributesToCSV {
testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0);
testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1);
MockFlowFile flowFile = flowFilesForRelationship.get(0);
MockFlowFile flowFile = flowFilesForRelationship.getFirst();
assertNull(flowFile.getAttribute(CoreAttributes.MIME_TYPE.key()));
@ -379,12 +365,10 @@ public class TestAttributesToCSV {
testRunner.setProperty(AttributesToCSV.ATTRIBUTES_LIST, "beach-name,beach-location,beach-endorsement,uuid");
testRunner.setProperty(AttributesToCSV.NULL_VALUE_FOR_EMPTY_STRING, "false");
Map<String, String> attrs = new HashMap<String, String>() {{
put("beach-name", "Malibu Beach");
put("beach-location", "California, US");
put("beach-endorsement", "This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim");
put("attribute-should-be-eliminated", "This should not be in CSVData!");
}};
Map<String, String> attrs = Map.of("beach-name", "Malibu Beach",
"beach-location", "California, US",
"beach-endorsement", "This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim",
"attribute-should-be-eliminated", "This should not be in CSVData!");
testRunner.enqueue(new byte[0], attrs);
testRunner.run();
@ -394,7 +378,7 @@ public class TestAttributesToCSV {
testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0);
testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1);
MockFlowFile flowFile = flowFilesForRelationship.get(0);
MockFlowFile flowFile = flowFilesForRelationship.getFirst();
assertNull(flowFile.getAttribute(CoreAttributes.MIME_TYPE.key()));
@ -409,8 +393,8 @@ public class TestAttributesToCSV {
assertTrue(CSVDataValues.contains("\"This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim\""));
assertTrue(!CSVDataValues.contains(flowFile.getAttribute("filename")));
assertTrue(!CSVDataValues.contains(flowFile.getAttribute("path")));
assertFalse(CSVDataValues.contains(flowFile.getAttribute("filename")));
assertFalse(CSVDataValues.contains(flowFile.getAttribute("path")));
assertTrue(CSVDataValues.contains(flowFile.getAttribute("uuid")));
}
@ -422,13 +406,11 @@ public class TestAttributesToCSV {
testRunner.setProperty(AttributesToCSV.ATTRIBUTES_LIST, "${myAttribs}");
testRunner.setProperty(AttributesToCSV.NULL_VALUE_FOR_EMPTY_STRING, "false");
Map<String, String> attrs = new HashMap<String, String>() {{
put("beach-name", "Malibu Beach");
put("beach-location", "California, US");
put("beach-endorsement", "This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim");
put("attribute-should-be-eliminated", "This should not be in CSVData!");
put("myAttribs", "beach-name,beach-location,beach-endorsement");
}};
Map<String, String> attrs = Map.of("beach-name", "Malibu Beach",
"beach-location", "California, US",
"beach-endorsement", "This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim",
"attribute-should-be-eliminated", "This should not be in CSVData!",
"myAttribs", "beach-name,beach-location,beach-endorsement");
testRunner.enqueue(new byte[0], attrs);
testRunner.run();
@ -439,7 +421,7 @@ public class TestAttributesToCSV {
testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1);
//Test flow file 0 with ATTRIBUTE_LIST populated from expression language
MockFlowFile flowFile = flowFilesForRelationship.get(0);
MockFlowFile flowFile = flowFilesForRelationship.getFirst();
assertNull(flowFile.getAttribute(CoreAttributes.MIME_TYPE.key()));
@ -458,7 +440,7 @@ public class TestAttributesToCSV {
assertTrue(CSVDataValues.contains(flowFile.getAttribute("uuid")));
//Test flow file 1 with ATTRIBUTE_LIST populated from expression language containing commas (output should be he same)
flowFile = flowFilesForRelationship.get(0);
flowFile = flowFilesForRelationship.getFirst();
assertNull(flowFile.getAttribute(CoreAttributes.MIME_TYPE.key()));
@ -487,13 +469,11 @@ public class TestAttributesToCSV {
testRunner.setProperty(AttributesToCSV.NULL_VALUE_FOR_EMPTY_STRING, "false");
Map<String, String> attrsCommaInName = new HashMap<String, String>() {{
put("beach,name", "Malibu Beach");
put("beach,location", "California, US");
put("beach,endorsement", "This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim");
put("attribute-should-be-eliminated", "This should not be in CSVData!");
put("myAttribs", "\"beach,name\",\"beach,location\",\"beach,endorsement\"");
}};
Map<String, String> attrsCommaInName = Map.of("beach,name", "Malibu Beach",
"beach,location", "California, US",
"beach,endorsement", "This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim",
"attribute-should-be-eliminated", "This should not be in CSVData!",
"myAttribs", "\"beach,name\",\"beach,location\",\"beach,endorsement\"");
testRunner.enqueue(new byte[0], attrsCommaInName);
testRunner.run();
@ -504,7 +484,7 @@ public class TestAttributesToCSV {
testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1);
//Test flow file 0 with ATTRIBUTE_LIST populated from expression language
MockFlowFile flowFile = flowFilesForRelationship.get(0);
MockFlowFile flowFile = flowFilesForRelationship.getFirst();
assertNull(flowFile.getAttribute(CoreAttributes.MIME_TYPE.key()));
@ -523,7 +503,7 @@ public class TestAttributesToCSV {
assertTrue(CSVDataValues.contains(flowFile.getAttribute("uuid")));
//Test flow file 1 with ATTRIBUTE_LIST populated from expression language containing commas (output should be he same)
flowFile = flowFilesForRelationship.get(0);
flowFile = flowFilesForRelationship.getFirst();
assertNull(flowFile.getAttribute(CoreAttributes.MIME_TYPE.key()));
@ -552,13 +532,11 @@ public class TestAttributesToCSV {
testRunner.setProperty(AttributesToCSV.ATTRIBUTES_LIST, "${myAttribs}");
testRunner.setProperty(AttributesToCSV.NULL_VALUE_FOR_EMPTY_STRING, "false");
Map<String, String> attrs = new HashMap<String, String>() {{
put("beach-name", "Malibu Beach");
put("beach-location", "California, US");
put("beach-endorsement", "This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim");
put("attribute-should-be-eliminated", "This should not be in CSVData!");
put("myAttribs", "beach-name,beach-location,beach-endorsement");
}};
Map<String, String> attrs = Map.of("beach-name", "Malibu Beach",
"beach-location", "California, US",
"beach-endorsement", "This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim",
"attribute-should-be-eliminated", "This should not be in CSVData!",
"myAttribs", "beach-name,beach-location,beach-endorsement");
testRunner.enqueue(new byte[0], attrs);
testRunner.run();
@ -568,7 +546,7 @@ public class TestAttributesToCSV {
testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0);
testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1);
MockFlowFile flowFile = flowFilesForRelationship.get(0);
MockFlowFile flowFile = flowFilesForRelationship.getFirst();
assertNull(flowFile.getAttribute(CoreAttributes.MIME_TYPE.key()));
@ -583,9 +561,9 @@ public class TestAttributesToCSV {
assertTrue(CSVDataValues.contains("\"This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim\""));
assertTrue(!CSVDataValues.contains(flowFile.getAttribute("filename")));
assertTrue(!CSVDataValues.contains(flowFile.getAttribute("path")));
assertTrue(!CSVDataValues.contains(flowFile.getAttribute("uuid")));
assertFalse(CSVDataValues.contains(flowFile.getAttribute("filename")));
assertFalse(CSVDataValues.contains(flowFile.getAttribute("path")));
assertFalse(CSVDataValues.contains(flowFile.getAttribute("uuid")));
}
@Test
@ -596,13 +574,11 @@ public class TestAttributesToCSV {
testRunner.setProperty(AttributesToCSV.ATTRIBUTES_REGEX, "${myRegEx}");
testRunner.setProperty(AttributesToCSV.NULL_VALUE_FOR_EMPTY_STRING, "false");
Map<String, String> attrs = new HashMap<String, String>() {{
put("beach-name", "Malibu Beach");
put("beach-location", "California, US");
put("beach-endorsement", "This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim");
put("attribute-should-be-eliminated", "This should not be in CSVData!");
put("myRegEx", "beach-.*");
}};
Map<String, String> attrs = Map.of("beach-name", "Malibu Beach",
"beach-location", "California, US",
"beach-endorsement", "This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim",
"attribute-should-be-eliminated", "This should not be in CSVData!",
"myRegEx", "beach-.*");
testRunner.enqueue(new byte[0], attrs);
testRunner.run();
@ -612,7 +588,7 @@ public class TestAttributesToCSV {
testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0);
testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1);
MockFlowFile flowFile = flowFilesForRelationship.get(0);
MockFlowFile flowFile = flowFilesForRelationship.getFirst();
assertNull(flowFile.getAttribute(CoreAttributes.MIME_TYPE.key()));
@ -627,9 +603,9 @@ public class TestAttributesToCSV {
assertTrue(CSVDataValues.contains("\"This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim\""));
assertTrue(!CSVDataValues.contains(flowFile.getAttribute("filename")));
assertTrue(!CSVDataValues.contains(flowFile.getAttribute("path")));
assertTrue(!CSVDataValues.contains(flowFile.getAttribute("uuid")));
assertFalse(CSVDataValues.contains(flowFile.getAttribute("filename")));
assertFalse(CSVDataValues.contains(flowFile.getAttribute("path")));
assertFalse(CSVDataValues.contains(flowFile.getAttribute("uuid")));
}
@Test
@ -641,15 +617,13 @@ public class TestAttributesToCSV {
testRunner.setProperty(AttributesToCSV.ATTRIBUTES_LIST, "moreInfo1,moreInfo2");
testRunner.setProperty(AttributesToCSV.NULL_VALUE_FOR_EMPTY_STRING, "false");
Map<String, String> attrs = new HashMap<String, String>() {{
put("beach-name", "Malibu Beach");
put("beach-location", "California, US");
put("beach-endorsement", "This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim");
put("attribute-should-be-eliminated", "This should not be in CSVData!");
put("myRegEx", "beach-.*");
put("moreInfo1", "A+ Rating");
put("moreInfo2", "Avg Temp: 61f");
}};
Map<String, String> attrs = Map.of("beach-name", "Malibu Beach",
"beach-location", "California, US",
"beach-endorsement", "This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim",
"attribute-should-be-eliminated", "This should not be in CSVData!",
"myRegEx", "beach-.*",
"moreInfo1", "A+ Rating",
"moreInfo2", "Avg Temp: 61f");
testRunner.enqueue(new byte[0], attrs);
testRunner.run();
@ -659,7 +633,7 @@ public class TestAttributesToCSV {
testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0);
testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1);
MockFlowFile flowFile = flowFilesForRelationship.get(0);
MockFlowFile flowFile = flowFilesForRelationship.getFirst();
assertNull(flowFile.getAttribute(CoreAttributes.MIME_TYPE.key()));
@ -675,9 +649,9 @@ public class TestAttributesToCSV {
assertTrue(CSVDataValues.contains("A+ Rating"));
assertTrue(CSVDataValues.contains("Avg Temp: 61f"));
assertTrue(!CSVDataValues.contains(flowFile.getAttribute("filename")));
assertTrue(!CSVDataValues.contains(flowFile.getAttribute("path")));
assertTrue(!CSVDataValues.contains(flowFile.getAttribute("uuid")));
assertFalse(CSVDataValues.contains(flowFile.getAttribute("filename")));
assertFalse(CSVDataValues.contains(flowFile.getAttribute("path")));
assertFalse(CSVDataValues.contains(flowFile.getAttribute("uuid")));
}
@ -690,30 +664,29 @@ public class TestAttributesToCSV {
testRunner.setProperty(AttributesToCSV.INCLUDE_SCHEMA, "true");
testRunner.setProperty(AttributesToCSV.ATTRIBUTES_REGEX, "beach-.*");
Map<String, String> attrs = new LinkedHashMap<String, String>() {{
put("beach-name", "Malibu Beach");
put("beach-location", "California, US");
put("attribute-should-be-eliminated", "This should not be in CSVData!");
}};
Map<String, String> attrs = new LinkedHashMap<>();
attrs.put("beach-name", "Malibu Beach");
attrs.put("beach-location", "California, US");
attrs.put("attribute-should-be-eliminated", "This should not be in CSVData!");
testRunner.enqueue(new byte[0], attrs);
testRunner.run();
testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS).get(0)
testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS).getFirst()
.assertAttributeExists("CSVData");
testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS).get(0)
testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS).getFirst()
.assertAttributeExists("CSVSchema");
testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1);
testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0);
testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS)
.get(0).assertAttributeEquals("CSVData", "Malibu Beach,\"California, US\"");
.getFirst().assertAttributeEquals("CSVData", "Malibu Beach,\"California, US\"");
testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS)
.get(0).assertAttributeEquals("CSVSchema", "beach-name,beach-location");
.getFirst().assertAttributeEquals("CSVSchema", "beach-name,beach-location");
}
@Test
public void testSchemaToContent() throws Exception {
public void testSchemaToContent() {
final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToCSV());
//set the destination of the csv string to be an attribute
testRunner.setProperty(AttributesToCSV.DESTINATION, OUTPUT_OVERWRITE_CONTENT);
@ -722,11 +695,10 @@ public class TestAttributesToCSV {
testRunner.setProperty(AttributesToCSV.INCLUDE_SCHEMA, "true");
testRunner.setProperty(AttributesToCSV.ATTRIBUTES_REGEX, "beach-.*");
Map<String, String> attrs = new LinkedHashMap<String, String>() {{
put("beach-name", "Malibu Beach");
put("beach-location", "California, US");
put("attribute-should-be-eliminated", "This should not be in CSVData!");
}};
Map<String, String> attrs = new LinkedHashMap<>();
attrs.put("beach-name", "Malibu Beach");
attrs.put("beach-location", "California, US");
attrs.put("attribute-should-be-eliminated", "This should not be in CSVData!");
testRunner.enqueue(new byte[0], attrs);
testRunner.run();
@ -734,13 +706,13 @@ public class TestAttributesToCSV {
testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1);
testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0);
MockFlowFile flowFile = testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS).get(0);
MockFlowFile flowFile = testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS).getFirst();
flowFile.assertAttributeNotExists("CSVData");
flowFile.assertAttributeNotExists("CSVSchema");
final byte[] contentData = testRunner.getContentAsByteArray(flowFile);
final String contentDataString = new String(contentData, "UTF-8");
final String contentDataString = new String(contentData, StandardCharsets.UTF_8);
assertEquals(contentDataString.split(newline)[0], "beach-name,beach-location");
assertEquals(contentDataString.split(newline)[1], "Malibu Beach,\"California, US\"");
}
@ -755,11 +727,10 @@ public class TestAttributesToCSV {
testRunner.setProperty(AttributesToCSV.INCLUDE_SCHEMA, "true");
testRunner.setProperty(AttributesToCSV.ATTRIBUTES_REGEX, "beach-.*");
Map<String, String> attrs = new LinkedHashMap<String, String>() {{
put("beach-name", "Malibu Beach");
put("beach-location", "California, US");
put("attribute-should-be-eliminated", "This should not be in CSVData!");
}};
Map<String, String> attrs = new LinkedHashMap<>();
attrs.put("beach-name", "Malibu Beach");
attrs.put("beach-location", "California, US");
attrs.put("attribute-should-be-eliminated", "This should not be in CSVData!");
testRunner.enqueue(new byte[0], attrs);
testRunner.run();
@ -767,7 +738,7 @@ public class TestAttributesToCSV {
testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1);
testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0);
MockFlowFile flowFile = testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS).get(0);
MockFlowFile flowFile = testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS).getFirst();
flowFile.assertAttributeExists("CSVData");
flowFile.assertAttributeExists("CSVSchema");
@ -780,7 +751,7 @@ public class TestAttributesToCSV {
}
@Test
public void testSchemaWithCoreAttribuesToContent() throws Exception {
public void testSchemaWithCoreAttribuesToContent() {
final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToCSV());
//set the destination of the csv string to be an attribute
testRunner.setProperty(AttributesToCSV.DESTINATION, OUTPUT_OVERWRITE_CONTENT);
@ -789,11 +760,10 @@ public class TestAttributesToCSV {
testRunner.setProperty(AttributesToCSV.INCLUDE_SCHEMA, "true");
testRunner.setProperty(AttributesToCSV.ATTRIBUTES_REGEX, "beach-.*");
Map<String, String> attrs = new LinkedHashMap<String, String>() {{
put("beach-name", "Malibu Beach");
put("beach-location", "California, US");
put("attribute-should-be-eliminated", "This should not be in CSVData!");
}};
Map<String, String> attrs = new LinkedHashMap<>();
attrs.put("beach-name", "Malibu Beach");
attrs.put("beach-location", "California, US");
attrs.put("attribute-should-be-eliminated", "This should not be in CSVData!");
testRunner.enqueue(new byte[0], attrs);
testRunner.run();
@ -801,7 +771,7 @@ public class TestAttributesToCSV {
testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1);
testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0);
MockFlowFile flowFile = testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS).get(0);
MockFlowFile flowFile = testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS).getFirst();
flowFile.assertAttributeNotExists("CSVData");
flowFile.assertAttributeNotExists("CSVSchema");
@ -811,7 +781,7 @@ public class TestAttributesToCSV {
final byte[] contentData = testRunner.getContentAsByteArray(flowFile);
final String contentDataString = new String(contentData, "UTF-8");
final String contentDataString = new String(contentData, StandardCharsets.UTF_8);
assertEquals(contentDataString.split(newline)[0], "beach-name,beach-location,path,filename,uuid");
assertEquals(contentDataString.split(newline)[1], "Malibu Beach,\"California, US\"," + path + "," + filename + "," + uuid);
}

View File

@ -50,7 +50,7 @@ public class TestDebugFlow {
String filename = "testFile" + (n + 1) + ".txt";
String content = "Hello World " + (n + 1) + "!";
contents.put(n, content);
attribs.put(n, new HashMap<String, String>());
attribs.put(n, new HashMap<>());
attribs.get(n).put(CoreAttributes.FILENAME.key(), filename);
attribs.get(n).put(CoreAttributes.UUID.key(), "TESTING-FILE-" + (n + 1) + "-TESTING");
namesToContent.put(filename, content);

View File

@ -21,8 +21,8 @@ import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.junit.jupiter.api.Test;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static org.apache.nifi.processors.standard.DuplicateFlowFile.COPY_INDEX_ATTRIBUTE;
@ -51,9 +51,7 @@ public class TestDuplicateFlowFile {
final TestRunner runner = TestRunners.newTestRunner(DuplicateFlowFile.class);
runner.setProperty(DuplicateFlowFile.NUM_COPIES, "${num.copies}");
runner.enqueue("hello".getBytes(), new HashMap<String, String>() {{
put("num.copies", "100");
}});
runner.enqueue("hello".getBytes(), Map.of("num.copies", "100"));
runner.run();
runner.assertAllFlowFilesTransferred(DuplicateFlowFile.REL_SUCCESS, 101);

View File

@ -21,7 +21,6 @@ import org.apache.avro.file.DataFileStream;
import org.apache.avro.generic.GenericDatumReader;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.io.DatumReader;
import org.apache.commons.compress.compressors.CompressorException;
import org.apache.nifi.controller.AbstractControllerService;
import org.apache.nifi.dbcp.DBCPService;
import org.apache.nifi.flowfile.attributes.CoreAttributes;
@ -116,7 +115,7 @@ public class TestExecuteSQL {
}
@Test
public void testIncomingConnectionWithNoFlowFile() throws InitializationException {
public void testIncomingConnectionWithNoFlowFile() {
runner.setIncomingConnection(true);
runner.setProperty(ExecuteSQL.SQL_SELECT_QUERY, "SELECT * FROM persons");
runner.run();
@ -125,7 +124,7 @@ public class TestExecuteSQL {
}
@Test
public void testIncomingConnectionWithNoFlowFileAndNoQuery() throws InitializationException {
public void testIncomingConnectionWithNoFlowFileAndNoQuery() {
runner.setIncomingConnection(true);
runner.run();
runner.assertTransferCount(ExecuteSQL.REL_SUCCESS, 0);
@ -133,55 +132,52 @@ public class TestExecuteSQL {
}
@Test
public void testNoIncomingConnectionAndNoQuery() throws InitializationException {
public void testNoIncomingConnectionAndNoQuery() {
runner.setIncomingConnection(false);
assertThrows(AssertionError.class, () -> {
runner.run();
});
assertThrows(AssertionError.class, () -> runner.run());
}
@Test
public void testNoIncomingConnection() throws ClassNotFoundException, SQLException, InitializationException, IOException {
public void testNoIncomingConnection() throws SQLException, IOException {
runner.setIncomingConnection(false);
invokeOnTrigger(null, QUERY_WITHOUT_EL, false, null, true);
}
@Test
public void testNoTimeLimit() throws InitializationException, ClassNotFoundException, SQLException, IOException {
public void testNoTimeLimit() throws SQLException, IOException {
invokeOnTrigger(null, QUERY_WITH_EL, true, null, true);
}
@Test
public void testSelectQueryInFlowFile() throws InitializationException, ClassNotFoundException, SQLException, IOException {
public void testSelectQueryInFlowFile() throws SQLException, IOException {
invokeOnTrigger(null, QUERY_WITHOUT_EL, true, null, false);
}
@Test
public void testSelectQueryInFlowFileWithParameters() throws InitializationException, ClassNotFoundException, SQLException, IOException {
Map<String, String> sqlParams = new HashMap<String, String>() {{
put("sql.args.1.type", "4");
put("sql.args.1.value", "20");
put("sql.args.2.type", "4");
put("sql.args.2.value", "5");
}};
public void testSelectQueryInFlowFileWithParameters() throws SQLException, IOException {
Map<String, String> sqlParams = new HashMap<>();
sqlParams.put("sql.args.1.type", "4");
sqlParams.put("sql.args.1.value", "20");
sqlParams.put("sql.args.2.type", "4");
sqlParams.put("sql.args.2.value", "5");
invokeOnTrigger(null, QUERY_WITHOUT_EL_WITH_PARAMS, true, sqlParams, false);
}
@Test
public void testQueryTimeout() throws InitializationException, ClassNotFoundException, SQLException, IOException {
public void testQueryTimeout() throws SQLException, IOException {
// Does to seem to have any effect when using embedded Derby
invokeOnTrigger(1, QUERY_WITH_EL, true, null, true); // 1 second max time
}
@Test
public void testAutoCommitFalse() throws InitializationException, ClassNotFoundException, SQLException, IOException {
public void testAutoCommitFalse() throws SQLException, IOException {
runner.setProperty(ExecuteSQL.AUTO_COMMIT, "false");
invokeOnTrigger(null, QUERY_WITHOUT_EL, true, null, false);
}
@Test
public void testAutoCommitTrue() throws InitializationException, ClassNotFoundException, SQLException, IOException {
public void testAutoCommitTrue() throws SQLException, IOException {
runner.setProperty(ExecuteSQL.AUTO_COMMIT, "true");
invokeOnTrigger(null, QUERY_WITHOUT_EL, true, null, false);
}
@ -198,7 +194,7 @@ public class TestExecuteSQL {
try {
stmt.execute("drop table TEST_NULL_INT");
} catch (final SQLException sqle) {
} catch (final SQLException ignored) {
}
stmt.execute("create table TEST_NULL_INT (id integer not null, val1 integer, val2 integer, constraint my_pk primary key (id))");
@ -211,12 +207,12 @@ public class TestExecuteSQL {
runner.run();
runner.assertAllFlowFilesTransferred(ExecuteSQL.REL_SUCCESS, 1);
runner.getFlowFilesForRelationship(ExecuteSQL.REL_SUCCESS).get(0).assertAttributeEquals(ExecuteSQL.RESULT_ROW_COUNT, "2");
runner.getFlowFilesForRelationship(ExecuteSQL.REL_SUCCESS).get(0).assertAttributeEquals(ExecuteSQL.RESULTSET_INDEX, "0");
runner.getFlowFilesForRelationship(ExecuteSQL.REL_SUCCESS).getFirst().assertAttributeEquals(ExecuteSQL.RESULT_ROW_COUNT, "2");
runner.getFlowFilesForRelationship(ExecuteSQL.REL_SUCCESS).getFirst().assertAttributeEquals(ExecuteSQL.RESULTSET_INDEX, "0");
}
@Test
public void testCompression() throws SQLException, CompressorException, IOException {
public void testCompression() throws SQLException, IOException {
// remove previous test database, if any
final File dbLocation = new File(DB_LOCATION);
dbLocation.delete();
@ -227,7 +223,7 @@ public class TestExecuteSQL {
try {
stmt.execute("drop table TEST_NULL_INT");
} catch (final SQLException sqle) {
} catch (final SQLException ignored) {
}
stmt.execute("create table TEST_NULL_INT (id integer not null, val1 integer, val2 integer, constraint my_pk primary key (id))");
@ -242,9 +238,9 @@ public class TestExecuteSQL {
runner.assertAllFlowFilesTransferred(ExecuteSQL.REL_SUCCESS, 1);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(ExecuteSQL.REL_SUCCESS).get(0);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(ExecuteSQL.REL_SUCCESS).getFirst();
try (DataFileStream<GenericRecord> dfs = new DataFileStream<>(new ByteArrayInputStream(flowFile.toByteArray()), new GenericDatumReader<GenericRecord>())) {
try (DataFileStream<GenericRecord> dfs = new DataFileStream<>(new ByteArrayInputStream(flowFile.toByteArray()), new GenericDatumReader<>())) {
assertEquals(AvroUtil.CodecType.BZIP2.name().toLowerCase(), dfs.getMetaString(DataFileConstants.CODEC).toLowerCase());
}
}
@ -261,7 +257,7 @@ public class TestExecuteSQL {
try {
stmt.execute("drop table TEST_NULL_INT");
} catch (final SQLException sqle) {
} catch (final SQLException ignored) {
}
stmt.execute("create table TEST_NULL_INT (id integer not null, val1 integer, val2 integer, constraint my_pk primary key (id))");
@ -280,7 +276,7 @@ public class TestExecuteSQL {
runner.assertAllFlowFilesContainAttribute(ExecuteSQL.REL_SUCCESS, FragmentAttributes.FRAGMENT_INDEX.key());
runner.assertAllFlowFilesContainAttribute(ExecuteSQL.REL_SUCCESS, FragmentAttributes.FRAGMENT_ID.key());
MockFlowFile firstFlowFile = runner.getFlowFilesForRelationship(ExecuteSQL.REL_SUCCESS).get(0);
MockFlowFile firstFlowFile = runner.getFlowFilesForRelationship(ExecuteSQL.REL_SUCCESS).getFirst();
firstFlowFile.assertAttributeEquals(ExecuteSQL.RESULT_ROW_COUNT, "5");
firstFlowFile.assertAttributeNotExists(FragmentAttributes.FRAGMENT_COUNT.key());
@ -306,7 +302,7 @@ public class TestExecuteSQL {
try {
stmt.execute("drop table TEST_NULL_INT");
} catch (final SQLException sqle) {
} catch (final SQLException ignored) {
}
stmt.execute("create table TEST_NULL_INT (id integer not null, val1 integer, val2 integer, constraint my_pk primary key (id))");
@ -332,7 +328,7 @@ public class TestExecuteSQL {
runner.assertAllFlowFilesContainAttribute(ExecuteSQL.REL_SUCCESS, FragmentAttributes.FRAGMENT_INDEX.key());
runner.assertAllFlowFilesContainAttribute(ExecuteSQL.REL_SUCCESS, FragmentAttributes.FRAGMENT_ID.key());
MockFlowFile firstFlowFile = runner.getFlowFilesForRelationship(ExecuteSQL.REL_SUCCESS).get(0);
MockFlowFile firstFlowFile = runner.getFlowFilesForRelationship(ExecuteSQL.REL_SUCCESS).getFirst();
firstFlowFile.assertAttributeEquals(ExecuteSQL.RESULT_ROW_COUNT, "5");
firstFlowFile.assertAttributeNotExists(FragmentAttributes.FRAGMENT_COUNT.key());
@ -362,7 +358,7 @@ public class TestExecuteSQL {
try {
stmt.execute("drop table TEST_NULL_INT");
} catch (final SQLException sqle) {
} catch (final SQLException ignored) {
}
stmt.execute("create table TEST_NULL_INT (id integer not null, val1 integer, val2 integer, constraint my_pk primary key (id))");
@ -383,7 +379,7 @@ public class TestExecuteSQL {
runner.assertAllFlowFilesContainAttribute(ExecuteSQL.REL_SUCCESS, FragmentAttributes.FRAGMENT_ID.key());
runner.assertAllFlowFilesContainAttribute(ExecuteSQL.REL_SUCCESS, FragmentAttributes.FRAGMENT_COUNT.key());
MockFlowFile firstFlowFile = runner.getFlowFilesForRelationship(ExecuteSQL.REL_SUCCESS).get(0);
MockFlowFile firstFlowFile = runner.getFlowFilesForRelationship(ExecuteSQL.REL_SUCCESS).getFirst();
firstFlowFile.assertAttributeEquals(ExecuteSQL.RESULT_ROW_COUNT, "5");
firstFlowFile.assertAttributeEquals(FragmentAttributes.FRAGMENT_INDEX.key(), "0");
@ -408,7 +404,7 @@ public class TestExecuteSQL {
try {
stmt.execute("drop table TEST_NULL_INT");
} catch (final SQLException sqle) {
} catch (final SQLException ignored) {
}
stmt.execute("create table TEST_NULL_INT (id integer not null, val1 integer, val2 integer, constraint my_pk primary key (id))");
@ -418,7 +414,7 @@ public class TestExecuteSQL {
runner.run();
runner.assertAllFlowFilesTransferred(ExecuteSQL.REL_SUCCESS, 1);
runner.getFlowFilesForRelationship(ExecuteSQL.REL_SUCCESS).get(0).assertAttributeEquals(ExecuteSQL.RESULT_ROW_COUNT, "0");
runner.getFlowFilesForRelationship(ExecuteSQL.REL_SUCCESS).getFirst().assertAttributeEquals(ExecuteSQL.RESULT_ROW_COUNT, "0");
}
@Test
@ -433,7 +429,7 @@ public class TestExecuteSQL {
try {
stmt.execute("drop table TEST_NULL_INT");
} catch (final SQLException sqle) {
} catch (final SQLException ignored) {
}
stmt.execute("create table TEST_NULL_INT (id integer not null, val1 integer, val2 integer, constraint my_pk primary key (id))");
@ -444,7 +440,7 @@ public class TestExecuteSQL {
runner.run();
runner.assertAllFlowFilesTransferred(ExecuteSQL.REL_SUCCESS, 1);
MockFlowFile firstFlowFile = runner.getFlowFilesForRelationship(ExecuteSQL.REL_SUCCESS).get(0);
MockFlowFile firstFlowFile = runner.getFlowFilesForRelationship(ExecuteSQL.REL_SUCCESS).getFirst();
firstFlowFile.assertAttributeEquals(ExecuteSQL.RESULT_ROW_COUNT, "0");
final InputStream in = new ByteArrayInputStream(firstFlowFile.toByteArray());
final DatumReader<GenericRecord> datumReader = new GenericDatumReader<>();
@ -476,7 +472,7 @@ public class TestExecuteSQL {
try {
stmt.execute("drop table host1");
stmt.execute("drop table host2");
} catch (final SQLException sqle) {
} catch (final SQLException ignored) {
}
stmt.execute("create table host1 (id integer not null, host varchar(45))");
@ -489,7 +485,7 @@ public class TestExecuteSQL {
runner.run();
runner.assertAllFlowFilesTransferred(ExecuteSQL.REL_SUCCESS, 1);
runner.getFlowFilesForRelationship(ExecuteSQL.REL_SUCCESS).get(0).assertAttributeEquals(ExecuteSQL.RESULT_ROW_COUNT, "1");
runner.getFlowFilesForRelationship(ExecuteSQL.REL_SUCCESS).getFirst().assertAttributeEquals(ExecuteSQL.RESULT_ROW_COUNT, "1");
}
@Test
@ -504,7 +500,7 @@ public class TestExecuteSQL {
try {
stmt.execute("drop table TEST_NO_ROWS");
} catch (final SQLException sqle) {
} catch (final SQLException ignored) {
}
stmt.execute("create table TEST_NO_ROWS (id integer)");
@ -547,15 +543,15 @@ public class TestExecuteSQL {
runner.assertTransferCount(ExecuteSQL.REL_SUCCESS, 0);
// Assert exception message has been put to flow file attribute
MockFlowFile failedFlowFile = runner.getFlowFilesForRelationship(ExecuteSQL.REL_FAILURE).get(0);
MockFlowFile failedFlowFile = runner.getFlowFilesForRelationship(ExecuteSQL.REL_FAILURE).getFirst();
assertEquals("java.sql.SQLException: test execute statement failed", failedFlowFile.getAttribute(ExecuteSQL.RESULT_ERROR_MESSAGE));
}
public void invokeOnTrigger(final Integer queryTimeout, final String query, final boolean incomingFlowFile, final Map<String, String> attrs, final boolean setQueryProperty)
throws InitializationException, ClassNotFoundException, SQLException, IOException {
throws SQLException, IOException {
if (queryTimeout != null) {
runner.setProperty(ExecuteSQL.QUERY_TIMEOUT, queryTimeout.toString() + " secs");
runner.setProperty(ExecuteSQL.QUERY_TIMEOUT, queryTimeout + " secs");
}
// remove previous test database, if any
@ -599,13 +595,13 @@ public class TestExecuteSQL {
runner.assertAllFlowFilesContainAttribute(ExecuteSQL.REL_SUCCESS, ExecuteSQL.RESULT_ROW_COUNT);
final List<MockFlowFile> flowfiles = runner.getFlowFilesForRelationship(ExecuteSQL.REL_SUCCESS);
final long executionTime = Long.parseLong(flowfiles.get(0).getAttribute(ExecuteSQL.RESULT_QUERY_EXECUTION_TIME));
final long fetchTime = Long.parseLong(flowfiles.get(0).getAttribute(ExecuteSQL.RESULT_QUERY_FETCH_TIME));
final long durationTime = Long.parseLong(flowfiles.get(0).getAttribute(ExecuteSQL.RESULT_QUERY_DURATION));
final long executionTime = Long.parseLong(flowfiles.getFirst().getAttribute(ExecuteSQL.RESULT_QUERY_EXECUTION_TIME));
final long fetchTime = Long.parseLong(flowfiles.getFirst().getAttribute(ExecuteSQL.RESULT_QUERY_FETCH_TIME));
final long durationTime = Long.parseLong(flowfiles.getFirst().getAttribute(ExecuteSQL.RESULT_QUERY_DURATION));
assertEquals(durationTime, fetchTime + executionTime);
final InputStream in = new ByteArrayInputStream(flowfiles.get(0).toByteArray());
final InputStream in = new ByteArrayInputStream(flowfiles.getFirst().toByteArray());
final DatumReader<GenericRecord> datumReader = new GenericDatumReader<>();
try (DataFileStream<GenericRecord> dataFileReader = new DataFileStream<>(in, datumReader)) {
GenericRecord record = null;
@ -635,7 +631,7 @@ public class TestExecuteSQL {
try {
stmt.execute("drop table TEST_NULL_INT");
} catch (final SQLException sqle) {
} catch (final SQLException ignored) {
}
stmt.execute("create table TEST_NULL_INT (id integer not null, val1 integer, val2 integer, constraint my_pk primary key (id))");
@ -648,7 +644,7 @@ public class TestExecuteSQL {
runner.run();
runner.assertAllFlowFilesTransferred(ExecuteSQL.REL_SUCCESS, 1);
MockFlowFile firstFlowFile = runner.getFlowFilesForRelationship(ExecuteSQL.REL_SUCCESS).get(0);
MockFlowFile firstFlowFile = runner.getFlowFilesForRelationship(ExecuteSQL.REL_SUCCESS).getFirst();
firstFlowFile.assertAttributeEquals(ExecuteSQL.RESULT_ROW_COUNT, "1");
final InputStream in = new ByteArrayInputStream(firstFlowFile.toByteArray());
@ -680,7 +676,7 @@ public class TestExecuteSQL {
try {
stmt.execute("drop table TEST_NULL_INT");
} catch (final SQLException sqle) {
} catch (final SQLException ignored) {
}
stmt.execute("create table TEST_NULL_INT (id integer not null, val1 integer, val2 integer, constraint my_pk primary key (id))");
@ -694,7 +690,7 @@ public class TestExecuteSQL {
runner.run();
runner.assertAllFlowFilesTransferred(ExecuteSQL.REL_SUCCESS, 1);
MockFlowFile firstFlowFile = runner.getFlowFilesForRelationship(ExecuteSQL.REL_SUCCESS).get(0);
MockFlowFile firstFlowFile = runner.getFlowFilesForRelationship(ExecuteSQL.REL_SUCCESS).getFirst();
firstFlowFile.assertAttributeEquals(ExecuteSQL.RESULT_ROW_COUNT, "1");
final InputStream in = new ByteArrayInputStream(firstFlowFile.toByteArray());
@ -726,7 +722,7 @@ public class TestExecuteSQL {
try {
stmt.execute("drop table TEST_NULL_INT");
} catch (final SQLException sqle) {
} catch (final SQLException ignored) {
}
stmt.execute("create table TEST_NULL_INT (id integer not null, val1 integer, val2 integer, constraint my_pk primary key (id))");
@ -753,7 +749,7 @@ public class TestExecuteSQL {
try {
stmt.execute("drop table TEST_NULL_INT");
} catch (final SQLException sqle) {
} catch (final SQLException ignored) {
}
stmt.execute("create table TEST_NULL_INT (id integer not null, val1 integer, val2 integer, constraint my_pk primary key (id))");
@ -767,7 +763,7 @@ public class TestExecuteSQL {
runner.run();
runner.assertAllFlowFilesTransferred(ExecuteSQL.REL_FAILURE, 1);
MockFlowFile firstFlowFile = runner.getFlowFilesForRelationship(ExecuteSQL.REL_FAILURE).get(0);
MockFlowFile firstFlowFile = runner.getFlowFilesForRelationship(ExecuteSQL.REL_FAILURE).getFirst();
firstFlowFile.assertContentEquals("test");
}

View File

@ -298,7 +298,7 @@ public class TestForkRecord {
fields.add(new RecordField("accounts", accountsType));
final RecordSchema schema = new SimpleRecordSchema(fields);
final List<RecordField> fieldsWrite = new ArrayList<RecordField>();
final List<RecordField> fieldsWrite = new ArrayList<>();
fieldsWrite.add(new RecordField("id", RecordFieldType.INT.getDataType()));
fieldsWrite.add(new RecordField("amount", RecordFieldType.DOUBLE.getDataType()));
final RecordSchema schemaWrite = new SimpleRecordSchema(fieldsWrite);

View File

@ -23,7 +23,6 @@ import org.apache.nifi.controller.AbstractControllerService;
import org.apache.nifi.dbcp.DBCPService;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processors.standard.db.impl.DerbyDatabaseAdapter;
import org.apache.nifi.reporting.InitializationException;
import org.apache.nifi.util.MockFlowFile;
import org.apache.nifi.util.MockProcessSession;
import org.apache.nifi.util.MockSessionFactory;
@ -156,7 +155,7 @@ public class TestGenerateTableFetch {
// Verify the expected FlowFile
runner.assertAllFlowFilesTransferred(REL_SUCCESS, 1);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(0);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).getFirst();
String query = new String(flowFile.toByteArray());
assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE WHERE ID <= 2 ORDER BY ID FETCH NEXT 10000 ROWS ONLY", query);
flowFile.assertAttributeEquals(FRAGMENT_INDEX, "0");
@ -192,7 +191,7 @@ public class TestGenerateTableFetch {
assertEquals(ff2.getAttribute(FRAGMENT_COUNT), "2");
// Verify first flow file's contents
flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(0);
flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).getFirst();
query = new String(flowFile.toByteArray());
assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE WHERE ID > 2 AND ID <= 5 ORDER BY ID FETCH NEXT 2 ROWS ONLY", query);
resultSet = stmt.executeQuery(query);
@ -215,7 +214,7 @@ public class TestGenerateTableFetch {
stmt.execute("insert into TEST_QUERY_DB_TABLE (id, name, scale, created_on) VALUES (6, 'Mr. NiFi', 1.0, '2012-01-01 03:23:34.234')");
runner.run();
runner.assertAllFlowFilesTransferred(REL_SUCCESS, 1);
flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(0);
flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).getFirst();
query = new String(flowFile.toByteArray());
assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE WHERE ID > 5 AND ID <= 6 ORDER BY ID FETCH NEXT 2 ROWS ONLY", query);
resultSet = stmt.executeQuery(query);
@ -249,7 +248,7 @@ public class TestGenerateTableFetch {
}
@Test
public void testAddedRowsTwoTables() throws ClassNotFoundException, SQLException, InitializationException, IOException {
public void testAddedRowsTwoTables() throws SQLException {
// load test data to database
final Connection con = ((DBCPService) runner.getControllerService("dbcp")).getConnection();
@ -278,7 +277,7 @@ public class TestGenerateTableFetch {
runner.run();
runner.assertAllFlowFilesTransferred(REL_SUCCESS, 1);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(0);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).getFirst();
String query = new String(flowFile.toByteArray());
assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE WHERE ID <= 2 ORDER BY ID FETCH NEXT 10000 ROWS ONLY", query);
ResultSet resultSet = stmt.executeQuery(query);
@ -304,7 +303,7 @@ public class TestGenerateTableFetch {
runner.run();
runner.assertAllFlowFilesTransferred(REL_SUCCESS, 1);
flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(0);
flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).getFirst();
query = new String(flowFile.toByteArray());
assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE2 WHERE ID <= 2 ORDER BY ID FETCH NEXT 10000 ROWS ONLY", query);
resultSet = stmt.executeQuery(query);
@ -324,7 +323,7 @@ public class TestGenerateTableFetch {
runner.assertAllFlowFilesTransferred(REL_SUCCESS, 2);
// Verify first flow file's contents
flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(0);
flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).getFirst();
query = new String(flowFile.toByteArray());
assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE2 WHERE ID > 2 AND ID <= 5 ORDER BY ID FETCH NEXT 2 ROWS ONLY", query);
resultSet = stmt.executeQuery(query);
@ -345,7 +344,7 @@ public class TestGenerateTableFetch {
}
@Test
public void testAddedRowsRightBounded() throws ClassNotFoundException, SQLException, InitializationException, IOException {
public void testAddedRowsRightBounded() throws SQLException, IOException {
// load test data to database
final Connection con = ((DBCPService) runner.getControllerService("dbcp")).getConnection();
@ -368,7 +367,7 @@ public class TestGenerateTableFetch {
runner.run();
runner.assertAllFlowFilesTransferred(REL_SUCCESS, 1);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(0);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).getFirst();
String query = new String(flowFile.toByteArray());
assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE WHERE ID <= 2 ORDER BY ID FETCH NEXT 10000 ROWS ONLY", query);
ResultSet resultSet = stmt.executeQuery(query);
@ -393,7 +392,7 @@ public class TestGenerateTableFetch {
runner.assertAllFlowFilesTransferred(REL_SUCCESS, 2);
// Verify first flow file's contents
flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(0);
flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).getFirst();
query = new String(flowFile.toByteArray());
assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE WHERE ID > 2 AND ID <= 5 ORDER BY ID FETCH NEXT 2 ROWS ONLY", query);
resultSet = stmt.executeQuery(query);
@ -416,7 +415,7 @@ public class TestGenerateTableFetch {
stmt.execute("insert into TEST_QUERY_DB_TABLE (id, name, scale, created_on) VALUES (6, 'Mr. NiFi', 1.0, '2012-01-01 03:23:34.234')");
runner.run();
runner.assertAllFlowFilesTransferred(REL_SUCCESS, 1);
flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(0);
flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).getFirst();
query = new String(flowFile.toByteArray());
assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE WHERE ID > 5 AND ID <= 6 ORDER BY ID FETCH NEXT 2 ROWS ONLY", query);
resultSet = stmt.executeQuery(query);
@ -443,7 +442,7 @@ public class TestGenerateTableFetch {
}
@Test
public void testAddedRowsTimestampRightBounded() throws ClassNotFoundException, SQLException, InitializationException, IOException {
public void testAddedRowsTimestampRightBounded() throws SQLException {
// load test data to database
final Connection con = ((DBCPService) runner.getControllerService("dbcp")).getConnection();
@ -466,7 +465,7 @@ public class TestGenerateTableFetch {
runner.run();
runner.assertAllFlowFilesTransferred(REL_SUCCESS, 1);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(0);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).getFirst();
String query = new String(flowFile.toByteArray());
assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE WHERE created_on <= '2010-01-01 00:00:00.0' ORDER BY created_on FETCH NEXT 10000 ROWS ONLY", query);
ResultSet resultSet = stmt.executeQuery(query);
@ -493,7 +492,7 @@ public class TestGenerateTableFetch {
runner.assertAllFlowFilesTransferred(REL_SUCCESS, 2);
// Verify first flow file's contents
flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(0);
flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).getFirst();
query = new String(flowFile.toByteArray());
assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE WHERE created_on > '2010-01-01 00:00:00.0' AND "
+ "created_on <= '2011-01-01 04:23:34.236' ORDER BY created_on FETCH NEXT 2 ROWS ONLY", query);
@ -518,7 +517,7 @@ public class TestGenerateTableFetch {
stmt.execute("insert into TEST_QUERY_DB_TABLE (id, name, scale, created_on) VALUES (8, 'Mr. NiFi', 1.0, '2012-01-01 03:23:34.234')");
runner.run();
runner.assertAllFlowFilesTransferred(REL_SUCCESS, 1);
flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(0);
flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).getFirst();
query = new String(flowFile.toByteArray());
assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE WHERE created_on > '2011-01-01 04:23:34.236' AND created_on <= '2012-01-01 03:23:34.234' ORDER BY created_on FETCH NEXT 2 ROWS ONLY", query);
resultSet = stmt.executeQuery(query);
@ -529,7 +528,7 @@ public class TestGenerateTableFetch {
}
@Test
public void testOnePartition() throws ClassNotFoundException, SQLException, InitializationException, IOException {
public void testOnePartition() throws SQLException {
// load test data to database
final Connection con = ((DBCPService) runner.getControllerService("dbcp")).getConnection();
@ -554,7 +553,7 @@ public class TestGenerateTableFetch {
runner.run();
runner.assertAllFlowFilesTransferred(GenerateTableFetch.REL_SUCCESS, 1);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(GenerateTableFetch.REL_SUCCESS).get(0);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(GenerateTableFetch.REL_SUCCESS).getFirst();
flowFile.assertContentEquals("SELECT * FROM TEST_QUERY_DB_TABLE WHERE ID <= 2");
flowFile.assertAttributeExists("generatetablefetch.limit");
flowFile.assertAttributeEquals("generatetablefetch.limit", null);
@ -591,7 +590,7 @@ public class TestGenerateTableFetch {
runner.setProperty(GenerateTableFetch.OUTPUT_EMPTY_FLOWFILE_ON_ZERO_RESULTS, "true");
runner.run();
runner.assertAllFlowFilesTransferred(GenerateTableFetch.REL_SUCCESS, 1);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(GenerateTableFetch.REL_SUCCESS).get(0);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(GenerateTableFetch.REL_SUCCESS).getFirst();
assertEquals("TEST_QUERY_DB_TABLE", flowFile.getAttribute("generatetablefetch.tableName"));
assertEquals("ID,BUCKET", flowFile.getAttribute("generatetablefetch.columnNames"));
assertEquals("1=1", flowFile.getAttribute("generatetablefetch.whereClause"));
@ -603,7 +602,7 @@ public class TestGenerateTableFetch {
}
@Test
public void testMultiplePartitions() throws ClassNotFoundException, SQLException, InitializationException, IOException {
public void testMultiplePartitions() throws SQLException {
// load test data to database
final Connection con = ((DBCPService) runner.getControllerService("dbcp")).getConnection();
@ -654,7 +653,7 @@ public class TestGenerateTableFetch {
}
@Test
public void testMultiplePartitionsIncomingFlowFiles() throws ClassNotFoundException, SQLException, InitializationException, IOException {
public void testMultiplePartitionsIncomingFlowFiles() throws SQLException {
// load test data to database
final Connection con = ((DBCPService) runner.getControllerService("dbcp")).getConnection();
@ -684,21 +683,12 @@ public class TestGenerateTableFetch {
runner.setIncomingConnection(true);
runner.setProperty(GenerateTableFetch.PARTITION_SIZE, "${partSize}");
runner.enqueue("".getBytes(), new HashMap<String, String>() {{
put("tableName", "TEST_QUERY_DB_TABLE1");
put("partSize", "1");
}});
runner.enqueue("".getBytes(), Map.of("tableName", "TEST_QUERY_DB_TABLE1", "partSize", "1"));
runner.enqueue("".getBytes(), new HashMap<String, String>() {{
put("tableName", "TEST_QUERY_DB_TABLE2");
put("partSize", "2");
}});
runner.enqueue("".getBytes(), Map.of("tableName", "TEST_QUERY_DB_TABLE2", "partSize", "2"));
// The table does not exist, expect the original flow file to be routed to failure
runner.enqueue("".getBytes(), new HashMap<String, String>() {{
put("tableName", "TEST_QUERY_DB_TABLE3");
put("partSize", "1");
}});
runner.enqueue("".getBytes(), Map.of("tableName", "TEST_QUERY_DB_TABLE3", "partSize", "1"));
runner.run(3);
runner.assertTransferCount(AbstractDatabaseFetchProcessor.REL_SUCCESS, 3);
@ -747,15 +737,11 @@ public class TestGenerateTableFetch {
runner.setProperty(GenerateTableFetch.TABLE_NAME, "TEST_QUERY_DB_TABLE");
runner.setIncomingConnection(true);
runner.setProperty(GenerateTableFetch.MAX_VALUE_COLUMN_NAMES, "${maxValueCol}");
runner.enqueue("".getBytes(), new HashMap<String, String>() {{
put("maxValueCol", "id");
}});
runner.enqueue("".getBytes(), Map.of("maxValueCol", "id"));
// Pre-populate the state with a key for column name (not fully-qualified)
StateManager stateManager = runner.getStateManager();
stateManager.setState(new HashMap<String, String>() {{
put("id", "0");
}}, Scope.CLUSTER);
stateManager.setState(Map.of("id", "0"), Scope.CLUSTER);
// Pre-populate the column type map with an entry for id (not fully-qualified)
processor.columnTypeMap.put("id", 4);
@ -763,7 +749,7 @@ public class TestGenerateTableFetch {
runner.run();
runner.assertAllFlowFilesTransferred(REL_SUCCESS, 1);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(0);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).getFirst();
assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE WHERE id > 0 AND id <= 1 ORDER BY id FETCH NEXT 10000 ROWS ONLY", new String(flowFile.toByteArray()));
}
@ -786,16 +772,11 @@ public class TestGenerateTableFetch {
runner.setProperty(GenerateTableFetch.TABLE_NAME, "${tableName}");
runner.setIncomingConnection(true);
runner.setProperty(GenerateTableFetch.MAX_VALUE_COLUMN_NAMES, "${maxValueCol}");
runner.enqueue("".getBytes(), new HashMap<String, String>() {{
put("tableName", "TEST_QUERY_DB_TABLE");
put("maxValueCol", "id");
}});
runner.enqueue("".getBytes(), Map.of("tableName", "TEST_QUERY_DB_TABLE", "maxValueCol", "id"));
// Pre-populate the state with a key for column name (not fully-qualified)
StateManager stateManager = runner.getStateManager();
stateManager.setState(new HashMap<String, String>() {{
put("id", "0");
}}, Scope.CLUSTER);
stateManager.setState(Map.of("id", "0"), Scope.CLUSTER);
// Pre-populate the column type map with an entry for id (not fully-qualified)
processor.columnTypeMap.put("id", 4);
@ -803,11 +784,11 @@ public class TestGenerateTableFetch {
runner.run();
runner.assertAllFlowFilesTransferred(REL_SUCCESS, 1);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(0);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).getFirst();
// Note there is no WHERE clause here. Because we are using dynamic tables, the old state key/value is not retrieved
assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE WHERE id <= 1 ORDER BY id FETCH NEXT 10000 ROWS ONLY", new String(flowFile.toByteArray()));
assertEquals("TEST_QUERY_DB_TABLE", flowFile.getAttribute("generatetablefetch.tableName"));
assertEquals(null, flowFile.getAttribute("generatetablefetch.columnNames"));
assertNull(flowFile.getAttribute("generatetablefetch.columnNames"));
assertEquals("id <= 1", flowFile.getAttribute("generatetablefetch.whereClause"));
assertEquals("id", flowFile.getAttribute("generatetablefetch.maxColumnNames"));
assertEquals("10000", flowFile.getAttribute("generatetablefetch.limit"));
@ -816,17 +797,14 @@ public class TestGenerateTableFetch {
runner.clearTransferState();
stmt.execute("insert into TEST_QUERY_DB_TABLE (id, bucket) VALUES (2, 0)");
runner.enqueue("".getBytes(), new HashMap<String, String>() {{
put("tableName", "TEST_QUERY_DB_TABLE");
put("maxValueCol", "id");
}});
runner.enqueue("".getBytes(), Map.of("tableName", "TEST_QUERY_DB_TABLE", "maxValueCol", "id"));
runner.run();
runner.assertAllFlowFilesTransferred(REL_SUCCESS, 1);
flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(0);
flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).getFirst();
assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE WHERE id > 1 AND id <= 2 ORDER BY id FETCH NEXT 10000 ROWS ONLY", new String(flowFile.toByteArray()));
assertEquals("TEST_QUERY_DB_TABLE", flowFile.getAttribute("generatetablefetch.tableName"));
assertEquals(null, flowFile.getAttribute("generatetablefetch.columnNames"));
assertNull(flowFile.getAttribute("generatetablefetch.columnNames"));
assertEquals("id > 1 AND id <= 2", flowFile.getAttribute("generatetablefetch.whereClause"));
assertEquals("id", flowFile.getAttribute("generatetablefetch.maxColumnNames"));
assertEquals("10000", flowFile.getAttribute("generatetablefetch.limit"));
@ -852,15 +830,11 @@ public class TestGenerateTableFetch {
runner.setProperty(GenerateTableFetch.TABLE_NAME, "${tableName}");
runner.setIncomingConnection(true);
runner.setProperty(GenerateTableFetch.MAX_VALUE_COLUMN_NAMES, "id");
runner.enqueue("".getBytes(), new HashMap<String, String>() {{
put("tableName", "TEST_QUERY_DB_TABLE");
}});
runner.enqueue("".getBytes(), Map.of("tableName", "TEST_QUERY_DB_TABLE"));
// Pre-populate the state with a key for column name (not fully-qualified)
StateManager stateManager = runner.getStateManager();
stateManager.setState(new HashMap<String, String>() {{
put("id", "0");
}}, Scope.CLUSTER);
stateManager.setState(Map.of("id", "0"), Scope.CLUSTER);
// Pre-populate the column type map with an entry for id (not fully-qualified)
processor.columnTypeMap.put("id", 4);
@ -868,21 +842,18 @@ public class TestGenerateTableFetch {
runner.run();
runner.assertAllFlowFilesTransferred(REL_SUCCESS, 1);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(0);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).getFirst();
// Note there is no WHERE clause here. Because we are using dynamic tables, the old state key/value is not retrieved
assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE WHERE id <= 1 ORDER BY id FETCH NEXT 10000 ROWS ONLY", new String(flowFile.toByteArray()));
runner.clearTransferState();
stmt.execute("insert into TEST_QUERY_DB_TABLE (id, bucket) VALUES (2, 0)");
runner.enqueue("".getBytes(), new HashMap<String, String>() {{
put("tableName", "TEST_QUERY_DB_TABLE");
put("maxValueCol", "id");
}});
runner.enqueue("".getBytes(), Map.of("tableName", "TEST_QUERY_DB_TABLE", "maxValueCol", "id"));
runner.run();
runner.assertAllFlowFilesTransferred(REL_SUCCESS, 1);
flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(0);
flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).getFirst();
assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE WHERE id > 1 AND id <= 2 ORDER BY id FETCH NEXT 10000 ROWS ONLY", new String(flowFile.toByteArray()));
}
@ -911,24 +882,21 @@ public class TestGenerateTableFetch {
// Pre-populate the state with a key for column name (not fully-qualified)
StateManager stateManager = runner.getStateManager();
stateManager.setState(new HashMap<String, String>() {{
put("id", "0");
}}, Scope.CLUSTER);
stateManager.setState(Map.of("id", "0"), Scope.CLUSTER);
// Pre-populate the column type map with an entry for id (not fully-qualified)
processor.columnTypeMap.put("id", 4);
runner.run();
runner.assertAllFlowFilesTransferred(REL_SUCCESS, 1);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(0);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).getFirst();
// Note there is no WHERE clause here. Because we are using dynamic tables (i.e. Expression Language,
// even when not referring to flow file attributes), the old state key/value is not retrieved
assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE WHERE id <= 1 ORDER BY id FETCH NEXT 10000 ROWS ONLY", new String(flowFile.toByteArray()));
}
@Test
public void testRidiculousRowCount() throws ClassNotFoundException, SQLException, InitializationException, IOException {
public void testRidiculousRowCount() throws SQLException {
long rowCount = Long.parseLong(Integer.toString(Integer.MAX_VALUE)) + 100;
int partitionSize = 1000000;
int expectedFileCount = (int) (rowCount / partitionSize) + 1;
@ -962,14 +930,14 @@ public class TestGenerateTableFetch {
runner.run();
runner.assertAllFlowFilesTransferred(REL_SUCCESS, expectedFileCount);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(0);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).getFirst();
String query = new String(flowFile.toByteArray());
assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE WHERE 1=1 ORDER BY ID FETCH NEXT 1000000 ROWS ONLY", query);
runner.clearTransferState();
}
@Test
public void testInitialMaxValue() throws ClassNotFoundException, SQLException, InitializationException, IOException {
public void testInitialMaxValue() throws SQLException {
// load test data to database
final Connection con = ((DBCPService) runner.getControllerService("dbcp")).getConnection();
@ -993,7 +961,7 @@ public class TestGenerateTableFetch {
runner.run();
runner.assertAllFlowFilesTransferred(REL_SUCCESS, 1);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(0);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).getFirst();
String query = new String(flowFile.toByteArray());
assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE WHERE ID > 1 AND ID <= 2 ORDER BY ID FETCH NEXT 10000 ROWS ONLY", query);
ResultSet resultSet = stmt.executeQuery(query);
@ -1017,7 +985,7 @@ public class TestGenerateTableFetch {
runner.assertAllFlowFilesTransferred(REL_SUCCESS, 2);
// Verify first flow file's contents
flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(0);
flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).getFirst();
query = new String(flowFile.toByteArray());
assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE WHERE ID > 2 AND ID <= 5 ORDER BY ID FETCH NEXT 2 ROWS ONLY", query);
resultSet = stmt.executeQuery(query);
@ -1038,7 +1006,7 @@ public class TestGenerateTableFetch {
}
@Test
public void testInitialMaxValueWithEL() throws ClassNotFoundException, SQLException, InitializationException, IOException {
public void testInitialMaxValueWithEL() throws SQLException {
// load test data to database
final Connection con = ((DBCPService) runner.getControllerService("dbcp")).getConnection();
@ -1063,7 +1031,7 @@ public class TestGenerateTableFetch {
runner.run();
runner.assertAllFlowFilesTransferred(REL_SUCCESS, 1);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(0);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).getFirst();
String query = new String(flowFile.toByteArray());
assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE WHERE ID > 1 AND ID <= 2 ORDER BY ID FETCH NEXT 10000 ROWS ONLY", query);
ResultSet resultSet = stmt.executeQuery(query);
@ -1079,7 +1047,7 @@ public class TestGenerateTableFetch {
}
@Test
public void testInitialMaxValueWithELAndIncoming() throws ClassNotFoundException, SQLException, InitializationException, IOException {
public void testInitialMaxValueWithELAndIncoming() throws SQLException {
// load test data to database
final Connection con = ((DBCPService) runner.getControllerService("dbcp")).getConnection();
@ -1099,14 +1067,12 @@ public class TestGenerateTableFetch {
runner.setProperty(GenerateTableFetch.TABLE_NAME, "TEST_QUERY_DB_TABLE");
runner.setProperty(GenerateTableFetch.MAX_VALUE_COLUMN_NAMES, "ID");
runner.setProperty("initial.maxvalue.ID", "${maxval.id}");
Map<String, String> attrs = new HashMap<String, String>() {{
put("maxval.id", "1");
}};
Map<String, String> attrs = Map.of("maxval.id", "1");
runner.setIncomingConnection(true);
runner.enqueue(new byte[0], attrs);
runner.run();
runner.assertAllFlowFilesTransferred(REL_SUCCESS, 1);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(0);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).getFirst();
String query = new String(flowFile.toByteArray());
assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE WHERE ID > 1 AND ID <= 2 ORDER BY ID FETCH NEXT 10000 ROWS ONLY", query);
ResultSet resultSet = stmt.executeQuery(query);
@ -1123,7 +1089,7 @@ public class TestGenerateTableFetch {
}
@Test
public void testInitialMaxValueWithELAndMultipleTables() throws ClassNotFoundException, SQLException, InitializationException, IOException {
public void testInitialMaxValueWithELAndMultipleTables() throws SQLException {
// load test data to database
final Connection con = ((DBCPService) runner.getControllerService("dbcp")).getConnection();
@ -1143,15 +1109,13 @@ public class TestGenerateTableFetch {
runner.setProperty(GenerateTableFetch.TABLE_NAME, "${table.name}");
runner.setProperty(GenerateTableFetch.MAX_VALUE_COLUMN_NAMES, "ID");
runner.setProperty("initial.maxvalue.ID", "${maxval.id}");
Map<String, String> attrs = new HashMap<String, String>() {{
put("maxval.id", "1");
put("table.name", "TEST_QUERY_DB_TABLE");
}};
runner.setIncomingConnection(true);
Map<String, String> attrs = new HashMap<>();
attrs.put("maxval.id", "1");
attrs.put("table.name", "TEST_QUERY_DB_TABLE"); runner.setIncomingConnection(true);
runner.enqueue(new byte[0], attrs);
runner.run();
runner.assertAllFlowFilesTransferred(REL_SUCCESS, 1);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(0);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).getFirst();
String query = new String(flowFile.toByteArray());
assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE WHERE ID > 1 AND ID <= 2 ORDER BY ID FETCH NEXT 10000 ROWS ONLY", query);
ResultSet resultSet = stmt.executeQuery(query);
@ -1182,7 +1146,7 @@ public class TestGenerateTableFetch {
runner.enqueue(new byte[0], attrs);
runner.run();
runner.assertAllFlowFilesTransferred(REL_SUCCESS, 1);
flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(0);
flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).getFirst();
query = new String(flowFile.toByteArray());
assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE2 WHERE ID > 1 AND ID <= 2 ORDER BY ID FETCH NEXT 10000 ROWS ONLY", query);
resultSet = stmt.executeQuery(query);
@ -1199,7 +1163,7 @@ public class TestGenerateTableFetch {
}
@Test
public void testNoDuplicateWithRightBounded() throws ClassNotFoundException, SQLException, InitializationException, IOException {
public void testNoDuplicateWithRightBounded() throws SQLException {
// load test data to database
final Connection con = ((DBCPService) runner.getControllerService("dbcp")).getConnection();
@ -1222,7 +1186,7 @@ public class TestGenerateTableFetch {
runner.run();
runner.assertAllFlowFilesTransferred(REL_SUCCESS, 1);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(0);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).getFirst();
String query = new String(flowFile.toByteArray());
// we now insert a row before the query issued by GFT is actually executed by, let's say, ExecuteSQL processor
@ -1240,7 +1204,7 @@ public class TestGenerateTableFetch {
// Run again
runner.run();
runner.assertAllFlowFilesTransferred(REL_SUCCESS, 1);
flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(0);
flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).getFirst();
query = new String(flowFile.toByteArray());
resultSet = stmt.executeQuery(query);
@ -1256,7 +1220,7 @@ public class TestGenerateTableFetch {
}
@Test
public void testAddedRowsWithCustomWhereClause() throws ClassNotFoundException, SQLException, InitializationException, IOException {
public void testAddedRowsWithCustomWhereClause() throws SQLException, IOException {
// load test data to database
final Connection con = ((DBCPService) runner.getControllerService("dbcp")).getConnection();
@ -1280,7 +1244,7 @@ public class TestGenerateTableFetch {
runner.run();
runner.assertAllFlowFilesTransferred(REL_SUCCESS, 1);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(0);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).getFirst();
String query = new String(flowFile.toByteArray());
assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE WHERE (type = 'male' OR type IS NULL)"
+ " AND ID <= 2 ORDER BY ID FETCH NEXT 10000 ROWS ONLY", query);
@ -1305,7 +1269,7 @@ public class TestGenerateTableFetch {
runner.assertAllFlowFilesTransferred(REL_SUCCESS, 2);
// Verify first flow file's contents
flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(0);
flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).getFirst();
query = new String(flowFile.toByteArray());
assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE WHERE ID > 2 AND (type = 'male' OR type IS NULL)"
+ " AND ID <= 5 ORDER BY ID FETCH NEXT 1 ROWS ONLY", query);
@ -1329,7 +1293,7 @@ public class TestGenerateTableFetch {
stmt.execute("insert into TEST_QUERY_DB_TABLE (id, type, name, scale, created_on) VALUES (6, 'male', 'Mr. NiFi', 1.0, '2012-01-01 03:23:34.234')");
runner.run();
runner.assertAllFlowFilesTransferred(REL_SUCCESS, 1);
flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(0);
flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).getFirst();
query = new String(flowFile.toByteArray());
assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE WHERE ID > 5 AND (type = 'male' OR type IS NULL)"
+ " AND ID <= 6 ORDER BY ID FETCH NEXT 1 ROWS ONLY", query);
@ -1372,7 +1336,7 @@ public class TestGenerateTableFetch {
}
@Test
public void testColumnTypeMissing() throws ClassNotFoundException, SQLException, InitializationException, IOException {
public void testColumnTypeMissing() throws SQLException {
// Load test data to database
final Connection con = ((DBCPService) runner.getControllerService("dbcp")).getConnection();
Statement stmt = con.createStatement();
@ -1393,13 +1357,10 @@ public class TestGenerateTableFetch {
runner.setProperty(GenerateTableFetch.TABLE_NAME, "${tableName}");
runner.setIncomingConnection(true);
runner.setProperty(GenerateTableFetch.MAX_VALUE_COLUMN_NAMES, "${maxValueCol}");
runner.enqueue("".getBytes(), new HashMap<String, String>() {{
put("tableName", "TEST_QUERY_DB_TABLE");
put("maxValueCol", "id");
}});
runner.enqueue("".getBytes(), Map.of("tableName", "TEST_QUERY_DB_TABLE", "maxValueCol", "id"));
runner.run();
runner.assertAllFlowFilesTransferred(REL_SUCCESS, 1);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(0);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).getFirst();
String query = new String(flowFile.toByteArray());
assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE WHERE id <= 1 ORDER BY id FETCH NEXT 10000 ROWS ONLY", query);
runner.clearTransferState();
@ -1412,22 +1373,19 @@ public class TestGenerateTableFetch {
stmt.execute("insert into TEST_QUERY_DB_TABLE (id, bucket) VALUES (2, 0)");
// Re-launch FlowFile to se if re-cache column type works
runner.enqueue("".getBytes(), new HashMap<String, String>() {{
put("tableName", "TEST_QUERY_DB_TABLE");
put("maxValueCol", "id");
}});
runner.enqueue("".getBytes(), Map.of("tableName", "TEST_QUERY_DB_TABLE", "maxValueCol", "id"));
// It should re-cache column type
runner.run();
runner.assertAllFlowFilesTransferred(REL_SUCCESS, 1);
flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(0);
flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).getFirst();
query = new String(flowFile.toByteArray());
assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE WHERE id > 1 AND id <= 2 ORDER BY id FETCH NEXT 10000 ROWS ONLY", query);
runner.clearTransferState();
}
@Test
public void testMultipleColumnTypeMissing() throws ClassNotFoundException, SQLException, InitializationException, IOException {
public void testMultipleColumnTypeMissing() throws SQLException {
// Load test data to database
final Connection con = ((DBCPService) runner.getControllerService("dbcp")).getConnection();
@ -1450,15 +1408,9 @@ public class TestGenerateTableFetch {
runner.setIncomingConnection(true);
runner.setProperty(GenerateTableFetch.MAX_VALUE_COLUMN_NAMES, "${maxValueCol}");
runner.enqueue("".getBytes(), new HashMap<String, String>() {{
put("tableName", "TEST_QUERY_DB_TABLE");
put("maxValueCol", "id");
}});
runner.enqueue("".getBytes(), Map.of("tableName", "TEST_QUERY_DB_TABLE", "maxValueCol", "id"));
runner.enqueue("".getBytes(), new HashMap<String, String>() {{
put("tableName", "TEST_QUERY_DB_TABLE_2");
put("maxValueCol", "id");
}});
runner.enqueue("".getBytes(), Map.of("tableName", "TEST_QUERY_DB_TABLE_2", "maxValueCol", "id"));
runner.run(2);
runner.assertAllFlowFilesTransferred(REL_SUCCESS, 2);
@ -1473,10 +1425,7 @@ public class TestGenerateTableFetch {
stmt.execute("insert into TEST_QUERY_DB_TABLE (id, bucket) VALUES (2, 0)");
// Re-launch FlowFile to se if re-cache column type works
runner.enqueue("".getBytes(), new HashMap<String, String>() {{
put("tableName", "TEST_QUERY_DB_TABLE");
put("maxValueCol", "id");
}});
runner.enqueue("".getBytes(), Map.of("tableName", "TEST_QUERY_DB_TABLE", "maxValueCol", "id"));
// It should re-cache column type
runner.run();
@ -1486,7 +1435,7 @@ public class TestGenerateTableFetch {
}
@Test
public void testUseColumnValuesForPartitioning() throws ClassNotFoundException, SQLException, InitializationException, IOException {
public void testUseColumnValuesForPartitioning() throws SQLException {
// load test data to database
final Connection con = ((DBCPService) runner.getControllerService("dbcp")).getConnection();
@ -1512,7 +1461,7 @@ public class TestGenerateTableFetch {
runner.run();
runner.assertAllFlowFilesTransferred(REL_SUCCESS, 2);
// First flow file
MockFlowFile flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(0);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).getFirst();
String query = new String(flowFile.toByteArray());
assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE WHERE ID <= 12 AND ID >= 10 AND ID < 12", query);
ResultSet resultSet = stmt.executeQuery(query);
@ -1543,7 +1492,7 @@ public class TestGenerateTableFetch {
runner.assertAllFlowFilesTransferred(REL_SUCCESS, 3);
// Verify first flow file's contents
flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(0);
flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).getFirst();
query = new String(flowFile.toByteArray());
assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE WHERE ID > 12 AND ID <= 24 AND ID >= 20 AND ID < 22", query);
resultSet = stmt.executeQuery(query);
@ -1572,7 +1521,7 @@ public class TestGenerateTableFetch {
}
@Test
public void testUseColumnValuesForPartitioningNoMaxValueColumn() throws ClassNotFoundException, SQLException, InitializationException, IOException {
public void testUseColumnValuesForPartitioningNoMaxValueColumn() throws SQLException {
// load test data to database
final Connection con = ((DBCPService) runner.getControllerService("dbcp")).getConnection();
@ -1597,7 +1546,7 @@ public class TestGenerateTableFetch {
runner.run();
runner.assertAllFlowFilesTransferred(REL_SUCCESS, 2);
// First flow file
MockFlowFile flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(0);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).getFirst();
String query = new String(flowFile.toByteArray());
assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE WHERE 1=1 AND ID >= 10 AND ID < 12", query);
ResultSet resultSet = stmt.executeQuery(query);
@ -1622,7 +1571,7 @@ public class TestGenerateTableFetch {
}
@Test
public void testCustomOrderByColumn() throws SQLException, IOException {
public void testCustomOrderByColumn() throws SQLException {
// load test data to database
final Connection con = ((DBCPService) runner.getControllerService("dbcp")).getConnection();
@ -1646,7 +1595,7 @@ public class TestGenerateTableFetch {
runner.run();
runner.assertAllFlowFilesTransferred(REL_SUCCESS, 2);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).get(0);
MockFlowFile flowFile = runner.getFlowFilesForRelationship(REL_SUCCESS).getFirst();
String query = new String(flowFile.toByteArray());
assertEquals("SELECT * FROM TEST_QUERY_DB_TABLE WHERE 1=1 ORDER BY SCALE FETCH NEXT 2 ROWS ONLY", query);
flowFile.assertAttributeEquals(FRAGMENT_INDEX, "0");

View File

@ -45,6 +45,7 @@ import java.io.ByteArrayOutputStream;
import java.io.File;
import java.io.IOException;
import java.io.InputStream;
import java.nio.charset.StandardCharsets;
import java.nio.file.Paths;
import java.util.Collection;
import java.util.HashMap;
@ -81,10 +82,10 @@ public class TestMergeContent {
runner.assertTransferCount(MergeContent.REL_ORIGINAL, 1);
runner.assertTransferCount(MergeContent.REL_MERGED, 1);
runner.assertTransferCount(MergeContent.REL_FAILURE, 0);
assertEquals(runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).get(0).getAttribute(CoreAttributes.UUID.key()),
runner.getFlowFilesForRelationship(MergeContent.REL_ORIGINAL).get(0).getAttribute(MergeContent.MERGE_UUID_ATTRIBUTE));
assertEquals(runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).getFirst().getAttribute(CoreAttributes.UUID.key()),
runner.getFlowFilesForRelationship(MergeContent.REL_ORIGINAL).getFirst().getAttribute(MergeContent.MERGE_UUID_ATTRIBUTE));
final MockFlowFile bundle = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).get(0);
final MockFlowFile bundle = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).getFirst();
assertEquals(1024 * 6, bundle.getSize());
// Queue should not be empty because the first FlowFile will be transferred back to the input queue
@ -130,7 +131,7 @@ public class TestMergeContent {
runner.assertTransferCount(MergeContent.REL_FAILURE, 0);
runner.assertTransferCount(MergeContent.REL_ORIGINAL, 3);
final MockFlowFile bundle = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).get(0);
final MockFlowFile bundle = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).getFirst();
bundle.assertAttributeEquals(CoreAttributes.MIME_TYPE.key(), "application/avro-binary");
// create a reader for the merged content
@ -180,7 +181,7 @@ public class TestMergeContent {
runner.assertTransferCount(MergeContent.REL_FAILURE, 1);
runner.assertTransferCount(MergeContent.REL_ORIGINAL, 3);
final MockFlowFile bundle = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).get(0);
final MockFlowFile bundle = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).getFirst();
bundle.assertAttributeEquals(CoreAttributes.MIME_TYPE.key(), "application/avro-binary");
final byte[] data = runner.getContentAsByteArray(bundle);
@ -189,7 +190,7 @@ public class TestMergeContent {
assertTrue(users.containsKey("Alyssa"));
assertTrue(users.containsKey("John"));
final MockFlowFile failure = runner.getFlowFilesForRelationship(MergeContent.REL_FAILURE).get(0);
final MockFlowFile failure = runner.getFlowFilesForRelationship(MergeContent.REL_FAILURE).getFirst();
final byte[] failureData = runner.getContentAsByteArray(failure);
final Map<String, GenericRecord> places = getGenericRecordMap(failureData, schema2, "name");
assertEquals(1, places.size());
@ -209,26 +210,19 @@ public class TestMergeContent {
final GenericRecord user1 = new GenericData.Record(schema);
user1.put("name", "Alyssa");
user1.put("favorite_number", 256);
final Map<String, String> userMeta1 = new HashMap<String, String>() {{
put("test_metadata1", "Test 1");
}};
final Map<String, String> userMeta1 = Map.of("test_metadata1", "Test 1");
final GenericRecord user2 = new GenericData.Record(schema);
user2.put("name", "Ben");
user2.put("favorite_number", 7);
user2.put("favorite_color", "red");
final Map<String, String> userMeta2 = new HashMap<String, String>() {{
put("test_metadata1", "Test 2"); // Test non-matching values
}};
final Map<String, String> userMeta2 = Map.of("test_metadata1", "Test 2"); // Test non-matching values
final GenericRecord user3 = new GenericData.Record(schema);
user3.put("name", "John");
user3.put("favorite_number", 5);
user3.put("favorite_color", "blue");
final Map<String, String> userMeta3 = new HashMap<String, String>() {{
put("test_metadata1", "Test 1");
put("test_metadata2", "Test"); // Test unique
}};
final Map<String, String> userMeta3 = Map.of("test_metadata1", "Test 1", "test_metadata2", "Test"); // Test unique
final DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<>(schema);
final ByteArrayOutputStream out1 = serializeAvroRecord(schema, user1, datumWriter, userMeta1);
@ -245,7 +239,7 @@ public class TestMergeContent {
runner.assertTransferCount(MergeContent.REL_FAILURE, 2);
runner.assertTransferCount(MergeContent.REL_ORIGINAL, 3);
final MockFlowFile bundle = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).get(0);
final MockFlowFile bundle = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).getFirst();
bundle.assertAttributeEquals(CoreAttributes.MIME_TYPE.key(), "application/avro-binary");
// create a reader for the merged content
@ -269,26 +263,19 @@ public class TestMergeContent {
final GenericRecord user1 = new GenericData.Record(schema);
user1.put("name", "Alyssa");
user1.put("favorite_number", 256);
final Map<String, String> userMeta1 = new HashMap<String, String>() {{
put("test_metadata1", "Test 1");
}};
final Map<String, String> userMeta1 = Map.of("test_metadata1", "Test 1");
final GenericRecord user2 = new GenericData.Record(schema);
user2.put("name", "Ben");
user2.put("favorite_number", 7);
user2.put("favorite_color", "red");
final Map<String, String> userMeta2 = new HashMap<String, String>() {{
put("test_metadata1", "Test 2"); // Test non-matching values
}};
final Map<String, String> userMeta2 = Map.of("test_metadata1", "Test 2"); // Test non-matching values
final GenericRecord user3 = new GenericData.Record(schema);
user3.put("name", "John");
user3.put("favorite_number", 5);
user3.put("favorite_color", "blue");
final Map<String, String> userMeta3 = new HashMap<String, String>() {{
put("test_metadata1", "Test 1");
put("test_metadata2", "Test"); // Test unique
}};
final Map<String, String> userMeta3 = Map.of("test_metadata1", "Test 1", "test_metadata2", "Test"); // Test unique
final DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<>(schema);
final ByteArrayOutputStream out1 = serializeAvroRecord(schema, user1, datumWriter, userMeta1);
@ -305,7 +292,7 @@ public class TestMergeContent {
runner.assertTransferCount(MergeContent.REL_FAILURE, 0);
runner.assertTransferCount(MergeContent.REL_ORIGINAL, 3);
final MockFlowFile bundle = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).get(0);
final MockFlowFile bundle = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).getFirst();
bundle.assertAttributeEquals(CoreAttributes.MIME_TYPE.key(), "application/avro-binary");
// create a reader for the merged content
@ -331,26 +318,19 @@ public class TestMergeContent {
final GenericRecord user1 = new GenericData.Record(schema);
user1.put("name", "Alyssa");
user1.put("favorite_number", 256);
final Map<String, String> userMeta1 = new HashMap<String, String>() {{
put("test_metadata1", "Test 1");
}};
final Map<String, String> userMeta1 = Map.of("test_metadata1", "Test 1");
final GenericRecord user2 = new GenericData.Record(schema);
user2.put("name", "Ben");
user2.put("favorite_number", 7);
user2.put("favorite_color", "red");
final Map<String, String> userMeta2 = new HashMap<String, String>() {{
put("test_metadata1", "Test 2"); // Test non-matching values
}};
final Map<String, String> userMeta2 = Map.of("test_metadata1", "Test 2"); // Test non-matching values
final GenericRecord user3 = new GenericData.Record(schema);
user3.put("name", "John");
user3.put("favorite_number", 5);
user3.put("favorite_color", "blue");
final Map<String, String> userMeta3 = new HashMap<String, String>() {{
put("test_metadata1", "Test 1");
put("test_metadata2", "Test"); // Test unique
}};
final Map<String, String> userMeta3 = Map.of("test_metadata1", "Test 1", "test_metadata2", "Test"); // Test unique
final DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<>(schema);
final ByteArrayOutputStream out1 = serializeAvroRecord(schema, user1, datumWriter, userMeta1);
@ -367,7 +347,7 @@ public class TestMergeContent {
runner.assertTransferCount(MergeContent.REL_FAILURE, 0);
runner.assertTransferCount(MergeContent.REL_ORIGINAL, 3);
final MockFlowFile bundle = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).get(0);
final MockFlowFile bundle = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).getFirst();
bundle.assertAttributeEquals(CoreAttributes.MIME_TYPE.key(), "application/avro-binary");
// create a reader for the merged content
@ -393,26 +373,19 @@ public class TestMergeContent {
final GenericRecord user1 = new GenericData.Record(schema);
user1.put("name", "Alyssa");
user1.put("favorite_number", 256);
final Map<String, String> userMeta1 = new HashMap<String, String>() {{
put("test_metadata1", "Test 1");
}};
final Map<String, String> userMeta1 = Map.of("test_metadata1", "Test 1");
final GenericRecord user2 = new GenericData.Record(schema);
user2.put("name", "Ben");
user2.put("favorite_number", 7);
user2.put("favorite_color", "red");
final Map<String, String> userMeta2 = new HashMap<String, String>() {{
put("test_metadata1", "Test 2"); // Test non-matching values
}};
final Map<String, String> userMeta2 = Map.of("test_metadata1", "Test 2"); // Test non-matching values
final GenericRecord user3 = new GenericData.Record(schema);
user3.put("name", "John");
user3.put("favorite_number", 5);
user3.put("favorite_color", "blue");
final Map<String, String> userMeta3 = new HashMap<String, String>() {{
put("test_metadata1", "Test 1");
put("test_metadata2", "Test"); // Test unique
}};
final Map<String, String> userMeta3 = Map.of("test_metadata1", "Test 1", "test_metadata2", "Test"); // Test unique
final DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<>(schema);
final ByteArrayOutputStream out1 = serializeAvroRecord(schema, user1, datumWriter, userMeta1);
@ -429,7 +402,7 @@ public class TestMergeContent {
runner.assertTransferCount(MergeContent.REL_FAILURE, 1);
runner.assertTransferCount(MergeContent.REL_ORIGINAL, 3);
final MockFlowFile bundle = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).get(0);
final MockFlowFile bundle = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).getFirst();
bundle.assertAttributeEquals(CoreAttributes.MIME_TYPE.key(), "application/avro-binary");
// create a reader for the merged content
@ -486,11 +459,11 @@ public class TestMergeContent {
runner.assertTransferCount(MergeContent.REL_FAILURE, 0);
runner.assertTransferCount(MergeContent.REL_ORIGINAL, 3);
final MockFlowFile bundle = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).get(0);
bundle.assertContentEquals("Hello, World!".getBytes("UTF-8"));
final MockFlowFile bundle = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).getFirst();
bundle.assertContentEquals("Hello, World!".getBytes(StandardCharsets.UTF_8));
bundle.assertAttributeEquals(CoreAttributes.MIME_TYPE.key(), "application/plain-text");
runner.getFlowFilesForRelationship(MergeContent.REL_ORIGINAL).stream().forEach(
runner.getFlowFilesForRelationship(MergeContent.REL_ORIGINAL).forEach(
ff -> assertEquals(bundle.getAttribute(CoreAttributes.UUID.key()), ff.getAttribute(MergeContent.MERGE_UUID_ATTRIBUTE)));
}
@ -509,7 +482,7 @@ public class TestMergeContent {
runner.assertTransferCount(MergeContent.REL_FAILURE, 0);
runner.assertTransferCount(MergeContent.REL_ORIGINAL, 3);
final MockFlowFile bundle = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).get(0);
final MockFlowFile bundle = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).getFirst();
bundle.assertContentEquals("Hello, World!");
bundle.assertAttributeEquals(CoreAttributes.MIME_TYPE.key(), "application/plain-text");
}
@ -532,8 +505,8 @@ public class TestMergeContent {
runner.assertTransferCount(MergeContent.REL_FAILURE, 0);
runner.assertTransferCount(MergeContent.REL_ORIGINAL, 3);
final MockFlowFile bundle = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).get(0);
bundle.assertContentEquals("@Hello#, #World!$".getBytes("UTF-8"));
final MockFlowFile bundle = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).getFirst();
bundle.assertContentEquals("@Hello#, #World!$".getBytes(StandardCharsets.UTF_8));
bundle.assertAttributeEquals(CoreAttributes.MIME_TYPE.key(), "application/plain-text");
}
@ -553,13 +526,13 @@ public class TestMergeContent {
runner.assertTransferCount(MergeContent.REL_FAILURE, 0);
runner.assertTransferCount(MergeContent.REL_ORIGINAL, 3);
final MockFlowFile bundle = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).get(0);
bundle.assertContentEquals("@Hello, World!".getBytes("UTF-8"));
final MockFlowFile bundle = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).getFirst();
bundle.assertContentEquals("@Hello, World!".getBytes(StandardCharsets.UTF_8));
bundle.assertAttributeEquals(CoreAttributes.MIME_TYPE.key(), "application/plain-text");
}
@Test
public void testSimpleBinaryConcatWithFileDelimiters() throws IOException, InterruptedException {
public void testSimpleBinaryConcatWithFileDelimiters() throws IOException {
final TestRunner runner = TestRunners.newTestRunner(new MergeContent());
runner.setProperty(MergeContent.MAX_BIN_AGE, "1 sec");
runner.setProperty(MergeContent.MERGE_FORMAT, MergeContent.MERGE_FORMAT_CONCAT);
@ -575,9 +548,9 @@ public class TestMergeContent {
attributes.put("demarcator", "src/test/resources/TestMergeContent/demarcate");
attributes.put("footer", "src/test/resources/TestMergeContent/foot");
runner.enqueue("Hello".getBytes("UTF-8"), attributes);
runner.enqueue(", ".getBytes("UTF-8"), attributes);
runner.enqueue("World!".getBytes("UTF-8"), attributes);
runner.enqueue("Hello".getBytes(StandardCharsets.UTF_8), attributes);
runner.enqueue(", ".getBytes(StandardCharsets.UTF_8), attributes);
runner.enqueue("World!".getBytes(StandardCharsets.UTF_8), attributes);
runner.run(2);
runner.assertQueueEmpty();
@ -585,8 +558,8 @@ public class TestMergeContent {
runner.assertTransferCount(MergeContent.REL_FAILURE, 0);
runner.assertTransferCount(MergeContent.REL_ORIGINAL, 3);
final MockFlowFile bundle = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).get(0);
bundle.assertContentEquals("(|)Hello***, ***World!___".getBytes("UTF-8"));
final MockFlowFile bundle = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).getFirst();
bundle.assertContentEquals("(|)Hello***, ***World!___".getBytes(StandardCharsets.UTF_8));
bundle.assertAttributeEquals(CoreAttributes.MIME_TYPE.key(), "application/plain-text");
}
@ -631,8 +604,7 @@ public class TestMergeContent {
Collection<ValidationResult> results = new HashSet<>();
ProcessContext context = runner.getProcessContext();
if (context instanceof MockProcessContext) {
MockProcessContext mockContext = (MockProcessContext) context;
if (context instanceof MockProcessContext mockContext) {
results = mockContext.validate();
}
@ -662,7 +634,7 @@ public class TestMergeContent {
assertEquals(3, results.size());
for (ValidationResult vr : results) {
assertTrue(vr.toString().contains("is invalid because File " + new File(doesNotExistFile).toString() + " does not exist"));
assertTrue(vr.toString().contains("is invalid because File " + new File(doesNotExistFile) + " does not exist"));
}
}
@ -684,8 +656,8 @@ public class TestMergeContent {
runner.assertTransferCount(MergeContent.REL_FAILURE, 0);
runner.assertTransferCount(MergeContent.REL_ORIGINAL, 4);
final MockFlowFile bundle = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).get(0);
bundle.assertContentEquals("Hello, World!".getBytes("UTF-8"));
final MockFlowFile bundle = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).getFirst();
bundle.assertContentEquals("Hello, World!".getBytes(StandardCharsets.UTF_8));
bundle.assertAttributeEquals(CoreAttributes.MIME_TYPE.key(), "application/octet-stream");
}
@ -768,25 +740,25 @@ public class TestMergeContent {
runner.assertTransferCount(MergeContent.REL_FAILURE, 0);
runner.assertTransferCount(MergeContent.REL_ORIGINAL, 3);
final MockFlowFile bundle = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).get(0);
final MockFlowFile bundle = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).getFirst();
try (final InputStream rawIn = new ByteArrayInputStream(runner.getContentAsByteArray(bundle)); final ZipInputStream in = new ZipInputStream(rawIn)) {
assertNotNull(in.getNextEntry());
final byte[] part1 = IOUtils.toByteArray(in);
assertArrayEquals("Hello".getBytes("UTF-8"), part1);
assertArrayEquals("Hello".getBytes(StandardCharsets.UTF_8), part1);
in.getNextEntry();
final byte[] part2 = IOUtils.toByteArray(in);
assertArrayEquals(", ".getBytes("UTF-8"), part2);
assertArrayEquals(", ".getBytes(StandardCharsets.UTF_8), part2);
in.getNextEntry();
final byte[] part3 = IOUtils.toByteArray(in);
assertArrayEquals("World!".getBytes("UTF-8"), part3);
assertArrayEquals("World!".getBytes(StandardCharsets.UTF_8), part3);
}
bundle.assertAttributeEquals(CoreAttributes.MIME_TYPE.key(), "application/zip");
}
@Test
public void testZipException() throws IOException {
public void testZipException() {
final TestRunner runner = TestRunners.newTestRunner(new MergeContent());
runner.setProperty(MergeContent.MAX_BIN_AGE, "1 sec");
runner.setProperty(MergeContent.MERGE_FORMAT, MergeContent.MERGE_FORMAT_ZIP);
@ -795,9 +767,9 @@ public class TestMergeContent {
attributes.put(CoreAttributes.MIME_TYPE.key(), "application/plain-text");
attributes.put("filename", "duplicate-filename.txt");
runner.enqueue("Hello".getBytes("UTF-8"), attributes);
runner.enqueue(", ".getBytes("UTF-8"), attributes);
runner.enqueue("World!".getBytes("UTF-8"), attributes);
runner.enqueue("Hello".getBytes(StandardCharsets.UTF_8), attributes);
runner.enqueue(", ".getBytes(StandardCharsets.UTF_8), attributes);
runner.enqueue("World!".getBytes(StandardCharsets.UTF_8), attributes);
runner.run(2);
runner.assertQueueEmpty();
@ -816,11 +788,11 @@ public class TestMergeContent {
attributes.put(CoreAttributes.MIME_TYPE.key(), "application/plain-text");
attributes.put(CoreAttributes.FILENAME.key(), "AShortFileName");
runner.enqueue("Hello".getBytes("UTF-8"), attributes);
runner.enqueue("Hello".getBytes(StandardCharsets.UTF_8), attributes);
attributes.put(CoreAttributes.FILENAME.key(), "ALongerrrFileName");
runner.enqueue(", ".getBytes("UTF-8"), attributes);
runner.enqueue(", ".getBytes(StandardCharsets.UTF_8), attributes);
attributes.put(CoreAttributes.FILENAME.key(), "AReallyLongggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggFileName");
runner.enqueue("World!".getBytes("UTF-8"), attributes);
runner.enqueue("World!".getBytes(StandardCharsets.UTF_8), attributes);
runner.run(2);
runner.assertQueueEmpty();
@ -828,23 +800,23 @@ public class TestMergeContent {
runner.assertTransferCount(MergeContent.REL_FAILURE, 0);
runner.assertTransferCount(MergeContent.REL_ORIGINAL, 3);
final MockFlowFile bundle = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).get(0);
final MockFlowFile bundle = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).getFirst();
try (final InputStream rawIn = new ByteArrayInputStream(runner.getContentAsByteArray(bundle)); final TarArchiveInputStream in = new TarArchiveInputStream(rawIn)) {
ArchiveEntry entry = in.getNextEntry();
assertNotNull(entry);
assertEquals("AShortFileName", entry.getName());
final byte[] part1 = IOUtils.toByteArray(in);
assertArrayEquals("Hello".getBytes("UTF-8"), part1);
assertArrayEquals("Hello".getBytes(StandardCharsets.UTF_8), part1);
entry = in.getNextEntry();
assertEquals("ALongerrrFileName", entry.getName());
final byte[] part2 = IOUtils.toByteArray(in);
assertArrayEquals(", ".getBytes("UTF-8"), part2);
assertArrayEquals(", ".getBytes(StandardCharsets.UTF_8), part2);
entry = in.getNextEntry();
assertEquals("AReallyLongggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggFileName", entry.getName());
final byte[] part3 = IOUtils.toByteArray(in);
assertArrayEquals("World!".getBytes("UTF-8"), part3);
assertArrayEquals("World!".getBytes(StandardCharsets.UTF_8), part3);
}
bundle.assertAttributeEquals(CoreAttributes.MIME_TYPE.key(), "application/tar");
}
@ -867,7 +839,7 @@ public class TestMergeContent {
runner.assertTransferCount(MergeContent.REL_FAILURE, 0);
runner.assertTransferCount(MergeContent.REL_ORIGINAL, 2);
final MockFlowFile merged = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).get(0);
final MockFlowFile merged = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).getFirst();
merged.assertAttributeEquals(CoreAttributes.MIME_TYPE.key(), StandardFlowFileMediaType.VERSION_3.getMediaType());
}
@ -882,19 +854,19 @@ public class TestMergeContent {
attributes.put(MergeContent.FRAGMENT_COUNT_ATTRIBUTE, "4");
attributes.put(MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "1");
runner.enqueue("A Man ".getBytes("UTF-8"), attributes);
runner.enqueue("A Man ".getBytes(StandardCharsets.UTF_8), attributes);
attributes.put(MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "2");
runner.enqueue("A Plan ".getBytes("UTF-8"), attributes);
runner.enqueue("A Plan ".getBytes(StandardCharsets.UTF_8), attributes);
attributes.put(MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "3");
runner.enqueue("A Canal ".getBytes("UTF-8"), attributes);
runner.enqueue("A Canal ".getBytes(StandardCharsets.UTF_8), attributes);
attributes.put(MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "4");
runner.enqueue("Panama".getBytes("UTF-8"), attributes);
runner.enqueue("Panama".getBytes(StandardCharsets.UTF_8), attributes);
runner.run();
runner.assertTransferCount(MergeContent.REL_MERGED, 1);
final MockFlowFile assembled = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).get(0);
assembled.assertContentEquals("A Man A Plan A Canal Panama".getBytes("UTF-8"));
final MockFlowFile assembled = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).getFirst();
assembled.assertContentEquals("A Man A Plan A Canal Panama".getBytes(StandardCharsets.UTF_8));
}
@Test
@ -907,23 +879,23 @@ public class TestMergeContent {
attributes.put(MergeContent.FRAGMENT_ID_ATTRIBUTE, "1");
attributes.put(MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "1");
runner.enqueue("A Man ".getBytes("UTF-8"), attributes);
runner.enqueue("A Man ".getBytes(StandardCharsets.UTF_8), attributes);
attributes.put(MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "2");
runner.enqueue("A Plan ".getBytes("UTF-8"), attributes);
runner.enqueue("A Plan ".getBytes(StandardCharsets.UTF_8), attributes);
attributes.put(MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "3");
runner.enqueue("A Canal ".getBytes("UTF-8"), attributes);
runner.enqueue("A Canal ".getBytes(StandardCharsets.UTF_8), attributes);
attributes.put(MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "4");
attributes.put(MergeContent.FRAGMENT_COUNT_ATTRIBUTE, "4");
runner.enqueue("Panama".getBytes("UTF-8"), attributes);
runner.enqueue("Panama".getBytes(StandardCharsets.UTF_8), attributes);
runner.run();
runner.assertTransferCount(MergeContent.REL_MERGED, 1);
final MockFlowFile assembled = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).get(0);
assembled.assertContentEquals("A Man A Plan A Canal Panama".getBytes("UTF-8"));
final MockFlowFile assembled = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).getFirst();
assembled.assertContentEquals("A Man A Plan A Canal Panama".getBytes(StandardCharsets.UTF_8));
}
@Test
@ -934,53 +906,37 @@ public class TestMergeContent {
final String fragmentId = "Fragment Id";
runner.enqueue("Fragment 1 without count ".getBytes("UTF-8"), new HashMap<String, String>() {{
put(MergeContent.FRAGMENT_ID_ATTRIBUTE, fragmentId);
put(MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "1");
}});
runner.enqueue("Fragment 1 without count ".getBytes(StandardCharsets.UTF_8), Map.of(MergeContent.FRAGMENT_ID_ATTRIBUTE, fragmentId,
MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "1"));
runner.enqueue("Fragment 2 with count ".getBytes("UTF-8"), new HashMap<String, String>() {{
put(MergeContent.FRAGMENT_ID_ATTRIBUTE, fragmentId);
put(MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "2");
put(MergeContent.FRAGMENT_COUNT_ATTRIBUTE, "3");
}});
runner.enqueue("Fragment 2 with count ".getBytes(StandardCharsets.UTF_8), Map.of(MergeContent.FRAGMENT_ID_ATTRIBUTE, fragmentId,
MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "2", MergeContent.FRAGMENT_COUNT_ATTRIBUTE, "3"));
runner.enqueue("Fragment 3 without count".getBytes("UTF-8"), new HashMap<String, String>() {{
put(MergeContent.FRAGMENT_ID_ATTRIBUTE, fragmentId);
put(MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "3");
}});
runner.enqueue("Fragment 3 without count".getBytes(StandardCharsets.UTF_8), Map.of(MergeContent.FRAGMENT_ID_ATTRIBUTE, fragmentId, MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "3"));
runner.run();
runner.assertTransferCount(MergeContent.REL_MERGED, 1);
final MockFlowFile assembled = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).get(0);
assembled.assertContentEquals("Fragment 1 without count Fragment 2 with count Fragment 3 without count".getBytes("UTF-8"));
final MockFlowFile assembled = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).getFirst();
assembled.assertContentEquals("Fragment 1 without count Fragment 2 with count Fragment 3 without count".getBytes(StandardCharsets.UTF_8));
}
@Test
public void testDefragmentWithDifferentFragmentCounts() throws IOException {
public void testDefragmentWithDifferentFragmentCounts() {
final TestRunner runner = TestRunners.newTestRunner(new MergeContent());
runner.setProperty(MergeContent.MERGE_STRATEGY, MergeContent.MERGE_STRATEGY_DEFRAGMENT);
runner.setProperty(MergeContent.MAX_BIN_AGE, "1 min");
final String fragmentId = "Fragment Id";
runner.enqueue("Fragment 1 with count ".getBytes("UTF-8"), new HashMap<String, String>() {{
put(MergeContent.FRAGMENT_ID_ATTRIBUTE, fragmentId);
put(MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "1");
put(MergeContent.FRAGMENT_COUNT_ATTRIBUTE, "2");
}});
runner.enqueue("Fragment 1 with count ".getBytes(StandardCharsets.UTF_8), Map.of(MergeContent.FRAGMENT_ID_ATTRIBUTE,
fragmentId, MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "1", MergeContent.FRAGMENT_COUNT_ATTRIBUTE, "2"));
runner.enqueue("Fragment 2 with count ".getBytes("UTF-8"), new HashMap<String, String>() {{
put(MergeContent.FRAGMENT_ID_ATTRIBUTE, fragmentId);
put(MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "2");
put(MergeContent.FRAGMENT_COUNT_ATTRIBUTE, "3");
}});
runner.enqueue("Fragment 2 with count ".getBytes(StandardCharsets.UTF_8), Map.of(MergeContent.FRAGMENT_ID_ATTRIBUTE, fragmentId,
MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "2", MergeContent.FRAGMENT_COUNT_ATTRIBUTE, "3"));
runner.enqueue("Fragment 3 without count".getBytes("UTF-8"), new HashMap<String, String>() {{
put(MergeContent.FRAGMENT_ID_ATTRIBUTE, fragmentId);
put(MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "3");
}});
runner.enqueue("Fragment 3 without count".getBytes(StandardCharsets.UTF_8), Map.of(MergeContent.FRAGMENT_ID_ATTRIBUTE, fragmentId,
MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "3"));
runner.run();
@ -1000,22 +956,22 @@ public class TestMergeContent {
attributes.put(MergeContent.FRAGMENT_COUNT_ATTRIBUTE, "4");
attributes.put(MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "1");
runner.enqueue("A Man ".getBytes("UTF-8"), attributes);
runner.enqueue("A Man ".getBytes(StandardCharsets.UTF_8), attributes);
attributes.put(MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "2");
runner.enqueue("A Plan ".getBytes("UTF-8"), attributes);
runner.enqueue("A Plan ".getBytes(StandardCharsets.UTF_8), attributes);
// enqueue a duplicate fragment
runner.enqueue("A Plan ".getBytes("UTF-8"), attributes);
runner.enqueue("A Plan ".getBytes(StandardCharsets.UTF_8), attributes);
attributes.put(MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "3");
runner.enqueue("A Canal ".getBytes("UTF-8"), attributes);
runner.enqueue("A Canal ".getBytes(StandardCharsets.UTF_8), attributes);
attributes.put(MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "4");
runner.enqueue("Panama".getBytes("UTF-8"), attributes);
runner.enqueue("Panama".getBytes(StandardCharsets.UTF_8), attributes);
runner.run(1, false);
runner.assertTransferCount(MergeContent.REL_FAILURE, 0);
runner.assertTransferCount(MergeContent.REL_MERGED, 1);
final MockFlowFile assembled = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).get(0);
assembled.assertContentEquals("A Man A Plan A Canal Panama".getBytes("UTF-8"));
final MockFlowFile assembled = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).getFirst();
assembled.assertContentEquals("A Man A Plan A Canal Panama".getBytes(StandardCharsets.UTF_8));
runner.clearTransferState();
Thread.sleep(1_100L);
@ -1035,24 +991,24 @@ public class TestMergeContent {
attributes.put(MergeContent.FRAGMENT_COUNT_ATTRIBUTE, "4");
attributes.put(MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "1");
runner.enqueue("A Man ".getBytes("UTF-8"), attributes);
runner.enqueue("A Man ".getBytes(StandardCharsets.UTF_8), attributes);
attributes.put(MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "2");
runner.enqueue("A Plan ".getBytes("UTF-8"), attributes);
runner.enqueue("A Plan ".getBytes(StandardCharsets.UTF_8), attributes);
attributes.put(MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "3");
runner.enqueue("A Canal ".getBytes("UTF-8"), attributes);
runner.enqueue("A Canal ".getBytes(StandardCharsets.UTF_8), attributes);
attributes.put(MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "4");
runner.enqueue("Panama".getBytes("UTF-8"), attributes);
runner.enqueue("Panama".getBytes(StandardCharsets.UTF_8), attributes);
runner.run();
runner.assertTransferCount(MergeContent.REL_FAILURE, 0);
runner.assertTransferCount(MergeContent.REL_MERGED, 1);
final MockFlowFile assembled = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).get(0);
assembled.assertContentEquals("A Man A Plan A Canal Panama".getBytes("UTF-8"));
final MockFlowFile assembled = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).getFirst();
assembled.assertContentEquals("A Man A Plan A Canal Panama".getBytes(StandardCharsets.UTF_8));
}
@Test
public void testDefragmentWithTooFewFragments() throws IOException {
public void testDefragmentWithTooFewFragments() {
final TestRunner runner = TestRunners.newTestRunner(new MergeContent());
runner.setProperty(MergeContent.MERGE_STRATEGY, MergeContent.MERGE_STRATEGY_DEFRAGMENT);
runner.setProperty(MergeContent.MAX_BIN_AGE, "2 secs");
@ -1062,13 +1018,13 @@ public class TestMergeContent {
attributes.put(MergeContent.FRAGMENT_COUNT_ATTRIBUTE, "5");
attributes.put(MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "1");
runner.enqueue("A Man ".getBytes("UTF-8"), attributes);
runner.enqueue("A Man ".getBytes(StandardCharsets.UTF_8), attributes);
attributes.put(MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "2");
runner.enqueue("A Plan ".getBytes("UTF-8"), attributes);
runner.enqueue("A Plan ".getBytes(StandardCharsets.UTF_8), attributes);
attributes.put(MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "3");
runner.enqueue("A Canal ".getBytes("UTF-8"), attributes);
runner.enqueue("A Canal ".getBytes(StandardCharsets.UTF_8), attributes);
attributes.put(MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "4");
runner.enqueue("Panama".getBytes("UTF-8"), attributes);
runner.enqueue("Panama".getBytes(StandardCharsets.UTF_8), attributes);
runner.run(1, false);
@ -1076,7 +1032,7 @@ public class TestMergeContent {
try {
Thread.sleep(3000L);
break;
} catch (final InterruptedException ie) {
} catch (final InterruptedException ignore) {
}
}
runner.run(1);
@ -1095,21 +1051,21 @@ public class TestMergeContent {
attributes.put(MergeContent.FRAGMENT_COUNT_ATTRIBUTE, "4");
attributes.put(MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "1");
runner.enqueue("A Man ".getBytes("UTF-8"), attributes);
runner.enqueue("A Man ".getBytes(StandardCharsets.UTF_8), attributes);
attributes.put(MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "3");
runner.enqueue("A Canal ".getBytes("UTF-8"), attributes);
runner.enqueue("A Canal ".getBytes(StandardCharsets.UTF_8), attributes);
attributes.put(MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "4");
runner.enqueue("Panama".getBytes("UTF-8"), attributes);
runner.enqueue("Panama".getBytes(StandardCharsets.UTF_8), attributes);
attributes.put(MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "2");
runner.enqueue("A Plan ".getBytes("UTF-8"), attributes);
runner.enqueue("A Plan ".getBytes(StandardCharsets.UTF_8), attributes);
runner.run();
runner.assertTransferCount(MergeContent.REL_MERGED, 1);
final MockFlowFile assembled = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).get(0);
assembled.assertContentEquals("A Man A Plan A Canal Panama".getBytes("UTF-8"));
final MockFlowFile assembled = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).getFirst();
assembled.assertContentEquals("A Man A Plan A Canal Panama".getBytes(StandardCharsets.UTF_8));
runner.getFlowFilesForRelationship(MergeContent.REL_ORIGINAL).stream().forEach(
runner.getFlowFilesForRelationship(MergeContent.REL_ORIGINAL).forEach(
ff -> assertEquals(assembled.getAttribute(CoreAttributes.UUID.key()), ff.getAttribute(MergeContent.MERGE_UUID_ATTRIBUTE)));
}
@ -1128,27 +1084,27 @@ public class TestMergeContent {
secondAttrs.put(MergeContent.FRAGMENT_COUNT_ATTRIBUTE, "3");
attributes.put(MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "1");
runner.enqueue("A Man ".getBytes("UTF-8"), attributes);
runner.enqueue("A Man ".getBytes(StandardCharsets.UTF_8), attributes);
attributes.put(MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "2");
runner.enqueue("A Plan ".getBytes("UTF-8"), attributes);
runner.enqueue("A Plan ".getBytes(StandardCharsets.UTF_8), attributes);
secondAttrs.put(MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "1");
runner.enqueue("No x ".getBytes("UTF-8"), secondAttrs);
runner.enqueue("No x ".getBytes(StandardCharsets.UTF_8), secondAttrs);
secondAttrs.put(MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "2");
runner.enqueue("in ".getBytes("UTF-8"), secondAttrs);
runner.enqueue("in ".getBytes(StandardCharsets.UTF_8), secondAttrs);
secondAttrs.put(MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "3");
runner.enqueue("Nixon".getBytes("UTF-8"), secondAttrs);
runner.enqueue("Nixon".getBytes(StandardCharsets.UTF_8), secondAttrs);
attributes.put(MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "3");
runner.enqueue("A Canal ".getBytes("UTF-8"), attributes);
runner.enqueue("A Canal ".getBytes(StandardCharsets.UTF_8), attributes);
attributes.put(MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "4");
runner.enqueue("Panama".getBytes("UTF-8"), attributes);
runner.enqueue("Panama".getBytes(StandardCharsets.UTF_8), attributes);
runner.run(1);
runner.assertTransferCount(MergeContent.REL_MERGED, 2);
final MockFlowFile assembled = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).get(0);
assembled.assertContentEquals("A Man A Plan A Canal Panama".getBytes("UTF-8"));
assembled.assertContentEquals("A Man A Plan A Canal Panama".getBytes(StandardCharsets.UTF_8));
final MockFlowFile assembledTwo = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).get(1);
assembledTwo.assertContentEquals("No x in Nixon".getBytes("UTF-8"));
assembledTwo.assertContentEquals("No x in Nixon".getBytes(StandardCharsets.UTF_8));
}
@Test
@ -1163,19 +1119,19 @@ public class TestMergeContent {
attributes.put("fragment.index", "1");
attributes.put("segment.original.filename", "originalfilename");
runner.enqueue("A Man ".getBytes("UTF-8"), attributes);
runner.enqueue("A Man ".getBytes(StandardCharsets.UTF_8), attributes);
attributes.put("fragment.index", "2");
runner.enqueue("A Plan ".getBytes("UTF-8"), attributes);
runner.enqueue("A Plan ".getBytes(StandardCharsets.UTF_8), attributes);
attributes.put("fragment.index", "3");
runner.enqueue("A Canal ".getBytes("UTF-8"), attributes);
runner.enqueue("A Canal ".getBytes(StandardCharsets.UTF_8), attributes);
attributes.put("fragment.index", "4");
runner.enqueue("Panama".getBytes("UTF-8"), attributes);
runner.enqueue("Panama".getBytes(StandardCharsets.UTF_8), attributes);
runner.run();
runner.assertTransferCount(MergeContent.REL_MERGED, 1);
final MockFlowFile assembled = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).get(0);
assembled.assertContentEquals("A Man A Plan A Canal Panama".getBytes("UTF-8"));
final MockFlowFile assembled = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).getFirst();
assembled.assertContentEquals("A Man A Plan A Canal Panama".getBytes(StandardCharsets.UTF_8));
assembled.assertAttributeEquals(CoreAttributes.FILENAME.key(), "originalfilename");
}
@ -1188,28 +1144,28 @@ public class TestMergeContent {
attributes.put(MergeContent.FRAGMENT_ID_ATTRIBUTE, "1");
attributes.put(MergeContent.FRAGMENT_COUNT_ATTRIBUTE, "4");
attributes.put(MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "1");
runner.enqueue("A Man ".getBytes("UTF-8"), attributes);
runner.enqueue("A Man ".getBytes(StandardCharsets.UTF_8), attributes);
runner.run();
attributes.put(MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "2");
runner.enqueue("A Plan ".getBytes("UTF-8"), attributes);
runner.enqueue("A Plan ".getBytes(StandardCharsets.UTF_8), attributes);
runner.run();
attributes.put(MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "3");
runner.enqueue("A Canal ".getBytes("UTF-8"), attributes);
runner.enqueue("A Canal ".getBytes(StandardCharsets.UTF_8), attributes);
runner.run();
attributes.put(MergeContent.FRAGMENT_INDEX_ATTRIBUTE, "4");
runner.enqueue("Panama".getBytes("UTF-8"), attributes);
runner.enqueue("Panama".getBytes(StandardCharsets.UTF_8), attributes);
runner.run();
runner.assertTransferCount(MergeContent.REL_MERGED, 1);
final MockFlowFile assembled = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).get(0);
assembled.assertContentEquals("A Man A Plan A Canal Panama".getBytes("UTF-8"));
final MockFlowFile assembled = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).getFirst();
assembled.assertContentEquals("A Man A Plan A Canal Panama".getBytes(StandardCharsets.UTF_8));
}
@Test
public void testMergeBasedOnCorrelation() throws IOException, InterruptedException {
public void testMergeBasedOnCorrelation() {
final TestRunner runner = TestRunners.newTestRunner(new MergeContent());
runner.setProperty(MergeContent.MERGE_STRATEGY, MergeContent.MERGE_STRATEGY_BIN_PACK);
runner.setProperty(MergeContent.MAX_BIN_AGE, "1 min");
@ -1219,14 +1175,14 @@ public class TestMergeContent {
final Map<String, String> attributes = new HashMap<>();
attributes.put("attr", "b");
runner.enqueue("A Man ".getBytes("UTF-8"), attributes);
runner.enqueue("A Plan ".getBytes("UTF-8"), attributes);
runner.enqueue("A Man ".getBytes(StandardCharsets.UTF_8), attributes);
runner.enqueue("A Plan ".getBytes(StandardCharsets.UTF_8), attributes);
attributes.put("attr", "c");
runner.enqueue("A Canal ".getBytes("UTF-8"), attributes);
runner.enqueue("A Canal ".getBytes(StandardCharsets.UTF_8), attributes);
attributes.put("attr", "b");
runner.enqueue("Panama".getBytes("UTF-8"), attributes);
runner.enqueue("Panama".getBytes(StandardCharsets.UTF_8), attributes);
runner.run(2);
@ -1298,7 +1254,7 @@ public class TestMergeContent {
runner.run();
runner.assertTransferCount(MergeContent.REL_MERGED, 1);
final MockFlowFile outFile = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).get(0);
final MockFlowFile outFile = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).getFirst();
outFile.assertAttributeEquals("abc", "xyz");
outFile.assertAttributeEquals("hello", "good-bye");
@ -1328,7 +1284,7 @@ public class TestMergeContent {
runner.run();
runner.assertTransferCount(MergeContent.REL_MERGED, 1);
final MockFlowFile outFile = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).get(0);
final MockFlowFile outFile = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).getFirst();
outFile.assertAttributeEquals("abc", "xyz");
outFile.assertAttributeNotExists("hello");
@ -1358,8 +1314,8 @@ public class TestMergeContent {
runner.assertTransferCount(MergeContent.REL_FAILURE, 0);
runner.assertTransferCount(MergeContent.REL_ORIGINAL, 3);
final MockFlowFile bundle = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).get(0);
bundle.assertContentEquals("Hello, World!".getBytes("UTF-8"));
final MockFlowFile bundle = runner.getFlowFilesForRelationship(MergeContent.REL_MERGED).getFirst();
bundle.assertContentEquals("Hello, World!".getBytes(StandardCharsets.UTF_8));
bundle.assertAttributeEquals(MergeContent.MERGE_COUNT_ATTRIBUTE, "3");
bundle.assertAttributeExists(MergeContent.MERGE_BIN_AGE_ATTRIBUTE);
}
@ -1372,7 +1328,7 @@ public class TestMergeContent {
runner.setProperty(MergeContent.MAX_BIN_COUNT, "3");
for (int i = 0; i < 17; i++) {
runner.enqueue(String.valueOf(i) + "\n");
runner.enqueue(i + "\n");
}
runner.run(5);

View File

@ -23,9 +23,10 @@ import org.junit.jupiter.api.Test;
import java.io.File;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.HashMap;
import java.util.Map;
import static org.junit.jupiter.api.Assertions.assertEquals;
@ -59,8 +60,8 @@ public class TestModifyBytes {
runner.run();
runner.assertAllFlowFilesTransferred(ModifyBytes.REL_SUCCESS, 1);
final MockFlowFile out = runner.getFlowFilesForRelationship(ModifyBytes.REL_SUCCESS).get(0);
out.assertContentEquals("".getBytes("UTF-8"));
final MockFlowFile out = runner.getFlowFilesForRelationship(ModifyBytes.REL_SUCCESS).getFirst();
out.assertContentEquals("".getBytes(StandardCharsets.UTF_8));
}
@Test
@ -73,7 +74,7 @@ public class TestModifyBytes {
runner.run();
runner.assertAllFlowFilesTransferred(ModifyBytes.REL_SUCCESS, 1);
final MockFlowFile out = runner.getFlowFilesForRelationship(ModifyBytes.REL_SUCCESS).get(0);
final MockFlowFile out = runner.getFlowFilesForRelationship(ModifyBytes.REL_SUCCESS).getFirst();
out.assertContentEquals(testFile);
}
@ -87,7 +88,7 @@ public class TestModifyBytes {
runner.run();
runner.assertAllFlowFilesTransferred(ModifyBytes.REL_SUCCESS, 1);
final MockFlowFile out = runner.getFlowFilesForRelationship(ModifyBytes.REL_SUCCESS).get(0);
final MockFlowFile out = runner.getFlowFilesForRelationship(ModifyBytes.REL_SUCCESS).getFirst();
out.assertContentEquals(noHeaderFile);
}
@ -97,13 +98,11 @@ public class TestModifyBytes {
runner.setProperty(ModifyBytes.START_OFFSET, "${numBytes}"); //REMOVE - '<<<HEADER>>>'
runner.setProperty(ModifyBytes.END_OFFSET, "0 MB");
runner.enqueue(testFilePath, new HashMap<String, String>() {{
put("numBytes", "12 B");
}});
runner.enqueue(testFilePath, Map.of("numBytes", "12 B"));
runner.run();
runner.assertAllFlowFilesTransferred(ModifyBytes.REL_SUCCESS, 1);
final MockFlowFile out = runner.getFlowFilesForRelationship(ModifyBytes.REL_SUCCESS).get(0);
final MockFlowFile out = runner.getFlowFilesForRelationship(ModifyBytes.REL_SUCCESS).getFirst();
out.assertContentEquals(noHeaderFile);
}
@ -117,8 +116,8 @@ public class TestModifyBytes {
runner.run();
runner.assertAllFlowFilesTransferred(ModifyBytes.REL_SUCCESS, 1);
final MockFlowFile out = runner.getFlowFilesForRelationship(ModifyBytes.REL_SUCCESS).get(0);
out.assertContentEquals("<<<FOOTER>>>".getBytes("UTF-8"));
final MockFlowFile out = runner.getFlowFilesForRelationship(ModifyBytes.REL_SUCCESS).getFirst();
out.assertContentEquals("<<<FOOTER>>>".getBytes(StandardCharsets.UTF_8));
}
@Test
@ -131,8 +130,8 @@ public class TestModifyBytes {
runner.run();
runner.assertAllFlowFilesTransferred(ModifyBytes.REL_SUCCESS, 1);
final MockFlowFile out = runner.getFlowFilesForRelationship(ModifyBytes.REL_SUCCESS).get(0);
out.assertContentEquals("<<<HEADER>>>".getBytes("UTF-8"));
final MockFlowFile out = runner.getFlowFilesForRelationship(ModifyBytes.REL_SUCCESS).getFirst();
out.assertContentEquals("<<<HEADER>>>".getBytes(StandardCharsets.UTF_8));
}
@Test
@ -141,14 +140,12 @@ public class TestModifyBytes {
runner.setProperty(ModifyBytes.START_OFFSET, "0 B");
runner.setProperty(ModifyBytes.END_OFFSET, "${numBytes}");
runner.enqueue(testFilePath, new HashMap<String, String>() {{
put("numBytes", "181 B");
}});
runner.enqueue(testFilePath, Map.of("numBytes", "181 B"));
runner.run();
runner.assertAllFlowFilesTransferred(ModifyBytes.REL_SUCCESS, 1);
final MockFlowFile out = runner.getFlowFilesForRelationship(ModifyBytes.REL_SUCCESS).get(0);
out.assertContentEquals("<<<HEADER>>>".getBytes("UTF-8"));
final MockFlowFile out = runner.getFlowFilesForRelationship(ModifyBytes.REL_SUCCESS).getFirst();
out.assertContentEquals("<<<HEADER>>>".getBytes(StandardCharsets.UTF_8));
}
@Test
@ -161,7 +158,7 @@ public class TestModifyBytes {
runner.run();
runner.assertAllFlowFilesTransferred(ModifyBytes.REL_SUCCESS, 1);
final MockFlowFile out = runner.getFlowFilesForRelationship(ModifyBytes.REL_SUCCESS).get(0);
final MockFlowFile out = runner.getFlowFilesForRelationship(ModifyBytes.REL_SUCCESS).getFirst();
out.assertContentEquals(noFooterFile);
}
@ -175,7 +172,7 @@ public class TestModifyBytes {
runner.run();
runner.assertAllFlowFilesTransferred(ModifyBytes.REL_SUCCESS, 1);
final MockFlowFile out = runner.getFlowFilesForRelationship(ModifyBytes.REL_SUCCESS).get(0);
final MockFlowFile out = runner.getFlowFilesForRelationship(ModifyBytes.REL_SUCCESS).getFirst();
out.assertContentEquals(noFooterNoHeaderFile);
}
@ -189,8 +186,8 @@ public class TestModifyBytes {
runner.run();
runner.assertAllFlowFilesTransferred(ModifyBytes.REL_SUCCESS, 1);
final MockFlowFile out = runner.getFlowFilesForRelationship(ModifyBytes.REL_SUCCESS).get(0);
out.assertContentEquals("".getBytes("UTF-8"));
final MockFlowFile out = runner.getFlowFilesForRelationship(ModifyBytes.REL_SUCCESS).getFirst();
out.assertContentEquals("".getBytes(StandardCharsets.UTF_8));
}
@Test
@ -203,8 +200,8 @@ public class TestModifyBytes {
runner.run();
runner.assertAllFlowFilesTransferred(ModifyBytes.REL_SUCCESS, 1);
final MockFlowFile out = runner.getFlowFilesForRelationship(ModifyBytes.REL_SUCCESS).get(0);
out.assertContentEquals("Dew".getBytes("UTF-8"));
final MockFlowFile out = runner.getFlowFilesForRelationship(ModifyBytes.REL_SUCCESS).getFirst();
out.assertContentEquals("Dew".getBytes(StandardCharsets.UTF_8));
}
@Test
@ -218,7 +215,7 @@ public class TestModifyBytes {
runner.run();
runner.assertAllFlowFilesTransferred(ModifyBytes.REL_SUCCESS, 1);
final MockFlowFile out = runner.getFlowFilesForRelationship(ModifyBytes.REL_SUCCESS).get(0);
final MockFlowFile out = runner.getFlowFilesForRelationship(ModifyBytes.REL_SUCCESS).getFirst();
assertEquals(0L, out.getSize());
}
@ -233,7 +230,7 @@ public class TestModifyBytes {
runner.run();
runner.assertAllFlowFilesTransferred(ModifyBytes.REL_SUCCESS, 1);
final MockFlowFile out = runner.getFlowFilesForRelationship(ModifyBytes.REL_SUCCESS).get(0);
final MockFlowFile out = runner.getFlowFilesForRelationship(ModifyBytes.REL_SUCCESS).getFirst();
assertEquals(0L, out.getSize());
}
@ -248,12 +245,12 @@ public class TestModifyBytes {
runner.run();
runner.assertAllFlowFilesTransferred(ModifyBytes.REL_SUCCESS, 1);
final MockFlowFile out = runner.getFlowFilesForRelationship(ModifyBytes.REL_SUCCESS).get(0);
final MockFlowFile out = runner.getFlowFilesForRelationship(ModifyBytes.REL_SUCCESS).getFirst();
assertEquals(testFile.length() - 20, out.getSize());
}
@Test
public void testCheckAllowableValues() throws IOException {
public void testCheckAllowableValues() {
final TestRunner runner = TestRunners.newTestRunner(new ModifyBytes());
runner.setProperty(ModifyBytes.REMOVE_ALL, "maybe");
runner.assertNotValid();

View File

@ -315,7 +315,7 @@ public class TestPutFile {
private void emptyTestDirectory() throws IOException {
Files.walkFileTree(Paths.get("target/test/data/out/PutFile"), new FileVisitor<Path>() {
Files.walkFileTree(Paths.get("target/test/data/out/PutFile"), new FileVisitor<>() {
@Override
public FileVisitResult preVisitDirectory(Path dir, BasicFileAttributes attrs) throws IOException {

View File

@ -33,7 +33,7 @@ import java.nio.charset.StandardCharsets;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_COUNT;
import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_ID;
@ -151,11 +151,7 @@ public class TestSplitJson {
final TestRunner testRunner = TestRunners.newTestRunner(new SplitJson());
testRunner.setProperty(SplitJson.ARRAY_JSON_PATH_EXPRESSION, "$[*].name");
testRunner.enqueue(JSON_SNIPPET, new HashMap<String, String>() {
{
put(CoreAttributes.FILENAME.key(), "test.json");
}
});
testRunner.enqueue(JSON_SNIPPET, Map.of(CoreAttributes.FILENAME.key(), "test.json"));
testRunner.run();
testRunner.assertTransferCount(SplitJson.REL_ORIGINAL, 1);

View File

@ -23,17 +23,17 @@ import org.apache.nifi.util.TestRunners;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.xml.sax.InputSource;
import org.xml.sax.SAXException;
import org.xml.sax.helpers.DefaultHandler;
import javax.xml.parsers.SAXParser;
import javax.xml.parsers.SAXParserFactory;
import java.io.IOException;
import java.io.StringReader;
import java.nio.charset.StandardCharsets;
import java.nio.file.Paths;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_COUNT;
import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_ID;
@ -58,22 +58,16 @@ public class TestSplitXml {
runner.setProperty(SplitXml.SPLIT_DEPTH, "0");
runner.enqueue(Paths.get("src/test/resources/TestXml/xml-bundle-1"));
assertThrows(AssertionError.class, () -> {
runner.run();
});
assertThrows(AssertionError.class, runner::run);
}
@Test
public void testDepthOf1() throws Exception {
final TestRunner runner = TestRunners.newTestRunner(new SplitXml());
runner.enqueue(Paths.get("src/test/resources/TestXml/xml-bundle-1"), new HashMap<String, String>() {
{
put(CoreAttributes.FILENAME.key(), "test.xml");
}
});
runner.enqueue(Paths.get("src/test/resources/TestXml/xml-bundle-1"), Map.of(CoreAttributes.FILENAME.key(), "test.xml"));
runner.run();
runner.assertTransferCount(SplitXml.REL_ORIGINAL, 1);
final MockFlowFile originalFlowFile = runner.getFlowFilesForRelationship(SplitXml.REL_ORIGINAL).get(0);
final MockFlowFile originalFlowFile = runner.getFlowFilesForRelationship(SplitXml.REL_ORIGINAL).getFirst();
originalFlowFile.assertAttributeExists(FRAGMENT_ID.key());
originalFlowFile.assertAttributeEquals(FRAGMENT_COUNT.key(), "6");
runner.assertTransferCount(SplitXml.REL_SPLIT, 6);
@ -95,7 +89,7 @@ public class TestSplitXml {
runner.enqueue(Paths.get("src/test/resources/TestXml/xml-bundle-1"));
runner.run();
runner.assertTransferCount(SplitXml.REL_ORIGINAL, 1);
runner.getFlowFilesForRelationship(SplitXml.REL_ORIGINAL).get(0).assertAttributeEquals(FRAGMENT_COUNT.key(), "12");
runner.getFlowFilesForRelationship(SplitXml.REL_ORIGINAL).getFirst().assertAttributeEquals(FRAGMENT_COUNT.key(), "12");
runner.assertTransferCount(SplitXml.REL_SPLIT, 12);
parseFlowFiles(runner.getFlowFilesForRelationship(SplitXml.REL_ORIGINAL));
@ -109,7 +103,7 @@ public class TestSplitXml {
runner.enqueue(Paths.get("src/test/resources/TestXml/xml-bundle-1"));
runner.run();
runner.assertTransferCount(SplitXml.REL_ORIGINAL, 1);
runner.getFlowFilesForRelationship(SplitXml.REL_ORIGINAL).get(0).assertAttributeEquals(FRAGMENT_COUNT.key(), "12");
runner.getFlowFilesForRelationship(SplitXml.REL_ORIGINAL).getFirst().assertAttributeEquals(FRAGMENT_COUNT.key(), "12");
runner.assertTransferCount(SplitXml.REL_SPLIT, 12);
parseFlowFiles(runner.getFlowFilesForRelationship(SplitXml.REL_ORIGINAL));
@ -129,7 +123,7 @@ public class TestSplitXml {
runner.enqueue(Paths.get("src/test/resources/TestXml/namespace.xml"));
runner.run();
runner.assertTransferCount(SplitXml.REL_ORIGINAL, 1);
runner.getFlowFilesForRelationship(SplitXml.REL_ORIGINAL).get(0).assertAttributeEquals(FRAGMENT_COUNT.key(), "2");
runner.getFlowFilesForRelationship(SplitXml.REL_ORIGINAL).getFirst().assertAttributeEquals(FRAGMENT_COUNT.key(), "2");
runner.assertTransferCount(SplitXml.REL_SPLIT, 2);
parseFlowFiles(runner.getFlowFilesForRelationship(SplitXml.REL_ORIGINAL));
@ -141,10 +135,10 @@ public class TestSplitXml {
split2.assertContentEquals(Paths.get("src/test/resources/TestXml/namespaceSplit2.xml"));
}
public void parseFlowFiles(List<MockFlowFile> flowfiles) throws Exception, SAXException {
public void parseFlowFiles(List<MockFlowFile> flowfiles) throws Exception {
for (MockFlowFile out : flowfiles) {
final byte[] outData = out.toByteArray();
final String outXml = new String(outData, "UTF-8");
final String outXml = new String(outData, StandardCharsets.UTF_8);
saxParser.parse(new InputSource(new StringReader(outXml)), new DefaultHandler());
}
}

View File

@ -33,7 +33,7 @@ public class TestUpdateCounter {
final TestRunner firstrunner = TestRunners.newTestRunner(new UpdateCounter());
firstrunner.setProperty(UpdateCounter.COUNTER_NAME, "firewall");
firstrunner.setProperty(UpdateCounter.DELTA, "1");
Map<String, String> attributes = new HashMap<String, String>();
Map<String, String> attributes = new HashMap<>();
firstrunner.enqueue("", attributes);
firstrunner.run();
firstrunner.assertAllFlowFilesTransferred(UpdateCounter.SUCCESS, 1);

View File

@ -170,7 +170,7 @@ public class PersistentSetCache implements SetCache {
@Override
public SetRecord deserializeRecord(DataInputStream in, int version) throws IOException {
return deserializeEdit(in, new HashMap<Object, SetRecord>(), version);
return deserializeEdit(in, new HashMap<>(), version);
}
@Override

View File

@ -64,7 +64,7 @@ public class TestDistributedMapCacheLookupService {
static final class EphemeralMapCacheClientService extends AbstractControllerService implements DistributedMapCacheClient {
private Map<String, String> map = new HashMap<String, String>();
private Map<String, String> map = new HashMap<>();
@OnEnabled
public void onEnabled(final ConfigurationContext context) {

View File

@ -37,7 +37,7 @@ public class EmbeddedAvroSchemaAccessStrategy implements SchemaAccessStrategy {
@Override
public RecordSchema getSchema(Map<String, String> variables, final InputStream contentStream, final RecordSchema readSchema) throws SchemaNotFoundException, IOException {
final DataFileStream<GenericRecord> dataFileStream = new DataFileStream<>(contentStream, new GenericDatumReader<GenericRecord>());
final DataFileStream<GenericRecord> dataFileStream = new DataFileStream<>(contentStream, new GenericDatumReader<>());
final Schema avroSchema = dataFileStream.getSchema();
final RecordSchema recordSchema = AvroTypeUtil.createSchema(avroSchema);
return recordSchema;

View File

@ -38,6 +38,7 @@ import java.nio.charset.StandardCharsets;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import static org.junit.jupiter.api.Assertions.assertArrayEquals;
import static org.junit.jupiter.api.Assertions.assertEquals;
@ -86,29 +87,29 @@ public class TestGrokReader {
new RecordField(RAW_FIELD, RecordFieldType.STRING.getDataType())
));
final Record expectedFirstRecord = new MapRecord(expectedSchema, new HashMap<String, Object>() {{
put(TIMESTAMP_FIELD, "1021-09-09 09:03:06");
put(FACILITY_FIELD, null);
put("priority", null);
put("logsource", "127.0.0.1");
put(PROGRAM_FIELD, "nifi");
put("pid", "1000");
put("message", " LogMessage");
put(STACKTRACE_FIELD, null);
put(RAW_FIELD, "1021-09-09 09:03:06 127.0.0.1 nifi[1000]: LogMessage");
}});
final Map<String, Object> expectedFirstMapValues = new HashMap<>();
expectedFirstMapValues.put(TIMESTAMP_FIELD, "1021-09-09 09:03:06");
expectedFirstMapValues.put(FACILITY_FIELD, null);
expectedFirstMapValues.put("priority", null);
expectedFirstMapValues.put("logsource", "127.0.0.1");
expectedFirstMapValues.put(PROGRAM_FIELD, "nifi");
expectedFirstMapValues.put("pid", "1000");
expectedFirstMapValues.put("message", " LogMessage");
expectedFirstMapValues.put(STACKTRACE_FIELD, null);
expectedFirstMapValues.put(RAW_FIELD, "1021-09-09 09:03:06 127.0.0.1 nifi[1000]: LogMessage");
final Record expectedFirstRecord = new MapRecord(expectedSchema, expectedFirstMapValues);
final Record expectedSecondRecord = new MapRecord(expectedSchema, new HashMap<String, Object>() {{
put(TIMESTAMP_FIELD, "October 19 19:13:16");
put(FACILITY_FIELD, null);
put("priority", null);
put("logsource", "127.0.0.1");
put(PROGRAM_FIELD, "nifi");
put("pid", "1000");
put(MESSAGE_FIELD, " LogMessage2");
put(STACKTRACE_FIELD, null);
put(RAW_FIELD, "October 19 19:13:16 127.0.0.1 nifi[1000]: LogMessage2");
}});
final Map<String, Object> expectedSecondMapValues = new HashMap<>();
expectedSecondMapValues.put(TIMESTAMP_FIELD, "October 19 19:13:16");
expectedSecondMapValues.put(FACILITY_FIELD, null);
expectedSecondMapValues.put("priority", null);
expectedSecondMapValues.put("logsource", "127.0.0.1");
expectedSecondMapValues.put(PROGRAM_FIELD, "nifi");
expectedSecondMapValues.put("pid", "1000");
expectedSecondMapValues.put(MESSAGE_FIELD, " LogMessage2");
expectedSecondMapValues.put(STACKTRACE_FIELD, null);
expectedSecondMapValues.put(RAW_FIELD, "October 19 19:13:16 127.0.0.1 nifi[1000]: LogMessage2");
final Record expectedSecondRecord = new MapRecord(expectedSchema, expectedSecondMapValues);
final GrokReader grokReader = new GrokReader();
runner.addControllerService(GrokReader.class.getSimpleName(), grokReader);

View File

@ -39,7 +39,6 @@ import org.apache.nifi.serialization.record.RecordSet;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
@ -98,17 +97,12 @@ public class TestFreeFormTextRecordSetWriterProcessor extends AbstractProcessor
@Override
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
return new ArrayList<PropertyDescriptor>() {{
add(WRITER);
add(MULTIPLE_RECORDS);
}};
return List.of(WRITER, MULTIPLE_RECORDS);
}
@Override
public Set<Relationship> getRelationships() {
return new HashSet<Relationship>() {{
add(SUCCESS);
}};
return Set.of(SUCCESS);
}
protected static RecordSet getRecordSet(boolean multipleRecords) {

View File

@ -182,16 +182,14 @@ public class TestWriteXMLResultUtils {
RecordSchema outerSchema = new SimpleRecordSchema(fields, SCHEMA_IDENTIFIER_PERSON);
Record innerRecord1 = new MapRecord(innerSchema, RECORD_FIELDS_ADDRESS_1);
Record outerRecord1 = new MapRecord(outerSchema, new HashMap<String, Object>() {{
putAll(RECORD_FIELDS_PERSON_1);
put(FIELD_ADDRESS, innerRecord1);
}});
Map<String, Object> outerRecord1MapRecordValues = new HashMap<>(RECORD_FIELDS_PERSON_1);
outerRecord1MapRecordValues.put(FIELD_ADDRESS, innerRecord1);
Record outerRecord1 = new MapRecord(outerSchema, outerRecord1MapRecordValues);
Record innerRecord2 = new MapRecord(innerSchema, RECORD_FIELDS_ADDRESS_2);
Record outerRecord2 = new MapRecord(outerSchema, new HashMap<String, Object>() {{
putAll(RECORD_FIELDS_PERSON_2);
put(FIELD_ADDRESS, innerRecord2);
}});
Map<String, Object> outerRecord2MapRecordValues = new HashMap<>(RECORD_FIELDS_PERSON_2);
outerRecord2MapRecordValues.put(FIELD_ADDRESS, innerRecord2);
Record outerRecord2 = new MapRecord(outerSchema, outerRecord2MapRecordValues);
List<Record> records = new ArrayList<>();
records.add(outerRecord1);
@ -217,16 +215,14 @@ public class TestWriteXMLResultUtils {
recordWithoutStreet2.remove(FIELD_STREET);
Record innerRecord1 = new MapRecord(innerSchema, recordWithoutStreet1);
Record outerRecord1 = new MapRecord(outerSchema, new HashMap<String, Object>() {{
putAll(RECORD_FIELDS_PERSON_1);
put(FIELD_ADDRESS, innerRecord1);
}});
Map<String, Object> outerRecord1MapRecordValues = new HashMap<>(RECORD_FIELDS_PERSON_1);
outerRecord1MapRecordValues.put(FIELD_ADDRESS, innerRecord1);
Record outerRecord1 = new MapRecord(outerSchema, outerRecord1MapRecordValues);
Record innerRecord2 = new MapRecord(innerSchema, recordWithoutStreet2);
Record outerRecord2 = new MapRecord(outerSchema, new HashMap<String, Object>() {{
putAll(RECORD_FIELDS_PERSON_2);
put(FIELD_ADDRESS, innerRecord2);
}});
Map<String, Object> outerRecord2MapRecordValues = new HashMap<>(RECORD_FIELDS_PERSON_2);
outerRecord2MapRecordValues.put(FIELD_ADDRESS, innerRecord2);
Record outerRecord2 = new MapRecord(outerSchema, outerRecord2MapRecordValues);
List<Record> records = new ArrayList<>();
records.add(outerRecord1);
@ -254,16 +250,14 @@ public class TestWriteXMLResultUtils {
recordWithoutStreet2.remove(FIELD_CITY);
Record innerRecord1 = new MapRecord(innerSchema, recordWithoutStreet1);
Record outerRecord1 = new MapRecord(outerSchema, new HashMap<String, Object>() {{
putAll(RECORD_FIELDS_PERSON_1);
put(FIELD_ADDRESS, innerRecord1);
}});
Map<String, Object> outerRecord1MapRecordValues = new HashMap<>(RECORD_FIELDS_PERSON_1);
outerRecord1MapRecordValues.put(FIELD_ADDRESS, innerRecord1);
Record outerRecord1 = new MapRecord(outerSchema, outerRecord1MapRecordValues);
Record innerRecord2 = new MapRecord(innerSchema, recordWithoutStreet2);
Record outerRecord2 = new MapRecord(outerSchema, new HashMap<String, Object>() {{
putAll(RECORD_FIELDS_PERSON_2);
put(FIELD_ADDRESS, innerRecord2);
}});
Map<String, Object> outerRecord2MapRecordValues = new HashMap<>(RECORD_FIELDS_PERSON_2);
outerRecord2MapRecordValues.put(FIELD_ADDRESS, innerRecord2);
Record outerRecord2 = new MapRecord(outerSchema, outerRecord2MapRecordValues);
List<Record> records = new ArrayList<>();
records.add(outerRecord1);
@ -283,16 +277,14 @@ public class TestWriteXMLResultUtils {
RecordSchema outerSchema = new SimpleRecordSchema(fields, SCHEMA_IDENTIFIER_PERSON);
Record innerRecord1 = new MapRecord(innerSchema, Collections.emptyMap());
Record outerRecord1 = new MapRecord(outerSchema, new HashMap<String, Object>() {{
putAll(RECORD_FIELDS_PERSON_1);
put(FIELD_ADDRESS, innerRecord1);
}});
Map<String, Object> outerRecord1MapRecordValues = new HashMap<>(RECORD_FIELDS_PERSON_1);
outerRecord1MapRecordValues.put(FIELD_ADDRESS, innerRecord1);
Record outerRecord1 = new MapRecord(outerSchema, outerRecord1MapRecordValues);
Record innerRecord2 = new MapRecord(innerSchema, Collections.emptyMap());
Record outerRecord2 = new MapRecord(outerSchema, new HashMap<String, Object>() {{
putAll(RECORD_FIELDS_PERSON_2);
put(FIELD_ADDRESS, innerRecord2);
}});
Map<String, Object> outerRecord2MapRecordValues = new HashMap<>(RECORD_FIELDS_PERSON_2);
outerRecord2MapRecordValues.put(FIELD_ADDRESS, innerRecord2);
Record outerRecord2 = new MapRecord(outerSchema, outerRecord2MapRecordValues);
List<Record> records = new ArrayList<>();
records.add(outerRecord1);
@ -311,17 +303,15 @@ public class TestWriteXMLResultUtils {
RecordSchema outerSchema = new SimpleRecordSchema(fields, SCHEMA_IDENTIFIER_PERSON);
Record innerRecord1 = new MapRecord(innerSchema, Collections.EMPTY_MAP);
Record outerRecord1 = new MapRecord(outerSchema, new HashMap<String, Object>() {{
putAll(RECORD_FIELDS_PERSON_1);
put(FIELD_ADDRESS, innerRecord1);
}});
Record innerRecord1 = new MapRecord(innerSchema, Map.of());
Map<String, Object> outerRecord1MapRecordValues = new HashMap<>(RECORD_FIELDS_PERSON_1);
outerRecord1MapRecordValues.put(FIELD_ADDRESS, innerRecord1);
Record outerRecord1 = new MapRecord(outerSchema, outerRecord1MapRecordValues);
Record innerRecord2 = new MapRecord(innerSchema, Collections.EMPTY_MAP);
Record outerRecord2 = new MapRecord(outerSchema, new HashMap<String, Object>() {{
putAll(RECORD_FIELDS_PERSON_2);
put(FIELD_ADDRESS, innerRecord2);
}});
Record innerRecord2 = new MapRecord(innerSchema, Map.of());
Map<String, Object> outerRecord2MapRecordValues = new HashMap<>(RECORD_FIELDS_PERSON_2);
outerRecord2MapRecordValues.put(FIELD_ADDRESS, innerRecord2);
Record outerRecord2 = new MapRecord(outerSchema, outerRecord2MapRecordValues);
List<Record> records = new ArrayList<>();
records.add(outerRecord1);
@ -363,12 +353,10 @@ public class TestWriteXMLResultUtils {
fields.add(new RecordField(FIELD_CHILDREN, arrayType));
fields.addAll(getSimpleRecordFields());
Map<String, Object> recordFields1 = new HashMap<>();
recordFields1.putAll(RECORD_FIELDS_PERSON_1);
Map<String, Object> recordFields1 = new HashMap<>(RECORD_FIELDS_PERSON_1);
recordFields1.put(FIELD_CHILDREN, children);
Map<String, Object> recordFields2 = new HashMap<>();
recordFields2.putAll(RECORD_FIELDS_PERSON_2);
Map<String, Object> recordFields2 = new HashMap<>(RECORD_FIELDS_PERSON_2);
recordFields2.put(FIELD_CHILDREN, children);
RecordSchema schema = new SimpleRecordSchema(fields, SCHEMA_IDENTIFIER_PERSON);
@ -413,12 +401,10 @@ public class TestWriteXMLResultUtils {
fields.add(new RecordField(FIELD_CHILDREN, mapType));
fields.addAll(getSimpleRecordFields());
Map<String, Object> recordFields1 = new HashMap<>();
recordFields1.putAll(RECORD_FIELDS_PERSON_1);
Map<String, Object> recordFields1 = new HashMap<>(RECORD_FIELDS_PERSON_1);
recordFields1.put(FIELD_CHILDREN, children);
Map<String, Object> recordFields2 = new HashMap<>();
recordFields2.putAll(RECORD_FIELDS_PERSON_2);
Map<String, Object> recordFields2 = new HashMap<>(RECORD_FIELDS_PERSON_2);
recordFields2.put(FIELD_CHILDREN, children);
RecordSchema schema = new SimpleRecordSchema(fields, SCHEMA_IDENTIFIER_PERSON);
@ -470,16 +456,14 @@ public class TestWriteXMLResultUtils {
RecordSchema outerSchema = new SimpleRecordSchema(fields, SCHEMA_IDENTIFIER_PERSON);
Record innerRecord1 = new MapRecord(new SimpleRecordSchema(Collections.emptyList()), RECORD_FIELDS_ADDRESS_1);
Record outerRecord1 = new MapRecord(outerSchema, new HashMap<String, Object>() {{
putAll(RECORD_FIELDS_PERSON_1);
put(FIELD_ADDRESS, innerRecord1);
}});
Map<String, Object> outerRecord1MapRecordValues = new HashMap<>(RECORD_FIELDS_PERSON_1);
outerRecord1MapRecordValues.put(FIELD_ADDRESS, innerRecord1);
Record outerRecord1 = new MapRecord(outerSchema, outerRecord1MapRecordValues);
Record innerRecord2 = new MapRecord(new SimpleRecordSchema(Collections.emptyList()), RECORD_FIELDS_ADDRESS_2);
Record outerRecord2 = new MapRecord(outerSchema, new HashMap<String, Object>() {{
putAll(RECORD_FIELDS_PERSON_2);
put(FIELD_ADDRESS, innerRecord2);
}});
Map<String, Object> outerRecord2MapRecordValues = new HashMap<>(RECORD_FIELDS_PERSON_2);
outerRecord2MapRecordValues.put(FIELD_ADDRESS, innerRecord2);
Record outerRecord2 = new MapRecord(outerSchema, outerRecord2MapRecordValues);
List<Record> records = new ArrayList<>();
records.add(outerRecord1);

View File

@ -33,7 +33,6 @@ import org.apache.nifi.util.StringUtils;
import java.io.InputStream;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
@ -88,6 +87,6 @@ public class TestXMLReaderProcessor extends AbstractProcessor {
@Override
public Set<Relationship> getRelationships() {
return new HashSet<Relationship>() {{ add(SUCCESS); }};
return Set.of(SUCCESS);
}
}

View File

@ -41,6 +41,7 @@ import java.util.Map;
import static org.junit.jupiter.api.Assertions.assertArrayEquals;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertInstanceOf;
import static org.junit.jupiter.api.Assertions.assertNotEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
@ -364,7 +365,7 @@ public class TestXMLRecordReader {
null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
Record first = reader.nextRecord(true, true);
assertEquals(null, first.getAsString("ID"));
assertNull(first.getAsString("ID"));
Record second = reader.nextRecord(false, false);
assertEquals("P2", second.getAsString("ID"));
@ -373,7 +374,7 @@ public class TestXMLRecordReader {
assertEquals("P3", third.getAsString("ID"));
Record fourth = reader.nextRecord(false, true);
assertEquals(null, fourth.getAsString("ID"));
assertNull(fourth.getAsString("ID"));
}
@Test
@ -463,9 +464,9 @@ public class TestXMLRecordReader {
null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
Record first = reader.nextRecord(true, true);
assertTrue(first.getValue("NAME") instanceof Record);
assertInstanceOf(Record.class, first.getValue("NAME"));
Record first_nested1 = (Record) first.getValue("NAME");
assertTrue(first.getValue("AGE") instanceof Record);
assertInstanceOf(Record.class, first.getValue("AGE"));
Record first_nested2 = (Record) first.getValue("AGE");
assertEquals("name1", first_nested1.getValue("ID"));
assertEquals("Cleve Butler", first_nested1.getValue("CONTENT"));
@ -473,9 +474,9 @@ public class TestXMLRecordReader {
assertEquals(42, first_nested2.getValue("CONTENT"));
Record second = reader.nextRecord(true, true);
assertTrue(second.getValue("NAME") instanceof Record);
assertInstanceOf(Record.class, second.getValue("NAME"));
Record second_nested1 = (Record) second.getValue("NAME");
assertTrue(second.getValue("AGE") instanceof Record);
assertInstanceOf(Record.class, second.getValue("AGE"));
Record second_nested2 = (Record) second.getValue("AGE");
assertEquals("name2", second_nested1.getValue("ID"));
assertEquals("Ainslie Fletcher", second_nested1.getValue("CONTENT"));
@ -490,9 +491,9 @@ public class TestXMLRecordReader {
null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
Record first = reader.nextRecord(false, false);
assertTrue(first.getValue("NAME") instanceof Record);
assertInstanceOf(Record.class, first.getValue("NAME"));
Record first_nested1 = (Record) first.getValue("NAME");
assertTrue(first.getValue("AGE") instanceof Record);
assertInstanceOf(Record.class, first.getValue("AGE"));
Record first_nested2 = (Record) first.getValue("AGE");
assertEquals("name1", first_nested1.getValue("ID"));
assertEquals("Cleve Butler", first_nested1.getValue("CONTENT"));
@ -501,9 +502,9 @@ public class TestXMLRecordReader {
assertEquals("USA", first.getValue("COUNTRY"));
Record second = reader.nextRecord(false, false);
assertTrue(second.getValue("NAME") instanceof Record);
assertInstanceOf(Record.class, second.getValue("NAME"));
Record second_nested1 = (Record) second.getValue("NAME");
assertTrue(second.getValue("AGE") instanceof Record);
assertInstanceOf(Record.class, second.getValue("AGE"));
Record second_nested2 = (Record) second.getValue("AGE");
assertEquals("name2", second_nested1.getValue("ID"));
assertEquals("Ainslie Fletcher", second_nested1.getValue("CONTENT"));
@ -569,7 +570,7 @@ public class TestXMLRecordReader {
null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
Record record = reader.nextRecord();
assertTrue(record.getValue("AGE") instanceof String);
assertInstanceOf(String.class, record.getValue("AGE"));
assertEquals("42", record.getValue("AGE"));
}
@ -582,7 +583,7 @@ public class TestXMLRecordReader {
null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
Record record = reader.nextRecord();
assertTrue(record.getValue("ADDRESS") instanceof Record);
assertInstanceOf(Record.class, record.getValue("ADDRESS"));
Record nested = (Record) record.getValue("ADDRESS");
assertEquals("292 West Street", nested.getValue("STREET"));
@ -980,7 +981,7 @@ public class TestXMLRecordReader {
Record valuesFourthRecord = reader.nextRecord();
assertArrayEquals(new Object[] {"Elenora Scrivens", 16, "USA"}, Arrays.copyOfRange(valuesFourthRecord.getValues(), 0, valuesFourthRecord.getValues().length - 1));
assertEquals(null, valuesFourthRecord.getValue("CHILD"));
assertNull(valuesFourthRecord.getValue("CHILD"));
}
@Test
@ -1015,7 +1016,7 @@ public class TestXMLRecordReader {
Record fourth = reader.nextRecord(false, false);
assertArrayEquals(new Object[] {"Elenora Scrivens", "16", "USA"}, Arrays.copyOfRange(fourth.getValues(), 0, fourth.getValues().length - 1));
assertEquals(null, fourth.getValue("CHILD"));
assertNull(fourth.getValue("CHILD"));
}
@Test
@ -1053,7 +1054,7 @@ public class TestXMLRecordReader {
Object[] valuesFourthRecord = fourthRecord.getValues();
assertArrayEquals(new Object[] {"Elenora Scrivens", 16, "USA"}, Arrays.copyOfRange(valuesFourthRecord, 0, valuesFourthRecord.length - 1));
assertEquals(null, fourthRecord.getValue("CHILDREN"));
assertNull(fourthRecord.getValue("CHILDREN"));
}
@Test
@ -1279,15 +1280,15 @@ public class TestXMLRecordReader {
Record first = records.get(0);
Record second = records.get(1);
assertTrue(first.getValue("NAME") instanceof String);
assertInstanceOf(String.class, first.getValue("NAME"));
assertEquals("Cleve Butler", first.getValue("NAME"));
assertTrue(first.getValue("AGE") instanceof Integer);
assertInstanceOf(Integer.class, first.getValue("AGE"));
assertEquals(42, first.getValue("AGE"));
assertEquals(2, first.toMap().size());
assertTrue(second.getValue("NAME") instanceof String);
assertInstanceOf(String.class, second.getValue("NAME"));
assertEquals("Ainslie Fletcher", second.getValue("NAME"));
assertTrue(second.getValue("AGE") instanceof Integer);
assertInstanceOf(Integer.class, second.getValue("AGE"));
assertEquals(33, second.getValue("AGE"));
assertEquals(2, second.toMap().size());
}
@ -1303,15 +1304,15 @@ public class TestXMLRecordReader {
Record first = records.get(0);
Record second = records.get(1);
assertTrue(first.getValue("NAME") instanceof String);
assertInstanceOf(String.class, first.getValue("NAME"));
assertEquals("Cleve Butler", first.getValue("NAME"));
assertTrue(first.getValue("AGE") instanceof String);
assertInstanceOf(String.class, first.getValue("AGE"));
assertEquals("42", first.getValue("AGE"));
assertEquals(2, first.toMap().size());
assertTrue(second.getValue("NAME") instanceof String);
assertInstanceOf(String.class, second.getValue("NAME"));
assertEquals("Ainslie Fletcher", second.getValue("NAME"));
assertTrue(second.getValue("AGE") instanceof String);
assertInstanceOf(String.class, second.getValue("AGE"));
assertEquals("33", second.getValue("AGE"));
assertEquals(2, second.toMap().size());
}
@ -1327,19 +1328,19 @@ public class TestXMLRecordReader {
Record first = records.get(0);
Record second = records.get(1);
assertTrue(first.getValue("NAME") instanceof String);
assertInstanceOf(String.class, first.getValue("NAME"));
assertEquals("Cleve Butler", first.getValue("NAME"));
assertTrue(first.getValue("AGE") instanceof Integer);
assertInstanceOf(Integer.class, first.getValue("AGE"));
assertEquals(42, first.getValue("AGE"));
assertTrue(first.getValue("COUNTRY") instanceof String);
assertInstanceOf(String.class, first.getValue("COUNTRY"));
assertEquals("USA", first.getValue("COUNTRY"));
assertEquals(3, first.toMap().size());
assertTrue(second.getValue("NAME") instanceof String);
assertInstanceOf(String.class, second.getValue("NAME"));
assertEquals("Ainslie Fletcher", second.getValue("NAME"));
assertTrue(second.getValue("AGE") instanceof Integer);
assertInstanceOf(Integer.class, second.getValue("AGE"));
assertEquals(33, second.getValue("AGE"));
assertTrue(second.getValue("COUNTRY") instanceof String);
assertInstanceOf(String.class, second.getValue("COUNTRY"));
assertEquals("UK", second.getValue("COUNTRY"));
assertEquals(3, second.toMap().size());
}
@ -1355,19 +1356,19 @@ public class TestXMLRecordReader {
Record first = records.get(0);
Record second = records.get(1);
assertTrue(first.getValue("NAME") instanceof String);
assertInstanceOf(String.class, first.getValue("NAME"));
assertEquals("Cleve Butler", first.getValue("NAME"));
assertTrue(first.getValue("AGE") instanceof String);
assertInstanceOf(String.class, first.getValue("AGE"));
assertEquals("42", first.getValue("AGE"));
assertTrue(first.getValue("COUNTRY") instanceof String);
assertInstanceOf(String.class, first.getValue("COUNTRY"));
assertEquals("USA", first.getValue("COUNTRY"));
assertEquals(3, first.toMap().size());
assertTrue(second.getValue("NAME") instanceof String);
assertInstanceOf(String.class, second.getValue("NAME"));
assertEquals("Ainslie Fletcher", second.getValue("NAME"));
assertTrue(second.getValue("AGE") instanceof String);
assertInstanceOf(String.class, second.getValue("AGE"));
assertEquals("33", second.getValue("AGE"));
assertTrue(second.getValue("COUNTRY") instanceof String);
assertInstanceOf(String.class, second.getValue("COUNTRY"));
assertEquals("UK", second.getValue("COUNTRY"));
assertEquals(3, second.toMap().size());
}
@ -1478,7 +1479,7 @@ public class TestXMLRecordReader {
private RecordSchema getSchemaWithNestedArray() {
final List<RecordField> fields = getSimpleRecordFields();
final DataType arrayType = RecordFieldType.ARRAY.getArrayDataType(RecordFieldType.STRING.getDataType());
final List<RecordField> nestedArrayField = new ArrayList<RecordField>() {{ add(new RecordField("CHILD", arrayType)); }};
final List<RecordField> nestedArrayField = List.of(new RecordField("CHILD", arrayType));
final DataType recordType = RecordFieldType.RECORD.getRecordDataType(new SimpleRecordSchema(nestedArrayField));
fields.add(new RecordField("CHILDREN", recordType));
@ -1520,11 +1521,10 @@ public class TestXMLRecordReader {
final DataType grandchild_arr1 = RecordFieldType.ARRAY.getArrayDataType(grandchild);
final DataType grandchildren = RecordFieldType.RECORD.getRecordDataType(new SimpleRecordSchema(
new ArrayList<RecordField>() {{ add(new RecordField("CHILD", grandchild_arr1)); }}));
List.of(new RecordField("CHILD", grandchild_arr1))));
final DataType grandchild_arr = RecordFieldType.ARRAY.getArrayDataType(grandchildren);
return new SimpleRecordSchema(
new ArrayList<RecordField>() {{ add(new RecordField("CHILDREN", grandchild_arr)); }});
return new SimpleRecordSchema(List.of(new RecordField("CHILDREN", grandchild_arr)));
}
private RecordSchema getSchemaForComplexData2() {
@ -1532,26 +1532,23 @@ public class TestXMLRecordReader {
final DataType grandchild_arr = RecordFieldType.ARRAY.getArrayDataType(grandchild);
final DataType grandchildren = RecordFieldType.RECORD.getRecordDataType(new SimpleRecordSchema(
new ArrayList<RecordField>() {{ add(new RecordField("CHILD", grandchild_arr)); }}));
List.of(new RecordField("CHILD", grandchild_arr))));
final DataType grandchildren_arr = RecordFieldType.ARRAY.getArrayDataType(grandchildren);
final DataType parent = RecordFieldType.RECORD.getRecordDataType(new SimpleRecordSchema(
new ArrayList<RecordField>() {{
add(new RecordField("CHILDREN", grandchildren_arr));
addAll(getSimpleFieldsForComplexData());
}}));
final List<RecordField> parentRecordFields = new ArrayList<>();
parentRecordFields.add(new RecordField("CHILDREN", grandchildren_arr));
parentRecordFields.addAll(getSimpleFieldsForComplexData());
final DataType parent = RecordFieldType.RECORD.getRecordDataType(new SimpleRecordSchema(parentRecordFields));
final DataType parent_arr = RecordFieldType.ARRAY.getArrayDataType(parent);
final DataType parents = RecordFieldType.RECORD.getRecordDataType(new SimpleRecordSchema(
new ArrayList<RecordField>() {{
add(new RecordField("CHILD", parent_arr));
}}));
List.of(new RecordField("CHILD", parent_arr))));
final DataType parents_arr = RecordFieldType.ARRAY.getArrayDataType(parents);
final List<RecordField> fields = new ArrayList<RecordField>() {{
add(new RecordField("CHILDREN", parents_arr));
addAll(getSimpleFieldsForComplexData());
}};
final List<RecordField> fields = new ArrayList<>();
fields.add(new RecordField("CHILDREN", parents_arr));
fields.addAll(getSimpleFieldsForComplexData());
return new SimpleRecordSchema(fields);
}

View File

@ -37,7 +37,6 @@ import org.apache.nifi.serialization.record.RecordSet;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
@ -90,12 +89,12 @@ public class TestXMLRecordSetWriterProcessor extends AbstractProcessor {
@Override
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
return new ArrayList<PropertyDescriptor>() {{ add(XML_WRITER); add(MULTIPLE_RECORDS); }};
return List.of(XML_WRITER, MULTIPLE_RECORDS);
}
@Override
public Set<Relationship> getRelationships() {
return new HashSet<Relationship>() {{ add(SUCCESS); }};
return Set.of(SUCCESS);
}
protected static RecordSet getRecordSet(boolean multipleRecords) {

Some files were not shown because too many files have changed in this diff Show More