NIFI-14067 Replaced anonymous classes with lambdas (#9571)

Signed-off-by: David Handermann <exceptionfactory@apache.org>
This commit is contained in:
dan-s1 2024-12-09 21:59:52 -05:00 committed by GitHub
parent 5c5b7ffa96
commit a5086a9eb2
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
45 changed files with 1191 additions and 1787 deletions

View File

@ -27,7 +27,6 @@ import org.slf4j.LoggerFactory;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.Comparator;
import java.util.Date; import java.util.Date;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
@ -204,16 +203,13 @@ public class StandardLineageResult implements ComputeLineageResult, ProgressiveR
Map<String, LineageNode> lastEventMap = new HashMap<>(); // maps FlowFile UUID to last event for that FlowFile Map<String, LineageNode> lastEventMap = new HashMap<>(); // maps FlowFile UUID to last event for that FlowFile
final List<ProvenanceEventRecord> sortedRecords = new ArrayList<>(relevantRecords); final List<ProvenanceEventRecord> sortedRecords = new ArrayList<>(relevantRecords);
sortedRecords.sort(new Comparator<ProvenanceEventRecord>() { sortedRecords.sort((o1, o2) -> {
@Override // Sort on Event Time, then Event ID.
public int compare(final ProvenanceEventRecord o1, final ProvenanceEventRecord o2) { final int eventTimeComparison = Long.compare(o1.getEventTime(), o2.getEventTime());
// Sort on Event Time, then Event ID. if (eventTimeComparison == 0) {
final int eventTimeComparison = Long.compare(o1.getEventTime(), o2.getEventTime()); return Long.compare(o1.getEventId(), o2.getEventId());
if (eventTimeComparison == 0) { } else {
return Long.compare(o1.getEventId(), o2.getEventId()); return eventTimeComparison;
} else {
return eventTimeComparison;
}
} }
}); });

View File

@ -23,7 +23,6 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator; import java.util.Comparator;
import java.util.List; import java.util.List;
import java.util.function.Supplier; import java.util.function.Supplier;
@ -121,12 +120,7 @@ public class IdentityMappingUtil {
} }
// sort the list by the key so users can control the ordering in nifi.properties // sort the list by the key so users can control the ordering in nifi.properties
Collections.sort(mappings, new Comparator<IdentityMapping>() { mappings.sort(Comparator.comparing(IdentityMapping::getKey));
@Override
public int compare(IdentityMapping m1, IdentityMapping m2) {
return m1.getKey().compareTo(m2.getKey());
}
});
return mappings; return mappings;
} }

View File

