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.Collection;
import java.util.Comparator;
import java.util.Date;
import java.util.HashMap;
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
final List<ProvenanceEventRecord> sortedRecords = new ArrayList<>(relevantRecords);
sortedRecords.sort(new Comparator<ProvenanceEventRecord>() {
@Override
public int compare(final ProvenanceEventRecord o1, final ProvenanceEventRecord o2) {
// Sort on Event Time, then Event ID.
final int eventTimeComparison = Long.compare(o1.getEventTime(), o2.getEventTime());
if (eventTimeComparison == 0) {
return Long.compare(o1.getEventId(), o2.getEventId());
} else {
return eventTimeComparison;
}
sortedRecords.sort((o1, o2) -> {
// Sort on Event Time, then Event ID.
final int eventTimeComparison = Long.compare(o1.getEventTime(), o2.getEventTime());
if (eventTimeComparison == 0) {
return Long.compare(o1.getEventId(), o2.getEventId());
} else {
return eventTimeComparison;
}
});

View File

@ -23,7 +23,6 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.List;
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
Collections.sort(mappings, new Comparator<IdentityMapping>() {
@Override
public int compare(IdentityMapping m1, IdentityMapping m2) {
return m1.getKey().compareTo(m2.getKey());
}
});
mappings.sort(Comparator.comparing(IdentityMapping::getKey));
return mappings;
}

View File

@ -163,12 +163,7 @@ public class RingBuffer<T> {
}
public List<T> asList() {
return getSelectedElements(new Filter<T>() {
@Override
public boolean select(final T value) {
return true;
}
});
return getSelectedElements(value -> true);
}
public T getOldestElement() {
@ -252,7 +247,7 @@ public class RingBuffer<T> {
}
}
public static interface Filter<S> {
public interface Filter<S> {
boolean select(S value);
}
@ -262,7 +257,7 @@ public class RingBuffer<T> {
*
* @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
@ -273,9 +268,9 @@ public class RingBuffer<T> {
boolean evaluate(S value);
}
public static enum IterationDirection {
public enum IterationDirection {
FORWARD,
BACKWARD;
BACKWARD
}
}

View File

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

View File

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

View File

@ -17,7 +17,6 @@
package org.apache.nifi.util.db;
import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import java.io.ByteArrayInputStream;
@ -38,12 +37,7 @@ class JdbcCommonTestUtils {
when(rs.getMetaData()).thenReturn(metadata);
final AtomicInteger counter = new AtomicInteger(1);
Mockito.doAnswer(new Answer<Boolean>() {
@Override
public Boolean answer(InvocationOnMock invocation) throws Throwable {
return counter.getAndDecrement() > 0;
}
}).when(rs).next();
Mockito.doAnswer((Answer<Boolean>) invocation -> counter.getAndDecrement() > 0).when(rs).next();
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) {
Collections.sort(ranges, new Comparator<Range>() {
@Override
public int compare(final Range o1, final Range o2) {
return Long.compare(o1.getStart(), o2.getStart());
}
});
ranges.sort(Comparator.comparingLong(Range::getStart));
for (int i = 0; i < ranges.size(); 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 {
try {
return ugi.doAs(new PrivilegedExceptionAction<FileSystem>() {
@Override
public FileSystem run() throws Exception {
return FileSystem.get(config);
}
});
return ugi.doAs((PrivilegedExceptionAction<FileSystem>) () -> FileSystem.get(config));
} catch (final InterruptedException 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,
final String charset, final int batchSize, final Consumer<List<JMSResponse>> messageSetConsumer) {
doWithJmsTemplate(destinationName, durable, shared, subscriptionName, messageSelector, new MessageReceiver() {
@Override
public void consume(Session session, MessageConsumer messageConsumer) throws JMSException {
final List<JMSResponse> jmsResponses = new ArrayList<>();
int batchCounter = 0;
doWithJmsTemplate(destinationName, durable, shared, subscriptionName, messageSelector, (session, messageConsumer) -> {
final List<JMSResponse> jmsResponses = new ArrayList<>();
int batchCounter = 0;
JMSResponse response;
while (batchCounter < batchSize && (response = receiveMessage(session, messageConsumer, charset, errorQueueName)) != null) {
response.setBatchOrder(batchCounter);
jmsResponses.add(response);
batchCounter++;
}
JMSResponse response;
while (batchCounter < batchSize && (response = receiveMessage(session, messageConsumer, charset, errorQueueName)) != null) {
response.setBatchOrder(batchCounter);
jmsResponses.add(response);
batchCounter++;
}
if (!jmsResponses.isEmpty()) {
// 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
// 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
// the JMSResponse.
messageSetConsumer.accept(jmsResponses);
}
if (!jmsResponses.isEmpty()) {
// 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
// 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
// the JMSResponse.
messageSetConsumer.accept(jmsResponses);
}
});
}
private void doWithJmsTemplate(String destinationName, boolean durable, boolean shared, String subscriptionName, String messageSelector, MessageReceiver messageReceiver) {
this.jmsTemplate.execute(new SessionCallback<Void>() {
@Override
public Void doInJms(final Session session) throws JMSException {
this.jmsTemplate.execute((SessionCallback<Void>) session -> {
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 {
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 {
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;
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);
}
return null;
JmsUtils.closeMessageConsumer(messageConsumer);
throw e;
}
return null;
}, 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.Test;
import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import java.io.IOException;
@ -81,6 +80,7 @@ import java.util.stream.Collectors;
import java.util.stream.Stream;
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.assertNull;
import static org.junit.jupiter.api.Assertions.assertTrue;
@ -237,11 +237,11 @@ class TestQueryNiFiReportingTask {
List<Map<String, Object>> rows = mockRecordSinkService.getRows();
assertEquals(4, rows.size());
// 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
Object id = row.get("id");
assertTrue(id instanceof String);
assertInstanceOf(String.class, id);
assertEquals("nested", id);
assertEquals(1001, row.get("queuedCount"));
// Validate the second row
@ -483,11 +483,11 @@ class TestQueryNiFiReportingTask {
List<Map<String, Object>> rows = mockRecordSinkService.getRows();
assertEquals(1, rows.size());
Map<String, Object> row = rows.get(0);
Map<String, Object> row = rows.getFirst();
assertEquals(11, row.size());
assertTrue(row.get(MetricNames.JVM_DAEMON_THREAD_COUNT.replace(".", "_")) instanceof Integer);
assertTrue(row.get(MetricNames.JVM_HEAP_USAGE.replace(".", "_")) instanceof Double);
assertInstanceOf(Integer.class, row.get(MetricNames.JVM_DAEMON_THREAD_COUNT.replace(".", "_")));
assertInstanceOf(Double.class, row.get(MetricNames.JVM_HEAP_USAGE.replace(".", "_")));
}
@Test
@ -501,7 +501,7 @@ class TestQueryNiFiReportingTask {
List<Map<String, Object>> rows = mockRecordSinkService.getRows();
assertEquals(4, rows.size());
// Validate the first row
Map<String, Object> row = rows.get(0);
Map<String, Object> row = rows.getFirst();
assertEquals(26, row.size());
assertEquals(1L, row.get("bytesRead"));
// Validate the second row
@ -538,7 +538,7 @@ class TestQueryNiFiReportingTask {
List<Map<String, Object>> rows = mockRecordSinkService.getRows();
assertEquals(1001, rows.size());
// Validate the first row
Map<String, Object> row = rows.get(0);
Map<String, Object> row = rows.getFirst();
assertEquals(24, row.size());
// Verify the first row contents
final Long firstEventId = (Long) row.get("eventId");
@ -549,11 +549,11 @@ class TestQueryNiFiReportingTask {
assertNull(row.get("previousContentPath"));
Object o = row.get("previousAttributes");
assertTrue(o instanceof Map);
assertInstanceOf(Map.class, o);
Map<String, String> previousAttributes = (Map<String, String>) o;
assertEquals("A", previousAttributes.get("test.value"));
o = row.get("updatedAttributes");
assertTrue(o instanceof Map);
assertInstanceOf(Map.class, o);
Map<String, String> updatedAttributes = (Map<String, String>) o;
assertEquals("B", updatedAttributes.get("test.value"));
@ -595,7 +595,7 @@ class TestQueryNiFiReportingTask {
}
// Validate the first row
Map<String, Object> row = rows.get(0);
Map<String, Object> row = rows.getFirst();
assertEquals(14, row.size());
assertNotNull(row.get("bulletinId"));
assertEquals("controller", row.get("bulletinCategory"));
@ -625,7 +625,7 @@ class TestQueryNiFiReportingTask {
List<Map<String, Object>> rows = mockRecordSinkService.getRows();
assertEquals(1, rows.size());
// Validate the first row
Map<String, Object> row = rows.get(0);
Map<String, Object> row = rows.getFirst();
assertEquals(22, row.size());
// Verify the first row contents
assertEquals(123, row.get("actionId"));
@ -720,13 +720,10 @@ class TestQueryNiFiReportingTask {
Mockito.when(eventAccess.getProvenanceRepository()).thenReturn(mockProvenanceRepository);
try {
Mockito.when(eventAccess.getProvenanceEvents(anyLong(), anyInt())).thenAnswer(new Answer<List<ProvenanceEventRecord>>() {
@Override
public List<ProvenanceEventRecord> answer(final InvocationOnMock invocation) throws Throwable {
final long startEventId = invocation.getArgument(0);
final int max = invocation.getArgument(1);
return mockProvenanceRepository.getEvents(startEventId, max);
}
Mockito.when(eventAccess.getProvenanceEvents(anyLong(), anyInt())).thenAnswer((Answer<List<ProvenanceEventRecord>>) invocation -> {
final long startEventId = invocation.getArgument(0);
final int max = invocation.getArgument(1);
return mockProvenanceRepository.getEvents(startEventId, max);
});
} catch (final IOException e) {
// 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.Relationship;
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.processors.standard.util.ArgumentUtils;
@ -57,7 +56,6 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
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
// or until the specified amount of time
FlowFile flowFile = session.create();
flowFile = session.write(flowFile, new OutputStreamCallback() {
@Override
public void process(final OutputStream flowFileOut) throws IOException {
try (final OutputStream out = new BufferedOutputStream(flowFileOut)) {
proxyOut.setDelegate(out);
flowFile = session.write(flowFile, flowFileOut -> {
try (final OutputStream out = new BufferedOutputStream(flowFileOut)) {
proxyOut.setDelegate(out);
if (batchNanos == null) {
// we are not creating batches; wait until process terminates.
// NB!!! Maybe get(long timeout, TimeUnit unit) should
// be used to avoid waiting forever.
try {
longRunningProcess.get();
} catch (final InterruptedException ie) {
// Ignore
} catch (final ExecutionException ee) {
getLogger().error("Process execution failed", ee.getCause());
}
} else {
// wait the allotted amount of time.
try {
TimeUnit.NANOSECONDS.sleep(batchNanos);
} catch (final InterruptedException ie) {
// Ignore
}
if (batchNanos == null) {
// we are not creating batches; wait until process terminates.
// NB!!! Maybe get(long timeout, TimeUnit unit) should
// be used to avoid waiting forever.
try {
longRunningProcess.get();
} catch (final InterruptedException ie) {
// Ignore
} catch (final ExecutionException ee) {
getLogger().error("Process execution failed", ee.getCause());
}
} else {
// wait the allotted amount of time.
try {
TimeUnit.NANOSECONDS.sleep(batchNanos);
} catch (final InterruptedException ie) {
// 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
if (!redirectErrorStream) {
executor.submit(new Runnable() {
@Override
public void run() {
try (final BufferedReader reader = new BufferedReader(new InputStreamReader(externalProcess.getErrorStream()))) {
reader.lines().filter(line -> line != null && line.length() > 0).forEach(getLogger()::warn);
} catch (final IOException ioe) {
}
executor.submit(() -> {
try (final BufferedReader reader = new BufferedReader(new InputStreamReader(externalProcess.getErrorStream()))) {
reader.lines().filter(line -> line != null && !line.isEmpty()).forEach(getLogger()::warn);
} catch (final IOException ignored) {
}
});
}
// Submit task to read output of Process and write to FlowFile.
failure = new AtomicBoolean(false);
final Future<?> future = executor.submit(new Callable<Object>() {
@Override
public Object call() throws IOException {
try {
if (batchNanos == null) {
// if we aren't batching, just copy the stream from the
// process to the flowfile.
try (final BufferedInputStream bufferedIn = new BufferedInputStream(externalProcess.getInputStream())) {
final byte[] buffer = new byte[4096];
int len;
while ((len = bufferedIn.read(buffer)) > 0) {
final Future<?> future = executor.submit(() -> {
try {
if (batchNanos == null) {
// if we aren't batching, just copy the stream from the
// process to the flowfile.
try (final BufferedInputStream bufferedIn = new BufferedInputStream(externalProcess.getInputStream())) {
final byte[] buffer = new byte[4096];
int len;
while ((len = bufferedIn.read(buffer)) > 0) {
// NB!!!! Maybe all data should be read from
// input stream in case of !isScheduled() to
// avoid subprocess deadlock?
// (we just don't write data to proxyOut)
// Or because we don't use this subprocess
// anymore anyway, we don't care?
if (!isScheduled()) {
return null;
}
proxyOut.write(buffer, 0, len);
// NB!!!! Maybe all data should be read from
// input stream in case of !isScheduled() to
// avoid subprocess deadlock?
// (we just don't write data to proxyOut)
// Or because we don't use this subprocess
// anymore anyway, we don't care?
if (!isScheduled()) {
return null;
}
}
} 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) {
if (!isScheduled()) {
return null;
}
proxyOut.write((line + "\n").getBytes(StandardCharsets.UTF_8));
}
proxyOut.write(buffer, 0, len);
}
}
} catch (final IOException ioe) {
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();
} 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) {
if (!isScheduled()) {
return null;
}
proxyOut.write((line + "\n").getBytes(StandardCharsets.UTF_8));
}
}
}
return null;
} catch (final IOException ioe) {
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;

View File

@ -57,8 +57,6 @@ import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.Relationship;
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.bin.Bin;
import org.apache.nifi.processor.util.bin.BinFiles;
@ -659,44 +657,41 @@ public class MergeContent extends BinFiles {
FlowFile bundle = session.create(bin.getContents());
final AtomicReference<String> bundleMimeTypeRef = new AtomicReference<>(null);
try {
bundle = session.write(bundle, new OutputStreamCallback() {
@Override
public void process(final OutputStream out) throws IOException {
final byte[] header = getDelimiterContent(context, contents, HEADER);
if (header != null) {
out.write(header);
}
bundle = session.write(bundle, out -> {
final byte[] header = getDelimiterContent(context, contents, HEADER);
if (header != null) {
out.write(header);
}
final byte[] demarcator = getDelimiterContent(context, contents, DEMARCATOR);
final byte[] demarcator = getDelimiterContent(context, contents, DEMARCATOR);
boolean isFirst = true;
final Iterator<FlowFile> itr = contents.iterator();
while (itr.hasNext()) {
final FlowFile flowFile = itr.next();
bin.getSession().read(flowFile, in -> StreamUtils.copy(in, out));
boolean isFirst = true;
final Iterator<FlowFile> itr = contents.iterator();
while (itr.hasNext()) {
final FlowFile flowFile = itr.next();
bin.getSession().read(flowFile, in -> StreamUtils.copy(in, out));
if (itr.hasNext()) {
if (demarcator != null) {
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);
}
if (itr.hasNext()) {
if (demarcator != null) {
out.write(demarcator);
}
}
final byte[] footer = getDelimiterContent(context, contents, FOOTER);
if (footer != null) {
out.write(footer);
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);
if (footer != null) {
out.write(footer);
}
});
} catch (final Exception e) {
removeFlowFileFromSession(session, bundle, context);
@ -802,49 +797,46 @@ public class MergeContent extends BinFiles {
try {
bundle = session.putAttribute(bundle, CoreAttributes.FILENAME.key(), createFilename(contents) + ".tar");
bundle = session.write(bundle, new OutputStreamCallback() {
@Override
public void process(final OutputStream rawOut) throws IOException {
try (final OutputStream bufferedOut = new BufferedOutputStream(rawOut);
final TarArchiveOutputStream out = new TarArchiveOutputStream(bufferedOut)) {
bundle = session.write(bundle, rawOut -> {
try (final OutputStream bufferedOut = new BufferedOutputStream(rawOut);
final TarArchiveOutputStream out = new TarArchiveOutputStream(bufferedOut)) {
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 (getMaxEntrySize(contents) >= TarConstants.MAXSIZE) {
out.setBigNumberMode(TarArchiveOutputStream.BIGNUMBER_POSIX);
}
for (final FlowFile flowFile : contents) {
final String path = keepPath ? getPath(flowFile) : "";
final String entryName = path + flowFile.getAttribute(CoreAttributes.FILENAME.key());
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 (getMaxEntrySize(contents) >= TarConstants.MAXSIZE) {
out.setBigNumberMode(TarArchiveOutputStream.BIGNUMBER_POSIX);
}
for (final FlowFile flowFile : contents) {
final String path = keepPath ? getPath(flowFile) : "";
final String entryName = path + flowFile.getAttribute(CoreAttributes.FILENAME.key());
final TarArchiveEntry tarEntry = new TarArchiveEntry(entryName);
tarEntry.setSize(flowFile.getSize());
final String permissionsVal = flowFile.getAttribute(TAR_PERMISSIONS_ATTRIBUTE);
if (permissionsVal != null) {
try {
tarEntry.setMode(Integer.parseInt(permissionsVal));
} catch (final Exception e) {
getLogger().debug("Attribute {} of {} is set to {}; expected 3 digits between 0-7, so ignoring",
new Object[] {TAR_PERMISSIONS_ATTRIBUTE, flowFile, permissionsVal});
}
final TarArchiveEntry tarEntry = new TarArchiveEntry(entryName);
tarEntry.setSize(flowFile.getSize());
final String permissionsVal = flowFile.getAttribute(TAR_PERMISSIONS_ATTRIBUTE);
if (permissionsVal != null) {
try {
tarEntry.setMode(Integer.parseInt(permissionsVal));
} catch (final Exception e) {
getLogger().debug("Attribute {} of {} is set to {}; expected 3 digits between 0-7, so ignoring",
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);
try {
bundle = session.write(bundle, new OutputStreamCallback() {
@Override
public void process(final OutputStream rawOut) throws IOException {
try (final OutputStream bufferedOut = new BufferedOutputStream(rawOut)) {
// we don't want the packager closing the stream. V1 creates a TAR Output Stream, which then gets
// closed, which in turn closes the underlying OutputStream, and we want to protect ourselves against that.
final OutputStream out = new NonCloseableOutputStream(bufferedOut);
bundle = session.write(bundle, rawOut -> {
try (final OutputStream bufferedOut = new BufferedOutputStream(rawOut)) {
// we don't want the packager closing the stream. V1 creates a TAR Output Stream, which then gets
// 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) {
bin.getSession().read(flowFile, new InputStreamCallback() {
@Override
public void process(final InputStream rawIn) throws IOException {
try (final InputStream in = new BufferedInputStream(rawIn)) {
final Map<String, String> attributes = new HashMap<>(flowFile.getAttributes());
packager.packageFlowFile(in, out, attributes, flowFile.getSize());
}
}
});
}
for (final FlowFile flowFile : contents) {
bin.getSession().read(flowFile, rawIn -> {
try (final InputStream in = new BufferedInputStream(rawIn)) {
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 {
bundle = session.putAttribute(bundle, CoreAttributes.FILENAME.key(), createFilename(contents) + ".zip");
bundle = session.write(bundle, new OutputStreamCallback() {
@Override
public void process(final OutputStream rawOut) throws IOException {
try (final OutputStream bufferedOut = new BufferedOutputStream(rawOut);
final ZipOutputStream out = new ZipOutputStream(bufferedOut)) {
out.setLevel(compressionLevel);
for (final FlowFile flowFile : contents) {
final String path = keepPath ? getPath(flowFile) : "";
final String entryName = path + flowFile.getAttribute(CoreAttributes.FILENAME.key());
final ZipEntry zipEntry = new ZipEntry(entryName);
zipEntry.setSize(flowFile.getSize());
try {
out.putNextEntry(zipEntry);
bundle = session.write(bundle, rawOut -> {
try (final OutputStream bufferedOut = new BufferedOutputStream(rawOut);
final ZipOutputStream out = new ZipOutputStream(bufferedOut)) {
out.setLevel(compressionLevel);
for (final FlowFile flowFile : contents) {
final String path = keepPath ? getPath(flowFile) : "";
final String entryName = path + flowFile.getAttribute(CoreAttributes.FILENAME.key());
final ZipEntry zipEntry = new ZipEntry(entryName);
zipEntry.setSize(flowFile.getSize());
try {
out.putNextEntry(zipEntry);
bin.getSession().exportTo(flowFile, out);
out.closeEntry();
unmerged.remove(flowFile);
} catch (ZipException e) {
getLogger().error("Encountered exception merging {}", flowFile, e);
}
bin.getSession().exportTo(flowFile, out);
out.closeEntry();
unmerged.remove(flowFile);
} catch (ZipException e) {
getLogger().error("Encountered exception merging {}", flowFile, e);
}
out.finish();
out.flush();
}
out.finish();
out.flush();
}
});
} 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
FlowFile bundle = session.create(contents);
try {
bundle = session.write(bundle, new OutputStreamCallback() {
@Override
public void process(final OutputStream rawOut) throws IOException {
try (final OutputStream out = new BufferedOutputStream(rawOut)) {
for (final FlowFile flowFile : contents) {
bin.getSession().read(flowFile, new InputStreamCallback() {
@Override
public void process(InputStream in) throws IOException {
boolean canMerge = true;
try (DataFileStream<GenericRecord> reader = new DataFileStream<>(in,
new GenericDatumReader<GenericRecord>())) {
if (schema.get() == null) {
// this is the first file - set up the writer, and store the
// Schema & metadata we'll use.
schema.set(reader.getSchema());
if (!METADATA_STRATEGY_IGNORE.getValue().equals(metadataStrategy)) {
for (String key : reader.getMetaKeys()) {
if (!DataFileWriter.isReservedMeta(key)) {
byte[] metadatum = reader.getMeta(key);
metadata.put(key, metadatum);
writer.setMeta(key, metadatum);
}
}
bundle = session.write(bundle, rawOut -> {
try (final OutputStream out = new BufferedOutputStream(rawOut)) {
for (final FlowFile flowFile : contents) {
bin.getSession().read(flowFile, in -> {
boolean canMerge = true;
try (DataFileStream<GenericRecord> reader = new DataFileStream<>(in,
new GenericDatumReader<>())) {
if (schema.get() == null) {
// this is the first file - set up the writer, and store the
// Schema & metadata we'll use.
schema.set(reader.getSchema());
if (!METADATA_STRATEGY_IGNORE.getValue().equals(metadataStrategy)) {
for (String key : reader.getMetaKeys()) {
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);
}
}
});
}
writer.flush();
} finally {
writer.close();
// write the Avro content from the current FlowFile to the merged OutputStream
if (canMerge) {
writer.appendAllFrom(reader, false);
}
}
});
}
writer.flush();
} finally {
writer.close();
}
});
} catch (final Exception e) {

View File

@ -33,8 +33,6 @@ import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.List;
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) {
final List<ConnectionStatus> connectionStatuses = new ArrayList<>();
populateConnectionStatuses(groupStatus, connectionStatuses);
connectionStatuses.sort(new Comparator<ConnectionStatus>() {
@Override
public int compare(final ConnectionStatus o1, final ConnectionStatus o2) {
if (o1 == null && o2 == null) {
return 0;
}
if (o1 == null) {
return 1;
}
if (o2 == null) {
return -1;
}
return -Long.compare(o1.getQueuedBytes(), o2.getQueuedBytes());
connectionStatuses.sort((o1, o2) -> {
if (o1 == null && o2 == null) {
return 0;
}
if (o1 == null) {
return 1;
}
if (o2 == null) {
return -1;
}
return -Long.compare(o1.getQueuedBytes(), o2.getQueuedBytes());
});
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) {
final List<ProcessorStatus> processorStatuses = new ArrayList<>();
populateProcessorStatuses(groupStatus, processorStatuses);
Collections.sort(processorStatuses, new Comparator<ProcessorStatus>() {
@Override
public int compare(final ProcessorStatus o1, final ProcessorStatus o2) {
if (o1 == null && o2 == null) {
return 0;
}
if (o1 == null) {
return 1;
}
if (o2 == null) {
return -1;
}
return -Long.compare(o1.getProcessingNanos(), o2.getProcessingNanos());
processorStatuses.sort((o1, o2) -> {
if (o1 == null && o2 == null) {
return 0;
}
if (o1 == null) {
return 1;
}
if (o2 == null) {
return -1;
}
return -Long.compare(o1.getProcessingNanos(), o2.getProcessingNanos());
});
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.fluent.FileBasedBuilderParameters;
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.lang3.StringUtils;
import org.apache.nifi.annotation.lifecycle.OnEnabled;
@ -49,7 +48,7 @@ import java.util.stream.Collectors;
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}.
*
*/
@ -105,14 +104,11 @@ public abstract class CommonsConfigurationLookupService<T extends FileBasedConfi
final FileBasedBuilderParameters params = new Parameters().fileBased().setFile(new File(config));
this.builder = new ReloadingFileBasedConfigurationBuilder<>(resultClass).configure(params);
builder.addEventListener(ConfigurationBuilderEvent.CONFIGURATION_REQUEST,
new EventListener<ConfigurationBuilderEvent>() {
@Override
public void onEvent(ConfigurationBuilderEvent event) {
event -> {
if (builder.getReloadingController().checkForReloading(null)) {
getLogger().debug("Reloading {}", config);
}
}
});
});
try {
// 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 java.text.Collator;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.List;
import java.util.Locale;
import java.util.Set;
@ -427,12 +425,9 @@ public class RuleResource {
}
// sort the rules
Collections.sort(ruleDtos, new Comparator<RuleDTO>() {
@Override
public int compare(RuleDTO r1, RuleDTO r2) {
final Collator collator = Collator.getInstance(Locale.US);
return collator.compare(r1.getName(), r2.getName());
}
ruleDtos.sort((r1, r2) -> {
final Collator collator = Collator.getInstance(Locale.US);
return collator.compare(r1.getName(), r2.getName());
});
// 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
*/
QUEUE_POSITION(new Comparator<FlowFileSummary>() {
@Override
public int compare(final FlowFileSummary o1, final FlowFileSummary o2) {
return Integer.compare(o1.getPosition(), o2.getPosition());
}
}),
QUEUE_POSITION(Comparator.comparingInt(FlowFileSummary::getPosition)),
/**
* Sort based on the UUID of the FlowFile
*/
FLOWFILE_UUID(new Comparator<FlowFileSummary>() {
@Override
public int compare(final FlowFileSummary o1, final FlowFileSummary o2) {
return o1.getUuid().compareTo(o2.getUuid());
}
}),
FLOWFILE_UUID(Comparator.comparing(FlowFileSummary::getUuid)),
/**
* Sort based on the 'filename' attribute of the FlowFile
*/
FILENAME(new Comparator<FlowFileSummary>() {
@Override
public int compare(final FlowFileSummary o1, final FlowFileSummary o2) {
return o1.getFilename().compareTo(o2.getFilename());
}
}),
FILENAME(Comparator.comparing(FlowFileSummary::getFilename)),
/**
* Sort based on the size of the FlowFile
*/
FLOWFILE_SIZE(new Comparator<FlowFileSummary>() {
@Override
public int compare(final FlowFileSummary o1, final FlowFileSummary o2) {
return Long.compare(o1.getSize(), o2.getSize());
}
}),
FLOWFILE_SIZE(Comparator.comparingLong(FlowFileSummary::getSize)),
/**
* Sort based on how long the FlowFile has been sitting in the queue
*/
QUEUED_DURATION(new Comparator<FlowFileSummary>() {
@Override
public int compare(final FlowFileSummary o1, final FlowFileSummary o2) {
return -Long.compare(o1.getLastQueuedTime(), o2.getLastQueuedTime());
}
}),
QUEUED_DURATION((o1, o2) -> -Long.compare(o1.getLastQueuedTime(), o2.getLastQueuedTime())),
/**
* Sort based on the age of the FlowFile. I.e., the time at which the FlowFile's
* "greatest ancestor" entered the flow
*/
FLOWFILE_AGE(new Comparator<FlowFileSummary>() {
@Override
public int compare(final FlowFileSummary o1, final FlowFileSummary o2) {
return Long.compare(o1.getLineageStartDate(), o2.getLineageStartDate());
}
}),
FLOWFILE_AGE(Comparator.comparingLong(FlowFileSummary::getLineageStartDate)),
/**
* Sort based on when the FlowFile's penalization ends
*/
PENALIZATION(new Comparator<FlowFileSummary>() {
@Override
public int compare(final FlowFileSummary o1, final FlowFileSummary o2) {
return Boolean.compare(o1.isPenalized(), o2.isPenalized());
}
});
PENALIZATION((o1, o2) -> Boolean.compare(o1.isPenalized(), o2.isPenalized()));
private final Comparator<FlowFileSummary> comparator;
private SortColumn(final Comparator<FlowFileSummary> comparator) {
SortColumn(final Comparator<FlowFileSummary> comparator) {
this.comparator = comparator;
}

View File

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

View File

@ -41,7 +41,6 @@ import java.nio.charset.CodingErrorAction;
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Comparator;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -152,34 +151,31 @@ public class LuceneUtil {
* list of {@link Document}s
*/
public static void sortDocsForRetrieval(final List<Document> documents) {
documents.sort(new Comparator<Document>() {
@Override
public int compare(final Document o1, final Document o2) {
final String filename1 = o1.get(FieldNames.STORAGE_FILENAME);
final String filename2 = o2.get(FieldNames.STORAGE_FILENAME);
documents.sort((o1, o2) -> {
final String filename1 = o1.get(FieldNames.STORAGE_FILENAME);
final String filename2 = o2.get(FieldNames.STORAGE_FILENAME);
final int filenameComp = filename1.compareTo(filename2);
if (filenameComp != 0) {
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 int filenameComp = filename1.compareTo(filename2);
if (filenameComp != 0) {
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);
});
}

View File

@ -32,7 +32,6 @@ import org.apache.nifi.provenance.toc.TocWriter;
import org.apache.nifi.provenance.util.DirectoryUtils;
import org.junit.jupiter.api.Test;
import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import java.io.File;
@ -45,7 +44,6 @@ import java.util.UUID;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.LinkedBlockingQueue;
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.assertTrue;
@ -76,13 +74,10 @@ public class TestWriteAheadStorePartition {
final Map<ProvenanceEventRecord, StorageSummary> reindexedEvents = new ConcurrentHashMap<>();
final EventIndex eventIndex = Mockito.mock(EventIndex.class);
Mockito.doAnswer(new Answer<Object>() {
@Override
public Object answer(final InvocationOnMock invocation) {
final Map<ProvenanceEventRecord, StorageSummary> events = invocation.getArgument(0);
reindexedEvents.putAll(events);
return null;
}
Mockito.doAnswer((Answer<Object>) invocation -> {
final Map<ProvenanceEventRecord, StorageSummary> events = invocation.getArgument(0);
reindexedEvents.putAll(events);
return null;
}).when(eventIndex).reindexEvents(Mockito.anyMap());
Mockito.doReturn(18L).when(eventIndex).getMinimumEventIdToReindex("1");
@ -91,7 +86,7 @@ public class TestWriteAheadStorePartition {
final List<Long> eventIdsReindexed = reindexedEvents.values().stream()
.map(StorageSummary::getEventId)
.sorted()
.collect(Collectors.toList());
.toList();
assertEquals(82, eventIdsReindexed.size());
for (int i = 0; i < eventIdsReindexed.size(); i++) {
@ -126,10 +121,10 @@ public class TestWriteAheadStorePartition {
partition.close();
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
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,
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.RingBuffer;
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.web.ResourceNotFoundException;
@ -171,16 +170,13 @@ public class VolatileProvenanceRepository implements ProvenanceRepository {
}
@Override
public List<ProvenanceEventRecord> getEvents(final long firstRecordId, final int maxRecords, final NiFiUser user) throws IOException {
return ringBuffer.getSelectedElements(new Filter<ProvenanceEventRecord>() {
@Override
public boolean select(final ProvenanceEventRecord value) {
if (!isAuthorized(value, user)) {
return false;
}
return value.getEventId() >= firstRecordId;
public List<ProvenanceEventRecord> getEvents(final long firstRecordId, final int maxRecords, final NiFiUser user) {
return ringBuffer.getSelectedElements(value -> {
if (!isAuthorized(value, user)) {
return false;
}
return value.getEventId() >= firstRecordId;
}, maxRecords);
}
@ -190,26 +186,16 @@ public class VolatileProvenanceRepository implements ProvenanceRepository {
return (newest == null) ? null : newest.getEventId();
}
public ProvenanceEventRecord getEvent(final String identifier) throws IOException {
final List<ProvenanceEventRecord> records = ringBuffer.getSelectedElements(new Filter<ProvenanceEventRecord>() {
@Override
public boolean select(final ProvenanceEventRecord event) {
return identifier.equals(event.getFlowFileUuid());
}
}, 1);
return records.isEmpty() ? null : records.get(0);
public ProvenanceEventRecord getEvent(final String identifier) {
final List<ProvenanceEventRecord> records = ringBuffer.getSelectedElements(event -> identifier.equals(event.getFlowFileUuid()), 1);
return records.isEmpty() ? null : records.getFirst();
}
@Override
public ProvenanceEventRecord getEvent(final long id) {
final List<ProvenanceEventRecord> records = ringBuffer.getSelectedElements(new Filter<ProvenanceEventRecord>() {
@Override
public boolean select(final ProvenanceEventRecord event) {
return event.getEventId() == id;
}
}, 1);
final List<ProvenanceEventRecord> records = ringBuffer.getSelectedElements(event -> event.getEventId() == id, 1);
return records.isEmpty() ? null : records.get(0);
return records.isEmpty() ? null : records.getFirst();
}
@Override
@ -282,127 +268,45 @@ public class VolatileProvenanceRepository implements ProvenanceRepository {
}
private Filter<ProvenanceEventRecord> createFilter(final Query query, final NiFiUser user) {
return new Filter<ProvenanceEventRecord>() {
@Override
public boolean select(final ProvenanceEventRecord event) {
if (!isAuthorized(event, user)) {
return event -> {
if (!isAuthorized(event, user)) {
return false;
}
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;
}
}
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;
}
}
if (query.getEndDate() != null && query.getEndDate().getTime() < event.getEventTime()) {
return false;
}
for (final SearchTerm searchTerm : query.getSearchTerms()) {
final SearchableField searchableField = searchTerm.getSearchableField();
final String searchValue = searchTerm.getValue();
final boolean excludeSearchValue = searchTerm.isInverted().booleanValue();
if (query.getMaxFileSize() != null) {
final long maxFileSize = DataUnit.parseDataSize(query.getMaxFileSize(), DataUnit.B).longValue();
if (event.getFileSize() > maxFileSize) {
return false;
}
}
if (searchableField.isAttribute()) {
final String attributeName = searchableField.getIdentifier();
if (query.getMinFileSize() != null) {
final long minFileSize = DataUnit.parseDataSize(query.getMinFileSize(), DataUnit.B).longValue();
if (event.getFileSize() < minFileSize) {
return false;
}
}
final String eventAttributeValue = event.getAttributes().get(attributeName);
for (final SearchTerm searchTerm : query.getSearchTerms()) {
final SearchableField searchableField = searchTerm.getSearchableField();
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 (searchValue.contains("?") || searchValue.contains("*")) {
if (eventAttributeValue == null || eventAttributeValue.isEmpty()) {
if (!excludeSearchValue) {
return false;
} 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("*")) {
final String regex = searchValue.replace("?", ".").replace("*", ".*");
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
|| patternMatches && excludeSearchValue) {
return false;
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 (!searchValue.equalsIgnoreCase(String.valueOf(fieldValue)) && !excludeSearchValue
|| searchValue.equalsIgnoreCase(String.valueOf(fieldValue)) && excludeSearchValue) {
} 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) {
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);
lineageSubmissionMap.put(result.getLineageIdentifier(), result);
final Filter<ProvenanceEventRecord> filter = new Filter<ProvenanceEventRecord>() {
@Override
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;
}
}
final Filter<ProvenanceEventRecord> filter = 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;
};
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
final AtomicInteger matchingCount = new AtomicInteger(0);
final List<ProvenanceEventRecord> matchingRecords = new ArrayList<>(maxRecords);
ringBuffer.forEach(new ForEachEvaluator<ProvenanceEventRecord>() {
@Override
public boolean evaluate(final ProvenanceEventRecord record) {
if (filter.select(record)) {
if (matchingCount.incrementAndGet() <= maxRecords) {
matchingRecords.add(record);
}
ringBuffer.forEach(record -> {
if (filter.select(record)) {
if (matchingCount.incrementAndGet() <= maxRecords) {
matchingRecords.add(record);
}
return true;
}
return true;
}, IterationDirection.BACKWARD);
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) {
TreeSet<T> components = new TreeSet<>(new Comparator<ComponentDTO>() {
@Override
public int compare(ComponentDTO c1, ComponentDTO c2) {
return c1.getId().compareTo(c2.getId());
}
});
TreeSet<T> components = new TreeSet<>(Comparator.comparing(ComponentDTO::getId));
components.addAll(dtos);
return components;
}
@ -298,12 +293,7 @@ public class FlowSnippetDTO {
}
private <T extends RemoteProcessGroupPortDTO> Set<T> orderedRemotePortsById(Set<T> dtos) {
TreeSet<T> components = new TreeSet<>(new Comparator<RemoteProcessGroupPortDTO>() {
@Override
public int compare(RemoteProcessGroupPortDTO c1, RemoteProcessGroupPortDTO c2) {
return UUID.fromString(c1.getId()).compareTo(UUID.fromString(c2.getId()));
}
});
TreeSet<T> components = new TreeSet<>(Comparator.comparing((RemoteProcessGroupPortDTO c) -> UUID.fromString(c.getId())));
components.addAll(dtos);
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.Test;
import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import java.io.File;
@ -144,8 +143,6 @@ public class FileAccessPolicyProviderTest {
private File restoreAuthorizations;
private File restoreTenants;
private File flow;
private File flowNoPorts;
private File flowWithDns;
private AuthorizerConfigurationContext configurationContext;
@ -170,10 +167,10 @@ public class FileAccessPolicyProviderTest {
flow = new File("src/test/resources/flow.json.gz");
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());
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());
properties = mock(NiFiProperties.class);
@ -248,12 +245,7 @@ public class FileAccessPolicyProviderTest {
});
final AccessPolicyProviderInitializationContext initializationContext = mock(AccessPolicyProviderInitializationContext.class);
when(initializationContext.getUserGroupProviderLookup()).thenReturn(new UserGroupProviderLookup() {
@Override
public UserGroupProvider getUserGroupProvider(String identifier) {
return userGroupProvider;
}
});
when(initializationContext.getUserGroupProviderLookup()).thenReturn(identifier -> userGroupProvider);
accessPolicyProvider = new FileAccessPolicyProvider();
accessPolicyProvider.setNiFiProperties(properties);
@ -261,7 +253,7 @@ public class FileAccessPolicyProviderTest {
}
@AfterEach
public void cleanup() throws Exception {
public void cleanup() {
deleteFile(primaryAuthorizations);
deleteFile(primaryTenants);
deleteFile(restoreAuthorizations);
@ -647,7 +639,7 @@ public class FileAccessPolicyProviderTest {
} else if (policy.getIdentifier().equals("policy-2")
&& policy.getResource().equals("/flow")
&& policy.getAction() == RequestAction.WRITE
&& policy.getGroups().size() == 0
&& policy.getGroups().isEmpty()
&& policy.getUsers().size() == 1
&& policy.getUsers().contains("user-2")) {
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()) {
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]);
}
});
when(nifiProperties.getProperty(anyString())).then((Answer<String>) invocationOnMock -> properties.getProperty((String) invocationOnMock.getArguments()[0]));
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.BeforeEach;
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.FileOutputStream;
@ -35,7 +32,6 @@ import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.HashMap;
import java.util.Map;
import java.util.Properties;
import java.util.Set;
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.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
@ -90,7 +85,6 @@ public class FileUserGroupProviderTest {
" </users>" +
"</tenants>";
private NiFiProperties properties;
private FileUserGroupProvider userGroupProvider;
private File primaryTenants;
private File restoreTenants;
@ -107,8 +101,8 @@ public class FileUserGroupProviderTest {
restoreTenants = new File("target/restore/users.xml");
FileUtils.ensureDirectoryExistAndCanAccess(restoreTenants.getParentFile());
properties = mock(NiFiProperties.class);
when(properties.getRestoreDirectory()).thenReturn(restoreTenants.getParentFile());
NiFiProperties properties1 = mock(NiFiProperties.class);
when(properties1.getRestoreDirectory()).thenReturn(restoreTenants.getParentFile());
configurationContext = mock(AuthorizerConfigurationContext.class);
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.setNiFiProperties(properties);
userGroupProvider.setNiFiProperties(properties1);
userGroupProvider.initialize(null);
}
@AfterEach
public void cleanup() throws Exception {
public void cleanup() {
deleteFile(primaryTenants);
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()) {
FileUtils.deleteFilesInDir(file, null, null, true, true);
}
return 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;
FileUtils.deleteFile(file, null, 10);
}
}

View File

@ -35,8 +35,6 @@ import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.EOFException;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.ByteBuffer;
/**
@ -65,89 +63,76 @@ public class JaxbProtocolContext<T> implements ProtocolContext<T> {
@Override
public ProtocolMessageMarshaller<T> createMarshaller() {
return new ProtocolMessageMarshaller<T>() {
return (msg, os) -> {
@Override
public void marshal(final T msg, final OutputStream os) throws IOException {
try {
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 Marshaller marshaller = jaxbCtx.createMarshaller();
final ByteArrayOutputStream msgBytes = new ByteArrayOutputStream();
marshaller.marshal(msg, msgBytes);
final DataOutputStream dos = new DataOutputStream(os);
final DataOutputStream dos = new DataOutputStream(os);
// write message protocol sentinel
dos.write(MESSAGE_PROTOCOL_START_SENTINEL);
// write message protocol sentinel
dos.write(MESSAGE_PROTOCOL_START_SENTINEL);
// write message size in bytes
dos.writeInt(msgBytes.size());
// write message size in bytes
dos.writeInt(msgBytes.size());
// write message
msgBytes.writeTo(dos);
// write message
msgBytes.writeTo(dos);
dos.flush();
} catch (final JAXBException je) {
throw new IOException("Failed marshalling protocol message due to: " + je, je);
}
dos.flush();
} catch (final JAXBException je) {
throw new IOException("Failed marshalling protocol message due to: " + je, je);
}
};
}
@Override
public ProtocolMessageUnmarshaller<T> createUnmarshaller() {
return new ProtocolMessageUnmarshaller<T>() {
return is -> {
@Override
public T unmarshal(final InputStream is) throws IOException {
try {
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();
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);
// check for the presence of the message protocol sentinel
final byte sentinel = (byte) dis.read();
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 = 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
protected void mergeResponses(ListingRequestDTO clientDto, Map<NodeIdentifier, ListingRequestDTO> dtoMap, Set<NodeResponse> successfulResponses, Set<NodeResponse> problematicResponses) {
final Comparator<FlowFileSummaryDTO> comparator = new Comparator<FlowFileSummaryDTO>() {
@Override
public int compare(final FlowFileSummaryDTO dto1, final FlowFileSummaryDTO dto2) {
int positionCompare = dto1.getPosition().compareTo(dto2.getPosition());
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 Comparator<FlowFileSummaryDTO> comparator = (dto1, dto2) -> {
int positionCompare = dto1.getPosition().compareTo(dto2.getPosition());
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 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.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.Date;
import java.util.HashMap;
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.
// 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
Collections.sort(allResults, new Comparator<ProvenanceEventDTO>() {
@Override
public int compare(final ProvenanceEventDTO o1, final ProvenanceEventDTO o2) {
final int eventTimeComparison = o1.getEventTime().compareTo(o2.getEventTime());
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());
allResults.sort((o1, o2) -> {
final int eventTimeComparison = o1.getEventTime().compareTo(o2.getEventTime());
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 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 java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
@ -32,21 +31,18 @@ public final class BulletinMerger {
private BulletinMerger() { }
public static Comparator<BulletinEntity> BULLETIN_COMPARATOR = new Comparator<BulletinEntity>() {
@Override
public int compare(BulletinEntity o1, BulletinEntity o2) {
if (o1 == null && o2 == null) {
return 0;
}
if (o1 == null) {
return 1;
}
if (o2 == null) {
return -1;
}
return -Long.compare(o1.getId(), o2.getId());
public static Comparator<BulletinEntity> BULLETIN_COMPARATOR = (o1, o2) -> {
if (o1 == null && o2 == null) {
return 0;
}
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);
// ensure the bulletins are sorted by time
Collections.sort(entities, (BulletinEntity o1, BulletinEntity o2) -> {
final int timeComparison = o1.getTimestamp().compareTo(o2.getTimestamp());
if (timeComparison != 0) {
return timeComparison;
}
return o1.getNodeAddress().compareTo(o2.getNodeAddress());
});
entities.sort(Comparator.comparing(BulletinEntity::getTimestamp).thenComparing(BulletinEntity::getNodeAddress));
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.Timeout;
import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import org.springframework.security.core.Authentication;
import org.springframework.security.core.context.SecurityContextHolder;
@ -315,12 +314,8 @@ public class TestThreadPoolRequestReplicator {
private ClusterCoordinator createClusterCoordinator() {
final ClusterCoordinator coordinator = mock(ClusterCoordinator.class);
when(coordinator.getConnectionStatus(Mockito.any(NodeIdentifier.class))).thenAnswer(new Answer<NodeConnectionStatus>() {
@Override
public NodeConnectionStatus answer(InvocationOnMock invocation) throws Throwable {
return new NodeConnectionStatus(invocation.getArgument(0), NodeConnectionState.CONNECTED);
}
});
when(coordinator.getConnectionStatus(Mockito.any(NodeIdentifier.class))).thenAnswer((Answer<NodeConnectionStatus>) invocation ->
new NodeConnectionStatus(invocation.getArgument(0), NodeConnectionState.CONNECTED));
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));
SecurityContextHolder.getContext().setAuthentication(authentication);
@ -381,23 +376,19 @@ public class TestThreadPoolRequestReplicator {
final CountDownLatch preNotifyLatch = new CountDownLatch(1);
final CountDownLatch postNotifyLatch = new CountDownLatch(1);
new Thread(new Runnable() {
@Override
public void run() {
synchronized (monitor) {
while (true) {
// If monitor is not notified, this will block indefinitely, and the test will timeout
try {
preNotifyLatch.countDown();
monitor.wait();
break;
} catch (InterruptedException e) {
continue;
}
new Thread(() -> {
synchronized (monitor) {
while (true) {
// If monitor is not notified, this will block indefinitely, and the test will timeout
try {
preNotifyLatch.countDown();
monitor.wait();
break;
} catch (InterruptedException ignored) {
}
postNotifyLatch.countDown();
}
postNotifyLatch.countDown();
}
}).start();
@ -431,23 +422,19 @@ public class TestThreadPoolRequestReplicator {
final CountDownLatch preNotifyLatch = new CountDownLatch(1);
final CountDownLatch postNotifyLatch = new CountDownLatch(1);
new Thread(new Runnable() {
@Override
public void run() {
synchronized (monitor) {
while (true) {
// If monitor is not notified, this will block indefinitely, and the test will timeout
try {
preNotifyLatch.countDown();
monitor.wait();
break;
} catch (InterruptedException e) {
continue;
}
new Thread(() -> {
synchronized (monitor) {
while (true) {
// If monitor is not notified, this will block indefinitely, and the test will timeout
try {
preNotifyLatch.countDown();
monitor.wait();
break;
} catch (InterruptedException ignored) {
}
postNotifyLatch.countDown();
}
postNotifyLatch.countDown();
}
}).start();
@ -485,23 +472,19 @@ public class TestThreadPoolRequestReplicator {
final CountDownLatch preNotifyLatch = new CountDownLatch(1);
final CountDownLatch postNotifyLatch = new CountDownLatch(1);
new Thread(new Runnable() {
@Override
public void run() {
synchronized (monitor) {
while (true) {
// If monitor is not notified, this will block indefinitely, and the test will timeout
try {
preNotifyLatch.countDown();
monitor.wait();
break;
} catch (InterruptedException e) {
continue;
}
new Thread(() -> {
synchronized (monitor) {
while (true) {
// If monitor is not notified, this will block indefinitely, and the test will timeout
try {
preNotifyLatch.countDown();
monitor.wait();
break;
} catch (InterruptedException ignored) {
}
postNotifyLatch.countDown();
}
postNotifyLatch.countDown();
}
}).start();
@ -557,9 +540,7 @@ public class TestThreadPoolRequestReplicator {
final StandardAsyncClusterResponse response) {
if (delayMillis > 0L) {
assertDoesNotThrow(() -> {
Thread.sleep(delayMillis);
}, "Thread Interrupted during test");
assertDoesNotThrow(() -> Thread.sleep(delayMillis), "Thread Interrupted during test");
}
if (failure != null) {
@ -581,9 +562,7 @@ public class TestThreadPoolRequestReplicator {
}
};
assertDoesNotThrow(() -> {
function.withReplicator(replicator);
});
assertDoesNotThrow(() -> function.withReplicator(replicator));
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.Timeout;
import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import java.io.IOException;
import java.util.ArrayList;
@ -181,13 +179,10 @@ public class TestNodeClusterCoordinator {
final ClusterCoordinationProtocolSenderListener senderListener = Mockito.mock(ClusterCoordinationProtocolSenderListener.class);
final AtomicReference<ReconnectionRequestMessage> requestRef = new AtomicReference<>();
when(senderListener.requestReconnection(any(ReconnectionRequestMessage.class))).thenAnswer(new Answer<Object>() {
@Override
public Object answer(InvocationOnMock invocation) throws Throwable {
final ReconnectionRequestMessage msg = invocation.getArgument(0);
requestRef.set(msg);
return null;
}
when(senderListener.requestReconnection(any(ReconnectionRequestMessage.class))).thenAnswer(invocation -> {
final ReconnectionRequestMessage msg = invocation.getArgument(0);
requestRef.set(msg);
return null;
});
final EventReporter eventReporter = Mockito.mock(EventReporter.class);

View File

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

View File

@ -97,60 +97,57 @@ public class VolatileBulletinRepository implements BulletinRepository {
private Filter<Bulletin> createFilter(final BulletinQuery bulletinQuery) {
final long fiveMinutesAgo = System.currentTimeMillis() - TimeUnit.MINUTES.toMillis(5);
return new Filter<Bulletin>() {
@Override
public boolean select(final Bulletin bulletin) {
// only include bulletins after the specified id
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;
return bulletin -> {
// only include bulletins after the specified id
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;
};
}
@ -205,12 +202,7 @@ public class VolatileBulletinRepository implements BulletinRepository {
final List<Bulletin> allComponentBulletins = new ArrayList<>();
for (final RingBuffer<Bulletin> ringBuffer : componentMap.values()) {
allComponentBulletins.addAll(ringBuffer.getSelectedElements(new Filter<Bulletin>() {
@Override
public boolean select(final Bulletin bulletin) {
return bulletin.getTimestamp().getTime() >= fiveMinutesAgo;
}
}, maxPerComponent));
allComponentBulletins.addAll(ringBuffer.getSelectedElements(bulletin -> bulletin.getTimestamp().getTime() >= fiveMinutesAgo, maxPerComponent));
}
return allComponentBulletins;
@ -225,12 +217,7 @@ public class VolatileBulletinRepository implements BulletinRepository {
public List<Bulletin> findBulletinsForController(final int max) {
final long fiveMinutesAgo = System.currentTimeMillis() - TimeUnit.MINUTES.toMillis(5);
final Filter<Bulletin> filter = new Filter<Bulletin>() {
@Override
public boolean select(final Bulletin bulletin) {
return bulletin.getTimestamp().getTime() >= fiveMinutesAgo;
}
};
final Filter<Bulletin> filter = bulletin -> bulletin.getTimestamp().getTime() >= fiveMinutesAgo;
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 ThreadFactory defaultThreadFactory = getThreadFactory();
setThreadFactory(new ThreadFactory() {
@Override
public Thread newThread(final Runnable r) {
final Thread t = defaultThreadFactory.newThread(r);
if (daemon) {
t.setDaemon(true);
}
t.setName(threadNamePrefix + " Thread-" + threadIndex.incrementAndGet());
return t;
setThreadFactory(r -> {
final Thread t = defaultThreadFactory.newThread(r);
if (daemon) {
t.setDaemon(true);
}
t.setName(threadNamePrefix + " Thread-" + threadIndex.incrementAndGet());
return t;
});
}
@ -103,28 +100,22 @@ public final class FlowEngine extends ScheduledThreadPoolExecutor {
}
private Runnable wrap(final Runnable runnable) {
return new Runnable() {
@Override
public void run() {
try {
runnable.run();
} catch (final Throwable t) {
logger.error("Uncaught Exception in Runnable task", t);
}
return () -> {
try {
runnable.run();
} catch (final Throwable t) {
logger.error("Uncaught Exception in Runnable task", t);
}
};
}
private <T> Callable<T> wrap(final Callable<T> callable) {
return new Callable<T>() {
@Override
public T call() throws Exception {
try {
return callable.call();
} catch (final Throwable t) {
logger.error("Uncaught Exception in Callable task", t);
throw t;
}
return () -> {
try {
return callable.call();
} catch (final Throwable 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() {
final Collator collator = Collator.getInstance(Locale.US);
return new Comparator<String>() {
@Override
public int compare(String s1, String s2) {
return collator.compare(s1, s2);
}
};
return collator::compare;
}
/**
@ -52,11 +47,6 @@ public class SortedStateUtils {
*/
public static Comparator<StateEntryDTO> getEntryDtoComparator() {
final Collator collator = Collator.getInstance(Locale.US);
return new Comparator<StateEntryDTO>() {
@Override
public int compare(StateEntryDTO o1, StateEntryDTO o2) {
return collator.compare(o1.getKey(), o2.getKey());
}
};
return (o1, o2) -> 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.Timeout;
import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import java.util.ArrayList;
@ -82,15 +81,12 @@ public class TestStandardFlowFileQueue {
provRepo = Mockito.mock(ProvenanceEventRepository.class);
Mockito.when(provRepo.eventBuilder()).thenReturn(new StandardProvenanceEventRecord.Builder());
Mockito.doAnswer(new Answer<Object>() {
@Override
public Object answer(final InvocationOnMock invocation) throws Throwable {
final Iterable<ProvenanceEventRecord> iterable = (Iterable<ProvenanceEventRecord>) invocation.getArguments()[0];
for (final ProvenanceEventRecord record : iterable) {
provRecords.add(record);
}
return null;
Mockito.doAnswer((Answer<Object>) invocation -> {
final Iterable<ProvenanceEventRecord> iterable = (Iterable<ProvenanceEventRecord>) invocation.getArguments()[0];
for (final ProvenanceEventRecord record : iterable) {
provRecords.add(record);
}
return null;
}).when(provRepo).registerEvents(Mockito.any(Iterable.class));
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.OutputStream;
import java.security.GeneralSecurityException;
import java.security.KeyManagementException;
import java.security.KeyPair;
import java.security.KeyPairGenerator;
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.CertificateException;
import java.security.cert.X509Certificate;
import java.time.Duration;
import java.util.ArrayList;
@ -156,12 +151,9 @@ public class LoadBalancedQueueIT {
new NodeConnectionStatus(invocation.getArgument(0, NodeIdentifier.class), NodeConnectionState.CONNECTED));
clusterEventListeners.clear();
doAnswer(new Answer() {
@Override
public Object answer(final InvocationOnMock invocation) {
clusterEventListeners.add(invocation.getArgument(0));
return null;
}
doAnswer(invocation -> {
clusterEventListeners.add(invocation.getArgument(0));
return null;
}).when(clusterCoordinator).registerEventListener(any(ClusterTopologyEventListener.class));
processScheduler = mock(ProcessScheduler.class);
@ -487,7 +479,7 @@ public class LoadBalancedQueueIT {
assertEquals(1, serverRepoRecords.size());
final RepositoryRecord serverRecord = serverRepoRecords.iterator().next();
final RepositoryRecord serverRecord = serverRepoRecords.getFirst();
final FlowFileRecord serverFlowFile = serverRecord.getCurrent();
assertEquals("test", serverFlowFile.getAttribute("integration"));
assertEquals("false", serverFlowFile.getAttribute("unit-test"));
@ -497,12 +489,12 @@ public class LoadBalancedQueueIT {
final byte[] serverFlowFileContent = serverClaimContents.get(serverContentClaim);
assertArrayEquals("hello".getBytes(), serverFlowFileContent);
while (clientRepoRecords.size() == 0) {
while (clientRepoRecords.isEmpty()) {
Thread.sleep(10L);
}
assertEquals(1, clientRepoRecords.size());
final RepositoryRecord clientRecord = clientRepoRecords.iterator().next();
final RepositoryRecord clientRecord = clientRepoRecords.getFirst();
assertEquals(RepositoryRecordType.DELETE, clientRecord.getType());
} finally {
flowFileQueue.stopLoadBalancing();
@ -568,12 +560,12 @@ public class LoadBalancedQueueIT {
flowFileQueue.startLoadBalancing();
while (clientRepoRecords.size() == 0) {
while (clientRepoRecords.isEmpty()) {
Thread.sleep(10L);
}
assertEquals(1, clientRepoRecords.size());
final RepositoryRecord clientRecord = clientRepoRecords.iterator().next();
final RepositoryRecord clientRecord = clientRepoRecords.getFirst();
assertEquals(RepositoryRecordType.CONTENTMISSING, clientRecord.getType());
} finally {
flowFileQueue.stopLoadBalancing();
@ -649,7 +641,7 @@ public class LoadBalancedQueueIT {
assertEquals(1, serverRepoRecords.size());
final RepositoryRecord serverRecord = serverRepoRecords.iterator().next();
final RepositoryRecord serverRecord = serverRepoRecords.getFirst();
final FlowFileRecord serverFlowFile = serverRecord.getCurrent();
assertEquals("test", serverFlowFile.getAttribute("integration"));
assertEquals("false", serverFlowFile.getAttribute("unit-test"));
@ -659,12 +651,12 @@ public class LoadBalancedQueueIT {
final byte[] serverFlowFileContent = serverClaimContents.get(serverContentClaim);
assertArrayEquals("hello".getBytes(), serverFlowFileContent);
while (clientRepoRecords.size() == 0) {
while (clientRepoRecords.isEmpty()) {
Thread.sleep(10L);
}
assertEquals(1, clientRepoRecords.size());
final RepositoryRecord clientRecord = clientRepoRecords.iterator().next();
final RepositoryRecord clientRecord = clientRepoRecords.getFirst();
assertEquals(RepositoryRecordType.DELETE, clientRecord.getType());
} finally {
flowFileQueue.stopLoadBalancing();
@ -740,7 +732,7 @@ public class LoadBalancedQueueIT {
assertEquals(1, serverRepoRecords.size());
final RepositoryRecord serverRecord = serverRepoRecords.iterator().next();
final RepositoryRecord serverRecord = serverRepoRecords.getFirst();
final FlowFileRecord serverFlowFile = serverRecord.getCurrent();
assertEquals("test", serverFlowFile.getAttribute("integration"));
assertEquals("false", serverFlowFile.getAttribute("unit-test"));
@ -750,12 +742,12 @@ public class LoadBalancedQueueIT {
final byte[] serverFlowFileContent = serverClaimContents.get(serverContentClaim);
assertArrayEquals("hello".getBytes(), serverFlowFileContent);
while (clientRepoRecords.size() == 0) {
while (clientRepoRecords.isEmpty()) {
Thread.sleep(10L);
}
assertEquals(1, clientRepoRecords.size());
final RepositoryRecord clientRecord = clientRepoRecords.iterator().next();
final RepositoryRecord clientRecord = clientRepoRecords.getFirst();
assertEquals(RepositoryRecordType.DELETE, clientRecord.getType());
} finally {
flowFileQueue.stopLoadBalancing();
@ -768,7 +760,7 @@ public class LoadBalancedQueueIT {
@Test
@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);
nodeIdentifiers.add(localNodeId);
@ -828,7 +820,7 @@ public class LoadBalancedQueueIT {
assertEquals(1, serverRepoRecords.size());
final RepositoryRecord serverRecord = serverRepoRecords.iterator().next();
final RepositoryRecord serverRecord = serverRepoRecords.getFirst();
final FlowFileRecord serverFlowFile = serverRecord.getCurrent();
assertEquals("test", serverFlowFile.getAttribute("integration"));
assertEquals("false", serverFlowFile.getAttribute("unit-test"));
@ -838,12 +830,12 @@ public class LoadBalancedQueueIT {
final byte[] serverFlowFileContent = serverClaimContents.get(serverContentClaim);
assertArrayEquals("hello".getBytes(), serverFlowFileContent);
while (clientRepoRecords.size() == 0) {
while (clientRepoRecords.isEmpty()) {
Thread.sleep(10L);
}
assertEquals(1, clientRepoRecords.size());
final RepositoryRecord clientRecord = clientRepoRecords.iterator().next();
final RepositoryRecord clientRecord = clientRepoRecords.getFirst();
assertEquals(RepositoryRecordType.DELETE, clientRecord.getType());
} finally {
flowFileQueue.stopLoadBalancing();
@ -857,7 +849,7 @@ public class LoadBalancedQueueIT {
@Test
@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);
nodeIdentifiers.add(localNodeId);
@ -915,7 +907,7 @@ public class LoadBalancedQueueIT {
assertEquals(i, serverRepoRecords.size());
final RepositoryRecord serverRecord = serverRepoRecords.iterator().next();
final RepositoryRecord serverRecord = serverRepoRecords.getFirst();
final FlowFileRecord serverFlowFile = serverRecord.getCurrent();
assertEquals("test", serverFlowFile.getAttribute("integration"));
assertEquals("false", serverFlowFile.getAttribute("unit-test"));
@ -930,7 +922,7 @@ public class LoadBalancedQueueIT {
}
assertEquals(i, clientRepoRecords.size());
final RepositoryRecord clientRecord = clientRepoRecords.iterator().next();
final RepositoryRecord clientRecord = clientRepoRecords.getFirst();
assertEquals(RepositoryRecordType.DELETE, clientRecord.getType());
}
} finally {
@ -945,7 +937,7 @@ public class LoadBalancedQueueIT {
@Test
@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);
nodeIdentifiers.add(localNodeId);
@ -1007,7 +999,7 @@ public class LoadBalancedQueueIT {
assertEquals(1, serverRepoRecords.size());
final RepositoryRecord serverRecord = serverRepoRecords.iterator().next();
final RepositoryRecord serverRecord = serverRepoRecords.getFirst();
final FlowFileRecord serverFlowFile = serverRecord.getCurrent();
assertEquals("test", serverFlowFile.getAttribute("integration"));
assertEquals("false", serverFlowFile.getAttribute("unit-test"));
@ -1017,12 +1009,12 @@ public class LoadBalancedQueueIT {
final byte[] serverFlowFileContent = serverClaimContents.get(serverContentClaim);
assertArrayEquals(payload, serverFlowFileContent);
while (clientRepoRecords.size() == 0) {
while (clientRepoRecords.isEmpty()) {
Thread.sleep(10L);
}
assertEquals(1, clientRepoRecords.size());
final RepositoryRecord clientRecord = clientRepoRecords.iterator().next();
final RepositoryRecord clientRecord = clientRepoRecords.getFirst();
assertEquals(RepositoryRecordType.DELETE, clientRecord.getType());
} finally {
flowFileQueue.stopLoadBalancing();
@ -1139,7 +1131,7 @@ public class LoadBalancedQueueIT {
assertEquals(1, serverRepoRecords.size());
final RepositoryRecord serverRecord = serverRepoRecords.iterator().next();
final RepositoryRecord serverRecord = serverRepoRecords.getFirst();
final FlowFileRecord serverFlowFile = serverRecord.getCurrent();
assertEquals("test", serverFlowFile.getAttribute("integration"));
assertEquals("false", serverFlowFile.getAttribute("unit-test"));
@ -1149,12 +1141,12 @@ public class LoadBalancedQueueIT {
final byte[] serverFlowFileContent = serverClaimContents.get(serverContentClaim);
assertArrayEquals("hello".getBytes(), serverFlowFileContent);
while (clientRepoRecords.size() == 0) {
while (clientRepoRecords.isEmpty()) {
Thread.sleep(10L);
}
assertEquals(1, clientRepoRecords.size());
final RepositoryRecord clientRecord = clientRepoRecords.iterator().next();
final RepositoryRecord clientRecord = clientRepoRecords.getFirst();
assertEquals(RepositoryRecordType.DELETE, clientRecord.getType());
} finally {
flowFileQueue.stopLoadBalancing();
@ -1325,12 +1317,7 @@ public class LoadBalancedQueueIT {
private ContentRepository createContentRepository(final ConcurrentMap<ContentClaim, byte[]> claimContents) throws IOException {
final ContentRepository contentRepo = mock(ContentRepository.class);
Mockito.doAnswer(new Answer<ContentClaim>() {
@Override
public ContentClaim answer(final InvocationOnMock invocation) {
return createContentClaim(null);
}
}).when(contentRepo).create(Mockito.anyBoolean());
Mockito.doAnswer((Answer<ContentClaim>) invocation -> createContentClaim(null)).when(contentRepo).create(Mockito.anyBoolean());
Mockito.doAnswer(new Answer<OutputStream>() {
@ -1351,21 +1338,18 @@ public class LoadBalancedQueueIT {
}).when(contentRepo).write(any(ContentClaim.class));
Mockito.doAnswer(new Answer<InputStream>() {
@Override
public InputStream answer(final InvocationOnMock invocation) {
final ContentClaim contentClaim = invocation.getArgument(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);
Mockito.doAnswer((Answer<InputStream>) invocation -> {
final ContentClaim contentClaim = invocation.getArgument(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);
}).when(contentRepo).read(Mockito.nullable(ContentClaim.class));
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.SwapSummary;
import org.apache.nifi.events.EventReporter;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.flowfile.FlowFilePrioritizer;
import org.apache.nifi.provenance.ProvenanceEventRepository;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout;
import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import java.io.IOException;
@ -70,7 +68,6 @@ import static org.mockito.Mockito.when;
public class TestSocketLoadBalancedFlowFileQueue {
private Connection connection;
private FlowFileRepository flowFileRepo;
private ContentRepository contentRepo;
private ProvenanceEventRepository provRepo;
@ -86,7 +83,7 @@ public class TestSocketLoadBalancedFlowFileQueue {
@BeforeEach
public void setup() {
MockFlowFileRecord.resetIdGenerator();
connection = mock(Connection.class);
Connection connection = mock(Connection.class);
when(connection.getIdentifier()).thenReturn("unit-test");
flowFileRepo = mock(FlowFileRepository.class);
@ -103,21 +100,13 @@ public class TestSocketLoadBalancedFlowFileQueue {
nodeIds.add(createNodeIdentifier("11111111-1111-1111-1111-111111111111"));
nodeIds.add(createNodeIdentifier("22222222-2222-2222-2222-222222222222"));
Mockito.doAnswer(new Answer<Set<NodeIdentifier>>() {
@Override
public Set<NodeIdentifier> answer(InvocationOnMock invocation) throws Throwable {
return new HashSet<>(nodeIds);
}
}).when(clusterCoordinator).getNodeIdentifiers();
Mockito.doAnswer((Answer<Set<NodeIdentifier>>) invocation -> new HashSet<>(nodeIds)).when(clusterCoordinator).getNodeIdentifiers();
when(clusterCoordinator.getLocalNodeIdentifier()).thenReturn(localNodeIdentifier);
doAnswer(new Answer() {
@Override
public Object answer(final InvocationOnMock invocation) throws Throwable {
clusterTopologyEventListener = invocation.getArgument(0);
return null;
}
doAnswer(invocation -> {
clusterTopologyEventListener = invocation.getArgument(0);
return null;
}).when(clusterCoordinator).registerEventListener(Mockito.any(ClusterTopologyEventListener.class));
final ProcessScheduler scheduler = mock(ProcessScheduler.class);
@ -148,7 +137,7 @@ public class TestSocketLoadBalancedFlowFileQueue {
assertFalse(queue.isEmpty());
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
// 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.
@ -174,13 +163,10 @@ public class TestSocketLoadBalancedFlowFileQueue {
@Test
public void testPriorities() {
final FlowFilePrioritizer iValuePrioritizer = new FlowFilePrioritizer() {
@Override
public int compare(final FlowFile o1, final FlowFile o2) {
final int i1 = Integer.parseInt(o1.getAttribute("i"));
final int i2 = Integer.parseInt(o2.getAttribute("i"));
return Integer.compare(i1, i2);
}
final FlowFilePrioritizer iValuePrioritizer = (o1, o2) -> {
final int i1 = Integer.parseInt(o1.getAttribute("i"));
final int i2 = Integer.parseInt(o2.getAttribute("i"));
return Integer.compare(i1, i2);
};
queue.setPriorities(Collections.singletonList(iValuePrioritizer));
@ -205,13 +191,10 @@ public class TestSocketLoadBalancedFlowFileQueue {
@Test
public void testPrioritiesWhenSetBeforeLocalNodeIdDetermined() {
final FlowFilePrioritizer iValuePrioritizer = new FlowFilePrioritizer() {
@Override
public int compare(final FlowFile o1, final FlowFile o2) {
final int i1 = Integer.parseInt(o1.getAttribute("i"));
final int i2 = Integer.parseInt(o2.getAttribute("i"));
return Integer.compare(i1, i2);
}
final FlowFilePrioritizer iValuePrioritizer = (o1, o2) -> {
final int i1 = Integer.parseInt(o1.getAttribute("i"));
final int i2 = Integer.parseInt(o2.getAttribute("i"));
return Integer.compare(i1, i2);
};
final ProcessScheduler scheduler = mock(ProcessScheduler.class);
@ -525,7 +508,7 @@ public class TestSocketLoadBalancedFlowFileQueue {
@Test
@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.
queue.setFlowFilePartitioner(new StaticSequencePartitioner(new int[] {0, 1, 0, 1}, false));
@ -545,7 +528,7 @@ public class TestSocketLoadBalancedFlowFileQueue {
@Test
@Timeout(10)
public void testDataInRemotePartitionForLocalIdIsMovedToLocalPartition() throws InterruptedException {
public void testDataInRemotePartitionForLocalIdIsMovedToLocalPartition() {
nodeIds.clear();
final NodeIdentifier id1 = createNodeIdentifier();

View File

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

View File

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

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

View File

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

View File

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

View File

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

View File

@ -23,7 +23,6 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.List;
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
Collections.sort(mappings, new Comparator<IdentityMapping>() {
@Override
public int compare(IdentityMapping m1, IdentityMapping m2) {
return m1.getKey().compareTo(m2.getKey());
}
});
mappings.sort(Comparator.comparing(IdentityMapping::getKey));
return mappings;
}

View File

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