@ -163,12 +163,7 @@ public class RingBuffer<T> {
} }
public List<T> asList() { public List<T> asList() {
return getSelectedElements(new Filter<T>() { return getSelectedElements(value -> true);
@Override
public boolean select(final T value) {
return true;
}
});
} }
public T getOldestElement() { public T getOldestElement() {
@ -252,7 +247,7 @@ public class RingBuffer<T> {
} }
} }
public static interface Filter<S> { public interface Filter<S> {
boolean select(S value); boolean select(S value);
} }
@ -262,7 +257,7 @@ public class RingBuffer<T> {
* *
* @param <S> the type to evaluate * @param <S> the type to evaluate
*/ */
public static interface ForEachEvaluator<S> { public interface ForEachEvaluator<S> {
/** /**
* Evaluates the given element and returns {@code true} if the next element should be evaluated, {@code false} otherwise * Evaluates the given element and returns {@code true} if the next element should be evaluated, {@code false} otherwise
@ -273,9 +268,9 @@ public class RingBuffer<T> {
boolean evaluate(S value); boolean evaluate(S value);
} }
public static enum IterationDirection { public enum IterationDirection {
FORWARD, FORWARD,
BACKWARD; BACKWARD
} }
} }

View File

@ -28,7 +28,6 @@ import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; import java.util.Collections;
import java.util.Comparator;
import java.util.HashSet; import java.util.HashSet;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -167,14 +166,11 @@ public class SequentialAccessWriteAheadLog<T> implements WriteAheadRepository<T>
} }
final List<File> orderedJournalFiles = Arrays.asList(journalFiles); final List<File> orderedJournalFiles = Arrays.asList(journalFiles);
Collections.sort(orderedJournalFiles, new Comparator<File>() { orderedJournalFiles.sort((o1, o2) -> {
@Override final long transactionId1 = getMinTransactionId(o1);
public int compare(final File o1, final File o2) { final long transactionId2 = getMinTransactionId(o2);
final long transactionId1 = getMinTransactionId(o1);
final long transactionId2 = getMinTransactionId(o2);
return Long.compare(transactionId1, transactionId2); return Long.compare(transactionId1, transactionId2);
}
}); });
final long snapshotTransactionId = snapshotRecovery.getMaxTransactionId(); final long snapshotTransactionId = snapshotRecovery.getMaxTransactionId();

View File

@ -29,11 +29,9 @@ import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.api.extension.ExtendWith;
import org.mockito.ArgumentCaptor; import org.mockito.ArgumentCaptor;
import org.mockito.Mock; import org.mockito.Mock;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.junit.jupiter.MockitoExtension; import org.mockito.junit.jupiter.MockitoExtension;
import org.mockito.junit.jupiter.MockitoSettings; import org.mockito.junit.jupiter.MockitoSettings;
import org.mockito.quality.Strictness; import org.mockito.quality.Strictness;
import org.mockito.stubbing.Answer;
import software.amazon.awssdk.awscore.exception.AwsErrorDetails; import software.amazon.awssdk.awscore.exception.AwsErrorDetails;
import software.amazon.awssdk.awscore.exception.AwsServiceException; import software.amazon.awssdk.awscore.exception.AwsServiceException;
import software.amazon.awssdk.services.dynamodb.DynamoDbClient; import software.amazon.awssdk.services.dynamodb.DynamoDbClient;
@ -46,7 +44,6 @@ import software.amazon.awssdk.services.dynamodb.model.WriteRequest;
import java.io.FileInputStream; import java.io.FileInputStream;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
@ -144,7 +141,7 @@ public class PutDynamoDBRecordTest {
final List<BatchWriteItemRequest> results = captor.getAllValues(); final List<BatchWriteItemRequest> results = captor.getAllValues();
Assertions.assertEquals(2, results.size()); Assertions.assertEquals(2, results.size());
final BatchWriteItemRequest result1 = results.get(0); final BatchWriteItemRequest result1 = results.getFirst();
assertTrue(result1.hasRequestItems()); assertTrue(result1.hasRequestItems());
assertNotNull(result1.requestItems().get(TABLE_NAME)); assertNotNull(result1.requestItems().get(TABLE_NAME));
assertItemsConvertedProperly(result1.requestItems().get(TABLE_NAME), 25); assertItemsConvertedProperly(result1.requestItems().get(TABLE_NAME), 25);
@ -155,7 +152,7 @@ public class PutDynamoDBRecordTest {
assertItemsConvertedProperly(result2.requestItems().get(TABLE_NAME), 4); assertItemsConvertedProperly(result2.requestItems().get(TABLE_NAME), 4);
runner.assertAllFlowFilesTransferred(PutDynamoDBRecord.REL_SUCCESS, 1); runner.assertAllFlowFilesTransferred(PutDynamoDBRecord.REL_SUCCESS, 1);
final MockFlowFile flowFile = runner.getFlowFilesForRelationship(PutDynamoDBRecord.REL_SUCCESS).get(0); final MockFlowFile flowFile = runner.getFlowFilesForRelationship(PutDynamoDBRecord.REL_SUCCESS).getFirst();
Assertions.assertEquals("2", flowFile.getAttribute(PutDynamoDBRecord.DYNAMODB_CHUNKS_PROCESSED_ATTRIBUTE)); Assertions.assertEquals("2", flowFile.getAttribute(PutDynamoDBRecord.DYNAMODB_CHUNKS_PROCESSED_ATTRIBUTE));
} }
@ -168,7 +165,7 @@ public class PutDynamoDBRecordTest {
runner.run(); runner.run();
runner.assertAllFlowFilesTransferred(PutDynamoDBRecord.REL_UNPROCESSED, 1); runner.assertAllFlowFilesTransferred(PutDynamoDBRecord.REL_UNPROCESSED, 1);
final MockFlowFile flowFile = runner.getFlowFilesForRelationship(PutDynamoDBRecord.REL_UNPROCESSED).get(0); final MockFlowFile flowFile = runner.getFlowFilesForRelationship(PutDynamoDBRecord.REL_UNPROCESSED).getFirst();
Assertions.assertEquals("1", flowFile.getAttribute(PutDynamoDBRecord.DYNAMODB_CHUNKS_PROCESSED_ATTRIBUTE)); Assertions.assertEquals("1", flowFile.getAttribute(PutDynamoDBRecord.DYNAMODB_CHUNKS_PROCESSED_ATTRIBUTE));
} }
@ -183,7 +180,7 @@ public class PutDynamoDBRecordTest {
Assertions.assertEquals(4, captor.getValue().requestItems().get(TABLE_NAME).size()); Assertions.assertEquals(4, captor.getValue().requestItems().get(TABLE_NAME).size());
runner.assertAllFlowFilesTransferred(PutDynamoDBRecord.REL_SUCCESS, 1); runner.assertAllFlowFilesTransferred(PutDynamoDBRecord.REL_SUCCESS, 1);
final MockFlowFile flowFile = runner.getFlowFilesForRelationship(PutDynamoDBRecord.REL_SUCCESS).get(0); final MockFlowFile flowFile = runner.getFlowFilesForRelationship(PutDynamoDBRecord.REL_SUCCESS).getFirst();
Assertions.assertEquals("2", flowFile.getAttribute(PutDynamoDBRecord.DYNAMODB_CHUNKS_PROCESSED_ATTRIBUTE)); Assertions.assertEquals("2", flowFile.getAttribute(PutDynamoDBRecord.DYNAMODB_CHUNKS_PROCESSED_ATTRIBUTE));
} }
@ -196,7 +193,7 @@ public class PutDynamoDBRecordTest {
runner.run(); runner.run();
runner.assertAllFlowFilesTransferred(PutDynamoDBRecord.REL_FAILURE, 1); runner.assertAllFlowFilesTransferred(PutDynamoDBRecord.REL_FAILURE, 1);
final MockFlowFile flowFile = runner.getFlowFilesForRelationship(PutDynamoDBRecord.REL_FAILURE).get(0); final MockFlowFile flowFile = runner.getFlowFilesForRelationship(PutDynamoDBRecord.REL_FAILURE).getFirst();
Assertions.assertEquals("0", flowFile.getAttribute(PutDynamoDBRecord.DYNAMODB_CHUNKS_PROCESSED_ATTRIBUTE)); Assertions.assertEquals("0", flowFile.getAttribute(PutDynamoDBRecord.DYNAMODB_CHUNKS_PROCESSED_ATTRIBUTE));
} }
@ -212,7 +209,7 @@ public class PutDynamoDBRecordTest {
final BatchWriteItemRequest result = captor.getValue(); final BatchWriteItemRequest result = captor.getValue();
Assertions.assertEquals(1, result.requestItems().get(TABLE_NAME).size()); Assertions.assertEquals(1, result.requestItems().get(TABLE_NAME).size());
final Map<String, AttributeValue> item = result.requestItems().get(TABLE_NAME).iterator().next().putRequest().item(); final Map<String, AttributeValue> item = result.requestItems().get(TABLE_NAME).getFirst().putRequest().item();
Assertions.assertEquals(4, item.size()); Assertions.assertEquals(4, item.size());
Assertions.assertEquals(string("P0"), item.get("partition")); Assertions.assertEquals(string("P0"), item.get("partition"));
assertTrue(item.containsKey("generated")); assertTrue(item.containsKey("generated"));
@ -317,7 +314,7 @@ public class PutDynamoDBRecordTest {
private void setInsertionError() { private void setInsertionError() {
final BatchWriteItemResponse outcome = mock(BatchWriteItemResponse.class); final BatchWriteItemResponse outcome = mock(BatchWriteItemResponse.class);
final Map<String, List<WriteRequest>> unprocessedItems = new HashMap<>(); final Map<String, List<WriteRequest>> unprocessedItems = new HashMap<>();
final List<WriteRequest> writeResults = Arrays.asList(mock(WriteRequest.class)); final List<WriteRequest> writeResults = Collections.singletonList(mock(WriteRequest.class));
unprocessedItems.put("test", writeResults); unprocessedItems.put("test", writeResults);
when(outcome.unprocessedItems()).thenReturn(unprocessedItems); when(outcome.unprocessedItems()).thenReturn(unprocessedItems);
when(outcome.hasUnprocessedItems()).thenReturn(true); when(outcome.hasUnprocessedItems()).thenReturn(true);
@ -332,17 +329,14 @@ public class PutDynamoDBRecordTest {
private void setExceedThroughputAtGivenChunk(final int chunkToFail) { private void setExceedThroughputAtGivenChunk(final int chunkToFail) {
final AtomicInteger numberOfCalls = new AtomicInteger(0); final AtomicInteger numberOfCalls = new AtomicInteger(0);
when(client.batchWriteItem(captor.capture())).then(new Answer<Object>() { when(client.batchWriteItem(captor.capture())).then(invocationOnMock -> {
@Override final int calls = numberOfCalls.incrementAndGet();
public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
final int calls = numberOfCalls.incrementAndGet();
if (calls >= chunkToFail) { if (calls >= chunkToFail) {
throw ProvisionedThroughputExceededException.builder().message("Throughput exceeded") throw ProvisionedThroughputExceededException.builder().message("Throughput exceeded")
.awsErrorDetails(AwsErrorDetails.builder().errorCode("error code").errorMessage("error message").build()).build(); .awsErrorDetails(AwsErrorDetails.builder().errorCode("error code").errorMessage("error message").build()).build();
} else { } else {
return mock(BatchWriteItemResponse.class); return mock(BatchWriteItemResponse.class);
}
} }
}); });
} }

View File

@ -17,7 +17,6 @@
package org.apache.nifi.util.db; package org.apache.nifi.util.db;
import org.mockito.Mockito; import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer; import org.mockito.stubbing.Answer;
import java.io.ByteArrayInputStream; import java.io.ByteArrayInputStream;
@ -38,12 +37,7 @@ class JdbcCommonTestUtils {
when(rs.getMetaData()).thenReturn(metadata); when(rs.getMetaData()).thenReturn(metadata);
final AtomicInteger counter = new AtomicInteger(1); final AtomicInteger counter = new AtomicInteger(1);
Mockito.doAnswer(new Answer<Boolean>() { Mockito.doAnswer((Answer<Boolean>) invocation -> counter.getAndDecrement() > 0).when(rs).next();
@Override
public Boolean answer(InvocationOnMock invocation) throws Throwable {
return counter.getAndDecrement() > 0;
}
}).when(rs).next();
return rs; return rs;
} }

View File

@ -379,12 +379,7 @@ public abstract class AbstractPutEventProcessor<T> extends AbstractSessionFactor
} }
private void transferRanges(final List<Range> ranges, final Relationship relationship) { private void transferRanges(final List<Range> ranges, final Relationship relationship) {
Collections.sort(ranges, new Comparator<Range>() { ranges.sort(Comparator.comparingLong(Range::getStart));
@Override
public int compare(final Range o1, final Range o2) {
return Long.compare(o1.getStart(), o2.getStart());
}
});
for (int i = 0; i < ranges.size(); i++) { for (int i = 0; i < ranges.size(); i++) {
Range range = ranges.get(i); Range range = ranges.get(i);

View File

@ -262,12 +262,7 @@ public class HDFSExternalResourceProvider implements ExternalResourceProvider {
private FileSystem getFileSystemAsUser(final Configuration config, final UserGroupInformation ugi) throws IOException { private FileSystem getFileSystemAsUser(final Configuration config, final UserGroupInformation ugi) throws IOException {
try { try {
return ugi.doAs(new PrivilegedExceptionAction<FileSystem>() { return ugi.doAs((PrivilegedExceptionAction<FileSystem>) () -> FileSystem.get(config));
@Override
public FileSystem run() throws Exception {
return FileSystem.get(config);
}
});
} catch (final InterruptedException e) { } catch (final InterruptedException e) {
throw new IOException("Unable to create file system: " + e.getMessage(), e); throw new IOException("Unable to create file system: " + e.getMessage(), e);
} }

View File

@ -91,56 +91,50 @@ class JMSConsumer extends JMSWorker {
*/ */
public void consumeMessageSet(final String destinationName, String errorQueueName, final boolean durable, final boolean shared, final String subscriptionName, final String messageSelector, public void consumeMessageSet(final String destinationName, String errorQueueName, final boolean durable, final boolean shared, final String subscriptionName, final String messageSelector,
final String charset, final int batchSize, final Consumer<List<JMSResponse>> messageSetConsumer) { final String charset, final int batchSize, final Consumer<List<JMSResponse>> messageSetConsumer) {
doWithJmsTemplate(destinationName, durable, shared, subscriptionName, messageSelector, new MessageReceiver() { doWithJmsTemplate(destinationName, durable, shared, subscriptionName, messageSelector, (session, messageConsumer) -> {
@Override final List<JMSResponse> jmsResponses = new ArrayList<>();
public void consume(Session session, MessageConsumer messageConsumer) throws JMSException { int batchCounter = 0;
final List<JMSResponse> jmsResponses = new ArrayList<>();
int batchCounter = 0;
JMSResponse response; JMSResponse response;
while (batchCounter < batchSize && (response = receiveMessage(session, messageConsumer, charset, errorQueueName)) != null) { while (batchCounter < batchSize && (response = receiveMessage(session, messageConsumer, charset, errorQueueName)) != null) {
response.setBatchOrder(batchCounter); response.setBatchOrder(batchCounter);
jmsResponses.add(response); jmsResponses.add(response);
batchCounter++; batchCounter++;
} }
if (!jmsResponses.isEmpty()) { if (!jmsResponses.isEmpty()) {
// Provide the JMSResponse to the processor to handle. It is the responsibility of the // Provide the JMSResponse to the processor to handle. It is the responsibility of the
// processor to handle acknowledgment of the message (if Client Acknowledge), and it is // processor to handle acknowledgment of the message (if Client Acknowledge), and it is
// the responsibility of the processor to handle closing the Message Consumer. // the responsibility of the processor to handle closing the Message Consumer.
// Both of these actions can be handled by calling the acknowledge() or reject() methods of // Both of these actions can be handled by calling the acknowledge() or reject() methods of
// the JMSResponse. // the JMSResponse.
messageSetConsumer.accept(jmsResponses); messageSetConsumer.accept(jmsResponses);
}
} }
}); });
} }
private void doWithJmsTemplate(String destinationName, boolean durable, boolean shared, String subscriptionName, String messageSelector, MessageReceiver messageReceiver) { private void doWithJmsTemplate(String destinationName, boolean durable, boolean shared, String subscriptionName, String messageSelector, MessageReceiver messageReceiver) {
this.jmsTemplate.execute(new SessionCallback<Void>() { this.jmsTemplate.execute((SessionCallback<Void>) session -> {
@Override
public Void doInJms(final Session session) throws JMSException {
final MessageConsumer messageConsumer = createMessageConsumer(session, destinationName, durable, shared, subscriptionName, messageSelector); final MessageConsumer messageConsumer = createMessageConsumer(session, destinationName, durable, shared, subscriptionName, messageSelector);
try {
messageReceiver.consume(session, messageConsumer);
} catch (Exception e) {
// We need to call recover to ensure that in the event of
// abrupt end or exception the current session will stop message
// delivery and restart with the oldest unacknowledged message
try { try {
messageReceiver.consume(session, messageConsumer); session.recover();
} catch (Exception e) { } catch (Exception e1) {
// We need to call recover to ensure that in the event of // likely the session is closed...need to catch this so that the root cause of failure is propagated
// abrupt end or exception the current session will stop message processLog.debug("Failed to recover JMS session while handling initial error. The recover error is: ", e1);
// delivery and restart with the oldest unacknowledged message
try {
session.recover();
} catch (Exception e1) {
// likely the session is closed...need to catch this so that the root cause of failure is propagated
processLog.debug("Failed to recover JMS session while handling initial error. The recover error is: ", e1);
}
JmsUtils.closeMessageConsumer(messageConsumer);
throw e;
} }
return null; JmsUtils.closeMessageConsumer(messageConsumer);
throw e;
} }
return null;
}, true); }, true);
} }

View File

@ -62,7 +62,6 @@ import org.apache.nifi.util.db.JdbcProperties;
import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import org.mockito.Mockito; import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer; import org.mockito.stubbing.Answer;
import java.io.IOException; import java.io.IOException;
@ -81,6 +80,7 @@ import java.util.stream.Collectors;
import java.util.stream.Stream; import java.util.stream.Stream;
import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertInstanceOf;
import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.assertTrue;
@ -237,11 +237,11 @@ class TestQueryNiFiReportingTask {
List<Map<String, Object>> rows = mockRecordSinkService.getRows(); List<Map<String, Object>> rows = mockRecordSinkService.getRows();
assertEquals(4, rows.size()); assertEquals(4, rows.size());
// Validate the first row // Validate the first row
Map<String, Object> row = rows.get(0); Map<String, Object> row = rows.getFirst();
assertEquals(3, row.size()); // Only projected 2 columns assertEquals(3, row.size()); // Only projected 2 columns
Object id = row.get("id"); Object id = row.get("id");
assertTrue(id instanceof String); assertInstanceOf(String.class, id);
assertEquals("nested", id); assertEquals("nested", id);
assertEquals(1001, row.get("queuedCount")); assertEquals(1001, row.get("queuedCount"));
// Validate the second row // Validate the second row
@ -483,11 +483,11 @@ class TestQueryNiFiReportingTask {
List<Map<String, Object>> rows = mockRecordSinkService.getRows(); List<Map<String, Object>> rows = mockRecordSinkService.getRows();
assertEquals(1, rows.size()); assertEquals(1, rows.size());
Map<String, Object> row = rows.get(0); Map<String, Object> row = rows.getFirst();
assertEquals(11, row.size()); assertEquals(11, row.size());
assertTrue(row.get(MetricNames.JVM_DAEMON_THREAD_COUNT.replace(".", "_")) instanceof Integer); assertInstanceOf(Integer.class, row.get(MetricNames.JVM_DAEMON_THREAD_COUNT.replace(".", "_")));
assertTrue(row.get(MetricNames.JVM_HEAP_USAGE.replace(".", "_")) instanceof Double); assertInstanceOf(Double.class, row.get(MetricNames.JVM_HEAP_USAGE.replace(".", "_")));
} }
@Test @Test
@ -501,7 +501,7 @@ class TestQueryNiFiReportingTask {
List<Map<String, Object>> rows = mockRecordSinkService.getRows(); List<Map<String, Object>> rows = mockRecordSinkService.getRows();
assertEquals(4, rows.size()); assertEquals(4, rows.size());
// Validate the first row // Validate the first row
Map<String, Object> row = rows.get(0); Map<String, Object> row = rows.getFirst();
assertEquals(26, row.size()); assertEquals(26, row.size());
assertEquals(1L, row.get("bytesRead")); assertEquals(1L, row.get("bytesRead"));
// Validate the second row // Validate the second row
@ -538,7 +538,7 @@ class TestQueryNiFiReportingTask {
List<Map<String, Object>> rows = mockRecordSinkService.getRows(); List<Map<String, Object>> rows = mockRecordSinkService.getRows();
assertEquals(1001, rows.size()); assertEquals(1001, rows.size());
// Validate the first row // Validate the first row
Map<String, Object> row = rows.get(0); Map<String, Object> row = rows.getFirst();
assertEquals(24, row.size()); assertEquals(24, row.size());
// Verify the first row contents // Verify the first row contents
final Long firstEventId = (Long) row.get("eventId"); final Long firstEventId = (Long) row.get("eventId");
@ -549,11 +549,11 @@ class TestQueryNiFiReportingTask {
assertNull(row.get("previousContentPath")); assertNull(row.get("previousContentPath"));
Object o = row.get("previousAttributes"); Object o = row.get("previousAttributes");
assertTrue(o instanceof Map); assertInstanceOf(Map.class, o);
Map<String, String> previousAttributes = (Map<String, String>) o; Map<String, String> previousAttributes = (Map<String, String>) o;
assertEquals("A", previousAttributes.get("test.value")); assertEquals("A", previousAttributes.get("test.value"));
o = row.get("updatedAttributes"); o = row.get("updatedAttributes");
assertTrue(o instanceof Map); assertInstanceOf(Map.class, o);
Map<String, String> updatedAttributes = (Map<String, String>) o; Map<String, String> updatedAttributes = (Map<String, String>) o;
assertEquals("B", updatedAttributes.get("test.value")); assertEquals("B", updatedAttributes.get("test.value"));
@ -595,7 +595,7 @@ class TestQueryNiFiReportingTask {
} }
// Validate the first row // Validate the first row
Map<String, Object> row = rows.get(0); Map<String, Object> row = rows.getFirst();
assertEquals(14, row.size()); assertEquals(14, row.size());
assertNotNull(row.get("bulletinId")); assertNotNull(row.get("bulletinId"));
assertEquals("controller", row.get("bulletinCategory")); assertEquals("controller", row.get("bulletinCategory"));
@ -625,7 +625,7 @@ class TestQueryNiFiReportingTask {
List<Map<String, Object>> rows = mockRecordSinkService.getRows(); List<Map<String, Object>> rows = mockRecordSinkService.getRows();
assertEquals(1, rows.size()); assertEquals(1, rows.size());
// Validate the first row // Validate the first row
Map<String, Object> row = rows.get(0); Map<String, Object> row = rows.getFirst();
assertEquals(22, row.size()); assertEquals(22, row.size());
// Verify the first row contents // Verify the first row contents
assertEquals(123, row.get("actionId")); assertEquals(123, row.get("actionId"));
@ -720,13 +720,10 @@ class TestQueryNiFiReportingTask {
Mockito.when(eventAccess.getProvenanceRepository()).thenReturn(mockProvenanceRepository); Mockito.when(eventAccess.getProvenanceRepository()).thenReturn(mockProvenanceRepository);
try { try {
Mockito.when(eventAccess.getProvenanceEvents(anyLong(), anyInt())).thenAnswer(new Answer<List<ProvenanceEventRecord>>() { Mockito.when(eventAccess.getProvenanceEvents(anyLong(), anyInt())).thenAnswer((Answer<List<ProvenanceEventRecord>>) invocation -> {
@Override final long startEventId = invocation.getArgument(0);
public List<ProvenanceEventRecord> answer(final InvocationOnMock invocation) throws Throwable { final int max = invocation.getArgument(1);
final long startEventId = invocation.getArgument(0); return mockProvenanceRepository.getEvents(startEventId, max);
final int max = invocation.getArgument(1);
return mockProvenanceRepository.getEvents(startEventId, max);
}
}); });
} catch (final IOException e) { } catch (final IOException e) {
// Won't happen // Won't happen

View File

@ -40,7 +40,6 @@ import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.io.OutputStreamCallback;
import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processors.standard.util.ArgumentUtils; import org.apache.nifi.processors.standard.util.ArgumentUtils;
@ -57,7 +56,6 @@ import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService; import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors; import java.util.concurrent.Executors;
@ -265,35 +263,32 @@ public class ExecuteProcess extends AbstractProcessor {
// as the delegate for the ProxyOuptutStream, then wait until the process finishes // as the delegate for the ProxyOuptutStream, then wait until the process finishes
// or until the specified amount of time // or until the specified amount of time
FlowFile flowFile = session.create(); FlowFile flowFile = session.create();
flowFile = session.write(flowFile, new OutputStreamCallback() { flowFile = session.write(flowFile, flowFileOut -> {
@Override try (final OutputStream out = new BufferedOutputStream(flowFileOut)) {
public void process(final OutputStream flowFileOut) throws IOException { proxyOut.setDelegate(out);
try (final OutputStream out = new BufferedOutputStream(flowFileOut)) {
proxyOut.setDelegate(out);
if (batchNanos == null) { if (batchNanos == null) {
// we are not creating batches; wait until process terminates. // we are not creating batches; wait until process terminates.
// NB!!! Maybe get(long timeout, TimeUnit unit) should // NB!!! Maybe get(long timeout, TimeUnit unit) should
// be used to avoid waiting forever. // be used to avoid waiting forever.
try { try {
longRunningProcess.get(); longRunningProcess.get();
} catch (final InterruptedException ie) { } catch (final InterruptedException ie) {
// Ignore // Ignore
} catch (final ExecutionException ee) { } catch (final ExecutionException ee) {
getLogger().error("Process execution failed", ee.getCause()); getLogger().error("Process execution failed", ee.getCause());
} }
} else { } else {
// wait the allotted amount of time. // wait the allotted amount of time.
try { try {
TimeUnit.NANOSECONDS.sleep(batchNanos); TimeUnit.NANOSECONDS.sleep(batchNanos);
} catch (final InterruptedException ie) { } catch (final InterruptedException ie) {
// Ignore // Ignore
}
} }
proxyOut.setDelegate(null); // prevent from writing to this
// stream
} }
proxyOut.setDelegate(null); // prevent from writing to this
// stream
} }
}); });
@ -358,82 +353,76 @@ public class ExecuteProcess extends AbstractProcessor {
// Submit task to read error stream from process // Submit task to read error stream from process
if (!redirectErrorStream) { if (!redirectErrorStream) {
executor.submit(new Runnable() { executor.submit(() -> {
@Override try (final BufferedReader reader = new BufferedReader(new InputStreamReader(externalProcess.getErrorStream()))) {
public void run() { reader.lines().filter(line -> line != null && !line.isEmpty()).forEach(getLogger()::warn);
try (final BufferedReader reader = new BufferedReader(new InputStreamReader(externalProcess.getErrorStream()))) { } catch (final IOException ignored) {
reader.lines().filter(line -> line != null && line.length() > 0).forEach(getLogger()::warn);
} catch (final IOException ioe) {
}
} }
}); });
} }
// Submit task to read output of Process and write to FlowFile. // Submit task to read output of Process and write to FlowFile.
failure = new AtomicBoolean(false); failure = new AtomicBoolean(false);
final Future<?> future = executor.submit(new Callable<Object>() { final Future<?> future = executor.submit(() -> {
@Override try {
public Object call() throws IOException { if (batchNanos == null) {
try { // if we aren't batching, just copy the stream from the
if (batchNanos == null) { // process to the flowfile.
// if we aren't batching, just copy the stream from the try (final BufferedInputStream bufferedIn = new BufferedInputStream(externalProcess.getInputStream())) {
// process to the flowfile. final byte[] buffer = new byte[4096];
try (final BufferedInputStream bufferedIn = new BufferedInputStream(externalProcess.getInputStream())) { int len;
final byte[] buffer = new byte[4096]; while ((len = bufferedIn.read(buffer)) > 0) {
int len;
while ((len = bufferedIn.read(buffer)) > 0) {
// NB!!!! Maybe all data should be read from // NB!!!! Maybe all data should be read from
// input stream in case of !isScheduled() to // input stream in case of !isScheduled() to
// avoid subprocess deadlock? // avoid subprocess deadlock?
// (we just don't write data to proxyOut) // (we just don't write data to proxyOut)
// Or because we don't use this subprocess // Or because we don't use this subprocess
// anymore anyway, we don't care? // anymore anyway, we don't care?
if (!isScheduled()) { if (!isScheduled()) {
return null; return null;
}
proxyOut.write(buffer, 0, len);
} }
}
} else {
// we are batching, which means that the output of the
// process is text. It doesn't make sense to grab
// arbitrary batches of bytes from some process and send
// it along as a piece of data, so we assume that
// setting a batch during means text.
// Also, we don't want that text to get split up in the
// middle of a line, so we use BufferedReader
// to read lines of text and write them as lines of text.
try (final BufferedReader reader = new BufferedReader(new InputStreamReader(externalProcess.getInputStream()))) {
String line;
while ((line = reader.readLine()) != null) { proxyOut.write(buffer, 0, len);
if (!isScheduled()) {
return null;
}
proxyOut.write((line + "\n").getBytes(StandardCharsets.UTF_8));
}
} }
} }
} catch (final IOException ioe) { } else {
failure.set(true); // we are batching, which means that the output of the
throw ioe; // process is text. It doesn't make sense to grab
} finally { // arbitrary batches of bytes from some process and send
try { // it along as a piece of data, so we assume that
// Since we are going to exit anyway, one sec gives it an extra chance to exit gracefully. // setting a batch during means text.
// In the future consider exposing it via configuration. // Also, we don't want that text to get split up in the
boolean terminated = externalProcess.waitFor(1000, TimeUnit.MILLISECONDS); // middle of a line, so we use BufferedReader
int exitCode = terminated ? externalProcess.exitValue() : -9999; // to read lines of text and write them as lines of text.
getLogger().info("Process finished with exit code {} ", exitCode); try (final BufferedReader reader = new BufferedReader(new InputStreamReader(externalProcess.getInputStream()))) {
} catch (InterruptedException e1) { String line;
Thread.currentThread().interrupt();
while ((line = reader.readLine()) != null) {
if (!isScheduled()) {
return null;
}
proxyOut.write((line + "\n").getBytes(StandardCharsets.UTF_8));
}
} }
} }
} catch (final IOException ioe) {
return null; failure.set(true);
throw ioe;
} finally {
try {
// Since we are going to exit anyway, one sec gives it an extra chance to exit gracefully.
// In the future consider exposing it via configuration.
boolean terminated = externalProcess.waitFor(1000, TimeUnit.MILLISECONDS);
int exitCode = terminated ? externalProcess.exitValue() : -9999;
getLogger().info("Process finished with exit code {} ", exitCode);
} catch (InterruptedException e1) {
Thread.currentThread().interrupt();
}
} }
return null;
}); });
return future; return future;

View File

@ -57,8 +57,6 @@ import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.io.InputStreamCallback;
import org.apache.nifi.processor.io.OutputStreamCallback;
import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processor.util.bin.Bin; import org.apache.nifi.processor.util.bin.Bin;
import org.apache.nifi.processor.util.bin.BinFiles; import org.apache.nifi.processor.util.bin.BinFiles;
@ -659,44 +657,41 @@ public class MergeContent extends BinFiles {
FlowFile bundle = session.create(bin.getContents()); FlowFile bundle = session.create(bin.getContents());
final AtomicReference<String> bundleMimeTypeRef = new AtomicReference<>(null); final AtomicReference<String> bundleMimeTypeRef = new AtomicReference<>(null);
try { try {
bundle = session.write(bundle, new OutputStreamCallback() { bundle = session.write(bundle, out -> {
@Override final byte[] header = getDelimiterContent(context, contents, HEADER);
public void process(final OutputStream out) throws IOException { if (header != null) {
final byte[] header = getDelimiterContent(context, contents, HEADER); out.write(header);
if (header != null) { }
out.write(header);
}
final byte[] demarcator = getDelimiterContent(context, contents, DEMARCATOR); final byte[] demarcator = getDelimiterContent(context, contents, DEMARCATOR);
boolean isFirst = true; boolean isFirst = true;
final Iterator<FlowFile> itr = contents.iterator(); final Iterator<FlowFile> itr = contents.iterator();
while (itr.hasNext()) { while (itr.hasNext()) {
final FlowFile flowFile = itr.next(); final FlowFile flowFile = itr.next();
bin.getSession().read(flowFile, in -> StreamUtils.copy(in, out)); bin.getSession().read(flowFile, in -> StreamUtils.copy(in, out));
if (itr.hasNext()) { if (itr.hasNext()) {
if (demarcator != null) { if (demarcator != null) {
out.write(demarcator); out.write(demarcator);
}
}
final String flowFileMimeType = flowFile.getAttribute(CoreAttributes.MIME_TYPE.key());
if (isFirst) {
bundleMimeTypeRef.set(flowFileMimeType);
isFirst = false;
} else {
if (bundleMimeTypeRef.get() != null && !bundleMimeTypeRef.get().equals(flowFileMimeType)) {
bundleMimeTypeRef.set(null);
}
} }
} }
final byte[] footer = getDelimiterContent(context, contents, FOOTER); final String flowFileMimeType = flowFile.getAttribute(CoreAttributes.MIME_TYPE.key());
if (footer != null) { if (isFirst) {
out.write(footer); bundleMimeTypeRef.set(flowFileMimeType);
isFirst = false;
} else {
if (bundleMimeTypeRef.get() != null && !bundleMimeTypeRef.get().equals(flowFileMimeType)) {
bundleMimeTypeRef.set(null);
}
} }
} }
final byte[] footer = getDelimiterContent(context, contents, FOOTER);
if (footer != null) {
out.write(footer);
}
}); });
} catch (final Exception e) { } catch (final Exception e) {
removeFlowFileFromSession(session, bundle, context); removeFlowFileFromSession(session, bundle, context);
@ -802,49 +797,46 @@ public class MergeContent extends BinFiles {
try { try {
bundle = session.putAttribute(bundle, CoreAttributes.FILENAME.key(), createFilename(contents) + ".tar"); bundle = session.putAttribute(bundle, CoreAttributes.FILENAME.key(), createFilename(contents) + ".tar");
bundle = session.write(bundle, new OutputStreamCallback() { bundle = session.write(bundle, rawOut -> {
@Override try (final OutputStream bufferedOut = new BufferedOutputStream(rawOut);
public void process(final OutputStream rawOut) throws IOException { final TarArchiveOutputStream out = new TarArchiveOutputStream(bufferedOut)) {
try (final OutputStream bufferedOut = new BufferedOutputStream(rawOut);
final TarArchiveOutputStream out = new TarArchiveOutputStream(bufferedOut)) {
out.setLongFileMode(TarArchiveOutputStream.LONGFILE_GNU); out.setLongFileMode(TarArchiveOutputStream.LONGFILE_GNU);
// if any one of the FlowFiles is larger than the default maximum tar entry size, then we set bigNumberMode to handle it // if any one of the FlowFiles is larger than the default maximum tar entry size, then we set bigNumberMode to handle it
if (getMaxEntrySize(contents) >= TarConstants.MAXSIZE) { if (getMaxEntrySize(contents) >= TarConstants.MAXSIZE) {
out.setBigNumberMode(TarArchiveOutputStream.BIGNUMBER_POSIX); out.setBigNumberMode(TarArchiveOutputStream.BIGNUMBER_POSIX);
} }
for (final FlowFile flowFile : contents) { for (final FlowFile flowFile : contents) {
final String path = keepPath ? getPath(flowFile) : ""; final String path = keepPath ? getPath(flowFile) : "";
final String entryName = path + flowFile.getAttribute(CoreAttributes.FILENAME.key()); final String entryName = path + flowFile.getAttribute(CoreAttributes.FILENAME.key());
final TarArchiveEntry tarEntry = new TarArchiveEntry(entryName); final TarArchiveEntry tarEntry = new TarArchiveEntry(entryName);
tarEntry.setSize(flowFile.getSize()); tarEntry.setSize(flowFile.getSize());
final String permissionsVal = flowFile.getAttribute(TAR_PERMISSIONS_ATTRIBUTE); final String permissionsVal = flowFile.getAttribute(TAR_PERMISSIONS_ATTRIBUTE);
if (permissionsVal != null) { if (permissionsVal != null) {
try { try {
tarEntry.setMode(Integer.parseInt(permissionsVal)); tarEntry.setMode(Integer.parseInt(permissionsVal));
} catch (final Exception e) { } catch (final Exception e) {
getLogger().debug("Attribute {} of {} is set to {}; expected 3 digits between 0-7, so ignoring", getLogger().debug("Attribute {} of {} is set to {}; expected 3 digits between 0-7, so ignoring",
new Object[] {TAR_PERMISSIONS_ATTRIBUTE, flowFile, permissionsVal}); new Object[] {TAR_PERMISSIONS_ATTRIBUTE, flowFile, permissionsVal});
}
} }
final String modTime = context.getProperty(TAR_MODIFIED_TIME)
.evaluateAttributeExpressions(flowFile).getValue();
if (StringUtils.isNotBlank(modTime)) {
try {
tarEntry.setModTime(Instant.parse(modTime).toEpochMilli());
} catch (final Exception e) {
getLogger().debug("Attribute {} of {} is set to {}; expected ISO8601 format, so ignoring",
new Object[] {TAR_MODIFIED_TIME, flowFile, modTime});
}
}
out.putArchiveEntry(tarEntry);
bin.getSession().exportTo(flowFile, out);
out.closeArchiveEntry();
} }
final String modTime = context.getProperty(TAR_MODIFIED_TIME)
.evaluateAttributeExpressions(flowFile).getValue();
if (StringUtils.isNotBlank(modTime)) {
try {
tarEntry.setModTime(Instant.parse(modTime).toEpochMilli());
} catch (final Exception e) {
getLogger().debug("Attribute {} of {} is set to {}; expected ISO8601 format, so ignoring",
new Object[] {TAR_MODIFIED_TIME, flowFile, modTime});
}
}
out.putArchiveEntry(tarEntry);
bin.getSession().exportTo(flowFile, out);
out.closeArchiveEntry();
} }
} }
}); });
@ -894,25 +886,19 @@ public class MergeContent extends BinFiles {
FlowFile bundle = session.create(contents); FlowFile bundle = session.create(contents);
try { try {
bundle = session.write(bundle, new OutputStreamCallback() { bundle = session.write(bundle, rawOut -> {
@Override try (final OutputStream bufferedOut = new BufferedOutputStream(rawOut)) {
public void process(final OutputStream rawOut) throws IOException { // we don't want the packager closing the stream. V1 creates a TAR Output Stream, which then gets
try (final OutputStream bufferedOut = new BufferedOutputStream(rawOut)) { // closed, which in turn closes the underlying OutputStream, and we want to protect ourselves against that.
// we don't want the packager closing the stream. V1 creates a TAR Output Stream, which then gets final OutputStream out = new NonCloseableOutputStream(bufferedOut);
// closed, which in turn closes the underlying OutputStream, and we want to protect ourselves against that.
final OutputStream out = new NonCloseableOutputStream(bufferedOut);
for (final FlowFile flowFile : contents) { for (final FlowFile flowFile : contents) {
bin.getSession().read(flowFile, new InputStreamCallback() { bin.getSession().read(flowFile, rawIn -> {
@Override try (final InputStream in = new BufferedInputStream(rawIn)) {
public void process(final InputStream rawIn) throws IOException { final Map<String, String> attributes = new HashMap<>(flowFile.getAttributes());
try (final InputStream in = new BufferedInputStream(rawIn)) { packager.packageFlowFile(in, out, attributes, flowFile.getSize());
final Map<String, String> attributes = new HashMap<>(flowFile.getAttributes()); }
packager.packageFlowFile(in, out, attributes, flowFile.getSize()); });
}
}
});
}
} }
} }
}); });
@ -959,31 +945,28 @@ public class MergeContent extends BinFiles {
try { try {
bundle = session.putAttribute(bundle, CoreAttributes.FILENAME.key(), createFilename(contents) + ".zip"); bundle = session.putAttribute(bundle, CoreAttributes.FILENAME.key(), createFilename(contents) + ".zip");
bundle = session.write(bundle, new OutputStreamCallback() { bundle = session.write(bundle, rawOut -> {
@Override try (final OutputStream bufferedOut = new BufferedOutputStream(rawOut);
public void process(final OutputStream rawOut) throws IOException { final ZipOutputStream out = new ZipOutputStream(bufferedOut)) {
try (final OutputStream bufferedOut = new BufferedOutputStream(rawOut); out.setLevel(compressionLevel);
final ZipOutputStream out = new ZipOutputStream(bufferedOut)) { for (final FlowFile flowFile : contents) {
out.setLevel(compressionLevel); final String path = keepPath ? getPath(flowFile) : "";
for (final FlowFile flowFile : contents) { final String entryName = path + flowFile.getAttribute(CoreAttributes.FILENAME.key());
final String path = keepPath ? getPath(flowFile) : ""; final ZipEntry zipEntry = new ZipEntry(entryName);
final String entryName = path + flowFile.getAttribute(CoreAttributes.FILENAME.key()); zipEntry.setSize(flowFile.getSize());
final ZipEntry zipEntry = new ZipEntry(entryName); try {
zipEntry.setSize(flowFile.getSize()); out.putNextEntry(zipEntry);
try {
out.putNextEntry(zipEntry);
bin.getSession().exportTo(flowFile, out); bin.getSession().exportTo(flowFile, out);
out.closeEntry(); out.closeEntry();
unmerged.remove(flowFile); unmerged.remove(flowFile);
} catch (ZipException e) { } catch (ZipException e) {
getLogger().error("Encountered exception merging {}", flowFile, e); getLogger().error("Encountered exception merging {}", flowFile, e);
}
} }
out.finish();
out.flush();
} }
out.finish();
out.flush();
} }
}); });
} catch (final Exception e) { } catch (final Exception e) {
@ -1024,90 +1007,84 @@ public class MergeContent extends BinFiles {
// we don't pass the parents to the #create method because the parents belong to different sessions // we don't pass the parents to the #create method because the parents belong to different sessions
FlowFile bundle = session.create(contents); FlowFile bundle = session.create(contents);
try { try {
bundle = session.write(bundle, new OutputStreamCallback() { bundle = session.write(bundle, rawOut -> {
@Override try (final OutputStream out = new BufferedOutputStream(rawOut)) {
public void process(final OutputStream rawOut) throws IOException { for (final FlowFile flowFile : contents) {
try (final OutputStream out = new BufferedOutputStream(rawOut)) { bin.getSession().read(flowFile, in -> {
for (final FlowFile flowFile : contents) { boolean canMerge = true;
bin.getSession().read(flowFile, new InputStreamCallback() { try (DataFileStream<GenericRecord> reader = new DataFileStream<>(in,
@Override new GenericDatumReader<>())) {
public void process(InputStream in) throws IOException { if (schema.get() == null) {
boolean canMerge = true; // this is the first file - set up the writer, and store the
try (DataFileStream<GenericRecord> reader = new DataFileStream<>(in, // Schema & metadata we'll use.
new GenericDatumReader<GenericRecord>())) { schema.set(reader.getSchema());
if (schema.get() == null) { if (!METADATA_STRATEGY_IGNORE.getValue().equals(metadataStrategy)) {
// this is the first file - set up the writer, and store the for (String key : reader.getMetaKeys()) {
// Schema & metadata we'll use. if (!DataFileWriter.isReservedMeta(key)) {
schema.set(reader.getSchema()); byte[] metadatum = reader.getMeta(key);
if (!METADATA_STRATEGY_IGNORE.getValue().equals(metadataStrategy)) { metadata.put(key, metadatum);
for (String key : reader.getMetaKeys()) { writer.setMeta(key, metadatum);
if (!DataFileWriter.isReservedMeta(key)) {
byte[] metadatum = reader.getMeta(key);
metadata.put(key, metadatum);
writer.setMeta(key, metadatum);
}
}
} }
inputCodec.set(reader.getMetaString(DataFileConstants.CODEC));
if (inputCodec.get() == null) {
inputCodec.set(DataFileConstants.NULL_CODEC);
}
writer.setCodec(CodecFactory.fromString(inputCodec.get()));
writer.create(schema.get(), out);
} else {
// check that we're appending to the same schema
if (!schema.get().equals(reader.getSchema())) {
getLogger().debug("Input file {} has different schema - {}, not merging",
new Object[] {flowFile.getId(), reader.getSchema().getName()});
canMerge = false;
unmerged.add(flowFile);
}
if (METADATA_STRATEGY_DO_NOT_MERGE.getValue().equals(metadataStrategy)
|| METADATA_STRATEGY_ALL_COMMON.getValue().equals(metadataStrategy)) {
// check that we're appending to the same metadata
for (String key : reader.getMetaKeys()) {
if (!DataFileWriter.isReservedMeta(key)) {
byte[] metadatum = reader.getMeta(key);
byte[] writersMetadatum = metadata.get(key);
if (!Arrays.equals(metadatum, writersMetadatum)) {
// Ignore additional metadata if ALL_COMMON is the strategy, otherwise don't merge
if (!METADATA_STRATEGY_ALL_COMMON.getValue().equals(metadataStrategy) || writersMetadatum != null) {
getLogger().debug("Input file {} has different non-reserved metadata, not merging",
new Object[] {flowFile.getId()});
canMerge = false;
unmerged.add(flowFile);
}
}
}
}
} // else the metadata in the first FlowFile was either ignored or retained in the if-clause above
// check that we're appending to the same codec
String thisCodec = reader.getMetaString(DataFileConstants.CODEC);
if (thisCodec == null) {
thisCodec = DataFileConstants.NULL_CODEC;
}
if (!inputCodec.get().equals(thisCodec)) {
getLogger().debug("Input file {} has different codec, not merging",
new Object[] {flowFile.getId()});
canMerge = false;
unmerged.add(flowFile);
}
}
// write the Avro content from the current FlowFile to the merged OutputStream
if (canMerge) {
writer.appendAllFrom(reader, false);
} }
} }
inputCodec.set(reader.getMetaString(DataFileConstants.CODEC));
if (inputCodec.get() == null) {
inputCodec.set(DataFileConstants.NULL_CODEC);
}
writer.setCodec(CodecFactory.fromString(inputCodec.get()));
writer.create(schema.get(), out);
} else {
// check that we're appending to the same schema
if (!schema.get().equals(reader.getSchema())) {
getLogger().debug("Input file {} has different schema - {}, not merging",
new Object[] {flowFile.getId(), reader.getSchema().getName()});
canMerge = false;
unmerged.add(flowFile);
}
if (METADATA_STRATEGY_DO_NOT_MERGE.getValue().equals(metadataStrategy)
|| METADATA_STRATEGY_ALL_COMMON.getValue().equals(metadataStrategy)) {
// check that we're appending to the same metadata
for (String key : reader.getMetaKeys()) {
if (!DataFileWriter.isReservedMeta(key)) {
byte[] metadatum = reader.getMeta(key);
byte[] writersMetadatum = metadata.get(key);
if (!Arrays.equals(metadatum, writersMetadatum)) {
// Ignore additional metadata if ALL_COMMON is the strategy, otherwise don't merge
if (!METADATA_STRATEGY_ALL_COMMON.getValue().equals(metadataStrategy) || writersMetadatum != null) {
getLogger().debug("Input file {} has different non-reserved metadata, not merging",
new Object[] {flowFile.getId()});
canMerge = false;
unmerged.add(flowFile);
}
}
}
}
} // else the metadata in the first FlowFile was either ignored or retained in the if-clause above
// check that we're appending to the same codec
String thisCodec = reader.getMetaString(DataFileConstants.CODEC);
if (thisCodec == null) {
thisCodec = DataFileConstants.NULL_CODEC;
}
if (!inputCodec.get().equals(thisCodec)) {
getLogger().debug("Input file {} has different codec, not merging",
new Object[] {flowFile.getId()});
canMerge = false;
unmerged.add(flowFile);
}
} }
});
} // write the Avro content from the current FlowFile to the merged OutputStream
writer.flush(); if (canMerge) {
} finally { writer.appendAllFrom(reader, false);
writer.close(); }
}
});
} }
writer.flush();
} finally {
writer.close();
} }
}); });
} catch (final Exception e) { } catch (final Exception e) {

View File

@ -33,8 +33,6 @@ import org.slf4j.LoggerFactory;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -255,21 +253,18 @@ public class ControllerStatusReportingTask extends AbstractReportingTask {
private void printConnectionStatus(final ProcessGroupStatus groupStatus, final StringBuilder builder, final boolean showDeltas, final long divisor) { private void printConnectionStatus(final ProcessGroupStatus groupStatus, final StringBuilder builder, final boolean showDeltas, final long divisor) {
final List<ConnectionStatus> connectionStatuses = new ArrayList<>(); final List<ConnectionStatus> connectionStatuses = new ArrayList<>();
populateConnectionStatuses(groupStatus, connectionStatuses); populateConnectionStatuses(groupStatus, connectionStatuses);
connectionStatuses.sort(new Comparator<ConnectionStatus>() { connectionStatuses.sort((o1, o2) -> {
@Override if (o1 == null && o2 == null) {
public int compare(final ConnectionStatus o1, final ConnectionStatus o2) { return 0;
if (o1 == null && o2 == null) {
return 0;
}
if (o1 == null) {
return 1;
}
if (o2 == null) {
return -1;
}
return -Long.compare(o1.getQueuedBytes(), o2.getQueuedBytes());
} }
if (o1 == null) {
return 1;
}
if (o2 == null) {
return -1;
}
return -Long.compare(o1.getQueuedBytes(), o2.getQueuedBytes());
}); });
for (final ConnectionStatus connectionStatus : connectionStatuses) { for (final ConnectionStatus connectionStatus : connectionStatuses) {
@ -359,21 +354,18 @@ public class ControllerStatusReportingTask extends AbstractReportingTask {
private void printProcessorStatus(final ProcessGroupStatus groupStatus, final StringBuilder builder, final boolean showDeltas, final long divisor) { private void printProcessorStatus(final ProcessGroupStatus groupStatus, final StringBuilder builder, final boolean showDeltas, final long divisor) {
final List<ProcessorStatus> processorStatuses = new ArrayList<>(); final List<ProcessorStatus> processorStatuses = new ArrayList<>();
populateProcessorStatuses(groupStatus, processorStatuses); populateProcessorStatuses(groupStatus, processorStatuses);
Collections.sort(processorStatuses, new Comparator<ProcessorStatus>() { processorStatuses.sort((o1, o2) -> {
@Override if (o1 == null && o2 == null) {
public int compare(final ProcessorStatus o1, final ProcessorStatus o2) { return 0;
if (o1 == null && o2 == null) {
return 0;
}
if (o1 == null) {
return 1;
}
if (o2 == null) {
return -1;
}
return -Long.compare(o1.getProcessingNanos(), o2.getProcessingNanos());
} }
if (o1 == null) {
return 1;
}
if (o2 == null) {
return -1;
}
return -Long.compare(o1.getProcessingNanos(), o2.getProcessingNanos());
}); });
for (final ProcessorStatus processorStatus : processorStatuses) { for (final ProcessorStatus processorStatus : processorStatuses) {

View File

@ -22,7 +22,6 @@ import org.apache.commons.configuration2.builder.ConfigurationBuilderEvent;
import org.apache.commons.configuration2.builder.ReloadingFileBasedConfigurationBuilder; import org.apache.commons.configuration2.builder.ReloadingFileBasedConfigurationBuilder;
import org.apache.commons.configuration2.builder.fluent.FileBasedBuilderParameters; import org.apache.commons.configuration2.builder.fluent.FileBasedBuilderParameters;
import org.apache.commons.configuration2.builder.fluent.Parameters; import org.apache.commons.configuration2.builder.fluent.Parameters;
import org.apache.commons.configuration2.event.EventListener;
import org.apache.commons.configuration2.ex.ConfigurationException; import org.apache.commons.configuration2.ex.ConfigurationException;
import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.annotation.lifecycle.OnEnabled; import org.apache.nifi.annotation.lifecycle.OnEnabled;
@ -49,7 +48,7 @@ import java.util.stream.Collectors;
import java.util.stream.Stream; import java.util.stream.Stream;
/** /**
* This abstract class defines a generic {@link LookupService} backed by an * This abstract class defines a generic {@link org.apache.nifi.lookup.LookupService} backed by an
* Apache Commons Configuration {@link FileBasedConfiguration}. * Apache Commons Configuration {@link FileBasedConfiguration}.
* *
*/ */
@ -105,14 +104,11 @@ public abstract class CommonsConfigurationLookupService<T extends FileBasedConfi
final FileBasedBuilderParameters params = new Parameters().fileBased().setFile(new File(config)); final FileBasedBuilderParameters params = new Parameters().fileBased().setFile(new File(config));
this.builder = new ReloadingFileBasedConfigurationBuilder<>(resultClass).configure(params); this.builder = new ReloadingFileBasedConfigurationBuilder<>(resultClass).configure(params);
builder.addEventListener(ConfigurationBuilderEvent.CONFIGURATION_REQUEST, builder.addEventListener(ConfigurationBuilderEvent.CONFIGURATION_REQUEST,
new EventListener<ConfigurationBuilderEvent>() { event -> {
@Override
public void onEvent(ConfigurationBuilderEvent event) {
if (builder.getReloadingController().checkForReloading(null)) { if (builder.getReloadingController().checkForReloading(null)) {
getLogger().debug("Reloading {}", config); getLogger().debug("Reloading {}", config);
} }
} });
});
try { try {
// Try getting configuration to see if there is any issue, for example wrong file format. // Try getting configuration to see if there is any issue, for example wrong file format.

View File

@ -66,8 +66,6 @@ import jakarta.ws.rs.core.UriBuilder;
import jakarta.ws.rs.core.UriInfo; import jakarta.ws.rs.core.UriInfo;
import java.text.Collator; import java.text.Collator;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.List; import java.util.List;
import java.util.Locale; import java.util.Locale;
import java.util.Set; import java.util.Set;
@ -427,12 +425,9 @@ public class RuleResource {
} }
// sort the rules // sort the rules
Collections.sort(ruleDtos, new Comparator<RuleDTO>() { ruleDtos.sort((r1, r2) -> {
@Override final Collator collator = Collator.getInstance(Locale.US);
public int compare(RuleDTO r1, RuleDTO r2) { return collator.compare(r1.getName(), r2.getName());
final Collator collator = Collator.getInstance(Locale.US);
return collator.compare(r1.getName(), r2.getName());
}
}); });
// create the response entity // create the response entity

View File

@ -27,77 +27,42 @@ public enum SortColumn implements Comparator<FlowFileSummary> {
/** /**
* Sort based on the current position in the queue * Sort based on the current position in the queue
*/ */
QUEUE_POSITION(new Comparator<FlowFileSummary>() { QUEUE_POSITION(Comparator.comparingInt(FlowFileSummary::getPosition)),
@Override
public int compare(final FlowFileSummary o1, final FlowFileSummary o2) {
return Integer.compare(o1.getPosition(), o2.getPosition());
}
}),
/** /**
* Sort based on the UUID of the FlowFile * Sort based on the UUID of the FlowFile
*/ */
FLOWFILE_UUID(new Comparator<FlowFileSummary>() { FLOWFILE_UUID(Comparator.comparing(FlowFileSummary::getUuid)),
@Override
public int compare(final FlowFileSummary o1, final FlowFileSummary o2) {
return o1.getUuid().compareTo(o2.getUuid());
}
}),
/** /**
* Sort based on the 'filename' attribute of the FlowFile * Sort based on the 'filename' attribute of the FlowFile
*/ */
FILENAME(new Comparator<FlowFileSummary>() { FILENAME(Comparator.comparing(FlowFileSummary::getFilename)),
@Override
public int compare(final FlowFileSummary o1, final FlowFileSummary o2) {
return o1.getFilename().compareTo(o2.getFilename());
}
}),
/** /**
* Sort based on the size of the FlowFile * Sort based on the size of the FlowFile
*/ */
FLOWFILE_SIZE(new Comparator<FlowFileSummary>() { FLOWFILE_SIZE(Comparator.comparingLong(FlowFileSummary::getSize)),
@Override
public int compare(final FlowFileSummary o1, final FlowFileSummary o2) {
return Long.compare(o1.getSize(), o2.getSize());
}
}),
/** /**
* Sort based on how long the FlowFile has been sitting in the queue * Sort based on how long the FlowFile has been sitting in the queue
*/ */
QUEUED_DURATION(new Comparator<FlowFileSummary>() { QUEUED_DURATION((o1, o2) -> -Long.compare(o1.getLastQueuedTime(), o2.getLastQueuedTime())),
@Override
public int compare(final FlowFileSummary o1, final FlowFileSummary o2) {
return -Long.compare(o1.getLastQueuedTime(), o2.getLastQueuedTime());
}
}),
/** /**
* Sort based on the age of the FlowFile. I.e., the time at which the FlowFile's * Sort based on the age of the FlowFile. I.e., the time at which the FlowFile's
* "greatest ancestor" entered the flow * "greatest ancestor" entered the flow
*/ */
FLOWFILE_AGE(new Comparator<FlowFileSummary>() { FLOWFILE_AGE(Comparator.comparingLong(FlowFileSummary::getLineageStartDate)),
@Override
public int compare(final FlowFileSummary o1, final FlowFileSummary o2) {
return Long.compare(o1.getLineageStartDate(), o2.getLineageStartDate());
}
}),
/** /**
* Sort based on when the FlowFile's penalization ends * Sort based on when the FlowFile's penalization ends
*/ */
PENALIZATION(new Comparator<FlowFileSummary>() { PENALIZATION((o1, o2) -> Boolean.compare(o1.isPenalized(), o2.isPenalized()));
@Override
public int compare(final FlowFileSummary o1, final FlowFileSummary o2) {
return Boolean.compare(o1.isPenalized(), o2.isPenalized());
}
});
private final Comparator<FlowFileSummary> comparator; private final Comparator<FlowFileSummary> comparator;
private SortColumn(final Comparator<FlowFileSummary> comparator) { SortColumn(final Comparator<FlowFileSummary> comparator) {
this.comparator = comparator; this.comparator = comparator;
} }

View File

@ -24,13 +24,11 @@ import org.slf4j.LoggerFactory;
import java.io.EOFException; import java.io.EOFException;
import java.io.File; import java.io.File;
import java.io.FileFilter;
import java.io.FileNotFoundException; import java.io.FileNotFoundException;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections; import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.List; import java.util.List;
@ -66,12 +64,7 @@ public class IndexConfiguration {
for (final File storageDirectory : repoConfig.getStorageDirectories().values()) { for (final File storageDirectory : repoConfig.getStorageDirectories().values()) {
final List<File> indexDirectories = new ArrayList<>(); final List<File> indexDirectories = new ArrayList<>();
final File[] matching = storageDirectory.listFiles(new FileFilter() { final File[] matching = storageDirectory.listFiles(pathname -> pathname.isDirectory() && indexNamePattern.matcher(pathname.getName()).matches());
@Override
public boolean accept(final File pathname) {
return pathname.isDirectory() && indexNamePattern.matcher(pathname.getName()).matches();
}
});
if (matching != null) { if (matching != null) {
indexDirectories.addAll(Arrays.asList(matching)); indexDirectories.addAll(Arrays.asList(matching));
@ -220,13 +213,10 @@ public class IndexConfiguration {
try { try {
// Sort directories so that we return the newest index first // Sort directories so that we return the newest index first
final List<File> sortedIndexDirectories = getIndexDirectories(); final List<File> sortedIndexDirectories = getIndexDirectories();
Collections.sort(sortedIndexDirectories, new Comparator<File>() { sortedIndexDirectories.sort((o1, o2) -> {
@Override final long epochTimestamp1 = getIndexStartTime(o1);
public int compare(final File o1, final File o2) { final long epochTimestamp2 = getIndexStartTime(o2);
final long epochTimestamp1 = getIndexStartTime(o1); return Long.compare(epochTimestamp2, epochTimestamp1);
final long epochTimestamp2 = getIndexStartTime(o2);
return Long.compare(epochTimestamp2, epochTimestamp1);
}
}); });
for (final File indexDir : sortedIndexDirectories) { for (final File indexDir : sortedIndexDirectories) {
@ -273,13 +263,10 @@ public class IndexConfiguration {
} }
final List<File> sortedIndexDirectories = new ArrayList<>(indices); final List<File> sortedIndexDirectories = new ArrayList<>(indices);
Collections.sort(sortedIndexDirectories, new Comparator<File>() { sortedIndexDirectories.sort((o1, o2) -> {
@Override final long epochTimestamp1 = getIndexStartTime(o1);
public int compare(final File o1, final File o2) { final long epochTimestamp2 = getIndexStartTime(o2);
final long epochTimestamp1 = getIndexStartTime(o1); return Long.compare(epochTimestamp1, epochTimestamp2);
final long epochTimestamp2 = getIndexStartTime(o2);
return Long.compare(epochTimestamp1, epochTimestamp2);
}
}); });
final Long firstEntryTime = getFirstEntryTime(provenanceLogFile); final Long firstEntryTime = getFirstEntryTime(provenanceLogFile);

View File

@ -41,7 +41,6 @@ import java.nio.charset.CodingErrorAction;
import java.nio.file.Path; import java.nio.file.Path;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.Comparator;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -152,34 +151,31 @@ public class LuceneUtil {
* list of {@link Document}s * list of {@link Document}s
*/ */
public static void sortDocsForRetrieval(final List<Document> documents) { public static void sortDocsForRetrieval(final List<Document> documents) {
documents.sort(new Comparator<Document>() { documents.sort((o1, o2) -> {
@Override final String filename1 = o1.get(FieldNames.STORAGE_FILENAME);
public int compare(final Document o1, final Document o2) { final String filename2 = o2.get(FieldNames.STORAGE_FILENAME);
final String filename1 = o1.get(FieldNames.STORAGE_FILENAME);
final String filename2 = o2.get(FieldNames.STORAGE_FILENAME);
final int filenameComp = filename1.compareTo(filename2); final int filenameComp = filename1.compareTo(filename2);
if (filenameComp != 0) { if (filenameComp != 0) {
return filenameComp; return filenameComp;
}
final IndexableField fileOffset1 = o1.getField(FieldNames.BLOCK_INDEX);
final IndexableField fileOffset2 = o1.getField(FieldNames.BLOCK_INDEX);
if (fileOffset1 != null && fileOffset2 != null) {
final int blockIndexResult = Long.compare(fileOffset1.numericValue().longValue(), fileOffset2.numericValue().longValue());
if (blockIndexResult != 0) {
return blockIndexResult;
}
final long eventId1 = o1.getField(SearchableFields.Identifier.getSearchableFieldName()).numericValue().longValue();
final long eventId2 = o2.getField(SearchableFields.Identifier.getSearchableFieldName()).numericValue().longValue();
return Long.compare(eventId1, eventId2);
}
final long offset1 = o1.getField(FieldNames.STORAGE_FILE_OFFSET).numericValue().longValue();
final long offset2 = o2.getField(FieldNames.STORAGE_FILE_OFFSET).numericValue().longValue();
return Long.compare(offset1, offset2);
} }
final IndexableField fileOffset1 = o1.getField(FieldNames.BLOCK_INDEX);
final IndexableField fileOffset2 = o1.getField(FieldNames.BLOCK_INDEX);
if (fileOffset1 != null && fileOffset2 != null) {
final int blockIndexResult = Long.compare(fileOffset1.numericValue().longValue(), fileOffset2.numericValue().longValue());
if (blockIndexResult != 0) {
return blockIndexResult;
}
final long eventId1 = o1.getField(SearchableFields.Identifier.getSearchableFieldName()).numericValue().longValue();
final long eventId2 = o2.getField(SearchableFields.Identifier.getSearchableFieldName()).numericValue().longValue();
return Long.compare(eventId1, eventId2);
}
final long offset1 = o1.getField(FieldNames.STORAGE_FILE_OFFSET).numericValue().longValue();
final long offset2 = o2.getField(FieldNames.STORAGE_FILE_OFFSET).numericValue().longValue();
return Long.compare(offset1, offset2);
}); });
} }

View File

@ -32,7 +32,6 @@ import org.apache.nifi.provenance.toc.TocWriter;
import org.apache.nifi.provenance.util.DirectoryUtils; import org.apache.nifi.provenance.util.DirectoryUtils;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import org.mockito.Mockito; import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer; import org.mockito.stubbing.Answer;
import java.io.File; import java.io.File;
@ -45,7 +44,6 @@ import java.util.UUID;
import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLong;
import java.util.stream.Collectors;
import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.assertTrue;
@ -76,13 +74,10 @@ public class TestWriteAheadStorePartition {
final Map<ProvenanceEventRecord, StorageSummary> reindexedEvents = new ConcurrentHashMap<>(); final Map<ProvenanceEventRecord, StorageSummary> reindexedEvents = new ConcurrentHashMap<>();
final EventIndex eventIndex = Mockito.mock(EventIndex.class); final EventIndex eventIndex = Mockito.mock(EventIndex.class);
Mockito.doAnswer(new Answer<Object>() { Mockito.doAnswer((Answer<Object>) invocation -> {
@Override final Map<ProvenanceEventRecord, StorageSummary> events = invocation.getArgument(0);
public Object answer(final InvocationOnMock invocation) { reindexedEvents.putAll(events);
final Map<ProvenanceEventRecord, StorageSummary> events = invocation.getArgument(0); return null;
reindexedEvents.putAll(events);
return null;
}
}).when(eventIndex).reindexEvents(Mockito.anyMap()); }).when(eventIndex).reindexEvents(Mockito.anyMap());
Mockito.doReturn(18L).when(eventIndex).getMinimumEventIdToReindex("1"); Mockito.doReturn(18L).when(eventIndex).getMinimumEventIdToReindex("1");
@ -91,7 +86,7 @@ public class TestWriteAheadStorePartition {
final List<Long> eventIdsReindexed = reindexedEvents.values().stream() final List<Long> eventIdsReindexed = reindexedEvents.values().stream()
.map(StorageSummary::getEventId) .map(StorageSummary::getEventId)
.sorted() .sorted()
.collect(Collectors.toList()); .toList();
assertEquals(82, eventIdsReindexed.size()); assertEquals(82, eventIdsReindexed.size());
for (int i = 0; i < eventIdsReindexed.size(); i++) { for (int i = 0; i < eventIdsReindexed.size(); i++) {
@ -126,10 +121,10 @@ public class TestWriteAheadStorePartition {
partition.close(); partition.close();
final List<File> fileList = Arrays.asList(storageDirectory.listFiles(DirectoryUtils.EVENT_FILE_FILTER)); final List<File> fileList = Arrays.asList(storageDirectory.listFiles(DirectoryUtils.EVENT_FILE_FILTER));
Collections.sort(fileList, DirectoryUtils.LARGEST_ID_FIRST); fileList.sort(DirectoryUtils.LARGEST_ID_FIRST);
// Create new empty prov file with largest id // Create new empty prov file with largest id
assertTrue(new File(storageDirectory, "1" + fileList.get(0).getName()).createNewFile()); assertTrue(new File(storageDirectory, "1" + fileList.getFirst().getName()).createNewFile());
partition = new WriteAheadStorePartition(storageDirectory, partitionName, repoConfig, recordWriterFactory, partition = new WriteAheadStorePartition(storageDirectory, partitionName, repoConfig, recordWriterFactory,
recordReaderFactory, new LinkedBlockingQueue<>(), new AtomicLong(0L), EventReporter.NO_OP, Mockito.mock(EventFileManager.class)); recordReaderFactory, new LinkedBlockingQueue<>(), new AtomicLong(0L), EventReporter.NO_OP, Mockito.mock(EventFileManager.class));

View File

@ -38,7 +38,6 @@ import org.apache.nifi.provenance.search.SearchableField;
import org.apache.nifi.util.NiFiProperties; import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.util.RingBuffer; import org.apache.nifi.util.RingBuffer;
import org.apache.nifi.util.RingBuffer.Filter; import org.apache.nifi.util.RingBuffer.Filter;
import org.apache.nifi.util.RingBuffer.ForEachEvaluator;
import org.apache.nifi.util.RingBuffer.IterationDirection; import org.apache.nifi.util.RingBuffer.IterationDirection;
import org.apache.nifi.web.ResourceNotFoundException; import org.apache.nifi.web.ResourceNotFoundException;
@ -171,16 +170,13 @@ public class VolatileProvenanceRepository implements ProvenanceRepository {
} }
@Override @Override
public List<ProvenanceEventRecord> getEvents(final long firstRecordId, final int maxRecords, final NiFiUser user) throws IOException { public List<ProvenanceEventRecord> getEvents(final long firstRecordId, final int maxRecords, final NiFiUser user) {
return ringBuffer.getSelectedElements(new Filter<ProvenanceEventRecord>() { return ringBuffer.getSelectedElements(value -> {
@Override if (!isAuthorized(value, user)) {
public boolean select(final ProvenanceEventRecord value) { return false;
if (!isAuthorized(value, user)) {
return false;
}
return value.getEventId() >= firstRecordId;
} }
return value.getEventId() >= firstRecordId;
}, maxRecords); }, maxRecords);
} }
@ -190,26 +186,16 @@ public class VolatileProvenanceRepository implements ProvenanceRepository {
return (newest == null) ? null : newest.getEventId(); return (newest == null) ? null : newest.getEventId();
} }
public ProvenanceEventRecord getEvent(final String identifier) throws IOException { public ProvenanceEventRecord getEvent(final String identifier) {
final List<ProvenanceEventRecord> records = ringBuffer.getSelectedElements(new Filter<ProvenanceEventRecord>() { final List<ProvenanceEventRecord> records = ringBuffer.getSelectedElements(event -> identifier.equals(event.getFlowFileUuid()), 1);
@Override return records.isEmpty() ? null : records.getFirst();
public boolean select(final ProvenanceEventRecord event) {
return identifier.equals(event.getFlowFileUuid());
}
}, 1);
return records.isEmpty() ? null : records.get(0);
} }
@Override @Override
public ProvenanceEventRecord getEvent(final long id) { public ProvenanceEventRecord getEvent(final long id) {
final List<ProvenanceEventRecord> records = ringBuffer.getSelectedElements(new Filter<ProvenanceEventRecord>() { final List<ProvenanceEventRecord> records = ringBuffer.getSelectedElements(event -> event.getEventId() == id, 1);
@Override
public boolean select(final ProvenanceEventRecord event) {
return event.getEventId() == id;
}
}, 1);
return records.isEmpty() ? null : records.get(0); return records.isEmpty() ? null : records.getFirst();
} }
@Override @Override
@ -282,127 +268,45 @@ public class VolatileProvenanceRepository implements ProvenanceRepository {
} }
private Filter<ProvenanceEventRecord> createFilter(final Query query, final NiFiUser user) { private Filter<ProvenanceEventRecord> createFilter(final Query query, final NiFiUser user) {
return new Filter<ProvenanceEventRecord>() { return event -> {
@Override if (!isAuthorized(event, user)) {
public boolean select(final ProvenanceEventRecord event) { return false;
if (!isAuthorized(event, user)) { }
if (query.getStartDate() != null && query.getStartDate().getTime() > event.getEventTime()) {
return false;
}
if (query.getEndDate() != null && query.getEndDate().getTime() < event.getEventTime()) {
return false;
}
if (query.getMaxFileSize() != null) {
final long maxFileSize = DataUnit.parseDataSize(query.getMaxFileSize(), DataUnit.B).longValue();
if (event.getFileSize() > maxFileSize) {
return false; return false;
} }
}
if (query.getStartDate() != null && query.getStartDate().getTime() > event.getEventTime()) { if (query.getMinFileSize() != null) {
final long minFileSize = DataUnit.parseDataSize(query.getMinFileSize(), DataUnit.B).longValue();
if (event.getFileSize() < minFileSize) {
return false; return false;
} }
}
if (query.getEndDate() != null && query.getEndDate().getTime() < event.getEventTime()) { for (final SearchTerm searchTerm : query.getSearchTerms()) {
return false; final SearchableField searchableField = searchTerm.getSearchableField();
} final String searchValue = searchTerm.getValue();
final boolean excludeSearchValue = searchTerm.isInverted().booleanValue();
if (query.getMaxFileSize() != null) { if (searchableField.isAttribute()) {
final long maxFileSize = DataUnit.parseDataSize(query.getMaxFileSize(), DataUnit.B).longValue(); final String attributeName = searchableField.getIdentifier();
if (event.getFileSize() > maxFileSize) {
return false;
}
}
if (query.getMinFileSize() != null) { final String eventAttributeValue = event.getAttributes().get(attributeName);
final long minFileSize = DataUnit.parseDataSize(query.getMinFileSize(), DataUnit.B).longValue();
if (event.getFileSize() < minFileSize) {
return false;
}
}
for (final SearchTerm searchTerm : query.getSearchTerms()) { if (searchValue.contains("?") || searchValue.contains("*")) {
final SearchableField searchableField = searchTerm.getSearchableField(); if (eventAttributeValue == null || eventAttributeValue.isEmpty()) {
final String searchValue = searchTerm.getValue();
final boolean excludeSearchValue = searchTerm.isInverted().booleanValue();
if (searchableField.isAttribute()) {
final String attributeName = searchableField.getIdentifier();
final String eventAttributeValue = event.getAttributes().get(attributeName);
if (searchValue.contains("?") || searchValue.contains("*")) {
if (eventAttributeValue == null || eventAttributeValue.isEmpty()) {
if (!excludeSearchValue) {
return false;
} else {
continue;
}
}
final String regex = searchValue.replace("?", ".").replace("*", ".*");
final Pattern pattern = Pattern.compile(regex, Pattern.CASE_INSENSITIVE);
final boolean patternMatches = pattern.matcher(eventAttributeValue).matches();
if ((!patternMatches && !excludeSearchValue)
|| (patternMatches && excludeSearchValue)) {
return false;
}
} else if (!searchValue.equalsIgnoreCase(eventAttributeValue) && !excludeSearchValue
|| searchValue.equalsIgnoreCase(eventAttributeValue) && excludeSearchValue) {
return false;
}
} else {
// if FlowFileUUID, search parent & child UUID's also.
if (searchableField.equals(SearchableFields.FlowFileUUID)) {
if (searchValue.contains("?") || searchValue.contains("*")) {
final String regex = searchValue.replace("?", ".").replace("*", ".*");
final Pattern pattern = Pattern.compile(regex, Pattern.CASE_INSENSITIVE);
final boolean patternMatches = pattern.matcher(event.getFlowFileUuid()).matches();
if (!excludeSearchValue) {
if (patternMatches) {
continue;
}
boolean found = false;
for (final String uuid : event.getParentUuids()) {
if (pattern.matcher(uuid).matches()) {
found = true;
break;
}
}
for (final String uuid : event.getChildUuids()) {
if (pattern.matcher(uuid).matches()) {
found = true;
break;
}
}
if (found) {
continue;
}
} else {
if (patternMatches) {
return false;
}
for (final String uuid : event.getParentUuids()) {
if (pattern.matcher(uuid).matches()) {
return false;
}
}
for (final String uuid : event.getChildUuids()) {
if (pattern.matcher(uuid).matches()) {
return false;
}
}
continue;
}
} else if (!excludeSearchValue
&& (event.getFlowFileUuid().equals(searchValue) || event.getParentUuids().contains(searchValue) || event.getChildUuids().contains(searchValue))) {
continue;
} else if (excludeSearchValue
&& (!event.getFlowFileUuid().equals(searchValue) && !event.getParentUuids().contains(searchValue) && !event.getChildUuids().contains(searchValue))) {
continue;
}
return false;
}
final Object fieldValue = getFieldValue(event, searchableField);
if (fieldValue == null) {
if (!excludeSearchValue) { if (!excludeSearchValue) {
return false; return false;
} else { } else {
@ -410,24 +314,103 @@ public class VolatileProvenanceRepository implements ProvenanceRepository {
} }
} }
final String regex = searchValue.replace("?", ".").replace("*", ".*");
final Pattern pattern = Pattern.compile(regex, Pattern.CASE_INSENSITIVE);
final boolean patternMatches = pattern.matcher(eventAttributeValue).matches();
if ((!patternMatches && !excludeSearchValue)
|| (patternMatches && excludeSearchValue)) {
return false;
}
} else if (!searchValue.equalsIgnoreCase(eventAttributeValue) && !excludeSearchValue
|| searchValue.equalsIgnoreCase(eventAttributeValue) && excludeSearchValue) {
return false;
}
} else {
// if FlowFileUUID, search parent & child UUID's also.
if (searchableField.equals(SearchableFields.FlowFileUUID)) {
if (searchValue.contains("?") || searchValue.contains("*")) { if (searchValue.contains("?") || searchValue.contains("*")) {
final String regex = searchValue.replace("?", ".").replace("*", ".*"); final String regex = searchValue.replace("?", ".").replace("*", ".*");
final Pattern pattern = Pattern.compile(regex, Pattern.CASE_INSENSITIVE); final Pattern pattern = Pattern.compile(regex, Pattern.CASE_INSENSITIVE);
final boolean patternMatches = pattern.matcher(String.valueOf(fieldValue)).matches(); final boolean patternMatches = pattern.matcher(event.getFlowFileUuid()).matches();
if (!patternMatches && !excludeSearchValue if (!excludeSearchValue) {
|| patternMatches && excludeSearchValue) { if (patternMatches) {
return false; continue;
}
boolean found = false;
for (final String uuid : event.getParentUuids()) {
if (pattern.matcher(uuid).matches()) {
found = true;
break;
}
}
for (final String uuid : event.getChildUuids()) {
if (pattern.matcher(uuid).matches()) {
found = true;
break;
}
}
if (found) {
continue;
}
} else {
if (patternMatches) {
return false;
}
for (final String uuid : event.getParentUuids()) {
if (pattern.matcher(uuid).matches()) {
return false;
}
}
for (final String uuid : event.getChildUuids()) {
if (pattern.matcher(uuid).matches()) {
return false;
}
}
continue;
} }
} else if (!searchValue.equalsIgnoreCase(String.valueOf(fieldValue)) && !excludeSearchValue } else if (!excludeSearchValue
|| searchValue.equalsIgnoreCase(String.valueOf(fieldValue)) && excludeSearchValue) { && (event.getFlowFileUuid().equals(searchValue) || event.getParentUuids().contains(searchValue) || event.getChildUuids().contains(searchValue))) {
continue;
} else if (excludeSearchValue
&& (!event.getFlowFileUuid().equals(searchValue) && !event.getParentUuids().contains(searchValue) && !event.getChildUuids().contains(searchValue))) {
continue;
}
return false;
}
final Object fieldValue = getFieldValue(event, searchableField);
if (fieldValue == null) {
if (!excludeSearchValue) {
return false; return false;
} else {
continue;
} }
} }
}
return true; if (searchValue.contains("?") || searchValue.contains("*")) {
final String regex = searchValue.replace("?", ".").replace("*", ".*");
final Pattern pattern = Pattern.compile(regex, Pattern.CASE_INSENSITIVE);
final boolean patternMatches = pattern.matcher(String.valueOf(fieldValue)).matches();
if (!patternMatches && !excludeSearchValue
|| patternMatches && excludeSearchValue) {
return false;
}
} else if (!searchValue.equalsIgnoreCase(String.valueOf(fieldValue)) && !excludeSearchValue
|| searchValue.equalsIgnoreCase(String.valueOf(fieldValue)) && excludeSearchValue) {
return false;
}
}
} }
return true;
}; };
} }
@ -670,31 +653,28 @@ public class VolatileProvenanceRepository implements ProvenanceRepository {
final AsyncLineageSubmission result = new AsyncLineageSubmission(computationType, eventId, flowFileUuids, 1, userId); final AsyncLineageSubmission result = new AsyncLineageSubmission(computationType, eventId, flowFileUuids, 1, userId);
lineageSubmissionMap.put(result.getLineageIdentifier(), result); lineageSubmissionMap.put(result.getLineageIdentifier(), result);
final Filter<ProvenanceEventRecord> filter = new Filter<ProvenanceEventRecord>() { final Filter<ProvenanceEventRecord> filter = event -> {
@Override if (!isAuthorized(event, user)) {
public boolean select(final ProvenanceEventRecord event) {
if (!isAuthorized(event, user)) {
return false;
}
if (flowFileUuids.contains(event.getFlowFileUuid())) {
return true;
}
for (final String parentId : event.getParentUuids()) {
if (flowFileUuids.contains(parentId)) {
return true;
}
}
for (final String childId : event.getChildUuids()) {
if (flowFileUuids.contains(childId)) {
return true;
}
}
return false; return false;
} }
if (flowFileUuids.contains(event.getFlowFileUuid())) {
return true;
}
for (final String parentId : event.getParentUuids()) {
if (flowFileUuids.contains(parentId)) {
return true;
}
}
for (final String childId : event.getChildUuids()) {
if (flowFileUuids.contains(childId)) {
return true;
}
}
return false;
}; };
queryExecService.submit(new ComputeLineageRunnable(ringBuffer, filter, result)); queryExecService.submit(new ComputeLineageRunnable(ringBuffer, filter, result));
@ -721,18 +701,14 @@ public class VolatileProvenanceRepository implements ProvenanceRepository {
// Retrieve the most recent results and count the total number of matches // Retrieve the most recent results and count the total number of matches
final AtomicInteger matchingCount = new AtomicInteger(0); final AtomicInteger matchingCount = new AtomicInteger(0);
final List<ProvenanceEventRecord> matchingRecords = new ArrayList<>(maxRecords); final List<ProvenanceEventRecord> matchingRecords = new ArrayList<>(maxRecords);
ringBuffer.forEach(new ForEachEvaluator<ProvenanceEventRecord>() { ringBuffer.forEach(record -> {
@Override if (filter.select(record)) {
public boolean evaluate(final ProvenanceEventRecord record) { if (matchingCount.incrementAndGet() <= maxRecords) {
if (filter.select(record)) { matchingRecords.add(record);
if (matchingCount.incrementAndGet() <= maxRecords) {
matchingRecords.add(record);
}
} }
return true;
} }
return true;
}, IterationDirection.BACKWARD); }, IterationDirection.BACKWARD);
submission.getResult().update(matchingRecords, matchingCount.get()); submission.getResult().update(matchingRecords, matchingCount.get());

View File

@ -184,12 +184,7 @@ public class FlowSnippetDTO {
} }
private <T extends ComponentDTO> Set<T> orderedById(Set<T> dtos) { private <T extends ComponentDTO> Set<T> orderedById(Set<T> dtos) {
TreeSet<T> components = new TreeSet<>(new Comparator<ComponentDTO>() { TreeSet<T> components = new TreeSet<>(Comparator.comparing(ComponentDTO::getId));
@Override
public int compare(ComponentDTO c1, ComponentDTO c2) {
return c1.getId().compareTo(c2.getId());
}
});
components.addAll(dtos); components.addAll(dtos);
return components; return components;
} }
@ -298,12 +293,7 @@ public class FlowSnippetDTO {
} }
private <T extends RemoteProcessGroupPortDTO> Set<T> orderedRemotePortsById(Set<T> dtos) { private <T extends RemoteProcessGroupPortDTO> Set<T> orderedRemotePortsById(Set<T> dtos) {
TreeSet<T> components = new TreeSet<>(new Comparator<RemoteProcessGroupPortDTO>() { TreeSet<T> components = new TreeSet<>(Comparator.comparing((RemoteProcessGroupPortDTO c) -> UUID.fromString(c.getId())));
@Override
public int compare(RemoteProcessGroupPortDTO c1, RemoteProcessGroupPortDTO c2) {
return UUID.fromString(c1.getId()).compareTo(UUID.fromString(c2.getId()));
}
});
components.addAll(dtos); components.addAll(dtos);
return components; return components;
} }

View File

@ -27,7 +27,6 @@ import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import org.mockito.Mockito; import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer; import org.mockito.stubbing.Answer;
import java.io.File; import java.io.File;
@ -144,8 +143,6 @@ public class FileAccessPolicyProviderTest {
private File restoreAuthorizations; private File restoreAuthorizations;
private File restoreTenants; private File restoreTenants;
private File flow; private File flow;
private File flowNoPorts;
private File flowWithDns;
private AuthorizerConfigurationContext configurationContext; private AuthorizerConfigurationContext configurationContext;
@ -170,10 +167,10 @@ public class FileAccessPolicyProviderTest {
flow = new File("src/test/resources/flow.json.gz"); flow = new File("src/test/resources/flow.json.gz");
FileUtils.ensureDirectoryExistAndCanAccess(flow.getParentFile()); FileUtils.ensureDirectoryExistAndCanAccess(flow.getParentFile());
flowNoPorts = new File("src/test/resources/flow-no-ports.json.gz"); File flowNoPorts = new File("src/test/resources/flow-no-ports.json.gz");
FileUtils.ensureDirectoryExistAndCanAccess(flowNoPorts.getParentFile()); FileUtils.ensureDirectoryExistAndCanAccess(flowNoPorts.getParentFile());
flowWithDns = new File("src/test/resources/flow-with-dns.json.gz"); File flowWithDns = new File("src/test/resources/flow-with-dns.json.gz");
FileUtils.ensureDirectoryExistAndCanAccess(flowWithDns.getParentFile()); FileUtils.ensureDirectoryExistAndCanAccess(flowWithDns.getParentFile());
properties = mock(NiFiProperties.class); properties = mock(NiFiProperties.class);
@ -248,12 +245,7 @@ public class FileAccessPolicyProviderTest {
}); });
final AccessPolicyProviderInitializationContext initializationContext = mock(AccessPolicyProviderInitializationContext.class); final AccessPolicyProviderInitializationContext initializationContext = mock(AccessPolicyProviderInitializationContext.class);
when(initializationContext.getUserGroupProviderLookup()).thenReturn(new UserGroupProviderLookup() { when(initializationContext.getUserGroupProviderLookup()).thenReturn(identifier -> userGroupProvider);
@Override
public UserGroupProvider getUserGroupProvider(String identifier) {
return userGroupProvider;
}
});
accessPolicyProvider = new FileAccessPolicyProvider(); accessPolicyProvider = new FileAccessPolicyProvider();
accessPolicyProvider.setNiFiProperties(properties); accessPolicyProvider.setNiFiProperties(properties);
@ -261,7 +253,7 @@ public class FileAccessPolicyProviderTest {
} }
@AfterEach @AfterEach
public void cleanup() throws Exception { public void cleanup() {
deleteFile(primaryAuthorizations); deleteFile(primaryAuthorizations);
deleteFile(primaryTenants); deleteFile(primaryTenants);
deleteFile(restoreAuthorizations); deleteFile(restoreAuthorizations);
@ -647,7 +639,7 @@ public class FileAccessPolicyProviderTest {
} else if (policy.getIdentifier().equals("policy-2") } else if (policy.getIdentifier().equals("policy-2")
&& policy.getResource().equals("/flow") && policy.getResource().equals("/flow")
&& policy.getAction() == RequestAction.WRITE && policy.getAction() == RequestAction.WRITE
&& policy.getGroups().size() == 0 && policy.getGroups().isEmpty()
&& policy.getUsers().size() == 1 && policy.getUsers().size() == 1
&& policy.getUsers().contains("user-2")) { && policy.getUsers().contains("user-2")) {
foundPolicy2 = true; foundPolicy2 = true;
@ -865,23 +857,18 @@ public class FileAccessPolicyProviderTest {
} }
} }
private static boolean deleteFile(final File file) { private static void deleteFile(final File file) {
if (file.isDirectory()) { if (file.isDirectory()) {
FileUtils.deleteFilesInDir(file, null, null, true, true); FileUtils.deleteFilesInDir(file, null, null, true, true);
} }
return FileUtils.deleteFile(file, null, 10); FileUtils.deleteFile(file, null, 10);
} }
private NiFiProperties getNiFiProperties(final Properties properties) { private NiFiProperties getNiFiProperties(final Properties properties) {
final NiFiProperties nifiProperties = Mockito.mock(NiFiProperties.class); final NiFiProperties nifiProperties = Mockito.mock(NiFiProperties.class);
when(nifiProperties.getPropertyKeys()).thenReturn(properties.stringPropertyNames()); when(nifiProperties.getPropertyKeys()).thenReturn(properties.stringPropertyNames());
when(nifiProperties.getProperty(anyString())).then(new Answer<String>() { when(nifiProperties.getProperty(anyString())).then((Answer<String>) invocationOnMock -> properties.getProperty((String) invocationOnMock.getArguments()[0]));
@Override
public String answer(InvocationOnMock invocationOnMock) throws Throwable {
return properties.getProperty((String) invocationOnMock.getArguments()[0]);
}
});
return nifiProperties; return nifiProperties;
} }

View File

@ -25,9 +25,6 @@ import org.apache.nifi.util.file.FileUtils;
import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import java.io.File; import java.io.File;
import java.io.FileOutputStream; import java.io.FileOutputStream;
@ -35,7 +32,6 @@ import java.io.IOException;
import java.nio.charset.StandardCharsets; import java.nio.charset.StandardCharsets;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
import java.util.Properties;
import java.util.Set; import java.util.Set;
import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertEquals;
@ -43,7 +39,6 @@ import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.ArgumentMatchers.eq; import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.Mockito.mock; import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when; import static org.mockito.Mockito.when;
@ -90,7 +85,6 @@ public class FileUserGroupProviderTest {
" </users>" + " </users>" +
"</tenants>"; "</tenants>";
private NiFiProperties properties;
private FileUserGroupProvider userGroupProvider; private FileUserGroupProvider userGroupProvider;
private File primaryTenants; private File primaryTenants;
private File restoreTenants; private File restoreTenants;
@ -107,8 +101,8 @@ public class FileUserGroupProviderTest {
restoreTenants = new File("target/restore/users.xml"); restoreTenants = new File("target/restore/users.xml");
FileUtils.ensureDirectoryExistAndCanAccess(restoreTenants.getParentFile()); FileUtils.ensureDirectoryExistAndCanAccess(restoreTenants.getParentFile());
properties = mock(NiFiProperties.class); NiFiProperties properties1 = mock(NiFiProperties.class);
when(properties.getRestoreDirectory()).thenReturn(restoreTenants.getParentFile()); when(properties1.getRestoreDirectory()).thenReturn(restoreTenants.getParentFile());
configurationContext = mock(AuthorizerConfigurationContext.class); configurationContext = mock(AuthorizerConfigurationContext.class);
when(configurationContext.getProperty(eq(FileUserGroupProvider.PROP_TENANTS_FILE))).thenReturn(new StandardPropertyValue(primaryTenants.getPath(), null, ParameterLookup.EMPTY)); when(configurationContext.getProperty(eq(FileUserGroupProvider.PROP_TENANTS_FILE))).thenReturn(new StandardPropertyValue(primaryTenants.getPath(), null, ParameterLookup.EMPTY));
@ -135,12 +129,12 @@ public class FileUserGroupProviderTest {
}); });
userGroupProvider = new FileUserGroupProvider(); userGroupProvider = new FileUserGroupProvider();
userGroupProvider.setNiFiProperties(properties); userGroupProvider.setNiFiProperties(properties1);
userGroupProvider.initialize(null); userGroupProvider.initialize(null);
} }
@AfterEach @AfterEach
public void cleanup() throws Exception { public void cleanup() {
deleteFile(primaryTenants); deleteFile(primaryTenants);
deleteFile(restoreTenants); deleteFile(restoreTenants);
} }
@ -614,24 +608,11 @@ public class FileUserGroupProviderTest {
} }
} }
private static boolean deleteFile(final File file) { private static void deleteFile(final File file) {
if (file.isDirectory()) { if (file.isDirectory()) {
FileUtils.deleteFilesInDir(file, null, null, true, true); FileUtils.deleteFilesInDir(file, null, null, true, true);
} }
return FileUtils.deleteFile(file, null, 10); FileUtils.deleteFile(file, null, 10);
}
private NiFiProperties getNiFiProperties(final Properties properties) {
final NiFiProperties nifiProperties = Mockito.mock(NiFiProperties.class);
when(nifiProperties.getPropertyKeys()).thenReturn(properties.stringPropertyNames());
when(nifiProperties.getProperty(anyString())).then(new Answer<String>() {
@Override
public String answer(InvocationOnMock invocationOnMock) throws Throwable {
return properties.getProperty((String) invocationOnMock.getArguments()[0]);
}
});
return nifiProperties;
} }
} }

View File

@ -35,8 +35,6 @@ import java.io.DataInputStream;
import java.io.DataOutputStream; import java.io.DataOutputStream;
import java.io.EOFException; import java.io.EOFException;
import java.io.IOException; import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
/** /**
@ -65,89 +63,76 @@ public class JaxbProtocolContext<T> implements ProtocolContext<T> {
@Override @Override
public ProtocolMessageMarshaller<T> createMarshaller() { public ProtocolMessageMarshaller<T> createMarshaller() {
return new ProtocolMessageMarshaller<T>() { return (msg, os) -> {
@Override try {
public void marshal(final T msg, final OutputStream os) throws IOException {
try { // marshal message to output stream
final Marshaller marshaller = jaxbCtx.createMarshaller();
final ByteArrayOutputStream msgBytes = new ByteArrayOutputStream();
marshaller.marshal(msg, msgBytes);
// marshal message to output stream final DataOutputStream dos = new DataOutputStream(os);
final Marshaller marshaller = jaxbCtx.createMarshaller();
final ByteArrayOutputStream msgBytes = new ByteArrayOutputStream();
marshaller.marshal(msg, msgBytes);
final DataOutputStream dos = new DataOutputStream(os); // write message protocol sentinel
dos.write(MESSAGE_PROTOCOL_START_SENTINEL);
// write message protocol sentinel // write message size in bytes
dos.write(MESSAGE_PROTOCOL_START_SENTINEL); dos.writeInt(msgBytes.size());
// write message size in bytes // write message
dos.writeInt(msgBytes.size()); msgBytes.writeTo(dos);
// write message dos.flush();
msgBytes.writeTo(dos);
dos.flush();
} catch (final JAXBException je) {
throw new IOException("Failed marshalling protocol message due to: " + je, je);
}
} catch (final JAXBException je) {
throw new IOException("Failed marshalling protocol message due to: " + je, je);
} }
}; };
} }
@Override @Override
public ProtocolMessageUnmarshaller<T> createUnmarshaller() { public ProtocolMessageUnmarshaller<T> createUnmarshaller() {
return new ProtocolMessageUnmarshaller<T>() { return is -> {
@Override try {
public T unmarshal(final InputStream is) throws IOException {
try { final DataInputStream dis = new DataInputStream(is);
final DataInputStream dis = new DataInputStream(is); // check for the presence of the message protocol sentinel
final byte sentinel = (byte) dis.read();
// check for the presence of the message protocol sentinel if (sentinel == -1) {
final byte sentinel = (byte) dis.read(); throw new EOFException();
if (sentinel == -1) {
throw new EOFException();
}
if (MESSAGE_PROTOCOL_START_SENTINEL != sentinel) {
throw new IOException("Failed reading protocol message due to malformed header");
}
// read the message size
final int msgBytesSize = dis.readInt();
// read the message
final ByteBuffer buffer = ByteBuffer.allocate(msgBytesSize);
int totalBytesRead = 0;
do {
final int bytesToRead;
if ((msgBytesSize - totalBytesRead) >= BUF_SIZE) {
bytesToRead = BUF_SIZE;
} else {
bytesToRead = msgBytesSize - totalBytesRead;
}
totalBytesRead += dis.read(buffer.array(), totalBytesRead, bytesToRead);
} while (totalBytesRead < msgBytesSize);
// unmarshall message and return
final Unmarshaller unmarshaller = jaxbCtx.createUnmarshaller();
final byte[] msg = new byte[totalBytesRead];
buffer.get(msg);
final XMLStreamReaderProvider provider = new StandardXMLStreamReaderProvider();
final XMLStreamReader xsr = provider.getStreamReader(new StreamSource(new ByteArrayInputStream(msg)));
return (T) unmarshaller.unmarshal(xsr);
} catch (final JAXBException | ProcessingException e) {
throw new IOException("Failed unmarshalling protocol message due to: " + e, e);
} }
if (MESSAGE_PROTOCOL_START_SENTINEL != sentinel) {
throw new IOException("Failed reading protocol message due to malformed header");
}
// read the message size
final int msgBytesSize = dis.readInt();
// read the message
final ByteBuffer buffer = ByteBuffer.allocate(msgBytesSize);
int totalBytesRead = 0;
do {
final int bytesToRead = Math.min((msgBytesSize - totalBytesRead), BUF_SIZE);
totalBytesRead += dis.read(buffer.array(), totalBytesRead, bytesToRead);
} while (totalBytesRead < msgBytesSize);
// unmarshall message and return
final Unmarshaller unmarshaller = jaxbCtx.createUnmarshaller();
final byte[] msg = new byte[totalBytesRead];
buffer.get(msg);
final XMLStreamReaderProvider provider = new StandardXMLStreamReaderProvider();
final XMLStreamReader xsr = provider.getStreamReader(new StreamSource(new ByteArrayInputStream(msg)));
return (T) unmarshaller.unmarshal(xsr);
} catch (final JAXBException | ProcessingException e) {
throw new IOException("Failed unmarshalling protocol message due to: " + e, e);
} }
}; };
} }
} }

View File

@ -62,27 +62,24 @@ public class ListFlowFilesEndpointMerger extends AbstractSingleDTOEndpoint<Listi
@Override @Override
protected void mergeResponses(ListingRequestDTO clientDto, Map<NodeIdentifier, ListingRequestDTO> dtoMap, Set<NodeResponse> successfulResponses, Set<NodeResponse> problematicResponses) { protected void mergeResponses(ListingRequestDTO clientDto, Map<NodeIdentifier, ListingRequestDTO> dtoMap, Set<NodeResponse> successfulResponses, Set<NodeResponse> problematicResponses) {
final Comparator<FlowFileSummaryDTO> comparator = new Comparator<FlowFileSummaryDTO>() { final Comparator<FlowFileSummaryDTO> comparator = (dto1, dto2) -> {
@Override int positionCompare = dto1.getPosition().compareTo(dto2.getPosition());
public int compare(final FlowFileSummaryDTO dto1, final FlowFileSummaryDTO dto2) { if (positionCompare != 0) {
int positionCompare = dto1.getPosition().compareTo(dto2.getPosition()); return positionCompare;
if (positionCompare != 0) {
return positionCompare;
}
final String address1 = dto1.getClusterNodeAddress();
final String address2 = dto2.getClusterNodeAddress();
if (address1 == null && address2 == null) {
return 0;
}
if (address1 == null) {
return 1;
}
if (address2 == null) {
return -1;
}
return address1.compareTo(address2);
} }
final String address1 = dto1.getClusterNodeAddress();
final String address2 = dto2.getClusterNodeAddress();
if (address1 == null && address2 == null) {
return 0;
}
if (address1 == null) {
return 1;
}
if (address2 == null) {
return -1;
}
return address1.compareTo(address2);
}; };
final NavigableSet<FlowFileSummaryDTO> flowFileSummaries = new TreeSet<>(comparator); final NavigableSet<FlowFileSummaryDTO> flowFileSummaries = new TreeSet<>(comparator);

View File

@ -29,8 +29,6 @@ import org.apache.nifi.web.api.entity.ProvenanceEntity;
import java.net.URI; import java.net.URI;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.Date; import java.util.Date;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
@ -141,33 +139,30 @@ public class ProvenanceQueryEndpointMerger implements EndpointResponseMerger {
// grab only the first X number of them. We do a sort based on time, such that the newest are included. // grab only the first X number of them. We do a sort based on time, such that the newest are included.
// If 2 events have the same timestamp, we do a secondary sort based on Cluster Node Identifier. If those are // If 2 events have the same timestamp, we do a secondary sort based on Cluster Node Identifier. If those are
// equal, we perform a tertiary sort based on the the event id // equal, we perform a tertiary sort based on the the event id
Collections.sort(allResults, new Comparator<ProvenanceEventDTO>() { allResults.sort((o1, o2) -> {
@Override final int eventTimeComparison = o1.getEventTime().compareTo(o2.getEventTime());
public int compare(final ProvenanceEventDTO o1, final ProvenanceEventDTO o2) { if (eventTimeComparison != 0) {
final int eventTimeComparison = o1.getEventTime().compareTo(o2.getEventTime()); return -eventTimeComparison;
if (eventTimeComparison != 0) {
return -eventTimeComparison;
}
final String nodeId1 = o1.getClusterNodeId();
final String nodeId2 = o2.getClusterNodeId();
final int nodeIdComparison;
if (nodeId1 == null && nodeId2 == null) {
nodeIdComparison = 0;
} else if (nodeId1 == null) {
nodeIdComparison = 1;
} else if (nodeId2 == null) {
nodeIdComparison = -1;
} else {
nodeIdComparison = -nodeId1.compareTo(nodeId2);
}
if (nodeIdComparison != 0) {
return nodeIdComparison;
}
return -Long.compare(o1.getEventId(), o2.getEventId());
} }
final String nodeId1 = o1.getClusterNodeId();
final String nodeId2 = o2.getClusterNodeId();
final int nodeIdComparison;
if (nodeId1 == null && nodeId2 == null) {
nodeIdComparison = 0;
} else if (nodeId1 == null) {
nodeIdComparison = 1;
} else if (nodeId2 == null) {
nodeIdComparison = -1;
} else {
nodeIdComparison = -nodeId1.compareTo(nodeId2);
}
if (nodeIdComparison != 0) {
return nodeIdComparison;
}
return -Long.compare(o1.getEventId(), o2.getEventId());
}); });
final int maxResults = request.getMaxResults().intValue(); final int maxResults = request.getMaxResults().intValue();

View File

@ -20,7 +20,6 @@ import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.web.api.entity.BulletinEntity; import org.apache.nifi.web.api.entity.BulletinEntity;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator; import java.util.Comparator;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -32,21 +31,18 @@ public final class BulletinMerger {
private BulletinMerger() { } private BulletinMerger() { }
public static Comparator<BulletinEntity> BULLETIN_COMPARATOR = new Comparator<BulletinEntity>() { public static Comparator<BulletinEntity> BULLETIN_COMPARATOR = (o1, o2) -> {
@Override if (o1 == null && o2 == null) {
public int compare(BulletinEntity o1, BulletinEntity o2) { return 0;
if (o1 == null && o2 == null) {
return 0;
}
if (o1 == null) {
return 1;
}
if (o2 == null) {
return -1;
}
return -Long.compare(o1.getId(), o2.getId());
} }
if (o1 == null) {
return 1;
}
if (o2 == null) {
return -1;
}
return -Long.compare(o1.getId(), o2.getId());
}; };
/** /**
@ -112,14 +108,7 @@ public final class BulletinMerger {
.forEach(entities::add); .forEach(entities::add);
// ensure the bulletins are sorted by time // ensure the bulletins are sorted by time
Collections.sort(entities, (BulletinEntity o1, BulletinEntity o2) -> { entities.sort(Comparator.comparing(BulletinEntity::getTimestamp).thenComparing(BulletinEntity::getNodeAddress));
final int timeComparison = o1.getTimestamp().compareTo(o2.getTimestamp());
if (timeComparison != 0) {
return timeComparison;
}
return o1.getNodeAddress().compareTo(o2.getNodeAddress());
});
return entities; return entities;
} }

View File

@ -39,7 +39,6 @@ import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.Timeout;
import org.mockito.Mockito; import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer; import org.mockito.stubbing.Answer;
import org.springframework.security.core.Authentication; import org.springframework.security.core.Authentication;
import org.springframework.security.core.context.SecurityContextHolder; import org.springframework.security.core.context.SecurityContextHolder;
@ -315,12 +314,8 @@ public class TestThreadPoolRequestReplicator {
private ClusterCoordinator createClusterCoordinator() { private ClusterCoordinator createClusterCoordinator() {
final ClusterCoordinator coordinator = mock(ClusterCoordinator.class); final ClusterCoordinator coordinator = mock(ClusterCoordinator.class);
when(coordinator.getConnectionStatus(Mockito.any(NodeIdentifier.class))).thenAnswer(new Answer<NodeConnectionStatus>() { when(coordinator.getConnectionStatus(Mockito.any(NodeIdentifier.class))).thenAnswer((Answer<NodeConnectionStatus>) invocation ->
@Override new NodeConnectionStatus(invocation.getArgument(0), NodeConnectionState.CONNECTED));
public NodeConnectionStatus answer(InvocationOnMock invocation) throws Throwable {
return new NodeConnectionStatus(invocation.getArgument(0), NodeConnectionState.CONNECTED);
}
});
return coordinator; return coordinator;
} }
@ -361,7 +356,7 @@ public class TestThreadPoolRequestReplicator {
} }
}; };
final IllegalClusterStateException exception = assertThrows(IllegalClusterStateException.class, () -> { assertThrows(IllegalClusterStateException.class, () -> {
final Authentication authentication = new NiFiAuthenticationToken(new NiFiUserDetails(StandardNiFiUser.ANONYMOUS)); final Authentication authentication = new NiFiAuthenticationToken(new NiFiUserDetails(StandardNiFiUser.ANONYMOUS));
SecurityContextHolder.getContext().setAuthentication(authentication); SecurityContextHolder.getContext().setAuthentication(authentication);
@ -381,23 +376,19 @@ public class TestThreadPoolRequestReplicator {
final CountDownLatch preNotifyLatch = new CountDownLatch(1); final CountDownLatch preNotifyLatch = new CountDownLatch(1);
final CountDownLatch postNotifyLatch = new CountDownLatch(1); final CountDownLatch postNotifyLatch = new CountDownLatch(1);
new Thread(new Runnable() { new Thread(() -> {
@Override synchronized (monitor) {
public void run() { while (true) {
synchronized (monitor) { // If monitor is not notified, this will block indefinitely, and the test will timeout
while (true) { try {
// If monitor is not notified, this will block indefinitely, and the test will timeout preNotifyLatch.countDown();
try { monitor.wait();
preNotifyLatch.countDown(); break;
monitor.wait(); } catch (InterruptedException ignored) {
break;
} catch (InterruptedException e) {
continue;
}
} }
postNotifyLatch.countDown();
} }
postNotifyLatch.countDown();
} }
}).start(); }).start();
@ -431,23 +422,19 @@ public class TestThreadPoolRequestReplicator {
final CountDownLatch preNotifyLatch = new CountDownLatch(1); final CountDownLatch preNotifyLatch = new CountDownLatch(1);
final CountDownLatch postNotifyLatch = new CountDownLatch(1); final CountDownLatch postNotifyLatch = new CountDownLatch(1);
new Thread(new Runnable() { new Thread(() -> {
@Override synchronized (monitor) {
public void run() { while (true) {
synchronized (monitor) { // If monitor is not notified, this will block indefinitely, and the test will timeout
while (true) { try {
// If monitor is not notified, this will block indefinitely, and the test will timeout preNotifyLatch.countDown();
try { monitor.wait();
preNotifyLatch.countDown(); break;
monitor.wait(); } catch (InterruptedException ignored) {
break;
} catch (InterruptedException e) {
continue;
}
} }
postNotifyLatch.countDown();
} }
postNotifyLatch.countDown();
} }
}).start(); }).start();
@ -485,23 +472,19 @@ public class TestThreadPoolRequestReplicator {
final CountDownLatch preNotifyLatch = new CountDownLatch(1); final CountDownLatch preNotifyLatch = new CountDownLatch(1);
final CountDownLatch postNotifyLatch = new CountDownLatch(1); final CountDownLatch postNotifyLatch = new CountDownLatch(1);
new Thread(new Runnable() { new Thread(() -> {
@Override synchronized (monitor) {
public void run() { while (true) {
synchronized (monitor) { // If monitor is not notified, this will block indefinitely, and the test will timeout
while (true) { try {
// If monitor is not notified, this will block indefinitely, and the test will timeout preNotifyLatch.countDown();
try { monitor.wait();
preNotifyLatch.countDown(); break;
monitor.wait(); } catch (InterruptedException ignored) {
break;
} catch (InterruptedException e) {
continue;
}
} }
postNotifyLatch.countDown();
} }
postNotifyLatch.countDown();
} }
}).start(); }).start();
@ -557,9 +540,7 @@ public class TestThreadPoolRequestReplicator {
final StandardAsyncClusterResponse response) { final StandardAsyncClusterResponse response) {
if (delayMillis > 0L) { if (delayMillis > 0L) {
assertDoesNotThrow(() -> { assertDoesNotThrow(() -> Thread.sleep(delayMillis), "Thread Interrupted during test");
Thread.sleep(delayMillis);
}, "Thread Interrupted during test");
} }
if (failure != null) { if (failure != null) {
@ -581,9 +562,7 @@ public class TestThreadPoolRequestReplicator {
} }
}; };
assertDoesNotThrow(() -> { assertDoesNotThrow(() -> function.withReplicator(replicator));
function.withReplicator(replicator);
});
replicator.shutdown(); replicator.shutdown();
} }

View File

@ -42,8 +42,6 @@ import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.Timeout;
import org.mockito.Mockito; import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
@ -181,13 +179,10 @@ public class TestNodeClusterCoordinator {
final ClusterCoordinationProtocolSenderListener senderListener = Mockito.mock(ClusterCoordinationProtocolSenderListener.class); final ClusterCoordinationProtocolSenderListener senderListener = Mockito.mock(ClusterCoordinationProtocolSenderListener.class);
final AtomicReference<ReconnectionRequestMessage> requestRef = new AtomicReference<>(); final AtomicReference<ReconnectionRequestMessage> requestRef = new AtomicReference<>();
when(senderListener.requestReconnection(any(ReconnectionRequestMessage.class))).thenAnswer(new Answer<Object>() { when(senderListener.requestReconnection(any(ReconnectionRequestMessage.class))).thenAnswer(invocation -> {
@Override final ReconnectionRequestMessage msg = invocation.getArgument(0);
public Object answer(InvocationOnMock invocation) throws Throwable { requestRef.set(msg);
final ReconnectionRequestMessage msg = invocation.getArgument(0); return null;
requestRef.set(msg);
return null;
}
}); });
final EventReporter eventReporter = Mockito.mock(EventReporter.class); final EventReporter eventReporter = Mockito.mock(EventReporter.class);

View File

@ -1907,35 +1907,25 @@ public class StandardProcessSession implements ProcessSession, ProvenanceEventEn
return Collections.emptyList(); return Collections.emptyList();
} }
return get(new ConnectionPoller() { return get((connection, expiredRecords) -> connection.poll(new FlowFileFilter() {
@Override int polled = 0;
public List<FlowFileRecord> poll(final Connection connection, final Set<FlowFileRecord> expiredRecords) {
return connection.poll(new FlowFileFilter() {
int polled = 0;
@Override @Override
public FlowFileFilterResult filter(final FlowFile flowFile) { public FlowFileFilterResult filter(final FlowFile flowFile) {
if (++polled < maxResults) { if (++polled < maxResults) {
return FlowFileFilterResult.ACCEPT_AND_CONTINUE; return FlowFileFilterResult.ACCEPT_AND_CONTINUE;
} else { } else {
return FlowFileFilterResult.ACCEPT_AND_TERMINATE; return FlowFileFilterResult.ACCEPT_AND_TERMINATE;
} }
}
}, expiredRecords);
} }
}, false); }, expiredRecords), false);
} }
@Override @Override
public List<FlowFile> get(final FlowFileFilter filter) { public List<FlowFile> get(final FlowFileFilter filter) {
verifyTaskActive(); verifyTaskActive();
return get(new ConnectionPoller() { return get((connection, expiredRecords) -> connection.poll(filter, expiredRecords), true);
@Override
public List<FlowFileRecord> poll(final Connection connection, final Set<FlowFileRecord> expiredRecords) {
return connection.poll(filter, expiredRecords);
}
}, true);
} }
@ -2494,12 +2484,7 @@ public class StandardProcessSession implements ProcessSession, ProvenanceEventEn
public void expireFlowFiles() { public void expireFlowFiles() {
final Set<FlowFileRecord> expired = new HashSet<>(); final Set<FlowFileRecord> expired = new HashSet<>();
final FlowFileFilter filter = new FlowFileFilter() { final FlowFileFilter filter = flowFile -> FlowFileFilter.FlowFileFilterResult.REJECT_AND_CONTINUE;
@Override
public FlowFileFilterResult filter(final FlowFile flowFile) {
return FlowFileFilterResult.REJECT_AND_CONTINUE;
}
};
for (final Connection conn : context.getConnectable().getIncomingConnections()) { for (final Connection conn : context.getConnectable().getIncomingConnections()) {
do { do {
@ -2536,46 +2521,43 @@ public class StandardProcessSession implements ProcessSession, ProvenanceEventEn
} }
try { try {
final Iterable<ProvenanceEventRecord> iterable = new Iterable<ProvenanceEventRecord>() { final Iterable<ProvenanceEventRecord> iterable = () -> {
@Override final Iterator<ProvenanceEventRecord> expiredEventIterator = expiredReporter.getEvents().iterator();
public Iterator<ProvenanceEventRecord> iterator() { final Iterator<ProvenanceEventRecord> enrichingIterator = new Iterator<>() {
final Iterator<ProvenanceEventRecord> expiredEventIterator = expiredReporter.getEvents().iterator(); @Override
final Iterator<ProvenanceEventRecord> enrichingIterator = new Iterator<ProvenanceEventRecord>() { public boolean hasNext() {
@Override return expiredEventIterator.hasNext();
public boolean hasNext() { }
return expiredEventIterator.hasNext();
@Override
public ProvenanceEventRecord next() {
final ProvenanceEventRecord event = expiredEventIterator.next();
final ProvenanceEventBuilder enriched = context.createProvenanceEventBuilder().fromEvent(event);
final FlowFileRecord record = recordIdMap.get(event.getFlowFileUuid());
if (record == null) {
return null;
} }
@Override final ContentClaim claim = record.getContentClaim();
public ProvenanceEventRecord next() { if (claim != null) {
final ProvenanceEventRecord event = expiredEventIterator.next(); final ResourceClaim resourceClaim = claim.getResourceClaim();
final ProvenanceEventBuilder enriched = context.createProvenanceEventBuilder().fromEvent(event); enriched.setCurrentContentClaim(resourceClaim.getContainer(), resourceClaim.getSection(), resourceClaim.getId(),
final FlowFileRecord record = recordIdMap.get(event.getFlowFileUuid());
if (record == null) {
return null;
}
final ContentClaim claim = record.getContentClaim();
if (claim != null) {
final ResourceClaim resourceClaim = claim.getResourceClaim();
enriched.setCurrentContentClaim(resourceClaim.getContainer(), resourceClaim.getSection(), resourceClaim.getId(),
record.getContentClaimOffset() + claim.getOffset(), record.getSize()); record.getContentClaimOffset() + claim.getOffset(), record.getSize());
enriched.setPreviousContentClaim(resourceClaim.getContainer(), resourceClaim.getSection(), resourceClaim.getId(), enriched.setPreviousContentClaim(resourceClaim.getContainer(), resourceClaim.getSection(), resourceClaim.getId(),
record.getContentClaimOffset() + claim.getOffset(), record.getSize()); record.getContentClaimOffset() + claim.getOffset(), record.getSize());
}
enriched.setAttributes(record.getAttributes(), Collections.<String, String> emptyMap());
return enriched.build();
} }
@Override enriched.setAttributes(record.getAttributes(), Collections.<String, String>emptyMap());
public void remove() { return enriched.build();
throw new UnsupportedOperationException(); }
}
};
return enrichingIterator; @Override
} public void remove() {
throw new UnsupportedOperationException();
}
};
return enrichingIterator;
}; };
context.getProvenanceRepository().registerEvents(iterable); context.getProvenanceRepository().registerEvents(iterable);

View File

@ -97,60 +97,57 @@ public class VolatileBulletinRepository implements BulletinRepository {
private Filter<Bulletin> createFilter(final BulletinQuery bulletinQuery) { private Filter<Bulletin> createFilter(final BulletinQuery bulletinQuery) {
final long fiveMinutesAgo = System.currentTimeMillis() - TimeUnit.MINUTES.toMillis(5); final long fiveMinutesAgo = System.currentTimeMillis() - TimeUnit.MINUTES.toMillis(5);
return new Filter<Bulletin>() { return bulletin -> {
@Override // only include bulletins after the specified id
public boolean select(final Bulletin bulletin) { if (bulletinQuery.getAfter() != null && bulletin.getId() <= bulletinQuery.getAfter()) {
// only include bulletins after the specified id return false;
if (bulletinQuery.getAfter() != null && bulletin.getId() <= bulletinQuery.getAfter()) {
return false;
}
// if group pattern was specified see if it should be excluded
if (bulletinQuery.getGroupIdPattern() != null) {
// exclude if this bulletin doesnt have a group or if it doesnt match
if (bulletin.getGroupId() == null || !bulletinQuery.getGroupIdPattern().matcher(bulletin.getGroupId()).find()) {
return false;
}
}
// if a message pattern was specified see if it should be excluded
if (bulletinQuery.getMessagePattern() != null) {
// exclude if this bulletin doesnt have a message or if it doesnt match
if (bulletin.getMessage() == null || !bulletinQuery.getMessagePattern().matcher(bulletin.getMessage()).find()) {
return false;
}
}
// if a name pattern was specified see if it should be excluded
if (bulletinQuery.getNamePattern() != null) {
// exclude if this bulletin doesnt have a source name or if it doesnt match
if (bulletin.getSourceName() == null || !bulletinQuery.getNamePattern().matcher(bulletin.getSourceName()).find()) {
return false;
}
}
if (bulletin.getTimestamp().getTime() < fiveMinutesAgo) {
return false;
}
// if a source id was specified see if it should be excluded
if (bulletinQuery.getSourceIdPattern() != null) {
// exclude if this bulletin doesn't have a source id or if it doesn't match
if (bulletin.getSourceId() == null || !bulletinQuery.getSourceIdPattern().matcher(bulletin.getSourceId()).find()) {
return false;
}
}
// if a source component type was specified see if it should be excluded
if (bulletinQuery.getSourceType() != null) {
// exclude if this bulletin source type doesn't match
if (bulletin.getSourceType() == null || !bulletinQuery.getSourceType().equals(bulletin.getSourceType())) {
return false;
}
}
return true;
} }
// if group pattern was specified see if it should be excluded
if (bulletinQuery.getGroupIdPattern() != null) {
// exclude if this bulletin doesnt have a group or if it doesnt match
if (bulletin.getGroupId() == null || !bulletinQuery.getGroupIdPattern().matcher(bulletin.getGroupId()).find()) {
return false;
}
}
// if a message pattern was specified see if it should be excluded
if (bulletinQuery.getMessagePattern() != null) {
// exclude if this bulletin doesnt have a message or if it doesnt match
if (bulletin.getMessage() == null || !bulletinQuery.getMessagePattern().matcher(bulletin.getMessage()).find()) {
return false;
}
}
// if a name pattern was specified see if it should be excluded
if (bulletinQuery.getNamePattern() != null) {
// exclude if this bulletin doesnt have a source name or if it doesnt match
if (bulletin.getSourceName() == null || !bulletinQuery.getNamePattern().matcher(bulletin.getSourceName()).find()) {
return false;
}
}
if (bulletin.getTimestamp().getTime() < fiveMinutesAgo) {
return false;
}
// if a source id was specified see if it should be excluded
if (bulletinQuery.getSourceIdPattern() != null) {
// exclude if this bulletin doesn't have a source id or if it doesn't match
if (bulletin.getSourceId() == null || !bulletinQuery.getSourceIdPattern().matcher(bulletin.getSourceId()).find()) {
return false;
}
}
// if a source component type was specified see if it should be excluded
if (bulletinQuery.getSourceType() != null) {
// exclude if this bulletin source type doesn't match
if (bulletin.getSourceType() == null || !bulletinQuery.getSourceType().equals(bulletin.getSourceType())) {
return false;
}
}
return true;
}; };
} }
@ -205,12 +202,7 @@ public class VolatileBulletinRepository implements BulletinRepository {
final List<Bulletin> allComponentBulletins = new ArrayList<>(); final List<Bulletin> allComponentBulletins = new ArrayList<>();
for (final RingBuffer<Bulletin> ringBuffer : componentMap.values()) { for (final RingBuffer<Bulletin> ringBuffer : componentMap.values()) {
allComponentBulletins.addAll(ringBuffer.getSelectedElements(new Filter<Bulletin>() { allComponentBulletins.addAll(ringBuffer.getSelectedElements(bulletin -> bulletin.getTimestamp().getTime() >= fiveMinutesAgo, maxPerComponent));
@Override
public boolean select(final Bulletin bulletin) {
return bulletin.getTimestamp().getTime() >= fiveMinutesAgo;
}
}, maxPerComponent));
} }
return allComponentBulletins; return allComponentBulletins;
@ -225,12 +217,7 @@ public class VolatileBulletinRepository implements BulletinRepository {
public List<Bulletin> findBulletinsForController(final int max) { public List<Bulletin> findBulletinsForController(final int max) {
final long fiveMinutesAgo = System.currentTimeMillis() - TimeUnit.MINUTES.toMillis(5); final long fiveMinutesAgo = System.currentTimeMillis() - TimeUnit.MINUTES.toMillis(5);
final Filter<Bulletin> filter = new Filter<Bulletin>() { final Filter<Bulletin> filter = bulletin -> bulletin.getTimestamp().getTime() >= fiveMinutesAgo;
@Override
public boolean select(final Bulletin bulletin) {
return bulletin.getTimestamp().getTime() >= fiveMinutesAgo;
}
};
final List<Bulletin> controllerBulletins = new ArrayList<>(); final List<Bulletin> controllerBulletins = new ArrayList<>();

View File

@ -56,16 +56,13 @@ public final class FlowEngine extends ScheduledThreadPoolExecutor {
final AtomicInteger threadIndex = new AtomicInteger(0); final AtomicInteger threadIndex = new AtomicInteger(0);
final ThreadFactory defaultThreadFactory = getThreadFactory(); final ThreadFactory defaultThreadFactory = getThreadFactory();
setThreadFactory(new ThreadFactory() { setThreadFactory(r -> {
@Override final Thread t = defaultThreadFactory.newThread(r);
public Thread newThread(final Runnable r) { if (daemon) {
final Thread t = defaultThreadFactory.newThread(r); t.setDaemon(true);
if (daemon) {
t.setDaemon(true);
}
t.setName(threadNamePrefix + " Thread-" + threadIndex.incrementAndGet());
return t;
} }
t.setName(threadNamePrefix + " Thread-" + threadIndex.incrementAndGet());
return t;
}); });
} }
@ -103,28 +100,22 @@ public final class FlowEngine extends ScheduledThreadPoolExecutor {
} }
private Runnable wrap(final Runnable runnable) { private Runnable wrap(final Runnable runnable) {
return new Runnable() { return () -> {
@Override try {
public void run() { runnable.run();
try { } catch (final Throwable t) {
runnable.run(); logger.error("Uncaught Exception in Runnable task", t);
} catch (final Throwable t) {
logger.error("Uncaught Exception in Runnable task", t);
}
} }
}; };
} }
private <T> Callable<T> wrap(final Callable<T> callable) { private <T> Callable<T> wrap(final Callable<T> callable) {
return new Callable<T>() { return () -> {
@Override try {
public T call() throws Exception { return callable.call();
try { } catch (final Throwable t) {
return callable.call(); logger.error("Uncaught Exception in Callable task", t);
} catch (final Throwable t) { throw t;
logger.error("Uncaught Exception in Callable task", t);
throw t;
}
} }
}; };
} }

View File

@ -37,12 +37,7 @@ public class SortedStateUtils {
*/ */
public static Comparator<String> getKeyComparator() { public static Comparator<String> getKeyComparator() {
final Collator collator = Collator.getInstance(Locale.US); final Collator collator = Collator.getInstance(Locale.US);
return new Comparator<String>() { return collator::compare;
@Override
public int compare(String s1, String s2) {
return collator.compare(s1, s2);
}
};
} }
/** /**
@ -52,11 +47,6 @@ public class SortedStateUtils {
*/ */
public static Comparator<StateEntryDTO> getEntryDtoComparator() { public static Comparator<StateEntryDTO> getEntryDtoComparator() {
final Collator collator = Collator.getInstance(Locale.US); final Collator collator = Collator.getInstance(Locale.US);
return new Comparator<StateEntryDTO>() { return (o1, o2) -> collator.compare(o1.getKey(), o2.getKey());
@Override
public int compare(StateEntryDTO o1, StateEntryDTO o2) {
return collator.compare(o1.getKey(), o2.getKey());
}
};
} }
} }

View File

@ -37,7 +37,6 @@ import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.Timeout;
import org.mockito.Mockito; import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer; import org.mockito.stubbing.Answer;
import java.util.ArrayList; import java.util.ArrayList;
@ -82,15 +81,12 @@ public class TestStandardFlowFileQueue {
provRepo = Mockito.mock(ProvenanceEventRepository.class); provRepo = Mockito.mock(ProvenanceEventRepository.class);
Mockito.when(provRepo.eventBuilder()).thenReturn(new StandardProvenanceEventRecord.Builder()); Mockito.when(provRepo.eventBuilder()).thenReturn(new StandardProvenanceEventRecord.Builder());
Mockito.doAnswer(new Answer<Object>() { Mockito.doAnswer((Answer<Object>) invocation -> {
@Override final Iterable<ProvenanceEventRecord> iterable = (Iterable<ProvenanceEventRecord>) invocation.getArguments()[0];
public Object answer(final InvocationOnMock invocation) throws Throwable { for (final ProvenanceEventRecord record : iterable) {
final Iterable<ProvenanceEventRecord> iterable = (Iterable<ProvenanceEventRecord>) invocation.getArguments()[0]; provRecords.add(record);
for (final ProvenanceEventRecord record : iterable) {
provRecords.add(record);
}
return null;
} }
return null;
}).when(provRepo).registerEvents(Mockito.any(Iterable.class)); }).when(provRepo).registerEvents(Mockito.any(Iterable.class));
queue = new StandardFlowFileQueue("id", flowFileRepo, provRepo, scheduler, swapManager, null, 10000, "0 sec", 0L, "0 B"); queue = new StandardFlowFileQueue("id", flowFileRepo, provRepo, scheduler, swapManager, null, 10000, "0 sec", 0L, "0 B");

View File

@ -71,15 +71,10 @@ import java.io.IOException;
import java.io.InputStream; import java.io.InputStream;
import java.io.OutputStream; import java.io.OutputStream;
import java.security.GeneralSecurityException; import java.security.GeneralSecurityException;
import java.security.KeyManagementException;
import java.security.KeyPair; import java.security.KeyPair;
import java.security.KeyPairGenerator; import java.security.KeyPairGenerator;
import java.security.KeyStore; import java.security.KeyStore;
import java.security.KeyStoreException;
import java.security.NoSuchAlgorithmException;
import java.security.UnrecoverableKeyException;
import java.security.cert.Certificate; import java.security.cert.Certificate;
import java.security.cert.CertificateException;
import java.security.cert.X509Certificate; import java.security.cert.X509Certificate;
import java.time.Duration; import java.time.Duration;
import java.util.ArrayList; import java.util.ArrayList;
@ -156,12 +151,9 @@ public class LoadBalancedQueueIT {
new NodeConnectionStatus(invocation.getArgument(0, NodeIdentifier.class), NodeConnectionState.CONNECTED)); new NodeConnectionStatus(invocation.getArgument(0, NodeIdentifier.class), NodeConnectionState.CONNECTED));
clusterEventListeners.clear(); clusterEventListeners.clear();
doAnswer(new Answer() { doAnswer(invocation -> {
@Override clusterEventListeners.add(invocation.getArgument(0));
public Object answer(final InvocationOnMock invocation) { return null;
clusterEventListeners.add(invocation.getArgument(0));
return null;
}
}).when(clusterCoordinator).registerEventListener(any(ClusterTopologyEventListener.class)); }).when(clusterCoordinator).registerEventListener(any(ClusterTopologyEventListener.class));
processScheduler = mock(ProcessScheduler.class); processScheduler = mock(ProcessScheduler.class);
@ -487,7 +479,7 @@ public class LoadBalancedQueueIT {
assertEquals(1, serverRepoRecords.size()); assertEquals(1, serverRepoRecords.size());
final RepositoryRecord serverRecord = serverRepoRecords.iterator().next(); final RepositoryRecord serverRecord = serverRepoRecords.getFirst();
final FlowFileRecord serverFlowFile = serverRecord.getCurrent(); final FlowFileRecord serverFlowFile = serverRecord.getCurrent();
assertEquals("test", serverFlowFile.getAttribute("integration")); assertEquals("test", serverFlowFile.getAttribute("integration"));
assertEquals("false", serverFlowFile.getAttribute("unit-test")); assertEquals("false", serverFlowFile.getAttribute("unit-test"));
@ -497,12 +489,12 @@ public class LoadBalancedQueueIT {
final byte[] serverFlowFileContent = serverClaimContents.get(serverContentClaim); final byte[] serverFlowFileContent = serverClaimContents.get(serverContentClaim);
assertArrayEquals("hello".getBytes(), serverFlowFileContent); assertArrayEquals("hello".getBytes(), serverFlowFileContent);
while (clientRepoRecords.size() == 0) { while (clientRepoRecords.isEmpty()) {
Thread.sleep(10L); Thread.sleep(10L);
} }
assertEquals(1, clientRepoRecords.size()); assertEquals(1, clientRepoRecords.size());
final RepositoryRecord clientRecord = clientRepoRecords.iterator().next(); final RepositoryRecord clientRecord = clientRepoRecords.getFirst();
assertEquals(RepositoryRecordType.DELETE, clientRecord.getType()); assertEquals(RepositoryRecordType.DELETE, clientRecord.getType());
} finally { } finally {
flowFileQueue.stopLoadBalancing(); flowFileQueue.stopLoadBalancing();
@ -568,12 +560,12 @@ public class LoadBalancedQueueIT {
flowFileQueue.startLoadBalancing(); flowFileQueue.startLoadBalancing();
while (clientRepoRecords.size() == 0) { while (clientRepoRecords.isEmpty()) {
Thread.sleep(10L); Thread.sleep(10L);
} }
assertEquals(1, clientRepoRecords.size()); assertEquals(1, clientRepoRecords.size());
final RepositoryRecord clientRecord = clientRepoRecords.iterator().next(); final RepositoryRecord clientRecord = clientRepoRecords.getFirst();
assertEquals(RepositoryRecordType.CONTENTMISSING, clientRecord.getType()); assertEquals(RepositoryRecordType.CONTENTMISSING, clientRecord.getType());
} finally { } finally {
flowFileQueue.stopLoadBalancing(); flowFileQueue.stopLoadBalancing();
@ -649,7 +641,7 @@ public class LoadBalancedQueueIT {
assertEquals(1, serverRepoRecords.size()); assertEquals(1, serverRepoRecords.size());
final RepositoryRecord serverRecord = serverRepoRecords.iterator().next(); final RepositoryRecord serverRecord = serverRepoRecords.getFirst();
final FlowFileRecord serverFlowFile = serverRecord.getCurrent(); final FlowFileRecord serverFlowFile = serverRecord.getCurrent();
assertEquals("test", serverFlowFile.getAttribute("integration")); assertEquals("test", serverFlowFile.getAttribute("integration"));
assertEquals("false", serverFlowFile.getAttribute("unit-test")); assertEquals("false", serverFlowFile.getAttribute("unit-test"));
@ -659,12 +651,12 @@ public class LoadBalancedQueueIT {
final byte[] serverFlowFileContent = serverClaimContents.get(serverContentClaim); final byte[] serverFlowFileContent = serverClaimContents.get(serverContentClaim);
assertArrayEquals("hello".getBytes(), serverFlowFileContent); assertArrayEquals("hello".getBytes(), serverFlowFileContent);
while (clientRepoRecords.size() == 0) { while (clientRepoRecords.isEmpty()) {
Thread.sleep(10L); Thread.sleep(10L);
} }
assertEquals(1, clientRepoRecords.size()); assertEquals(1, clientRepoRecords.size());
final RepositoryRecord clientRecord = clientRepoRecords.iterator().next(); final RepositoryRecord clientRecord = clientRepoRecords.getFirst();
assertEquals(RepositoryRecordType.DELETE, clientRecord.getType()); assertEquals(RepositoryRecordType.DELETE, clientRecord.getType());
} finally { } finally {
flowFileQueue.stopLoadBalancing(); flowFileQueue.stopLoadBalancing();
@ -740,7 +732,7 @@ public class LoadBalancedQueueIT {
assertEquals(1, serverRepoRecords.size()); assertEquals(1, serverRepoRecords.size());
final RepositoryRecord serverRecord = serverRepoRecords.iterator().next(); final RepositoryRecord serverRecord = serverRepoRecords.getFirst();
final FlowFileRecord serverFlowFile = serverRecord.getCurrent(); final FlowFileRecord serverFlowFile = serverRecord.getCurrent();
assertEquals("test", serverFlowFile.getAttribute("integration")); assertEquals("test", serverFlowFile.getAttribute("integration"));
assertEquals("false", serverFlowFile.getAttribute("unit-test")); assertEquals("false", serverFlowFile.getAttribute("unit-test"));
@ -750,12 +742,12 @@ public class LoadBalancedQueueIT {
final byte[] serverFlowFileContent = serverClaimContents.get(serverContentClaim); final byte[] serverFlowFileContent = serverClaimContents.get(serverContentClaim);
assertArrayEquals("hello".getBytes(), serverFlowFileContent); assertArrayEquals("hello".getBytes(), serverFlowFileContent);
while (clientRepoRecords.size() == 0) { while (clientRepoRecords.isEmpty()) {
Thread.sleep(10L); Thread.sleep(10L);
} }
assertEquals(1, clientRepoRecords.size()); assertEquals(1, clientRepoRecords.size());
final RepositoryRecord clientRecord = clientRepoRecords.iterator().next(); final RepositoryRecord clientRecord = clientRepoRecords.getFirst();
assertEquals(RepositoryRecordType.DELETE, clientRecord.getType()); assertEquals(RepositoryRecordType.DELETE, clientRecord.getType());
} finally { } finally {
flowFileQueue.stopLoadBalancing(); flowFileQueue.stopLoadBalancing();
@ -768,7 +760,7 @@ public class LoadBalancedQueueIT {
@Test @Test
@Timeout(20) @Timeout(20)
public void testWithSSLContext() throws IOException, InterruptedException, UnrecoverableKeyException, CertificateException, NoSuchAlgorithmException, KeyStoreException, KeyManagementException { public void testWithSSLContext() throws IOException, InterruptedException {
localNodeId = new NodeIdentifier("unit-test-local", "localhost", 7090, "localhost", 7090, "localhost", 7090, null, null, null, false, null); localNodeId = new NodeIdentifier("unit-test-local", "localhost", 7090, "localhost", 7090, "localhost", 7090, null, null, null, false, null);
nodeIdentifiers.add(localNodeId); nodeIdentifiers.add(localNodeId);
@ -828,7 +820,7 @@ public class LoadBalancedQueueIT {
assertEquals(1, serverRepoRecords.size()); assertEquals(1, serverRepoRecords.size());
final RepositoryRecord serverRecord = serverRepoRecords.iterator().next(); final RepositoryRecord serverRecord = serverRepoRecords.getFirst();
final FlowFileRecord serverFlowFile = serverRecord.getCurrent(); final FlowFileRecord serverFlowFile = serverRecord.getCurrent();
assertEquals("test", serverFlowFile.getAttribute("integration")); assertEquals("test", serverFlowFile.getAttribute("integration"));
assertEquals("false", serverFlowFile.getAttribute("unit-test")); assertEquals("false", serverFlowFile.getAttribute("unit-test"));
@ -838,12 +830,12 @@ public class LoadBalancedQueueIT {
final byte[] serverFlowFileContent = serverClaimContents.get(serverContentClaim); final byte[] serverFlowFileContent = serverClaimContents.get(serverContentClaim);
assertArrayEquals("hello".getBytes(), serverFlowFileContent); assertArrayEquals("hello".getBytes(), serverFlowFileContent);
while (clientRepoRecords.size() == 0) { while (clientRepoRecords.isEmpty()) {
Thread.sleep(10L); Thread.sleep(10L);
} }
assertEquals(1, clientRepoRecords.size()); assertEquals(1, clientRepoRecords.size());
final RepositoryRecord clientRecord = clientRepoRecords.iterator().next(); final RepositoryRecord clientRecord = clientRepoRecords.getFirst();
assertEquals(RepositoryRecordType.DELETE, clientRecord.getType()); assertEquals(RepositoryRecordType.DELETE, clientRecord.getType());
} finally { } finally {
flowFileQueue.stopLoadBalancing(); flowFileQueue.stopLoadBalancing();
@ -857,7 +849,7 @@ public class LoadBalancedQueueIT {
@Test @Test
@Timeout(60) @Timeout(60)
public void testReusingClient() throws IOException, InterruptedException, UnrecoverableKeyException, CertificateException, NoSuchAlgorithmException, KeyStoreException, KeyManagementException { public void testReusingClient() throws IOException, InterruptedException {
localNodeId = new NodeIdentifier("unit-test-local", "localhost", 7090, "localhost", 7090, "localhost", 7090, null, null, null, false, null); localNodeId = new NodeIdentifier("unit-test-local", "localhost", 7090, "localhost", 7090, "localhost", 7090, null, null, null, false, null);
nodeIdentifiers.add(localNodeId); nodeIdentifiers.add(localNodeId);
@ -915,7 +907,7 @@ public class LoadBalancedQueueIT {
assertEquals(i, serverRepoRecords.size()); assertEquals(i, serverRepoRecords.size());
final RepositoryRecord serverRecord = serverRepoRecords.iterator().next(); final RepositoryRecord serverRecord = serverRepoRecords.getFirst();
final FlowFileRecord serverFlowFile = serverRecord.getCurrent(); final FlowFileRecord serverFlowFile = serverRecord.getCurrent();
assertEquals("test", serverFlowFile.getAttribute("integration")); assertEquals("test", serverFlowFile.getAttribute("integration"));
assertEquals("false", serverFlowFile.getAttribute("unit-test")); assertEquals("false", serverFlowFile.getAttribute("unit-test"));
@ -930,7 +922,7 @@ public class LoadBalancedQueueIT {
} }
assertEquals(i, clientRepoRecords.size()); assertEquals(i, clientRepoRecords.size());
final RepositoryRecord clientRecord = clientRepoRecords.iterator().next(); final RepositoryRecord clientRecord = clientRepoRecords.getFirst();
assertEquals(RepositoryRecordType.DELETE, clientRecord.getType()); assertEquals(RepositoryRecordType.DELETE, clientRecord.getType());
} }
} finally { } finally {
@ -945,7 +937,7 @@ public class LoadBalancedQueueIT {
@Test @Test
@Timeout(20) @Timeout(20)
public void testLargePayload() throws IOException, InterruptedException, UnrecoverableKeyException, CertificateException, NoSuchAlgorithmException, KeyStoreException, KeyManagementException { public void testLargePayload() throws IOException, InterruptedException {
localNodeId = new NodeIdentifier("unit-test-local", "localhost", 7090, "localhost", 7090, "localhost", 7090, null, null, null, false, null); localNodeId = new NodeIdentifier("unit-test-local", "localhost", 7090, "localhost", 7090, "localhost", 7090, null, null, null, false, null);
nodeIdentifiers.add(localNodeId); nodeIdentifiers.add(localNodeId);
@ -1007,7 +999,7 @@ public class LoadBalancedQueueIT {
assertEquals(1, serverRepoRecords.size()); assertEquals(1, serverRepoRecords.size());
final RepositoryRecord serverRecord = serverRepoRecords.iterator().next(); final RepositoryRecord serverRecord = serverRepoRecords.getFirst();
final FlowFileRecord serverFlowFile = serverRecord.getCurrent(); final FlowFileRecord serverFlowFile = serverRecord.getCurrent();
assertEquals("test", serverFlowFile.getAttribute("integration")); assertEquals("test", serverFlowFile.getAttribute("integration"));
assertEquals("false", serverFlowFile.getAttribute("unit-test")); assertEquals("false", serverFlowFile.getAttribute("unit-test"));
@ -1017,12 +1009,12 @@ public class LoadBalancedQueueIT {
final byte[] serverFlowFileContent = serverClaimContents.get(serverContentClaim); final byte[] serverFlowFileContent = serverClaimContents.get(serverContentClaim);
assertArrayEquals(payload, serverFlowFileContent); assertArrayEquals(payload, serverFlowFileContent);
while (clientRepoRecords.size() == 0) { while (clientRepoRecords.isEmpty()) {
Thread.sleep(10L); Thread.sleep(10L);
} }
assertEquals(1, clientRepoRecords.size()); assertEquals(1, clientRepoRecords.size());
final RepositoryRecord clientRecord = clientRepoRecords.iterator().next(); final RepositoryRecord clientRecord = clientRepoRecords.getFirst();
assertEquals(RepositoryRecordType.DELETE, clientRecord.getType()); assertEquals(RepositoryRecordType.DELETE, clientRecord.getType());
} finally { } finally {
flowFileQueue.stopLoadBalancing(); flowFileQueue.stopLoadBalancing();
@ -1139,7 +1131,7 @@ public class LoadBalancedQueueIT {
assertEquals(1, serverRepoRecords.size()); assertEquals(1, serverRepoRecords.size());
final RepositoryRecord serverRecord = serverRepoRecords.iterator().next(); final RepositoryRecord serverRecord = serverRepoRecords.getFirst();
final FlowFileRecord serverFlowFile = serverRecord.getCurrent(); final FlowFileRecord serverFlowFile = serverRecord.getCurrent();
assertEquals("test", serverFlowFile.getAttribute("integration")); assertEquals("test", serverFlowFile.getAttribute("integration"));
assertEquals("false", serverFlowFile.getAttribute("unit-test")); assertEquals("false", serverFlowFile.getAttribute("unit-test"));
@ -1149,12 +1141,12 @@ public class LoadBalancedQueueIT {
final byte[] serverFlowFileContent = serverClaimContents.get(serverContentClaim); final byte[] serverFlowFileContent = serverClaimContents.get(serverContentClaim);
assertArrayEquals("hello".getBytes(), serverFlowFileContent); assertArrayEquals("hello".getBytes(), serverFlowFileContent);
while (clientRepoRecords.size() == 0) { while (clientRepoRecords.isEmpty()) {
Thread.sleep(10L); Thread.sleep(10L);
} }
assertEquals(1, clientRepoRecords.size()); assertEquals(1, clientRepoRecords.size());
final RepositoryRecord clientRecord = clientRepoRecords.iterator().next(); final RepositoryRecord clientRecord = clientRepoRecords.getFirst();
assertEquals(RepositoryRecordType.DELETE, clientRecord.getType()); assertEquals(RepositoryRecordType.DELETE, clientRecord.getType());
} finally { } finally {
flowFileQueue.stopLoadBalancing(); flowFileQueue.stopLoadBalancing();
@ -1325,12 +1317,7 @@ public class LoadBalancedQueueIT {
private ContentRepository createContentRepository(final ConcurrentMap<ContentClaim, byte[]> claimContents) throws IOException { private ContentRepository createContentRepository(final ConcurrentMap<ContentClaim, byte[]> claimContents) throws IOException {
final ContentRepository contentRepo = mock(ContentRepository.class); final ContentRepository contentRepo = mock(ContentRepository.class);
Mockito.doAnswer(new Answer<ContentClaim>() { Mockito.doAnswer((Answer<ContentClaim>) invocation -> createContentClaim(null)).when(contentRepo).create(Mockito.anyBoolean());
@Override
public ContentClaim answer(final InvocationOnMock invocation) {
return createContentClaim(null);
}
}).when(contentRepo).create(Mockito.anyBoolean());
Mockito.doAnswer(new Answer<OutputStream>() { Mockito.doAnswer(new Answer<OutputStream>() {
@ -1351,21 +1338,18 @@ public class LoadBalancedQueueIT {
}).when(contentRepo).write(any(ContentClaim.class)); }).when(contentRepo).write(any(ContentClaim.class));
Mockito.doAnswer(new Answer<InputStream>() { Mockito.doAnswer((Answer<InputStream>) invocation -> {
@Override final ContentClaim contentClaim = invocation.getArgument(0);
public InputStream answer(final InvocationOnMock invocation) { if (contentClaim == null) {
final ContentClaim contentClaim = invocation.getArgument(0); return new ByteArrayInputStream(new byte[0]);
if (contentClaim == null) {
return new ByteArrayInputStream(new byte[0]);
}
final byte[] bytes = claimContents.get(contentClaim);
if (bytes == null) {
throw new ContentNotFoundException(contentClaim);
}
return new ByteArrayInputStream(bytes);
} }
final byte[] bytes = claimContents.get(contentClaim);
if (bytes == null) {
throw new ContentNotFoundException(contentClaim);
}
return new ByteArrayInputStream(bytes);
}).when(contentRepo).read(Mockito.nullable(ContentClaim.class)); }).when(contentRepo).read(Mockito.nullable(ContentClaim.class));
return contentRepo; return contentRepo;

View File

@ -36,14 +36,12 @@ import org.apache.nifi.controller.repository.FlowFileRecord;
import org.apache.nifi.controller.repository.FlowFileRepository; import org.apache.nifi.controller.repository.FlowFileRepository;
import org.apache.nifi.controller.repository.SwapSummary; import org.apache.nifi.controller.repository.SwapSummary;
import org.apache.nifi.events.EventReporter; import org.apache.nifi.events.EventReporter;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.flowfile.FlowFilePrioritizer; import org.apache.nifi.flowfile.FlowFilePrioritizer;
import org.apache.nifi.provenance.ProvenanceEventRepository; import org.apache.nifi.provenance.ProvenanceEventRepository;
import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.Timeout;
import org.mockito.Mockito; import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer; import org.mockito.stubbing.Answer;
import java.io.IOException; import java.io.IOException;
@ -70,7 +68,6 @@ import static org.mockito.Mockito.when;
public class TestSocketLoadBalancedFlowFileQueue { public class TestSocketLoadBalancedFlowFileQueue {
private Connection connection;
private FlowFileRepository flowFileRepo; private FlowFileRepository flowFileRepo;
private ContentRepository contentRepo; private ContentRepository contentRepo;
private ProvenanceEventRepository provRepo; private ProvenanceEventRepository provRepo;
@ -86,7 +83,7 @@ public class TestSocketLoadBalancedFlowFileQueue {
@BeforeEach @BeforeEach
public void setup() { public void setup() {
MockFlowFileRecord.resetIdGenerator(); MockFlowFileRecord.resetIdGenerator();
connection = mock(Connection.class); Connection connection = mock(Connection.class);
when(connection.getIdentifier()).thenReturn("unit-test"); when(connection.getIdentifier()).thenReturn("unit-test");
flowFileRepo = mock(FlowFileRepository.class); flowFileRepo = mock(FlowFileRepository.class);
@ -103,21 +100,13 @@ public class TestSocketLoadBalancedFlowFileQueue {
nodeIds.add(createNodeIdentifier("11111111-1111-1111-1111-111111111111")); nodeIds.add(createNodeIdentifier("11111111-1111-1111-1111-111111111111"));
nodeIds.add(createNodeIdentifier("22222222-2222-2222-2222-222222222222")); nodeIds.add(createNodeIdentifier("22222222-2222-2222-2222-222222222222"));
Mockito.doAnswer(new Answer<Set<NodeIdentifier>>() { Mockito.doAnswer((Answer<Set<NodeIdentifier>>) invocation -> new HashSet<>(nodeIds)).when(clusterCoordinator).getNodeIdentifiers();
@Override
public Set<NodeIdentifier> answer(InvocationOnMock invocation) throws Throwable {
return new HashSet<>(nodeIds);
}
}).when(clusterCoordinator).getNodeIdentifiers();
when(clusterCoordinator.getLocalNodeIdentifier()).thenReturn(localNodeIdentifier); when(clusterCoordinator.getLocalNodeIdentifier()).thenReturn(localNodeIdentifier);
doAnswer(new Answer() { doAnswer(invocation -> {
@Override clusterTopologyEventListener = invocation.getArgument(0);
public Object answer(final InvocationOnMock invocation) throws Throwable { return null;
clusterTopologyEventListener = invocation.getArgument(0);
return null;
}
}).when(clusterCoordinator).registerEventListener(Mockito.any(ClusterTopologyEventListener.class)); }).when(clusterCoordinator).registerEventListener(Mockito.any(ClusterTopologyEventListener.class));
final ProcessScheduler scheduler = mock(ProcessScheduler.class); final ProcessScheduler scheduler = mock(ProcessScheduler.class);
@ -148,7 +137,7 @@ public class TestSocketLoadBalancedFlowFileQueue {
assertFalse(queue.isEmpty()); assertFalse(queue.isEmpty());
assertSame(FlowFileAvailability.HEAD_OF_QUEUE_PENALIZED, queue.getFlowFileAvailability()); assertSame(FlowFileAvailability.HEAD_OF_QUEUE_PENALIZED, queue.getFlowFileAvailability());
// Adjust the penalty expiration so that it's not longer penalized. // Adjust the penalty expiration so that it's no longer penalized.
// This will not change the FlowFile Availability, however, because it has already stored the // This will not change the FlowFile Availability, however, because it has already stored the
// Penalty expiration date elsewhere. To trigger that to change, we need to add something to the queue // Penalty expiration date elsewhere. To trigger that to change, we need to add something to the queue
// or remove something. We don't want to remove the data yet, so we add a new FlowFile. // or remove something. We don't want to remove the data yet, so we add a new FlowFile.
@ -174,13 +163,10 @@ public class TestSocketLoadBalancedFlowFileQueue {
@Test @Test
public void testPriorities() { public void testPriorities() {
final FlowFilePrioritizer iValuePrioritizer = new FlowFilePrioritizer() { final FlowFilePrioritizer iValuePrioritizer = (o1, o2) -> {
@Override final int i1 = Integer.parseInt(o1.getAttribute("i"));
public int compare(final FlowFile o1, final FlowFile o2) { final int i2 = Integer.parseInt(o2.getAttribute("i"));
final int i1 = Integer.parseInt(o1.getAttribute("i")); return Integer.compare(i1, i2);
final int i2 = Integer.parseInt(o2.getAttribute("i"));
return Integer.compare(i1, i2);
}
}; };
queue.setPriorities(Collections.singletonList(iValuePrioritizer)); queue.setPriorities(Collections.singletonList(iValuePrioritizer));
@ -205,13 +191,10 @@ public class TestSocketLoadBalancedFlowFileQueue {
@Test @Test
public void testPrioritiesWhenSetBeforeLocalNodeIdDetermined() { public void testPrioritiesWhenSetBeforeLocalNodeIdDetermined() {
final FlowFilePrioritizer iValuePrioritizer = new FlowFilePrioritizer() { final FlowFilePrioritizer iValuePrioritizer = (o1, o2) -> {
@Override final int i1 = Integer.parseInt(o1.getAttribute("i"));
public int compare(final FlowFile o1, final FlowFile o2) { final int i2 = Integer.parseInt(o2.getAttribute("i"));
final int i1 = Integer.parseInt(o1.getAttribute("i")); return Integer.compare(i1, i2);
final int i2 = Integer.parseInt(o2.getAttribute("i"));
return Integer.compare(i1, i2);
}
}; };
final ProcessScheduler scheduler = mock(ProcessScheduler.class); final ProcessScheduler scheduler = mock(ProcessScheduler.class);
@ -525,7 +508,7 @@ public class TestSocketLoadBalancedFlowFileQueue {
@Test @Test
@Timeout(10) @Timeout(10)
public void testChangeInPartitionerTriggersRebalance() throws InterruptedException { public void testChangeInPartitionerTriggersRebalance() {
// Create partitioner that sends first 2 FlowFiles to Partition 0, next 2 to Partition 1, and then next 4 to Partition 3. // Create partitioner that sends first 2 FlowFiles to Partition 0, next 2 to Partition 1, and then next 4 to Partition 3.
queue.setFlowFilePartitioner(new StaticSequencePartitioner(new int[] {0, 1, 0, 1}, false)); queue.setFlowFilePartitioner(new StaticSequencePartitioner(new int[] {0, 1, 0, 1}, false));
@ -545,7 +528,7 @@ public class TestSocketLoadBalancedFlowFileQueue {
@Test @Test
@Timeout(10) @Timeout(10)
public void testDataInRemotePartitionForLocalIdIsMovedToLocalPartition() throws InterruptedException { public void testDataInRemotePartitionForLocalIdIsMovedToLocalPartition() {
nodeIds.clear(); nodeIds.clear();
final NodeIdentifier id1 = createNodeIdentifier(); final NodeIdentifier id1 = createNodeIdentifier();

View File

@ -107,19 +107,16 @@ public class TestStandardLoadBalanceProtocol {
flowController = Mockito.mock(FlowController.class); flowController = Mockito.mock(FlowController.class);
claimContents = new ConcurrentHashMap<>(); claimContents = new ConcurrentHashMap<>();
Mockito.doAnswer(new Answer<ContentClaim>() { Mockito.doAnswer((Answer<ContentClaim>) invocation -> {
@Override final ContentClaim contentClaim = Mockito.mock(ContentClaim.class);
public ContentClaim answer(final InvocationOnMock invocation) throws Throwable { final ResourceClaim resourceClaim = Mockito.mock(ResourceClaim.class);
final ContentClaim contentClaim = Mockito.mock(ContentClaim.class); when(contentClaim.getResourceClaim()).thenReturn(resourceClaim);
final ResourceClaim resourceClaim = Mockito.mock(ResourceClaim.class); return contentClaim;
when(contentClaim.getResourceClaim()).thenReturn(resourceClaim);
return contentClaim;
}
}).when(contentRepo).create(Mockito.anyBoolean()); }).when(contentRepo).create(Mockito.anyBoolean());
Mockito.doAnswer(new Answer<OutputStream>() { Mockito.doAnswer(new Answer<OutputStream>() {
@Override @Override
public OutputStream answer(final InvocationOnMock invocation) throws Throwable { public OutputStream answer(final InvocationOnMock invocation) {
final ContentClaim contentClaim = invocation.getArgument(0); final ContentClaim contentClaim = invocation.getArgument(0);
final ByteArrayOutputStream baos = new ByteArrayOutputStream() { final ByteArrayOutputStream baos = new ByteArrayOutputStream() {
@ -143,36 +140,24 @@ public class TestStandardLoadBalanceProtocol {
when(flowFileQueue.getLoadBalanceCompression()).thenReturn(LoadBalanceCompression.DO_NOT_COMPRESS); when(flowFileQueue.getLoadBalanceCompression()).thenReturn(LoadBalanceCompression.DO_NOT_COMPRESS);
when(connection.getFlowFileQueue()).thenReturn(flowFileQueue); when(connection.getFlowFileQueue()).thenReturn(flowFileQueue);
Mockito.doAnswer(new Answer<Void>() { Mockito.doAnswer((Answer<Void>) invocation -> {
@Override flowFileQueuePutRecords.addAll(invocation.getArgument(0));
public Void answer(final InvocationOnMock invocation) throws Throwable { return null;
flowFileQueuePutRecords.addAll(invocation.getArgument(0));
return null;
}
}).when(flowFileQueue).putAll(anyCollection()); }).when(flowFileQueue).putAll(anyCollection());
Mockito.doAnswer(new Answer<Void>() { Mockito.doAnswer((Answer<Void>) invocation -> {
@Override flowFileQueueReceiveRecords.addAll(invocation.getArgument(0));
public Void answer(final InvocationOnMock invocation) throws Throwable { return null;
flowFileQueueReceiveRecords.addAll(invocation.getArgument(0));
return null;
}
}).when(flowFileQueue).receiveFromPeer(anyCollection()); }).when(flowFileQueue).receiveFromPeer(anyCollection());
Mockito.doAnswer(new Answer<Void>() { Mockito.doAnswer((Answer<Void>) invocation -> {
@Override flowFileRepoUpdateRecords.addAll(invocation.getArgument(0));
public Void answer(final InvocationOnMock invocation) throws Throwable { return null;
flowFileRepoUpdateRecords.addAll(invocation.getArgument(0));
return null;
}
}).when(flowFileRepo).updateRepository(anyCollection()); }).when(flowFileRepo).updateRepository(anyCollection());
Mockito.doAnswer(new Answer<Void>() { Mockito.doAnswer((Answer<Void>) invocation -> {
@Override provRepoUpdateRecords.addAll(invocation.getArgument(0));
public Void answer(final InvocationOnMock invocation) throws Throwable { return null;
provRepoUpdateRecords.addAll(invocation.getArgument(0));
return null;
}
}).when(provenanceRepo).registerEvents(anyCollection()); }).when(provenanceRepo).registerEvents(anyCollection());
} }

View File

@ -51,8 +51,6 @@ import org.apache.nifi.processor.exception.FlowFileAccessException;
import org.apache.nifi.processor.exception.FlowFileHandlingException; import org.apache.nifi.processor.exception.FlowFileHandlingException;
import org.apache.nifi.processor.exception.MissingFlowFileException; import org.apache.nifi.processor.exception.MissingFlowFileException;
import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.io.InputStreamCallback;
import org.apache.nifi.processor.io.OutputStreamCallback;
import org.apache.nifi.processor.io.StreamCallback; import org.apache.nifi.processor.io.StreamCallback;
import org.apache.nifi.provenance.MockProvenanceRepository; import org.apache.nifi.provenance.MockProvenanceRepository;
import org.apache.nifi.provenance.ProvenanceEventRecord; import org.apache.nifi.provenance.ProvenanceEventRecord;
@ -201,18 +199,15 @@ public class StandardProcessSessionIT {
when(connectable.getBackoffMechanism()).thenReturn(BackoffMechanism.PENALIZE_FLOWFILE); when(connectable.getBackoffMechanism()).thenReturn(BackoffMechanism.PENALIZE_FLOWFILE);
when(connectable.getMaxBackoffPeriod()).thenReturn("1 sec"); when(connectable.getMaxBackoffPeriod()).thenReturn("1 sec");
Mockito.doAnswer(new Answer<Set<Connection>>() { Mockito.doAnswer((Answer<Set<Connection>>) invocation -> {
@Override final Object[] arguments = invocation.getArguments();
public Set<Connection> answer(final InvocationOnMock invocation) throws Throwable { final Relationship relationship = (Relationship) arguments[0];
final Object[] arguments = invocation.getArguments(); if (relationship == Relationship.SELF) {
final Relationship relationship = (Relationship) arguments[0]; return Collections.emptySet();
if (relationship == Relationship.SELF) { } else if (relationship == FAKE_RELATIONSHIP || relationship.equals(FAKE_RELATIONSHIP)) {
return Collections.emptySet(); return null;
} else if (relationship == FAKE_RELATIONSHIP || relationship.equals(FAKE_RELATIONSHIP)) { } else {
return null; return new HashSet<>(connList);
} else {
return new HashSet<>(connList);
}
} }
}).when(connectable).getConnections(Mockito.any(Relationship.class)); }).when(connectable).getConnections(Mockito.any(Relationship.class));
@ -264,39 +259,24 @@ public class StandardProcessSessionIT {
when(connection.getFlowFileQueue()).thenReturn(localFlowFileQueue); when(connection.getFlowFileQueue()).thenReturn(localFlowFileQueue);
Mockito.doAnswer(new Answer<Object>() { Mockito.doAnswer((Answer<Object>) invocation -> {
@Override localFlowFileQueue.put((FlowFileRecord) invocation.getArguments()[0]);
public Object answer(InvocationOnMock invocation) throws Throwable { return null;
localFlowFileQueue.put((FlowFileRecord) invocation.getArguments()[0]);
return null;
}
}).when(connection).enqueue(Mockito.any(FlowFileRecord.class)); }).when(connection).enqueue(Mockito.any(FlowFileRecord.class));
Mockito.doAnswer(new Answer<Object>() { Mockito.doAnswer((Answer<Object>) invocation -> {
@Override localFlowFileQueue.putAll((Collection<FlowFileRecord>) invocation.getArguments()[0]);
public Object answer(InvocationOnMock invocation) throws Throwable { return null;
localFlowFileQueue.putAll((Collection<FlowFileRecord>) invocation.getArguments()[0]);
return null;
}
}).when(connection).enqueue(Mockito.any(Collection.class)); }).when(connection).enqueue(Mockito.any(Collection.class));
final Connectable dest = Mockito.mock(Connectable.class); final Connectable dest = Mockito.mock(Connectable.class);
when(connection.getDestination()).thenReturn(dest); when(connection.getDestination()).thenReturn(dest);
when(connection.getSource()).thenReturn(dest); when(connection.getSource()).thenReturn(dest);
Mockito.doAnswer(new Answer<FlowFile>() { Mockito.doAnswer((Answer<FlowFile>) invocation -> localFlowFileQueue.poll(invocation.getArgument(0))).when(connection).poll(any(Set.class));
@Override
public FlowFile answer(InvocationOnMock invocation) throws Throwable {
return localFlowFileQueue.poll(invocation.getArgument(0));
}
}).when(connection).poll(any(Set.class));
Mockito.doAnswer(new Answer<List<FlowFileRecord>>() { Mockito.doAnswer((Answer<List<FlowFileRecord>>) invocation ->
@Override localFlowFileQueue.poll((FlowFileFilter) invocation.getArgument(0), invocation.getArgument(1))).when(connection).poll(any(FlowFileFilter.class), any(Set.class));
public List<FlowFileRecord> answer(InvocationOnMock invocation) throws Throwable {
return localFlowFileQueue.poll((FlowFileFilter) invocation.getArgument(0), invocation.getArgument(1));
}
}).when(connection).poll(any(FlowFileFilter.class), any(Set.class));
Mockito.when(connection.getIdentifier()).thenReturn("conn-uuid"); Mockito.when(connection.getIdentifier()).thenReturn("conn-uuid");
return connection; return connection;
@ -354,7 +334,7 @@ public class StandardProcessSessionIT {
"Expected a FlowFileHandlingException to be thrown because parent was forked and then migrated without children"); "Expected a FlowFileHandlingException to be thrown because parent was forked and then migrated without children");
assertThrows(FlowFileHandlingException.class, assertThrows(FlowFileHandlingException.class,
() -> session.migrate(secondSession, Arrays.asList(flowFile, children.get(0), children.get(1))), () -> session.migrate(secondSession, Arrays.asList(flowFile, children.getFirst(), children.get(1))),
"Expected a FlowFileHandlingException to be thrown because parent was forked and then migrated without children"); "Expected a FlowFileHandlingException to be thrown because parent was forked and then migrated without children");
// Should succeed when migrating all FlowFiles. // Should succeed when migrating all FlowFiles.
@ -407,7 +387,7 @@ public class StandardProcessSessionIT {
assertEquals(1, eventsByType.get(ProvenanceEventType.DROP).size()); assertEquals(1, eventsByType.get(ProvenanceEventType.DROP).size());
assertEquals(1, eventsByType.get(ProvenanceEventType.FORK).size()); assertEquals(1, eventsByType.get(ProvenanceEventType.FORK).size());
final ProvenanceEventRecord fork = eventsByType.get(ProvenanceEventType.FORK).get(0); final ProvenanceEventRecord fork = eventsByType.get(ProvenanceEventType.FORK).getFirst();
assertEquals(clone.getAttribute(CoreAttributes.UUID.key()), fork.getFlowFileUuid()); assertEquals(clone.getAttribute(CoreAttributes.UUID.key()), fork.getFlowFileUuid());
assertEquals(Collections.singletonList(clone.getAttribute(CoreAttributes.UUID.key())), fork.getParentUuids()); assertEquals(Collections.singletonList(clone.getAttribute(CoreAttributes.UUID.key())), fork.getParentUuids());
@ -556,17 +536,15 @@ public class StandardProcessSessionIT {
session.adjustCounter("b", 2, true); session.adjustCounter("b", 2, true);
session.commit(); session.commit();
context.getFlowFileEventRepository().reportTransferEvents(10L).getReportEntries().forEach((k, v) -> { context.getFlowFileEventRepository().reportTransferEvents(10L).getReportEntries().forEach((k, v) -> v.getCounters().forEach((key, value) -> {
v.getCounters().forEach((key, value) -> { if (key.equals("a")) {
if (key.equals("a")) { assertEquals(5L, (long) value);
assertEquals(5L, (long) value); }
}
if (key.equals("b")) { if (key.equals("b")) {
assertEquals(10L, (long) value); assertEquals(10L, (long) value);
} }
}); }));
});
} }
@Test @Test
@ -978,7 +956,7 @@ public class StandardProcessSessionIT {
} }
@Test @Test
public void testAppendToChildThrowsIOExceptionThenRemove() throws IOException { public void testAppendToChildThrowsIOExceptionThenRemove() {
final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder() final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder()
.id(1000L) .id(1000L)
.addAttribute("uuid", "12345678-1234-1234-1234-123456789012") .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
@ -1006,7 +984,7 @@ public class StandardProcessSessionIT {
} }
@Test @Test
public void testWriteForChildThrowsIOExceptionThenRemove() throws IOException { public void testWriteForChildThrowsIOExceptionThenRemove() {
final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder() final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder()
.id(1000L) .id(1000L)
.addAttribute("uuid", "12345678-1234-1234-1234-123456789012") .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
@ -1034,7 +1012,7 @@ public class StandardProcessSessionIT {
} }
@Test @Test
public void testModifyContentWithStreamCallbackHasCorrectSize() throws IOException { public void testModifyContentWithStreamCallbackHasCorrectSize() {
final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder() final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder()
.id(1000L) .id(1000L)
.addAttribute("uuid", "12345678-1234-1234-1234-123456789012") .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
@ -1053,7 +1031,7 @@ public class StandardProcessSessionIT {
} }
@Test @Test
public void testModifyContentWithOutputStreamCallbackHasCorrectSize() throws IOException { public void testModifyContentWithOutputStreamCallbackHasCorrectSize() {
final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder() final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder()
.id(1000L) .id(1000L)
.addAttribute("uuid", "12345678-1234-1234-1234-123456789012") .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
@ -1072,7 +1050,7 @@ public class StandardProcessSessionIT {
} }
@Test @Test
public void testModifyContentWithAppendHasCorrectSize() throws IOException { public void testModifyContentWithAppendHasCorrectSize() {
final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder() final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder()
.id(1000L) .id(1000L)
.addAttribute("uuid", "12345678-1234-1234-1234-123456789012") .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
@ -1107,12 +1085,7 @@ public class StandardProcessSessionIT {
// Get the FlowFile and write to it. This should leave us with 1 content claim. // Get the FlowFile and write to it. This should leave us with 1 content claim.
FlowFile flowFile = session.get(); FlowFile flowFile = session.get();
flowFile = session.putAttribute(flowFile, "filename", "1.txt"); flowFile = session.putAttribute(flowFile, "filename", "1.txt");
flowFile = session.write(flowFile, new OutputStreamCallback() { flowFile = session.write(flowFile, out -> out.write("Hello".getBytes(StandardCharsets.UTF_8)));
@Override
public void process(OutputStream out) throws IOException {
out.write("Hello".getBytes(StandardCharsets.UTF_8));
}
});
session.transfer(flowFile); session.transfer(flowFile);
session.commit(); session.commit();
assertEquals(1, contentRepo.getExistingClaims().size()); assertEquals(1, contentRepo.getExistingClaims().size());
@ -1179,24 +1152,14 @@ public class StandardProcessSessionIT {
FlowFile flowFile = session.get(); FlowFile flowFile = session.get();
assertNotNull(flowFile); assertNotNull(flowFile);
flowFile = session.putAttribute(flowFile, "filename", "1.txt"); flowFile = session.putAttribute(flowFile, "filename", "1.txt");
flowFile = session.write(flowFile, new OutputStreamCallback() { flowFile = session.write(flowFile, out -> out.write("Hello".getBytes(StandardCharsets.UTF_8)));
@Override
public void process(OutputStream out) throws IOException {
out.write("Hello".getBytes(StandardCharsets.UTF_8));
}
});
session.transfer(flowFile); session.transfer(flowFile);
session.commit(); session.commit();
assertEquals(1, contentRepo.getExistingClaims().size()); assertEquals(1, contentRepo.getExistingClaims().size());
flowFile = session.get(); flowFile = session.get();
assertNotNull(flowFile); assertNotNull(flowFile);
flowFile = session.write(flowFile, new OutputStreamCallback() { flowFile = session.write(flowFile, out -> out.write("Hello".getBytes(StandardCharsets.UTF_8)));
@Override
public void process(OutputStream out) throws IOException {
out.write("Hello".getBytes(StandardCharsets.UTF_8));
}
});
session.transfer(flowFile); session.transfer(flowFile);
session.commit(); session.commit();
@ -1225,13 +1188,13 @@ public class StandardProcessSessionIT {
} }
private void assertDisabled(final InputStream inputStream) { private void assertDisabled(final InputStream inputStream) {
assertThrows(FlowFileAccessException.class, () -> inputStream.read(), assertThrows(FlowFileAccessException.class, inputStream::read,
"Expected InputStream to be disabled; was able to call read()"); "Expected InputStream to be disabled; was able to call read()");
assertThrows(FlowFileAccessException.class, () -> inputStream.read(new byte[0]), assertThrows(FlowFileAccessException.class, () -> inputStream.read(new byte[0]),
"Expected InputStream to be disabled; was able to call read(byte[])"); "Expected InputStream to be disabled; was able to call read(byte[])");
assertThrows(FlowFileAccessException.class, () -> inputStream.read(new byte[0], 0, 0), assertThrows(FlowFileAccessException.class, () -> inputStream.read(new byte[0], 0, 0),
"Expected InputStream to be disabled; was able to call read(byte[], int, int)"); "Expected InputStream to be disabled; was able to call read(byte[], int, int)");
assertThrows(FlowFileAccessException.class, () -> inputStream.reset(), assertThrows(FlowFileAccessException.class, inputStream::reset,
"Expected InputStream to be disabled; was able to call reset()"); "Expected InputStream to be disabled; was able to call reset()");
assertThrows(FlowFileAccessException.class, () -> inputStream.skip(1L), assertThrows(FlowFileAccessException.class, () -> inputStream.skip(1L),
"Expected InputStream to be disabled; was able to call skip(long)"); "Expected InputStream to be disabled; was able to call skip(long)");
@ -1249,12 +1212,7 @@ public class StandardProcessSessionIT {
FlowFile flowFile = session.get(); FlowFile flowFile = session.get();
assertNotNull(flowFile); assertNotNull(flowFile);
final AtomicReference<OutputStream> outputStreamHolder = new AtomicReference<>(null); final AtomicReference<OutputStream> outputStreamHolder = new AtomicReference<>(null);
flowFile = session.append(flowFile, new OutputStreamCallback() { flowFile = session.append(flowFile, outputStreamHolder::set);
@Override
public void process(final OutputStream outputStream) throws IOException {
outputStreamHolder.set(outputStream);
}
});
assertDisabled(outputStreamHolder.get()); assertDisabled(outputStreamHolder.get());
} }
@ -1270,12 +1228,7 @@ public class StandardProcessSessionIT {
FlowFile flowFile = session.get(); FlowFile flowFile = session.get();
assertNotNull(flowFile); assertNotNull(flowFile);
flowFile = session.append(flowFile, new OutputStreamCallback() { flowFile = session.append(flowFile, out -> out.write("Hello World".getBytes()));
@Override
public void process(OutputStream out) throws IOException {
out.write("Hello World".getBytes());
}
});
// should be OK // should be OK
ByteArrayOutputStream os = new ByteArrayOutputStream(); ByteArrayOutputStream os = new ByteArrayOutputStream();
@ -1285,7 +1238,7 @@ public class StandardProcessSessionIT {
// should throw ProcessException because of IOException (from processor code) // should throw ProcessException because of IOException (from processor code)
FileOutputStream mock = Mockito.mock(FileOutputStream.class); FileOutputStream mock = Mockito.mock(FileOutputStream.class);
doThrow(new IOException()).when(mock).write((byte[]) notNull(), any(Integer.class), any(Integer.class)); doThrow(new IOException()).when(mock).write(notNull(), any(Integer.class), any(Integer.class));
final FlowFile finalFlowfile = flowFile; final FlowFile finalFlowfile = flowFile;
assertThrows(ProcessException.class, () -> session.exportTo(finalFlowfile, mock)); assertThrows(ProcessException.class, () -> session.exportTo(finalFlowfile, mock));
@ -1320,12 +1273,9 @@ public class StandardProcessSessionIT {
assertNotNull(flowFile); assertNotNull(flowFile);
final AtomicReference<InputStream> inputStreamHolder = new AtomicReference<>(null); final AtomicReference<InputStream> inputStreamHolder = new AtomicReference<>(null);
final AtomicReference<OutputStream> outputStreamHolder = new AtomicReference<>(null); final AtomicReference<OutputStream> outputStreamHolder = new AtomicReference<>(null);
flowFile = session.write(flowFile, new StreamCallback() { flowFile = session.write(flowFile, (input, output) -> {
@Override inputStreamHolder.set(input);
public void process(final InputStream input, final OutputStream output) throws IOException { outputStreamHolder.set(output);
inputStreamHolder.set(input);
outputStreamHolder.set(output);
}
}); });
assertDisabled(inputStreamHolder.get()); assertDisabled(inputStreamHolder.get());
assertDisabled(outputStreamHolder.get()); assertDisabled(outputStreamHolder.get());
@ -1343,17 +1293,12 @@ public class StandardProcessSessionIT {
FlowFile flowFile = session.get(); FlowFile flowFile = session.get();
assertNotNull(flowFile); assertNotNull(flowFile);
final AtomicReference<OutputStream> outputStreamHolder = new AtomicReference<>(null); final AtomicReference<OutputStream> outputStreamHolder = new AtomicReference<>(null);
flowFile = session.write(flowFile, new OutputStreamCallback() { flowFile = session.write(flowFile, outputStreamHolder::set);
@Override
public void process(final OutputStream out) throws IOException {
outputStreamHolder.set(out);
}
});
assertDisabled(outputStreamHolder.get()); assertDisabled(outputStreamHolder.get());
} }
@Test @Test
public void testCreateThenRollbackRemovesContent() throws IOException { public void testCreateThenRollbackRemovesContent() {
final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder() final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder()
.addAttribute("uuid", "12345678-1234-1234-1234-123456789012") .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
@ -1361,10 +1306,7 @@ public class StandardProcessSessionIT {
.build(); .build();
flowFileQueue.put(flowFileRecord); flowFileQueue.put(flowFileRecord);
final StreamCallback nop = new StreamCallback() { final StreamCallback nop = (in, out) -> {
@Override
public void process(InputStream in, OutputStream out) throws IOException {
}
}; };
session.create(); session.create();
@ -1663,35 +1605,20 @@ public class StandardProcessSessionIT {
} }
@Test @Test
public void testAppend() throws IOException { public void testAppend() {
FlowFile ff = session.create(); FlowFile ff = session.create();
ff = session.append(ff, out -> out.write("Hello".getBytes())); ff = session.append(ff, out -> out.write("Hello".getBytes()));
// do not allow the content repo to be read from; this ensures that we are // do not allow the content repo to be read from; this ensures that we are
// not copying the data each time we call append but instead are actually appending to the output stream // not copying the data each time we call append but instead are actually appending to the output stream
contentRepo.disableRead = true; contentRepo.disableRead = true;
ff = session.append(ff, new OutputStreamCallback() { ff = session.append(ff, out -> out.write(", ".getBytes()));
@Override
public void process(OutputStream out) throws IOException {
out.write(", ".getBytes());
}
});
ff = session.append(ff, new OutputStreamCallback() { ff = session.append(ff, out -> out.write("World".getBytes()));
@Override
public void process(OutputStream out) throws IOException {
out.write("World".getBytes());
}
});
contentRepo.disableRead = false; contentRepo.disableRead = false;
final byte[] buff = new byte["Hello, World".getBytes().length]; final byte[] buff = new byte["Hello, World".getBytes().length];
session.read(ff, new InputStreamCallback() { session.read(ff, in -> StreamUtils.fillBuffer(in, buff));
@Override
public void process(final InputStream in) throws IOException {
StreamUtils.fillBuffer(in, buff);
}
});
assertEquals("Hello, World", new String(buff)); assertEquals("Hello, World", new String(buff));
} }
@ -1730,12 +1657,7 @@ public class StandardProcessSessionIT {
public void testAppendDoesNotDecrementContentClaimIfNotNeeded() { public void testAppendDoesNotDecrementContentClaimIfNotNeeded() {
FlowFile flowFile = session.create(); FlowFile flowFile = session.create();
session.append(flowFile, new OutputStreamCallback() { session.append(flowFile, out -> out.write("hello".getBytes()));
@Override
public void process(OutputStream out) throws IOException {
out.write("hello".getBytes());
}
});
final Set<ContentClaim> existingClaims = contentRepo.getExistingClaims(); final Set<ContentClaim> existingClaims = contentRepo.getExistingClaims();
assertEquals(1, existingClaims.size()); assertEquals(1, existingClaims.size());
@ -1760,7 +1682,7 @@ public class StandardProcessSessionIT {
int iterations = 0; int iterations = 0;
@Override @Override
public List<FlowFileRecord> answer(InvocationOnMock invocation) throws Throwable { public List<FlowFileRecord> answer(InvocationOnMock invocation) {
if (iterations++ == 0) { if (iterations++ == 0) {
final Set<FlowFileRecord> expired = invocation.getArgument(1); final Set<FlowFileRecord> expired = invocation.getArgument(1);
expired.add(flowFileRecord); expired.add(flowFileRecord);
@ -1778,7 +1700,7 @@ public class StandardProcessSessionIT {
@Test @Test
@Disabled @Disabled
public void testManyFilesOpened() throws IOException { public void testManyFilesOpened() {
StandardProcessSession[] standardProcessSessions = new StandardProcessSession[100000]; StandardProcessSession[] standardProcessSessions = new StandardProcessSession[100000];
for (int i = 0; i < 70000; i++) { for (int i = 0; i < 70000; i++) {
@ -1787,12 +1709,7 @@ public class StandardProcessSessionIT {
FlowFile flowFile = standardProcessSessions[i].create(); FlowFile flowFile = standardProcessSessions[i].create();
final byte[] buff = new byte["Hello".getBytes().length]; final byte[] buff = new byte["Hello".getBytes().length];
flowFile = standardProcessSessions[i].append(flowFile, new OutputStreamCallback() { flowFile = standardProcessSessions[i].append(flowFile, out -> out.write("Hello".getBytes()));
@Override
public void process(OutputStream out) throws IOException {
out.write("Hello".getBytes());
}
});
try { try {
standardProcessSessions[i].read(flowFile, in -> StreamUtils.fillBuffer(in, buff)); standardProcessSessions[i].read(flowFile, in -> StreamUtils.fillBuffer(in, buff));
@ -1832,10 +1749,7 @@ public class StandardProcessSessionIT {
flowFileQueue.put(flowFileRecord); flowFileQueue.put(flowFileRecord);
FlowFile ff1 = session.get(); FlowFile ff1 = session.get();
ff1 = session.write(ff1, new OutputStreamCallback() { ff1 = session.write(ff1, out -> {
@Override
public void process(OutputStream out) throws IOException {
}
}); });
session.transfer(ff1); session.transfer(ff1);
session.commit(); session.commit();
@ -1869,10 +1783,7 @@ public class StandardProcessSessionIT {
flowFileQueue.put(flowFileRecord); flowFileQueue.put(flowFileRecord);
FlowFile ff1 = session.get(); FlowFile ff1 = session.get();
ff1 = session.write(ff1, new OutputStreamCallback() { ff1 = session.write(ff1, out -> {
@Override
public void process(OutputStream out) throws IOException {
}
}); });
session.transfer(ff1); session.transfer(ff1);
session.commit(); session.commit();
@ -1932,12 +1843,7 @@ public class StandardProcessSessionIT {
assertTrue(flowFileQueue.isActiveQueueEmpty()); assertTrue(flowFileQueue.isActiveQueueEmpty());
assertTrue(flowFileQueue.isUnacknowledgedFlowFile()); assertTrue(flowFileQueue.isUnacknowledgedFlowFile());
final FlowFile modified = session.write(originalFlowFile, new OutputStreamCallback() { final FlowFile modified = session.write(originalFlowFile, out -> out.write("Hello".getBytes()));
@Override
public void process(OutputStream out) throws IOException {
out.write("Hello".getBytes());
}
});
session.transfer(modified); session.transfer(modified);
@ -1968,12 +1874,7 @@ public class StandardProcessSessionIT {
assertTrue(flowFileQueue.isActiveQueueEmpty()); assertTrue(flowFileQueue.isActiveQueueEmpty());
assertTrue(flowFileQueue.isUnacknowledgedFlowFile()); assertTrue(flowFileQueue.isUnacknowledgedFlowFile());
final FlowFile modified = session.write(originalFlowFile, new OutputStreamCallback() { final FlowFile modified = session.write(originalFlowFile, out -> out.write("Hello".getBytes()));
@Override
public void process(OutputStream out) throws IOException {
out.write("Hello".getBytes());
}
});
session.transfer(modified); session.transfer(modified);
@ -1994,12 +1895,7 @@ public class StandardProcessSessionIT {
assertTrue(flowFileQueue.isActiveQueueEmpty()); assertTrue(flowFileQueue.isActiveQueueEmpty());
assertTrue(flowFileQueue.isUnacknowledgedFlowFile()); assertTrue(flowFileQueue.isUnacknowledgedFlowFile());
final FlowFile modifiedRound2 = session.write(originalRound2, new OutputStreamCallback() { final FlowFile modifiedRound2 = session.write(originalRound2, out -> out.write("Hello".getBytes()));
@Override
public void process(OutputStream out) throws IOException {
out.write("Hello".getBytes());
}
});
session.transfer(modifiedRound2); session.transfer(modifiedRound2);
@ -2025,17 +1921,14 @@ public class StandardProcessSessionIT {
assertFalse(events.isEmpty()); assertFalse(events.isEmpty());
assertEquals(1, events.size()); assertEquals(1, events.size());
final ProvenanceEventRecord event = events.get(0); final ProvenanceEventRecord event = events.getFirst();
assertEquals(ProvenanceEventType.CREATE, event.getEventType()); assertEquals(ProvenanceEventType.CREATE, event.getEventType());
} }
@Test @Test
public void testContentModifiedNotEmittedForCreate() throws IOException { public void testContentModifiedNotEmittedForCreate() throws IOException {
FlowFile newFlowFile = session.create(); FlowFile newFlowFile = session.create();
newFlowFile = session.write(newFlowFile, new OutputStreamCallback() { newFlowFile = session.write(newFlowFile, out -> {
@Override
public void process(OutputStream out) throws IOException {
}
}); });
session.transfer(newFlowFile, new Relationship.Builder().name("A").build()); session.transfer(newFlowFile, new Relationship.Builder().name("A").build());
session.commit(); session.commit();
@ -2044,7 +1937,7 @@ public class StandardProcessSessionIT {
assertFalse(events.isEmpty()); assertFalse(events.isEmpty());
assertEquals(1, events.size()); assertEquals(1, events.size());
final ProvenanceEventRecord event = events.get(0); final ProvenanceEventRecord event = events.getFirst();
assertEquals(ProvenanceEventType.CREATE, event.getEventType()); assertEquals(ProvenanceEventType.CREATE, event.getEventType());
} }
@ -2068,7 +1961,7 @@ public class StandardProcessSessionIT {
assertFalse(events.isEmpty()); assertFalse(events.isEmpty());
assertEquals(1, events.size()); assertEquals(1, events.size());
final ProvenanceEventRecord event = events.get(0); final ProvenanceEventRecord event = events.getFirst();
assertEquals(ProvenanceEventType.CONTENT_MODIFIED, event.getEventType()); assertEquals(ProvenanceEventType.CONTENT_MODIFIED, event.getEventType());
} }
@ -2104,7 +1997,7 @@ public class StandardProcessSessionIT {
final List<FlowFile> flowFiles = session.get(100); final List<FlowFile> flowFiles = session.get(100);
// FlowFile Queued times should not match yet // FlowFile Queued times should not match yet
assertNotEquals(flowFiles.get(0).getLastQueueDate(), flowFiles.get(99).getLastQueueDate(), assertNotEquals(flowFiles.getFirst().getLastQueueDate(), flowFiles.get(99).getLastQueueDate(),
"Queued times should not be equal."); "Queued times should not be equal.");
session.transfer(flowFiles, new Relationship.Builder().name("A").build()); session.transfer(flowFiles, new Relationship.Builder().name("A").build());
@ -2113,7 +2006,7 @@ public class StandardProcessSessionIT {
final List<FlowFile> flowFilesUpdated = session.get(100); final List<FlowFile> flowFilesUpdated = session.get(100);
// FlowFile Queued times should match // FlowFile Queued times should match
assertEquals(flowFilesUpdated.get(0).getLastQueueDate(), flowFilesUpdated.get(99).getLastQueueDate(), assertEquals(flowFilesUpdated.getFirst().getLastQueueDate(), flowFilesUpdated.get(99).getLastQueueDate(),
"Queued times should be equal."); "Queued times should be equal.");
} }
@ -2134,19 +2027,14 @@ public class StandardProcessSessionIT {
assertFalse(events.isEmpty()); assertFalse(events.isEmpty());
assertEquals(1, events.size()); assertEquals(1, events.size());
final ProvenanceEventRecord event = events.get(0); final ProvenanceEventRecord event = events.getFirst();
assertEquals(ProvenanceEventType.ATTRIBUTES_MODIFIED, event.getEventType()); assertEquals(ProvenanceEventType.ATTRIBUTES_MODIFIED, event.getEventType());
} }
@Test @Test
public void testReadFromInputStream() throws IOException { public void testReadFromInputStream() throws IOException {
FlowFile flowFile = session.create(); FlowFile flowFile = session.create();
flowFile = session.write(flowFile, new OutputStreamCallback() { flowFile = session.write(flowFile, out -> out.write("hello, world".getBytes()));
@Override
public void process(final OutputStream out) throws IOException {
out.write("hello, world".getBytes());
}
});
try (InputStream in = session.read(flowFile)) { try (InputStream in = session.read(flowFile)) {
final byte[] buffer = new byte[12]; final byte[] buffer = new byte[12];
@ -2161,12 +2049,7 @@ public class StandardProcessSessionIT {
@Test @Test
public void testReadFromInputStreamWithoutClosingThenRemove() throws IOException { public void testReadFromInputStreamWithoutClosingThenRemove() throws IOException {
FlowFile flowFile = session.create(); FlowFile flowFile = session.create();
flowFile = session.write(flowFile, new OutputStreamCallback() { flowFile = session.write(flowFile, out -> out.write("hello, world".getBytes()));
@Override
public void process(final OutputStream out) throws IOException {
out.write("hello, world".getBytes());
}
});
InputStream in = session.read(flowFile); InputStream in = session.read(flowFile);
final byte[] buffer = new byte[12]; final byte[] buffer = new byte[12];
@ -2339,7 +2222,7 @@ public class StandardProcessSessionIT {
assertEquals(-1, in.read()); assertEquals(-1, in.read());
} }
assertTrue(Arrays.equals(new byte[] {'1', '2', '3'}, buff)); assertArrayEquals(new byte[]{'1', '2', '3'}, buff);
newSession.remove(flowFile); newSession.remove(flowFile);
newSession.commit(); newSession.commit();
@ -2394,7 +2277,7 @@ public class StandardProcessSessionIT {
assertEquals(1, repoUpdates.size()); assertEquals(1, repoUpdates.size());
// Should be 4 transient claims because it was written to 5 times. So 4 transient + 1 actual claim. // Should be 4 transient claims because it was written to 5 times. So 4 transient + 1 actual claim.
final RepositoryRecord record = repoUpdates.get(0); final RepositoryRecord record = repoUpdates.getFirst();
assertEquals(RepositoryRecordType.CREATE, record.getType()); assertEquals(RepositoryRecordType.CREATE, record.getType());
final List<ContentClaim> transientClaims = record.getTransientClaims(); final List<ContentClaim> transientClaims = record.getTransientClaims();
assertEquals(4, transientClaims.size()); assertEquals(4, transientClaims.size());
@ -3157,7 +3040,7 @@ public class StandardProcessSessionIT {
} }
@Override @Override
public void close() throws IOException { public void close() {
} }
@Override @Override
@ -3166,7 +3049,7 @@ public class StandardProcessSessionIT {
} }
@Override @Override
public long getMaxFlowFileIdentifier() throws IOException { public long getMaxFlowFileIdentifier() {
return 0L; return 0L;
} }
@ -3197,12 +3080,12 @@ public class StandardProcessSessionIT {
} }
@Override @Override
public long getStorageCapacity() throws IOException { public long getStorageCapacity() {
return 0; return 0;
} }
@Override @Override
public long getUsableStorageSpace() throws IOException { public long getUsableStorageSpace() {
return 0; return 0;
} }
@ -3217,25 +3100,25 @@ public class StandardProcessSessionIT {
} }
@Override @Override
public long loadFlowFiles(QueueProvider queueProvider) throws IOException { public long loadFlowFiles(QueueProvider queueProvider) {
return 0; return 0;
} }
@Override @Override
public Set<String> findQueuesWithFlowFiles(final FlowFileSwapManager flowFileSwapManager) throws IOException { public Set<String> findQueuesWithFlowFiles(final FlowFileSwapManager flowFileSwapManager) {
return Collections.emptySet(); return Collections.emptySet();
} }
@Override @Override
public void swapFlowFilesIn(String swapLocation, List<FlowFileRecord> flowFileRecords, FlowFileQueue queue) throws IOException { public void swapFlowFilesIn(String swapLocation, List<FlowFileRecord> flowFileRecords, FlowFileQueue queue) {
} }
@Override @Override
public void swapFlowFilesOut(List<FlowFileRecord> swappedOut, FlowFileQueue queue, String swapLocation) throws IOException { public void swapFlowFilesOut(List<FlowFileRecord> swappedOut, FlowFileQueue queue, String swapLocation) {
} }
@Override @Override
public void initialize(ResourceClaimManager claimManager) throws IOException { public void initialize(ResourceClaimManager claimManager) {
} }
@Override @Override
@ -3277,7 +3160,7 @@ public class StandardProcessSessionIT {
claimantCounts.put(contentClaim, new AtomicInteger(1)); claimantCounts.put(contentClaim, new AtomicInteger(1));
final Path path = getPath(contentClaim); final Path path = getPath(contentClaim);
final Path parent = path.getParent(); final Path parent = path.getParent();
if (Files.exists(parent) == false) { if (!Files.exists(parent)) {
Files.createDirectories(parent); Files.createDirectories(parent);
} }
Files.createFile(getPath(contentClaim)); Files.createFile(getPath(contentClaim));
@ -3291,7 +3174,7 @@ public class StandardProcessSessionIT {
claimantCounts.put(contentClaim, new AtomicInteger(1)); claimantCounts.put(contentClaim, new AtomicInteger(1));
final Path path = getPath(contentClaim); final Path path = getPath(contentClaim);
final Path parent = path.getParent(); final Path parent = path.getParent();
if (Files.exists(parent) == false) { if (!Files.exists(parent)) {
Files.createDirectories(parent); Files.createDirectories(parent);
} }
@ -3325,7 +3208,7 @@ public class StandardProcessSessionIT {
} }
@Override @Override
public long getContainerCapacity(String containerName) throws IOException { public long getContainerCapacity(String containerName) {
return 0; return 0;
} }
@ -3335,7 +3218,7 @@ public class StandardProcessSessionIT {
} }
@Override @Override
public long getContainerUsableSpace(String containerName) throws IOException { public long getContainerUsableSpace(String containerName) {
return 0; return 0;
} }
@ -3371,7 +3254,7 @@ public class StandardProcessSessionIT {
} }
@Override @Override
public ContentClaim clone(ContentClaim original, boolean lossTolerant) throws IOException { public ContentClaim clone(ContentClaim original, boolean lossTolerant) {
return null; return null;
} }
@ -3404,12 +3287,12 @@ public class StandardProcessSessionIT {
} }
@Override @Override
public long exportTo(ContentClaim claim, Path destination, boolean append) throws IOException { public long exportTo(ContentClaim claim, Path destination, boolean append) {
throw new UnsupportedOperationException(); throw new UnsupportedOperationException();
} }
@Override @Override
public long exportTo(ContentClaim claim, Path destination, boolean append, long offset, long length) throws IOException { public long exportTo(ContentClaim claim, Path destination, boolean append, long offset, long length) {
throw new UnsupportedOperationException(); throw new UnsupportedOperationException();
} }
@ -3421,7 +3304,7 @@ public class StandardProcessSessionIT {
} }
@Override @Override
public long exportTo(ContentClaim claim, OutputStream destination, long offset, long length) throws IOException { public long exportTo(ContentClaim claim, OutputStream destination, long offset, long length) {
throw new UnsupportedOperationException(); throw new UnsupportedOperationException();
} }
@ -3491,12 +3374,12 @@ public class StandardProcessSessionIT {
} }
@Override @Override
public boolean isAccessible(ContentClaim contentClaim) throws IOException { public boolean isAccessible(ContentClaim contentClaim) {
return true; return true;
} }
@Override @Override
public void initialize(ContentRepositoryContext context) throws IOException { public void initialize(ContentRepositoryContext context) {
this.claimManager = context.getResourceClaimManager(); this.claimManager = context.getResourceClaimManager();
} }
} }

View File

@ -19,7 +19,6 @@ package org.apache.nifi.controller.status.history;
import java.util.Date; import java.util.Date;
import java.util.HashMap; import java.util.HashMap;
import java.util.LinkedHashSet; import java.util.LinkedHashSet;
import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
@ -106,31 +105,28 @@ public class StandardStatusSnapshot implements StatusSnapshot {
@Override @Override
public ValueReducer<StatusSnapshot, StatusSnapshot> getValueReducer() { public ValueReducer<StatusSnapshot, StatusSnapshot> getValueReducer() {
return new ValueReducer<StatusSnapshot, StatusSnapshot>() { return values -> {
@Override Date reducedTimestamp = null;
public StatusSnapshot reduce(final List<StatusSnapshot> values) { final Set<MetricDescriptor<?>> allDescriptors = new LinkedHashSet<>(getMetricDescriptors());
Date reducedTimestamp = null;
final Set<MetricDescriptor<?>> allDescriptors = new LinkedHashSet<>(getMetricDescriptors());
for (final StatusSnapshot statusSnapshot : values) { for (final StatusSnapshot statusSnapshot : values) {
if (reducedTimestamp == null) { if (reducedTimestamp == null) {
reducedTimestamp = statusSnapshot.getTimestamp(); reducedTimestamp = statusSnapshot.getTimestamp();
}
allDescriptors.addAll(statusSnapshot.getMetricDescriptors());
} }
allDescriptors.addAll(statusSnapshot.getMetricDescriptors());
final StandardStatusSnapshot reduced = new StandardStatusSnapshot(allDescriptors);
if (reducedTimestamp != null) {
reduced.setTimestamp(reducedTimestamp);
}
for (final MetricDescriptor<?> descriptor : allDescriptors) {
final Long descriptorValue = descriptor.getValueReducer().reduce(values);
reduced.addStatusMetric(descriptor, descriptorValue);
}
return reduced;
} }
final StandardStatusSnapshot reduced = new StandardStatusSnapshot(allDescriptors);
if (reducedTimestamp != null) {
reduced.setTimestamp(reducedTimestamp);
}
for (final MetricDescriptor<?> descriptor : allDescriptors) {
final Long descriptorValue = descriptor.getValueReducer().reduce(values);
reduced.addStatusMetric(descriptor, descriptorValue);
}
return reduced;
}; };
} }
} }

View File

@ -404,14 +404,12 @@ import org.apache.nifi.web.dao.ReportingTaskDAO;
import org.apache.nifi.web.dao.SnippetDAO; import org.apache.nifi.web.dao.SnippetDAO;
import org.apache.nifi.web.dao.UserDAO; import org.apache.nifi.web.dao.UserDAO;
import org.apache.nifi.web.dao.UserGroupDAO; import org.apache.nifi.web.dao.UserGroupDAO;
import org.apache.nifi.web.revision.DeleteRevisionTask;
import org.apache.nifi.web.revision.ExpiredRevisionClaimException; import org.apache.nifi.web.revision.ExpiredRevisionClaimException;
import org.apache.nifi.web.revision.RevisionClaim; import org.apache.nifi.web.revision.RevisionClaim;
import org.apache.nifi.web.revision.RevisionManager; import org.apache.nifi.web.revision.RevisionManager;
import org.apache.nifi.web.revision.RevisionUpdate; import org.apache.nifi.web.revision.RevisionUpdate;
import org.apache.nifi.web.revision.StandardRevisionClaim; import org.apache.nifi.web.revision.StandardRevisionClaim;
import org.apache.nifi.web.revision.StandardRevisionUpdate; import org.apache.nifi.web.revision.StandardRevisionUpdate;
import org.apache.nifi.web.revision.UpdateRevisionTask;
import org.apache.nifi.web.util.PredictionBasedParallelProcessingService; import org.apache.nifi.web.util.PredictionBasedParallelProcessingService;
import org.apache.nifi.web.util.SnippetUtils; import org.apache.nifi.web.util.SnippetUtils;
import org.slf4j.Logger; import org.slf4j.Logger;
@ -1183,27 +1181,24 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
final RevisionUpdate<SnippetDTO> snapshot; final RevisionUpdate<SnippetDTO> snapshot;
try { try {
snapshot = revisionManager.updateRevision(revisionClaim, user, new UpdateRevisionTask<SnippetDTO>() { snapshot = revisionManager.updateRevision(revisionClaim, user, () -> {
@Override // get the updated component
public RevisionUpdate<SnippetDTO> update() { final Snippet snippet = snippetDAO.updateSnippetComponents(snippetDto);
// get the updated component
final Snippet snippet = snippetDAO.updateSnippetComponents(snippetDto);
// drop the snippet // drop the snippet
snippetDAO.dropSnippet(snippet.getId()); snippetDAO.dropSnippet(snippet.getId());
// save updated controller // save updated controller
controllerFacade.save(); controllerFacade.save();
// increment the revisions // increment the revisions
final Set<Revision> updatedRevisions = revisions.stream().map(revision -> { final Set<Revision> updatedRevisions = revisions.stream().map(revision -> {
final Revision currentRevision = revisionManager.getRevision(revision.getComponentId()); final Revision currentRevision = revisionManager.getRevision(revision.getComponentId());
return currentRevision.incrementRevision(revision.getClientId()); return currentRevision.incrementRevision(revision.getClientId());
}).collect(Collectors.toSet()); }).collect(Collectors.toSet());
final SnippetDTO dto = dtoFactory.createSnippetDto(snippet); final SnippetDTO dto = dtoFactory.createSnippetDto(snippet);
return new StandardRevisionUpdate<>(dto, null, updatedRevisions); return new StandardRevisionUpdate<>(dto, null, updatedRevisions);
}
}); });
} catch (final ExpiredRevisionClaimException e) { } catch (final ExpiredRevisionClaimException e) {
throw new InvalidRevisionException("Failed to update Snippet", e); throw new InvalidRevisionException("Failed to update Snippet", e);
@ -1831,30 +1826,26 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
public ScheduleComponentsEntity enableComponents(String processGroupId, ScheduledState state, Map<String, Revision> componentRevisions) { public ScheduleComponentsEntity enableComponents(String processGroupId, ScheduledState state, Map<String, Revision> componentRevisions) {
final NiFiUser user = NiFiUserUtils.getNiFiUser(); final NiFiUser user = NiFiUserUtils.getNiFiUser();
final RevisionUpdate<ScheduleComponentsEntity> updatedComponent = revisionManager.updateRevision(new StandardRevisionClaim(componentRevisions.values()), user, new final RevisionUpdate<ScheduleComponentsEntity> updatedComponent = revisionManager.updateRevision(new StandardRevisionClaim(componentRevisions.values()), user, () -> {
UpdateRevisionTask<ScheduleComponentsEntity>() { // schedule the components
@Override processGroupDAO.enableComponents(processGroupId, state, componentRevisions.keySet());
public RevisionUpdate<ScheduleComponentsEntity> update() {
// schedule the components
processGroupDAO.enableComponents(processGroupId, state, componentRevisions.keySet());
// update the revisions // update the revisions
final Map<String, Revision> updatedRevisions = new HashMap<>(); final Map<String, Revision> updatedRevisions = new HashMap<>();
for (final Revision revision : componentRevisions.values()) { for (final Revision revision : componentRevisions.values()) {
final Revision currentRevision = revisionManager.getRevision(revision.getComponentId()); final Revision currentRevision = revisionManager.getRevision(revision.getComponentId());
updatedRevisions.put(revision.getComponentId(), currentRevision.incrementRevision(revision.getClientId())); updatedRevisions.put(revision.getComponentId(), currentRevision.incrementRevision(revision.getClientId()));
} }
// save // save
controllerFacade.save(); controllerFacade.save();
// gather details for response // gather details for response
final ScheduleComponentsEntity entity = new ScheduleComponentsEntity(); final ScheduleComponentsEntity entity = new ScheduleComponentsEntity();
entity.setId(processGroupId); entity.setId(processGroupId);
entity.setState(state.name()); entity.setState(state.name());
return new StandardRevisionUpdate<>(entity, null, new HashSet<>(updatedRevisions.values())); return new StandardRevisionUpdate<>(entity, null, new HashSet<>(updatedRevisions.values()));
} });
});
return updatedComponent.getComponent(); return updatedComponent.getComponent();
} }
@ -1862,30 +1853,26 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
@Override @Override
public ScheduleComponentsEntity scheduleComponents(final String processGroupId, final ScheduledState state, final Map<String, Revision> componentRevisions) { public ScheduleComponentsEntity scheduleComponents(final String processGroupId, final ScheduledState state, final Map<String, Revision> componentRevisions) {
final NiFiUser user = NiFiUserUtils.getNiFiUser(); final NiFiUser user = NiFiUserUtils.getNiFiUser();
final RevisionUpdate<ScheduleComponentsEntity> updatedComponent = revisionManager.updateRevision(new StandardRevisionClaim(componentRevisions.values()), user, new final RevisionUpdate<ScheduleComponentsEntity> updatedComponent = revisionManager.updateRevision(new StandardRevisionClaim(componentRevisions.values()), user, () -> {
UpdateRevisionTask<ScheduleComponentsEntity>() { // schedule the components
@Override processGroupDAO.scheduleComponents(processGroupId, state, componentRevisions.keySet());
public RevisionUpdate<ScheduleComponentsEntity> update() {
// schedule the components
processGroupDAO.scheduleComponents(processGroupId, state, componentRevisions.keySet());
// update the revisions // update the revisions
final Map<String, Revision> updatedRevisions = new HashMap<>(); final Map<String, Revision> updatedRevisions = new HashMap<>();
for (final Revision revision : componentRevisions.values()) { for (final Revision revision : componentRevisions.values()) {
final Revision currentRevision = revisionManager.getRevision(revision.getComponentId()); final Revision currentRevision = revisionManager.getRevision(revision.getComponentId());
updatedRevisions.put(revision.getComponentId(), currentRevision.incrementRevision(revision.getClientId())); updatedRevisions.put(revision.getComponentId(), currentRevision.incrementRevision(revision.getClientId()));
} }
// save // save
controllerFacade.save(); controllerFacade.save();
// gather details for response // gather details for response
final ScheduleComponentsEntity entity = new ScheduleComponentsEntity(); final ScheduleComponentsEntity entity = new ScheduleComponentsEntity();
entity.setId(processGroupId); entity.setId(processGroupId);
entity.setState(state.name()); entity.setState(state.name());
return new StandardRevisionUpdate<>(entity, null, new HashSet<>(updatedRevisions.values())); return new StandardRevisionUpdate<>(entity, null, new HashSet<>(updatedRevisions.values()));
} });
});
return updatedComponent.getComponent(); return updatedComponent.getComponent();
} }
@ -1894,9 +1881,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
public ActivateControllerServicesEntity activateControllerServices(final String processGroupId, final ControllerServiceState state, final Map<String, Revision> serviceRevisions) { public ActivateControllerServicesEntity activateControllerServices(final String processGroupId, final ControllerServiceState state, final Map<String, Revision> serviceRevisions) {
final NiFiUser user = NiFiUserUtils.getNiFiUser(); final NiFiUser user = NiFiUserUtils.getNiFiUser();
final RevisionUpdate<ActivateControllerServicesEntity> updatedComponent = revisionManager.updateRevision(new StandardRevisionClaim(serviceRevisions.values()), user, final RevisionUpdate<ActivateControllerServicesEntity> updatedComponent = revisionManager.updateRevision(new StandardRevisionClaim(serviceRevisions.values()), user,
new UpdateRevisionTask<ActivateControllerServicesEntity>() { () -> {
@Override
public RevisionUpdate<ActivateControllerServicesEntity> update() {
// schedule the components // schedule the components
processGroupDAO.activateControllerServices(processGroupId, state, serviceRevisions.keySet()); processGroupDAO.activateControllerServices(processGroupId, state, serviceRevisions.keySet());
@ -1915,8 +1900,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
entity.setId(processGroupId); entity.setId(processGroupId);
entity.setState(state.name()); entity.setState(state.name());
return new StandardRevisionUpdate<>(entity, null, new HashSet<>(updatedRevisions.values())); return new StandardRevisionUpdate<>(entity, null, new HashSet<>(updatedRevisions.values()));
} });
});
return updatedComponent.getComponent(); return updatedComponent.getComponent();
} }
@ -2229,24 +2213,21 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
final RevisionClaim claim = new StandardRevisionClaim(revision); final RevisionClaim claim = new StandardRevisionClaim(revision);
final NiFiUser user = NiFiUserUtils.getNiFiUser(); final NiFiUser user = NiFiUserUtils.getNiFiUser();
return revisionManager.deleteRevision(claim, user, new DeleteRevisionTask<D>() { return revisionManager.deleteRevision(claim, user, () -> {
@Override logger.debug("Attempting to delete component {} with claim {}", resource.getIdentifier(), claim);
public D performTask() {
logger.debug("Attempting to delete component {} with claim {}", resource.getIdentifier(), claim);
// run the delete action // run the delete action
deleteAction.run(); deleteAction.run();
// save the flow // save the flow
controllerFacade.save(); controllerFacade.save();
logger.debug("Deletion of component {} was successful", resource.getIdentifier()); logger.debug("Deletion of component {} was successful", resource.getIdentifier());
if (cleanUpPolicies) { if (cleanUpPolicies) {
cleanUpPolicies(resource); cleanUpPolicies(resource);
}
return dto;
} }
return dto;
}); });
} }
@ -2318,21 +2299,18 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
final NiFiUser user = NiFiUserUtils.getNiFiUser(); final NiFiUser user = NiFiUserUtils.getNiFiUser();
final RevisionClaim claim = new StandardRevisionClaim(revisions); final RevisionClaim claim = new StandardRevisionClaim(revisions);
final SnippetDTO dto = revisionManager.deleteRevision(claim, user, new DeleteRevisionTask<SnippetDTO>() { final SnippetDTO dto = revisionManager.deleteRevision(claim, user, () -> {
@Override // delete the components in the snippet
public SnippetDTO performTask() { snippetDAO.deleteSnippetComponents(snippetId);
// delete the components in the snippet
snippetDAO.deleteSnippetComponents(snippetId);
// drop the snippet // drop the snippet
snippetDAO.dropSnippet(snippetId); snippetDAO.dropSnippet(snippetId);
// save // save
controllerFacade.save(); controllerFacade.save();
// create the dto for the snippet that was just removed // create the dto for the snippet that was just removed
return dtoFactory.createSnippetDto(snippet); return dtoFactory.createSnippetDto(snippet);
}
}); });
// clean up component policies // clean up component policies
@ -3012,30 +2990,27 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
final NiFiUser user = NiFiUserUtils.getNiFiUser(); final NiFiUser user = NiFiUserUtils.getNiFiUser();
final RevisionUpdate<ControllerServiceReferencingComponentsEntity> update = revisionManager.updateRevision(claim, user, final RevisionUpdate<ControllerServiceReferencingComponentsEntity> update = revisionManager.updateRevision(claim, user,
new UpdateRevisionTask<ControllerServiceReferencingComponentsEntity>() { () -> {
@Override final Set<ComponentNode> updated = controllerServiceDAO.updateControllerServiceReferencingComponents(controllerServiceId, scheduledState, controllerServiceState);
public RevisionUpdate<ControllerServiceReferencingComponentsEntity> update() { controllerFacade.save();
final Set<ComponentNode> updated = controllerServiceDAO.updateControllerServiceReferencingComponents(controllerServiceId, scheduledState, controllerServiceState);
controllerFacade.save();
final ControllerServiceReference updatedReference = controllerServiceDAO.getControllerService(controllerServiceId).getReferences(); final ControllerServiceReference updatedReference = controllerServiceDAO.getControllerService(controllerServiceId).getReferences();
// get the revisions of the updated components // get the revisions of the updated components
final Map<String, Revision> updatedRevisions = new HashMap<>(); final Map<String, Revision> updatedRevisions = new HashMap<>();
for (final ComponentNode component : updated) { for (final ComponentNode component : updated) {
final Revision currentRevision = revisionManager.getRevision(component.getIdentifier()); final Revision currentRevision = revisionManager.getRevision(component.getIdentifier());
final Revision requestRevision = referenceRevisions.get(component.getIdentifier()); final Revision requestRevision = referenceRevisions.get(component.getIdentifier());
updatedRevisions.put(component.getIdentifier(), currentRevision.incrementRevision(requestRevision.getClientId())); updatedRevisions.put(component.getIdentifier(), currentRevision.incrementRevision(requestRevision.getClientId()));
}
// ensure the revision for all referencing components is included regardless of whether they were updated in this request
for (final ComponentNode component : updatedReference.findRecursiveReferences(ComponentNode.class)) {
updatedRevisions.putIfAbsent(component.getIdentifier(), revisionManager.getRevision(component.getIdentifier()));
}
final ControllerServiceReferencingComponentsEntity entity = createControllerServiceReferencingComponentsEntity(updatedReference, updatedRevisions);
return new StandardRevisionUpdate<>(entity, null, new HashSet<>(updatedRevisions.values()));
} }
// ensure the revision for all referencing components is included regardless of whether they were updated in this request
for (final ComponentNode component : updatedReference.findRecursiveReferences(ComponentNode.class)) {
updatedRevisions.putIfAbsent(component.getIdentifier(), revisionManager.getRevision(component.getIdentifier()));
}
final ControllerServiceReferencingComponentsEntity entity = createControllerServiceReferencingComponentsEntity(updatedReference, updatedRevisions);
return new StandardRevisionUpdate<>(entity, null, new HashSet<>(updatedRevisions.values()));
}); });
return update.getComponent(); return update.getComponent();
@ -4783,21 +4758,18 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
private List<BulletinEntity> pruneAndSortBulletins(final List<BulletinEntity> bulletinEntities, final int maxBulletins) { private List<BulletinEntity> pruneAndSortBulletins(final List<BulletinEntity> bulletinEntities, final int maxBulletins) {
// sort the bulletins // sort the bulletins
Collections.sort(bulletinEntities, new Comparator<BulletinEntity>() { Collections.sort(bulletinEntities, (o1, o2) -> {
@Override if (o1 == null && o2 == null) {
public int compare(BulletinEntity o1, BulletinEntity o2) { return 0;
if (o1 == null && o2 == null) {
return 0;
}
if (o1 == null) {
return 1;
}
if (o2 == null) {
return -1;
}
return -Long.compare(o1.getId(), o2.getId());
} }
if (o1 == null) {
return 1;
}
if (o2 == null) {
return -1;
}
return -Long.compare(o1.getId(), o2.getId());
}); });
// prune the response to only include the max number of bulletins // prune the response to only include the max number of bulletins
@ -6347,29 +6319,26 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
final RevisionClaim revisionClaim = new StandardRevisionClaim(revisions); final RevisionClaim revisionClaim = new StandardRevisionClaim(revisions);
final RevisionUpdate<ProcessGroupDTO> revisionUpdate = revisionManager.updateRevision(revisionClaim, user, new UpdateRevisionTask<ProcessGroupDTO>() { final RevisionUpdate<ProcessGroupDTO> revisionUpdate = revisionManager.updateRevision(revisionClaim, user, () -> {
@Override // update the Process Group
public RevisionUpdate<ProcessGroupDTO> update() { final VersionedExternalFlow externalFlow = createVersionedExternalFlow(proposedFlowSnapshot);
// update the Process Group processGroupDAO.updateProcessGroupFlow(groupId, externalFlow, versionControlInfo, componentIdSeed, verifyNotModified, updateSettings,
final VersionedExternalFlow externalFlow = createVersionedExternalFlow(proposedFlowSnapshot); updateDescendantVersionedFlows);
processGroupDAO.updateProcessGroupFlow(groupId, externalFlow, versionControlInfo, componentIdSeed, verifyNotModified, updateSettings,
updateDescendantVersionedFlows);
// update the revisions // update the revisions
final Set<Revision> updatedRevisions = revisions.stream() final Set<Revision> updatedRevisions = revisions.stream()
.map(rev -> revisionManager.getRevision(rev.getComponentId()).incrementRevision(revision.getClientId())) .map(rev -> revisionManager.getRevision(rev.getComponentId()).incrementRevision(revision.getClientId()))
.collect(Collectors.toSet()); .collect(Collectors.toSet());
// save // save
controllerFacade.save(); controllerFacade.save();
// gather details for response // gather details for response
final ProcessGroupDTO dto = dtoFactory.createProcessGroupDto(processGroup); final ProcessGroupDTO dto = dtoFactory.createProcessGroupDto(processGroup);
final Revision updatedRevision = revisionManager.getRevision(groupId).incrementRevision(revision.getClientId()); final Revision updatedRevision = revisionManager.getRevision(groupId).incrementRevision(revision.getClientId());
final FlowModification lastModification = new FlowModification(updatedRevision, user.getIdentity()); final FlowModification lastModification = new FlowModification(updatedRevision, user.getIdentity());
return new StandardRevisionUpdate<>(dto, lastModification, updatedRevisions); return new StandardRevisionUpdate<>(dto, lastModification, updatedRevisions);
}
}); });
final FlowModification lastModification = revisionUpdate.getLastModification(); final FlowModification lastModification = revisionUpdate.getLastModification();

View File

@ -1613,12 +1613,7 @@ public final class DtoFactory {
dto.setDefaultSchedulingPeriod(defaultSchedulingPeriod); dto.setDefaultSchedulingPeriod(defaultSchedulingPeriod);
// sort a copy of the properties // sort a copy of the properties
final Map<PropertyDescriptor, String> sortedProperties = new TreeMap<>(new Comparator<PropertyDescriptor>() { final Map<PropertyDescriptor, String> sortedProperties = new TreeMap<>((o1, o2) -> Collator.getInstance(Locale.US).compare(o1.getName(), o2.getName()));
@Override
public int compare(final PropertyDescriptor o1, final PropertyDescriptor o2) {
return Collator.getInstance(Locale.US).compare(o1.getName(), o2.getName());
}
});
sortedProperties.putAll(reportingTaskNode.getRawPropertyValues()); sortedProperties.putAll(reportingTaskNode.getRawPropertyValues());
// get the property order from the reporting task // get the property order from the reporting task
@ -1814,12 +1809,7 @@ public final class DtoFactory {
dto.setVersionedComponentId(controllerServiceNode.getVersionedComponentId().orElse(null)); dto.setVersionedComponentId(controllerServiceNode.getVersionedComponentId().orElse(null));
// sort a copy of the properties // sort a copy of the properties
final Map<PropertyDescriptor, String> sortedProperties = new TreeMap<>(new Comparator<PropertyDescriptor>() { final Map<PropertyDescriptor, String> sortedProperties = new TreeMap<>((o1, o2) -> Collator.getInstance(Locale.US).compare(o1.getName(), o2.getName()));
@Override
public int compare(final PropertyDescriptor o1, final PropertyDescriptor o2) {
return Collator.getInstance(Locale.US).compare(o1.getName(), o2.getName());
}
});
sortedProperties.putAll(controllerServiceNode.getRawPropertyValues()); sortedProperties.putAll(controllerServiceNode.getRawPropertyValues());
// get the property order from the controller service // get the property order from the controller service
@ -1942,12 +1932,7 @@ public final class DtoFactory {
} }
// process properties unconditionally since dynamic properties are available here and not in getPropertyDescriptors // process properties unconditionally since dynamic properties are available here and not in getPropertyDescriptors
final Map<PropertyDescriptor, String> sortedProperties = new TreeMap<>(new Comparator<PropertyDescriptor>() { final Map<PropertyDescriptor, String> sortedProperties = new TreeMap<>((o1, o2) -> Collator.getInstance(Locale.US).compare(o1.getName(), o2.getName()));
@Override
public int compare(final PropertyDescriptor o1, final PropertyDescriptor o2) {
return Collator.getInstance(Locale.US).compare(o1.getName(), o2.getName());
}
});
sortedProperties.putAll(component.getRawPropertyValues()); sortedProperties.putAll(component.getRawPropertyValues());
final Map<PropertyDescriptor, String> orderedProperties = new LinkedHashMap<>(); final Map<PropertyDescriptor, String> orderedProperties = new LinkedHashMap<>();
@ -3373,12 +3358,7 @@ public final class DtoFactory {
} }
// sort the relationships // sort the relationships
relationships.sort(new Comparator<RelationshipDTO>() { relationships.sort((r1, r2) -> Collator.getInstance(Locale.US).compare(r1.getName(), r2.getName()));
@Override
public int compare(final RelationshipDTO r1, final RelationshipDTO r2) {
return Collator.getInstance(Locale.US).compare(r1.getName(), r2.getName());
}
});
// set the relationships // set the relationships
dto.setRelationships(relationships); dto.setRelationships(relationships);
@ -3413,28 +3393,25 @@ public final class DtoFactory {
*/ */
public BulletinBoardDTO createBulletinBoardDto(final List<BulletinEntity> bulletins) { public BulletinBoardDTO createBulletinBoardDto(final List<BulletinEntity> bulletins) {
// sort the bulletins // sort the bulletins
Collections.sort(bulletins, new Comparator<BulletinEntity>() { Collections.sort(bulletins, (bulletin1, bulletin2) -> {
@Override if (bulletin1 == null && bulletin2 == null) {
public int compare(final BulletinEntity bulletin1, final BulletinEntity bulletin2) { return 0;
if (bulletin1 == null && bulletin2 == null) { } else if (bulletin1 == null) {
return 0; return 1;
} else if (bulletin1 == null) { } else if (bulletin2 == null) {
return 1; return -1;
} else if (bulletin2 == null) { }
return -1;
}
final Date timestamp1 = bulletin1.getTimestamp(); final Date timestamp1 = bulletin1.getTimestamp();
final Date timestamp2 = bulletin2.getTimestamp(); final Date timestamp2 = bulletin2.getTimestamp();
if (timestamp1 == null && timestamp2 == null) { if (timestamp1 == null && timestamp2 == null) {
return 0; return 0;
} else if (timestamp1 == null) { } else if (timestamp1 == null) {
return 1; return 1;
} else if (timestamp2 == null) { } else if (timestamp2 == null) {
return -1; return -1;
} else { } else {
return timestamp1.compareTo(timestamp2); return timestamp1.compareTo(timestamp2);
}
} }
}); });
@ -4913,12 +4890,7 @@ public final class DtoFactory {
// populate node events // populate node events
final List<NodeEvent> nodeEvents = new ArrayList<>(events); final List<NodeEvent> nodeEvents = new ArrayList<>(events);
Collections.sort(nodeEvents, new Comparator<NodeEvent>() { Collections.sort(nodeEvents, (event1, event2) -> new Date(event2.getTimestamp()).compareTo(new Date(event1.getTimestamp())));
@Override
public int compare(final NodeEvent event1, final NodeEvent event2) {
return new Date(event2.getTimestamp()).compareTo(new Date(event1.getTimestamp()));
}
});
// create the node event dtos // create the node event dtos
final List<NodeEventDTO> nodeEventDtos = new ArrayList<>(); final List<NodeEventDTO> nodeEventDtos = new ArrayList<>();

View File

@ -1650,12 +1650,7 @@ public class ControllerFacade implements Authorizable {
// only include all details if not summarizing // only include all details if not summarizing
if (!summarize) { if (!summarize) {
// convert the attributes // convert the attributes
final Comparator<AttributeDTO> attributeComparator = new Comparator<AttributeDTO>() { final Comparator<AttributeDTO> attributeComparator = (a1, a2) -> Collator.getInstance(Locale.US).compare(a1.getName(), a2.getName());
@Override
public int compare(AttributeDTO a1, AttributeDTO a2) {
return Collator.getInstance(Locale.US).compare(a1.getName(), a2.getName());
}
};
final SortedSet<AttributeDTO> attributes = new TreeSet<>(attributeComparator); final SortedSet<AttributeDTO> attributes = new TreeSet<>(attributeComparator);

View File

@ -225,7 +225,7 @@ public class StandardProcessorTestRunner implements TestRunner {
executorService.shutdown(); executorService.shutdown();
try { try {
executorService.awaitTermination(runWait, TimeUnit.MILLISECONDS); executorService.awaitTermination(runWait, TimeUnit.MILLISECONDS);
} catch (final InterruptedException e1) { } catch (final InterruptedException ignored) {
} }
} }
@ -242,7 +242,7 @@ public class StandardProcessorTestRunner implements TestRunner {
unscheduledRun = true; unscheduledRun = true;
unSchedule(); unSchedule();
} }
} catch (final InterruptedException | ExecutionException e) { } catch (final InterruptedException | ExecutionException ignored) {
} }
} }
@ -324,22 +324,12 @@ public class StandardProcessorTestRunner implements TestRunner {
@Override @Override
public void assertAllFlowFilesContainAttribute(final String attributeName) { public void assertAllFlowFilesContainAttribute(final String attributeName) {
assertAllFlowFiles(new FlowFileValidator() { assertAllFlowFiles(f -> Assertions.assertNotNull(f.getAttribute(attributeName)));
@Override
public void assertFlowFile(FlowFile f) {
Assertions.assertNotNull(f.getAttribute(attributeName));
}
});
} }
@Override @Override
public void assertAllFlowFilesContainAttribute(final Relationship relationship, final String attributeName) { public void assertAllFlowFilesContainAttribute(final Relationship relationship, final String attributeName) {
assertAllFlowFiles(relationship, new FlowFileValidator() { assertAllFlowFiles(relationship, f -> Assertions.assertNotNull(f.getAttribute(attributeName)));
@Override
public void assertFlowFile(FlowFile f) {
Assertions.assertNotNull(f.getAttribute(attributeName));
}
});
} }
@Override @Override

View File

@ -23,7 +23,6 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator; import java.util.Comparator;
import java.util.List; import java.util.List;
import java.util.function.Supplier; import java.util.function.Supplier;
@ -128,12 +127,7 @@ public class IdentityMappingUtil {
} }
// sort the list by the key so users can control the ordering in nifi-registry.properties // sort the list by the key so users can control the ordering in nifi-registry.properties
Collections.sort(mappings, new Comparator<IdentityMapping>() { mappings.sort(Comparator.comparing(IdentityMapping::getKey));
@Override
public int compare(IdentityMapping m1, IdentityMapping m2) {
return m1.getKey().compareTo(m2.getKey());
}
});
return mappings; return mappings;
} }

View File

@ -39,7 +39,6 @@ import java.net.SocketTimeoutException;
import java.nio.charset.StandardCharsets; import java.nio.charset.StandardCharsets;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator; import java.util.Comparator;
import java.util.List; import java.util.List;
import java.util.UUID; import java.util.UUID;
@ -173,37 +172,34 @@ public class BootstrapListener {
logger.debug("Received connection from Bootstrap"); logger.debug("Received connection from Bootstrap");
socket.setSoTimeout(5000); socket.setSoTimeout(5000);
executor.submit(new Runnable() { executor.submit(() -> {
@Override try {
public void run() { final BootstrapRequest request = readRequest(socket.getInputStream());
try { final BootstrapRequest.RequestType requestType = request.getRequestType();
final BootstrapRequest request = readRequest(socket.getInputStream());
final BootstrapRequest.RequestType requestType = request.getRequestType();
switch (requestType) { switch (requestType) {
case PING: case PING:
logger.debug("Received PING request from Bootstrap; responding"); logger.debug("Received PING request from Bootstrap; responding");
echoPing(socket.getOutputStream()); echoPing(socket.getOutputStream());
logger.debug("Responded to PING request from Bootstrap"); logger.debug("Responded to PING request from Bootstrap");
break; break;
case SHUTDOWN: case SHUTDOWN:
logger.info("Received SHUTDOWN request from Bootstrap"); logger.info("Received SHUTDOWN request from Bootstrap");
echoShutdown(socket.getOutputStream()); echoShutdown(socket.getOutputStream());
nifi.shutdownHook(); nifi.shutdownHook();
return; return;
case DUMP: case DUMP:
logger.info("Received DUMP request from Bootstrap"); logger.info("Received DUMP request from Bootstrap");
writeDump(socket.getOutputStream()); writeDump(socket.getOutputStream());
break; break;
} }
} catch (final Throwable t) { } catch (final Throwable t) {
logger.error("Failed to process request from Bootstrap", t); logger.error("Failed to process request from Bootstrap", t);
} finally { } finally {
try { try {
socket.close(); socket.close();
} catch (final IOException ioe) { } catch (final IOException ioe) {
logger.warn("Failed to close socket to Bootstrap", ioe); logger.warn("Failed to close socket to Bootstrap", ioe);
}
} }
} }
}); });
@ -226,12 +222,7 @@ public class BootstrapListener {
for (final ThreadInfo info : infos) { for (final ThreadInfo info : infos) {
sortedInfos.add(info); sortedInfos.add(info);
} }
Collections.sort(sortedInfos, new Comparator<ThreadInfo>() { sortedInfos.sort(Comparator.comparing(o -> o.getThreadName().toLowerCase()));
@Override
public int compare(ThreadInfo o1, ThreadInfo o2) {
return o1.getThreadName().toLowerCase().compareTo(o2.getThreadName().toLowerCase());
}
});
final StringBuilder sb = new StringBuilder(); final StringBuilder sb = new StringBuilder();
for (final ThreadInfo info : sortedInfos) { for (final ThreadInfo info : sortedInfos) {
@ -368,11 +359,11 @@ public class BootstrapListener {
private static class BootstrapRequest { private static class BootstrapRequest {
public static enum RequestType { public enum RequestType {
SHUTDOWN, SHUTDOWN,
DUMP, DUMP,
PING; PING
} }
private final RequestType requestType; private final RequestType requestType;