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

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -91,9 +91,7 @@ class JMSConsumer extends JMSWorker {
*/ */
public void consumeMessageSet(final String destinationName, String errorQueueName, final boolean durable, final boolean shared, final String subscriptionName, final String messageSelector, public void consumeMessageSet(final String destinationName, String errorQueueName, final boolean durable, final boolean shared, final String subscriptionName, final String messageSelector,
final String charset, final int batchSize, final Consumer<List<JMSResponse>> messageSetConsumer) { final String charset, final int batchSize, final Consumer<List<JMSResponse>> messageSetConsumer) {
doWithJmsTemplate(destinationName, durable, shared, subscriptionName, messageSelector, new MessageReceiver() { doWithJmsTemplate(destinationName, durable, shared, subscriptionName, messageSelector, (session, messageConsumer) -> {
@Override
public void consume(Session session, MessageConsumer messageConsumer) throws JMSException {
final List<JMSResponse> jmsResponses = new ArrayList<>(); final List<JMSResponse> jmsResponses = new ArrayList<>();
int batchCounter = 0; int batchCounter = 0;
@ -112,14 +110,11 @@ class JMSConsumer extends JMSWorker {
// the JMSResponse. // the JMSResponse.
messageSetConsumer.accept(jmsResponses); messageSetConsumer.accept(jmsResponses);
} }
}
}); });
} }
private void doWithJmsTemplate(String destinationName, boolean durable, boolean shared, String subscriptionName, String messageSelector, MessageReceiver messageReceiver) { private void doWithJmsTemplate(String destinationName, boolean durable, boolean shared, String subscriptionName, String messageSelector, MessageReceiver messageReceiver) {
this.jmsTemplate.execute(new SessionCallback<Void>() { this.jmsTemplate.execute((SessionCallback<Void>) session -> {
@Override
public Void doInJms(final Session session) throws JMSException {
final MessageConsumer messageConsumer = createMessageConsumer(session, destinationName, durable, shared, subscriptionName, messageSelector); final MessageConsumer messageConsumer = createMessageConsumer(session, destinationName, durable, shared, subscriptionName, messageSelector);
try { try {
@ -140,7 +135,6 @@ class JMSConsumer extends JMSWorker {
} }
return null; return null;
}
}, true); }, true);
} }

View File

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

View File

@ -40,7 +40,6 @@ import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.io.OutputStreamCallback;
import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processors.standard.util.ArgumentUtils; import org.apache.nifi.processors.standard.util.ArgumentUtils;
@ -57,7 +56,6 @@ import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService; import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors; import java.util.concurrent.Executors;
@ -265,9 +263,7 @@ public class ExecuteProcess extends AbstractProcessor {
// as the delegate for the ProxyOuptutStream, then wait until the process finishes // as the delegate for the ProxyOuptutStream, then wait until the process finishes
// or until the specified amount of time // or until the specified amount of time
FlowFile flowFile = session.create(); FlowFile flowFile = session.create();
flowFile = session.write(flowFile, new OutputStreamCallback() { flowFile = session.write(flowFile, flowFileOut -> {
@Override
public void process(final OutputStream flowFileOut) throws IOException {
try (final OutputStream out = new BufferedOutputStream(flowFileOut)) { try (final OutputStream out = new BufferedOutputStream(flowFileOut)) {
proxyOut.setDelegate(out); proxyOut.setDelegate(out);
@ -294,7 +290,6 @@ public class ExecuteProcess extends AbstractProcessor {
proxyOut.setDelegate(null); // prevent from writing to this proxyOut.setDelegate(null); // prevent from writing to this
// stream // stream
} }
}
}); });
if (flowFile.getSize() == 0L) { if (flowFile.getSize() == 0L) {
@ -358,22 +353,17 @@ public class ExecuteProcess extends AbstractProcessor {
// Submit task to read error stream from process // Submit task to read error stream from process
if (!redirectErrorStream) { if (!redirectErrorStream) {
executor.submit(new Runnable() { executor.submit(() -> {
@Override
public void run() {
try (final BufferedReader reader = new BufferedReader(new InputStreamReader(externalProcess.getErrorStream()))) { try (final BufferedReader reader = new BufferedReader(new InputStreamReader(externalProcess.getErrorStream()))) {
reader.lines().filter(line -> line != null && line.length() > 0).forEach(getLogger()::warn); reader.lines().filter(line -> line != null && !line.isEmpty()).forEach(getLogger()::warn);
} catch (final IOException ioe) { } catch (final IOException ignored) {
}
} }
}); });
} }
// Submit task to read output of Process and write to FlowFile. // Submit task to read output of Process and write to FlowFile.
failure = new AtomicBoolean(false); failure = new AtomicBoolean(false);
final Future<?> future = executor.submit(new Callable<Object>() { final Future<?> future = executor.submit(() -> {
@Override
public Object call() throws IOException {
try { try {
if (batchNanos == null) { if (batchNanos == null) {
// if we aren't batching, just copy the stream from the // if we aren't batching, just copy the stream from the
@ -433,7 +423,6 @@ public class ExecuteProcess extends AbstractProcessor {
} }
return null; return null;
}
}); });
return future; return future;

View File

@ -57,8 +57,6 @@ import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.io.InputStreamCallback;
import org.apache.nifi.processor.io.OutputStreamCallback;
import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processor.util.bin.Bin; import org.apache.nifi.processor.util.bin.Bin;
import org.apache.nifi.processor.util.bin.BinFiles; import org.apache.nifi.processor.util.bin.BinFiles;
@ -659,9 +657,7 @@ public class MergeContent extends BinFiles {
FlowFile bundle = session.create(bin.getContents()); FlowFile bundle = session.create(bin.getContents());
final AtomicReference<String> bundleMimeTypeRef = new AtomicReference<>(null); final AtomicReference<String> bundleMimeTypeRef = new AtomicReference<>(null);
try { try {
bundle = session.write(bundle, new OutputStreamCallback() { bundle = session.write(bundle, out -> {
@Override
public void process(final OutputStream out) throws IOException {
final byte[] header = getDelimiterContent(context, contents, HEADER); final byte[] header = getDelimiterContent(context, contents, HEADER);
if (header != null) { if (header != null) {
out.write(header); out.write(header);
@ -696,7 +692,6 @@ public class MergeContent extends BinFiles {
if (footer != null) { if (footer != null) {
out.write(footer); out.write(footer);
} }
}
}); });
} catch (final Exception e) { } catch (final Exception e) {
removeFlowFileFromSession(session, bundle, context); removeFlowFileFromSession(session, bundle, context);
@ -802,9 +797,7 @@ public class MergeContent extends BinFiles {
try { try {
bundle = session.putAttribute(bundle, CoreAttributes.FILENAME.key(), createFilename(contents) + ".tar"); bundle = session.putAttribute(bundle, CoreAttributes.FILENAME.key(), createFilename(contents) + ".tar");
bundle = session.write(bundle, new OutputStreamCallback() { bundle = session.write(bundle, rawOut -> {
@Override
public void process(final OutputStream rawOut) throws IOException {
try (final OutputStream bufferedOut = new BufferedOutputStream(rawOut); try (final OutputStream bufferedOut = new BufferedOutputStream(rawOut);
final TarArchiveOutputStream out = new TarArchiveOutputStream(bufferedOut)) { final TarArchiveOutputStream out = new TarArchiveOutputStream(bufferedOut)) {
@ -846,7 +839,6 @@ public class MergeContent extends BinFiles {
out.closeArchiveEntry(); out.closeArchiveEntry();
} }
} }
}
}); });
} catch (final Exception e) { } catch (final Exception e) {
removeFlowFileFromSession(session, bundle, context); removeFlowFileFromSession(session, bundle, context);
@ -894,27 +886,21 @@ public class MergeContent extends BinFiles {
FlowFile bundle = session.create(contents); FlowFile bundle = session.create(contents);
try { try {
bundle = session.write(bundle, new OutputStreamCallback() { bundle = session.write(bundle, rawOut -> {
@Override
public void process(final OutputStream rawOut) throws IOException {
try (final OutputStream bufferedOut = new BufferedOutputStream(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 // 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. // closed, which in turn closes the underlying OutputStream, and we want to protect ourselves against that.
final OutputStream out = new NonCloseableOutputStream(bufferedOut); final OutputStream out = new NonCloseableOutputStream(bufferedOut);
for (final FlowFile flowFile : contents) { for (final FlowFile flowFile : contents) {
bin.getSession().read(flowFile, new InputStreamCallback() { bin.getSession().read(flowFile, rawIn -> {
@Override
public void process(final InputStream rawIn) throws IOException {
try (final InputStream in = new BufferedInputStream(rawIn)) { try (final InputStream in = new BufferedInputStream(rawIn)) {
final Map<String, String> attributes = new HashMap<>(flowFile.getAttributes()); final Map<String, String> attributes = new HashMap<>(flowFile.getAttributes());
packager.packageFlowFile(in, out, attributes, flowFile.getSize()); packager.packageFlowFile(in, out, attributes, flowFile.getSize());
} }
}
}); });
} }
} }
}
}); });
} catch (final Exception e) { } catch (final Exception e) {
removeFlowFileFromSession(session, bundle, context); removeFlowFileFromSession(session, bundle, context);
@ -959,9 +945,7 @@ public class MergeContent extends BinFiles {
try { try {
bundle = session.putAttribute(bundle, CoreAttributes.FILENAME.key(), createFilename(contents) + ".zip"); bundle = session.putAttribute(bundle, CoreAttributes.FILENAME.key(), createFilename(contents) + ".zip");
bundle = session.write(bundle, new OutputStreamCallback() { bundle = session.write(bundle, rawOut -> {
@Override
public void process(final OutputStream rawOut) throws IOException {
try (final OutputStream bufferedOut = new BufferedOutputStream(rawOut); try (final OutputStream bufferedOut = new BufferedOutputStream(rawOut);
final ZipOutputStream out = new ZipOutputStream(bufferedOut)) { final ZipOutputStream out = new ZipOutputStream(bufferedOut)) {
out.setLevel(compressionLevel); out.setLevel(compressionLevel);
@ -984,7 +968,6 @@ public class MergeContent extends BinFiles {
out.finish(); out.finish();
out.flush(); out.flush();
} }
}
}); });
} catch (final Exception e) { } catch (final Exception e) {
removeFlowFileFromSession(session, bundle, context); removeFlowFileFromSession(session, bundle, context);
@ -1024,17 +1007,13 @@ public class MergeContent extends BinFiles {
// we don't pass the parents to the #create method because the parents belong to different sessions // we don't pass the parents to the #create method because the parents belong to different sessions
FlowFile bundle = session.create(contents); FlowFile bundle = session.create(contents);
try { try {
bundle = session.write(bundle, new OutputStreamCallback() { bundle = session.write(bundle, rawOut -> {
@Override
public void process(final OutputStream rawOut) throws IOException {
try (final OutputStream out = new BufferedOutputStream(rawOut)) { try (final OutputStream out = new BufferedOutputStream(rawOut)) {
for (final FlowFile flowFile : contents) { for (final FlowFile flowFile : contents) {
bin.getSession().read(flowFile, new InputStreamCallback() { bin.getSession().read(flowFile, in -> {
@Override
public void process(InputStream in) throws IOException {
boolean canMerge = true; boolean canMerge = true;
try (DataFileStream<GenericRecord> reader = new DataFileStream<>(in, try (DataFileStream<GenericRecord> reader = new DataFileStream<>(in,
new GenericDatumReader<GenericRecord>())) { new GenericDatumReader<>())) {
if (schema.get() == null) { if (schema.get() == null) {
// this is the first file - set up the writer, and store the // this is the first file - set up the writer, and store the
// Schema & metadata we'll use. // Schema & metadata we'll use.
@ -1101,14 +1080,12 @@ public class MergeContent extends BinFiles {
writer.appendAllFrom(reader, false); writer.appendAllFrom(reader, false);
} }
} }
}
}); });
} }
writer.flush(); writer.flush();
} finally { } finally {
writer.close(); writer.close();
} }
}
}); });
} catch (final Exception e) { } catch (final Exception e) {
removeFlowFileFromSession(session, bundle, context); removeFlowFileFromSession(session, bundle, context);

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -41,7 +41,6 @@ import java.nio.charset.CodingErrorAction;
import java.nio.file.Path; import java.nio.file.Path;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.Comparator;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -152,9 +151,7 @@ public class LuceneUtil {
* list of {@link Document}s * list of {@link Document}s
*/ */
public static void sortDocsForRetrieval(final List<Document> documents) { public static void sortDocsForRetrieval(final List<Document> documents) {
documents.sort(new Comparator<Document>() { documents.sort((o1, o2) -> {
@Override
public int compare(final Document o1, final Document o2) {
final String filename1 = o1.get(FieldNames.STORAGE_FILENAME); final String filename1 = o1.get(FieldNames.STORAGE_FILENAME);
final String filename2 = o2.get(FieldNames.STORAGE_FILENAME); final String filename2 = o2.get(FieldNames.STORAGE_FILENAME);
@ -179,7 +176,6 @@ public class LuceneUtil {
final long offset1 = o1.getField(FieldNames.STORAGE_FILE_OFFSET).numericValue().longValue(); final long offset1 = o1.getField(FieldNames.STORAGE_FILE_OFFSET).numericValue().longValue();
final long offset2 = o2.getField(FieldNames.STORAGE_FILE_OFFSET).numericValue().longValue(); final long offset2 = o2.getField(FieldNames.STORAGE_FILE_OFFSET).numericValue().longValue();
return Long.compare(offset1, offset2); return Long.compare(offset1, offset2);
}
}); });
} }

View File

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

View File

@ -38,7 +38,6 @@ import org.apache.nifi.provenance.search.SearchableField;
import org.apache.nifi.util.NiFiProperties; import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.util.RingBuffer; import org.apache.nifi.util.RingBuffer;
import org.apache.nifi.util.RingBuffer.Filter; import org.apache.nifi.util.RingBuffer.Filter;
import org.apache.nifi.util.RingBuffer.ForEachEvaluator;
import org.apache.nifi.util.RingBuffer.IterationDirection; import org.apache.nifi.util.RingBuffer.IterationDirection;
import org.apache.nifi.web.ResourceNotFoundException; import org.apache.nifi.web.ResourceNotFoundException;
@ -171,16 +170,13 @@ public class VolatileProvenanceRepository implements ProvenanceRepository {
} }
@Override @Override
public List<ProvenanceEventRecord> getEvents(final long firstRecordId, final int maxRecords, final NiFiUser user) throws IOException { public List<ProvenanceEventRecord> getEvents(final long firstRecordId, final int maxRecords, final NiFiUser user) {
return ringBuffer.getSelectedElements(new Filter<ProvenanceEventRecord>() { return ringBuffer.getSelectedElements(value -> {
@Override
public boolean select(final ProvenanceEventRecord value) {
if (!isAuthorized(value, user)) { if (!isAuthorized(value, user)) {
return false; return false;
} }
return value.getEventId() >= firstRecordId; return value.getEventId() >= firstRecordId;
}
}, maxRecords); }, maxRecords);
} }
@ -190,26 +186,16 @@ public class VolatileProvenanceRepository implements ProvenanceRepository {
return (newest == null) ? null : newest.getEventId(); return (newest == null) ? null : newest.getEventId();
} }
public ProvenanceEventRecord getEvent(final String identifier) throws IOException { public ProvenanceEventRecord getEvent(final String identifier) {
final List<ProvenanceEventRecord> records = ringBuffer.getSelectedElements(new Filter<ProvenanceEventRecord>() { final List<ProvenanceEventRecord> records = ringBuffer.getSelectedElements(event -> identifier.equals(event.getFlowFileUuid()), 1);
@Override return records.isEmpty() ? null : records.getFirst();
public boolean select(final ProvenanceEventRecord event) {
return identifier.equals(event.getFlowFileUuid());
}
}, 1);
return records.isEmpty() ? null : records.get(0);
} }
@Override @Override
public ProvenanceEventRecord getEvent(final long id) { public ProvenanceEventRecord getEvent(final long id) {
final List<ProvenanceEventRecord> records = ringBuffer.getSelectedElements(new Filter<ProvenanceEventRecord>() { final List<ProvenanceEventRecord> records = ringBuffer.getSelectedElements(event -> event.getEventId() == id, 1);
@Override
public boolean select(final ProvenanceEventRecord event) {
return event.getEventId() == id;
}
}, 1);
return records.isEmpty() ? null : records.get(0); return records.isEmpty() ? null : records.getFirst();
} }
@Override @Override
@ -282,9 +268,7 @@ public class VolatileProvenanceRepository implements ProvenanceRepository {
} }
private Filter<ProvenanceEventRecord> createFilter(final Query query, final NiFiUser user) { private Filter<ProvenanceEventRecord> createFilter(final Query query, final NiFiUser user) {
return new Filter<ProvenanceEventRecord>() { return event -> {
@Override
public boolean select(final ProvenanceEventRecord event) {
if (!isAuthorized(event, user)) { if (!isAuthorized(event, user)) {
return false; return false;
} }
@ -427,7 +411,6 @@ public class VolatileProvenanceRepository implements ProvenanceRepository {
} }
return true; return true;
}
}; };
} }
@ -670,9 +653,7 @@ public class VolatileProvenanceRepository implements ProvenanceRepository {
final AsyncLineageSubmission result = new AsyncLineageSubmission(computationType, eventId, flowFileUuids, 1, userId); final AsyncLineageSubmission result = new AsyncLineageSubmission(computationType, eventId, flowFileUuids, 1, userId);
lineageSubmissionMap.put(result.getLineageIdentifier(), result); lineageSubmissionMap.put(result.getLineageIdentifier(), result);
final Filter<ProvenanceEventRecord> filter = new Filter<ProvenanceEventRecord>() { final Filter<ProvenanceEventRecord> filter = event -> {
@Override
public boolean select(final ProvenanceEventRecord event) {
if (!isAuthorized(event, user)) { if (!isAuthorized(event, user)) {
return false; return false;
} }
@ -694,7 +675,6 @@ public class VolatileProvenanceRepository implements ProvenanceRepository {
} }
return false; return false;
}
}; };
queryExecService.submit(new ComputeLineageRunnable(ringBuffer, filter, result)); queryExecService.submit(new ComputeLineageRunnable(ringBuffer, filter, result));
@ -721,9 +701,7 @@ public class VolatileProvenanceRepository implements ProvenanceRepository {
// Retrieve the most recent results and count the total number of matches // Retrieve the most recent results and count the total number of matches
final AtomicInteger matchingCount = new AtomicInteger(0); final AtomicInteger matchingCount = new AtomicInteger(0);
final List<ProvenanceEventRecord> matchingRecords = new ArrayList<>(maxRecords); final List<ProvenanceEventRecord> matchingRecords = new ArrayList<>(maxRecords);
ringBuffer.forEach(new ForEachEvaluator<ProvenanceEventRecord>() { ringBuffer.forEach(record -> {
@Override
public boolean evaluate(final ProvenanceEventRecord record) {
if (filter.select(record)) { if (filter.select(record)) {
if (matchingCount.incrementAndGet() <= maxRecords) { if (matchingCount.incrementAndGet() <= maxRecords) {
matchingRecords.add(record); matchingRecords.add(record);
@ -731,8 +709,6 @@ public class VolatileProvenanceRepository implements ProvenanceRepository {
} }
return true; return true;
}
}, IterationDirection.BACKWARD); }, IterationDirection.BACKWARD);
submission.getResult().update(matchingRecords, matchingCount.get()); submission.getResult().update(matchingRecords, matchingCount.get());

View File

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

View File

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

View File

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

View File

@ -35,8 +35,6 @@ import java.io.DataInputStream;
import java.io.DataOutputStream; import java.io.DataOutputStream;
import java.io.EOFException; import java.io.EOFException;
import java.io.IOException; import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
/** /**
@ -65,10 +63,7 @@ public class JaxbProtocolContext<T> implements ProtocolContext<T> {
@Override @Override
public ProtocolMessageMarshaller<T> createMarshaller() { public ProtocolMessageMarshaller<T> createMarshaller() {
return new ProtocolMessageMarshaller<T>() { return (msg, os) -> {
@Override
public void marshal(final T msg, final OutputStream os) throws IOException {
try { try {
@ -94,16 +89,12 @@ public class JaxbProtocolContext<T> implements ProtocolContext<T> {
throw new IOException("Failed marshalling protocol message due to: " + je, je); throw new IOException("Failed marshalling protocol message due to: " + je, je);
} }
}
}; };
} }
@Override @Override
public ProtocolMessageUnmarshaller<T> createUnmarshaller() { public ProtocolMessageUnmarshaller<T> createUnmarshaller() {
return new ProtocolMessageUnmarshaller<T>() { return is -> {
@Override
public T unmarshal(final InputStream is) throws IOException {
try { try {
@ -126,12 +117,7 @@ public class JaxbProtocolContext<T> implements ProtocolContext<T> {
final ByteBuffer buffer = ByteBuffer.allocate(msgBytesSize); final ByteBuffer buffer = ByteBuffer.allocate(msgBytesSize);
int totalBytesRead = 0; int totalBytesRead = 0;
do { do {
final int bytesToRead; final int bytesToRead = Math.min((msgBytesSize - totalBytesRead), BUF_SIZE);
if ((msgBytesSize - totalBytesRead) >= BUF_SIZE) {
bytesToRead = BUF_SIZE;
} else {
bytesToRead = msgBytesSize - totalBytesRead;
}
totalBytesRead += dis.read(buffer.array(), totalBytesRead, bytesToRead); totalBytesRead += dis.read(buffer.array(), totalBytesRead, bytesToRead);
} while (totalBytesRead < msgBytesSize); } while (totalBytesRead < msgBytesSize);
@ -147,7 +133,6 @@ public class JaxbProtocolContext<T> implements ProtocolContext<T> {
throw new IOException("Failed unmarshalling protocol message due to: " + e, e); throw new IOException("Failed unmarshalling protocol message due to: " + e, e);
} }
}
}; };
} }
} }

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -1907,10 +1907,7 @@ public class StandardProcessSession implements ProcessSession, ProvenanceEventEn
return Collections.emptyList(); return Collections.emptyList();
} }
return get(new ConnectionPoller() { return get((connection, expiredRecords) -> connection.poll(new FlowFileFilter() {
@Override
public List<FlowFileRecord> poll(final Connection connection, final Set<FlowFileRecord> expiredRecords) {
return connection.poll(new FlowFileFilter() {
int polled = 0; int polled = 0;
@Override @Override
@ -1921,21 +1918,14 @@ public class StandardProcessSession implements ProcessSession, ProvenanceEventEn
return FlowFileFilterResult.ACCEPT_AND_TERMINATE; return FlowFileFilterResult.ACCEPT_AND_TERMINATE;
} }
} }
}, expiredRecords); }, expiredRecords), false);
}
}, false);
} }
@Override @Override
public List<FlowFile> get(final FlowFileFilter filter) { public List<FlowFile> get(final FlowFileFilter filter) {
verifyTaskActive(); verifyTaskActive();
return get(new ConnectionPoller() { return get((connection, expiredRecords) -> connection.poll(filter, expiredRecords), true);
@Override
public List<FlowFileRecord> poll(final Connection connection, final Set<FlowFileRecord> expiredRecords) {
return connection.poll(filter, expiredRecords);
}
}, true);
} }
@ -2494,12 +2484,7 @@ public class StandardProcessSession implements ProcessSession, ProvenanceEventEn
public void expireFlowFiles() { public void expireFlowFiles() {
final Set<FlowFileRecord> expired = new HashSet<>(); final Set<FlowFileRecord> expired = new HashSet<>();
final FlowFileFilter filter = new FlowFileFilter() { final FlowFileFilter filter = flowFile -> FlowFileFilter.FlowFileFilterResult.REJECT_AND_CONTINUE;
@Override
public FlowFileFilterResult filter(final FlowFile flowFile) {
return FlowFileFilterResult.REJECT_AND_CONTINUE;
}
};
for (final Connection conn : context.getConnectable().getIncomingConnections()) { for (final Connection conn : context.getConnectable().getIncomingConnections()) {
do { do {
@ -2536,11 +2521,9 @@ public class StandardProcessSession implements ProcessSession, ProvenanceEventEn
} }
try { try {
final Iterable<ProvenanceEventRecord> iterable = new Iterable<ProvenanceEventRecord>() { final Iterable<ProvenanceEventRecord> iterable = () -> {
@Override
public Iterator<ProvenanceEventRecord> iterator() {
final Iterator<ProvenanceEventRecord> expiredEventIterator = expiredReporter.getEvents().iterator(); final Iterator<ProvenanceEventRecord> expiredEventIterator = expiredReporter.getEvents().iterator();
final Iterator<ProvenanceEventRecord> enrichingIterator = new Iterator<ProvenanceEventRecord>() { final Iterator<ProvenanceEventRecord> enrichingIterator = new Iterator<>() {
@Override @Override
public boolean hasNext() { public boolean hasNext() {
return expiredEventIterator.hasNext(); return expiredEventIterator.hasNext();
@ -2564,7 +2547,7 @@ public class StandardProcessSession implements ProcessSession, ProvenanceEventEn
record.getContentClaimOffset() + claim.getOffset(), record.getSize()); record.getContentClaimOffset() + claim.getOffset(), record.getSize());
} }
enriched.setAttributes(record.getAttributes(), Collections.<String, String> emptyMap()); enriched.setAttributes(record.getAttributes(), Collections.<String, String>emptyMap());
return enriched.build(); return enriched.build();
} }
@ -2575,7 +2558,6 @@ public class StandardProcessSession implements ProcessSession, ProvenanceEventEn
}; };
return enrichingIterator; return enrichingIterator;
}
}; };
context.getProvenanceRepository().registerEvents(iterable); context.getProvenanceRepository().registerEvents(iterable);

View File

@ -97,9 +97,7 @@ public class VolatileBulletinRepository implements BulletinRepository {
private Filter<Bulletin> createFilter(final BulletinQuery bulletinQuery) { private Filter<Bulletin> createFilter(final BulletinQuery bulletinQuery) {
final long fiveMinutesAgo = System.currentTimeMillis() - TimeUnit.MINUTES.toMillis(5); final long fiveMinutesAgo = System.currentTimeMillis() - TimeUnit.MINUTES.toMillis(5);
return new Filter<Bulletin>() { return bulletin -> {
@Override
public boolean select(final Bulletin bulletin) {
// only include bulletins after the specified id // only include bulletins after the specified id
if (bulletinQuery.getAfter() != null && bulletin.getId() <= bulletinQuery.getAfter()) { if (bulletinQuery.getAfter() != null && bulletin.getId() <= bulletinQuery.getAfter()) {
return false; return false;
@ -150,7 +148,6 @@ public class VolatileBulletinRepository implements BulletinRepository {
} }
return true; return true;
}
}; };
} }
@ -205,12 +202,7 @@ public class VolatileBulletinRepository implements BulletinRepository {
final List<Bulletin> allComponentBulletins = new ArrayList<>(); final List<Bulletin> allComponentBulletins = new ArrayList<>();
for (final RingBuffer<Bulletin> ringBuffer : componentMap.values()) { for (final RingBuffer<Bulletin> ringBuffer : componentMap.values()) {
allComponentBulletins.addAll(ringBuffer.getSelectedElements(new Filter<Bulletin>() { allComponentBulletins.addAll(ringBuffer.getSelectedElements(bulletin -> bulletin.getTimestamp().getTime() >= fiveMinutesAgo, maxPerComponent));
@Override
public boolean select(final Bulletin bulletin) {
return bulletin.getTimestamp().getTime() >= fiveMinutesAgo;
}
}, maxPerComponent));
} }
return allComponentBulletins; return allComponentBulletins;
@ -225,12 +217,7 @@ public class VolatileBulletinRepository implements BulletinRepository {
public List<Bulletin> findBulletinsForController(final int max) { public List<Bulletin> findBulletinsForController(final int max) {
final long fiveMinutesAgo = System.currentTimeMillis() - TimeUnit.MINUTES.toMillis(5); final long fiveMinutesAgo = System.currentTimeMillis() - TimeUnit.MINUTES.toMillis(5);
final Filter<Bulletin> filter = new Filter<Bulletin>() { final Filter<Bulletin> filter = bulletin -> bulletin.getTimestamp().getTime() >= fiveMinutesAgo;
@Override
public boolean select(final Bulletin bulletin) {
return bulletin.getTimestamp().getTime() >= fiveMinutesAgo;
}
};
final List<Bulletin> controllerBulletins = new ArrayList<>(); final List<Bulletin> controllerBulletins = new ArrayList<>();

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -19,7 +19,6 @@ package org.apache.nifi.controller.status.history;
import java.util.Date; import java.util.Date;
import java.util.HashMap; import java.util.HashMap;
import java.util.LinkedHashSet; import java.util.LinkedHashSet;
import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
@ -106,9 +105,7 @@ public class StandardStatusSnapshot implements StatusSnapshot {
@Override @Override
public ValueReducer<StatusSnapshot, StatusSnapshot> getValueReducer() { public ValueReducer<StatusSnapshot, StatusSnapshot> getValueReducer() {
return new ValueReducer<StatusSnapshot, StatusSnapshot>() { return values -> {
@Override
public StatusSnapshot reduce(final List<StatusSnapshot> values) {
Date reducedTimestamp = null; Date reducedTimestamp = null;
final Set<MetricDescriptor<?>> allDescriptors = new LinkedHashSet<>(getMetricDescriptors()); final Set<MetricDescriptor<?>> allDescriptors = new LinkedHashSet<>(getMetricDescriptors());
@ -130,7 +127,6 @@ public class StandardStatusSnapshot implements StatusSnapshot {
} }
return reduced; return reduced;
}
}; };
} }
} }

View File

@ -404,14 +404,12 @@ import org.apache.nifi.web.dao.ReportingTaskDAO;
import org.apache.nifi.web.dao.SnippetDAO; import org.apache.nifi.web.dao.SnippetDAO;
import org.apache.nifi.web.dao.UserDAO; import org.apache.nifi.web.dao.UserDAO;
import org.apache.nifi.web.dao.UserGroupDAO; import org.apache.nifi.web.dao.UserGroupDAO;
import org.apache.nifi.web.revision.DeleteRevisionTask;
import org.apache.nifi.web.revision.ExpiredRevisionClaimException; import org.apache.nifi.web.revision.ExpiredRevisionClaimException;
import org.apache.nifi.web.revision.RevisionClaim; import org.apache.nifi.web.revision.RevisionClaim;
import org.apache.nifi.web.revision.RevisionManager; import org.apache.nifi.web.revision.RevisionManager;
import org.apache.nifi.web.revision.RevisionUpdate; import org.apache.nifi.web.revision.RevisionUpdate;
import org.apache.nifi.web.revision.StandardRevisionClaim; import org.apache.nifi.web.revision.StandardRevisionClaim;
import org.apache.nifi.web.revision.StandardRevisionUpdate; import org.apache.nifi.web.revision.StandardRevisionUpdate;
import org.apache.nifi.web.revision.UpdateRevisionTask;
import org.apache.nifi.web.util.PredictionBasedParallelProcessingService; import org.apache.nifi.web.util.PredictionBasedParallelProcessingService;
import org.apache.nifi.web.util.SnippetUtils; import org.apache.nifi.web.util.SnippetUtils;
import org.slf4j.Logger; import org.slf4j.Logger;
@ -1183,9 +1181,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
final RevisionUpdate<SnippetDTO> snapshot; final RevisionUpdate<SnippetDTO> snapshot;
try { try {
snapshot = revisionManager.updateRevision(revisionClaim, user, new UpdateRevisionTask<SnippetDTO>() { snapshot = revisionManager.updateRevision(revisionClaim, user, () -> {
@Override
public RevisionUpdate<SnippetDTO> update() {
// get the updated component // get the updated component
final Snippet snippet = snippetDAO.updateSnippetComponents(snippetDto); final Snippet snippet = snippetDAO.updateSnippetComponents(snippetDto);
@ -1203,7 +1199,6 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
final SnippetDTO dto = dtoFactory.createSnippetDto(snippet); final SnippetDTO dto = dtoFactory.createSnippetDto(snippet);
return new StandardRevisionUpdate<>(dto, null, updatedRevisions); return new StandardRevisionUpdate<>(dto, null, updatedRevisions);
}
}); });
} catch (final ExpiredRevisionClaimException e) { } catch (final ExpiredRevisionClaimException e) {
throw new InvalidRevisionException("Failed to update Snippet", e); throw new InvalidRevisionException("Failed to update Snippet", e);
@ -1831,10 +1826,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
public ScheduleComponentsEntity enableComponents(String processGroupId, ScheduledState state, Map<String, Revision> componentRevisions) { public ScheduleComponentsEntity enableComponents(String processGroupId, ScheduledState state, Map<String, Revision> componentRevisions) {
final NiFiUser user = NiFiUserUtils.getNiFiUser(); final NiFiUser user = NiFiUserUtils.getNiFiUser();
final RevisionUpdate<ScheduleComponentsEntity> updatedComponent = revisionManager.updateRevision(new StandardRevisionClaim(componentRevisions.values()), user, new final RevisionUpdate<ScheduleComponentsEntity> updatedComponent = revisionManager.updateRevision(new StandardRevisionClaim(componentRevisions.values()), user, () -> {
UpdateRevisionTask<ScheduleComponentsEntity>() {
@Override
public RevisionUpdate<ScheduleComponentsEntity> update() {
// schedule the components // schedule the components
processGroupDAO.enableComponents(processGroupId, state, componentRevisions.keySet()); processGroupDAO.enableComponents(processGroupId, state, componentRevisions.keySet());
@ -1853,7 +1845,6 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
entity.setId(processGroupId); entity.setId(processGroupId);
entity.setState(state.name()); entity.setState(state.name());
return new StandardRevisionUpdate<>(entity, null, new HashSet<>(updatedRevisions.values())); return new StandardRevisionUpdate<>(entity, null, new HashSet<>(updatedRevisions.values()));
}
}); });
return updatedComponent.getComponent(); return updatedComponent.getComponent();
@ -1862,10 +1853,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
@Override @Override
public ScheduleComponentsEntity scheduleComponents(final String processGroupId, final ScheduledState state, final Map<String, Revision> componentRevisions) { public ScheduleComponentsEntity scheduleComponents(final String processGroupId, final ScheduledState state, final Map<String, Revision> componentRevisions) {
final NiFiUser user = NiFiUserUtils.getNiFiUser(); final NiFiUser user = NiFiUserUtils.getNiFiUser();
final RevisionUpdate<ScheduleComponentsEntity> updatedComponent = revisionManager.updateRevision(new StandardRevisionClaim(componentRevisions.values()), user, new final RevisionUpdate<ScheduleComponentsEntity> updatedComponent = revisionManager.updateRevision(new StandardRevisionClaim(componentRevisions.values()), user, () -> {
UpdateRevisionTask<ScheduleComponentsEntity>() {
@Override
public RevisionUpdate<ScheduleComponentsEntity> update() {
// schedule the components // schedule the components
processGroupDAO.scheduleComponents(processGroupId, state, componentRevisions.keySet()); processGroupDAO.scheduleComponents(processGroupId, state, componentRevisions.keySet());
@ -1884,7 +1872,6 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
entity.setId(processGroupId); entity.setId(processGroupId);
entity.setState(state.name()); entity.setState(state.name());
return new StandardRevisionUpdate<>(entity, null, new HashSet<>(updatedRevisions.values())); return new StandardRevisionUpdate<>(entity, null, new HashSet<>(updatedRevisions.values()));
}
}); });
return updatedComponent.getComponent(); return updatedComponent.getComponent();
@ -1894,9 +1881,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
public ActivateControllerServicesEntity activateControllerServices(final String processGroupId, final ControllerServiceState state, final Map<String, Revision> serviceRevisions) { public ActivateControllerServicesEntity activateControllerServices(final String processGroupId, final ControllerServiceState state, final Map<String, Revision> serviceRevisions) {
final NiFiUser user = NiFiUserUtils.getNiFiUser(); final NiFiUser user = NiFiUserUtils.getNiFiUser();
final RevisionUpdate<ActivateControllerServicesEntity> updatedComponent = revisionManager.updateRevision(new StandardRevisionClaim(serviceRevisions.values()), user, final RevisionUpdate<ActivateControllerServicesEntity> updatedComponent = revisionManager.updateRevision(new StandardRevisionClaim(serviceRevisions.values()), user,
new UpdateRevisionTask<ActivateControllerServicesEntity>() { () -> {
@Override
public RevisionUpdate<ActivateControllerServicesEntity> update() {
// schedule the components // schedule the components
processGroupDAO.activateControllerServices(processGroupId, state, serviceRevisions.keySet()); processGroupDAO.activateControllerServices(processGroupId, state, serviceRevisions.keySet());
@ -1915,7 +1900,6 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
entity.setId(processGroupId); entity.setId(processGroupId);
entity.setState(state.name()); entity.setState(state.name());
return new StandardRevisionUpdate<>(entity, null, new HashSet<>(updatedRevisions.values())); return new StandardRevisionUpdate<>(entity, null, new HashSet<>(updatedRevisions.values()));
}
}); });
return updatedComponent.getComponent(); return updatedComponent.getComponent();
@ -2229,9 +2213,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
final RevisionClaim claim = new StandardRevisionClaim(revision); final RevisionClaim claim = new StandardRevisionClaim(revision);
final NiFiUser user = NiFiUserUtils.getNiFiUser(); final NiFiUser user = NiFiUserUtils.getNiFiUser();
return revisionManager.deleteRevision(claim, user, new DeleteRevisionTask<D>() { return revisionManager.deleteRevision(claim, user, () -> {
@Override
public D performTask() {
logger.debug("Attempting to delete component {} with claim {}", resource.getIdentifier(), claim); logger.debug("Attempting to delete component {} with claim {}", resource.getIdentifier(), claim);
// run the delete action // run the delete action
@ -2246,7 +2228,6 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
} }
return dto; return dto;
}
}); });
} }
@ -2318,9 +2299,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
final NiFiUser user = NiFiUserUtils.getNiFiUser(); final NiFiUser user = NiFiUserUtils.getNiFiUser();
final RevisionClaim claim = new StandardRevisionClaim(revisions); final RevisionClaim claim = new StandardRevisionClaim(revisions);
final SnippetDTO dto = revisionManager.deleteRevision(claim, user, new DeleteRevisionTask<SnippetDTO>() { final SnippetDTO dto = revisionManager.deleteRevision(claim, user, () -> {
@Override
public SnippetDTO performTask() {
// delete the components in the snippet // delete the components in the snippet
snippetDAO.deleteSnippetComponents(snippetId); snippetDAO.deleteSnippetComponents(snippetId);
@ -2332,7 +2311,6 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
// create the dto for the snippet that was just removed // create the dto for the snippet that was just removed
return dtoFactory.createSnippetDto(snippet); return dtoFactory.createSnippetDto(snippet);
}
}); });
// clean up component policies // clean up component policies
@ -3012,9 +2990,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
final NiFiUser user = NiFiUserUtils.getNiFiUser(); final NiFiUser user = NiFiUserUtils.getNiFiUser();
final RevisionUpdate<ControllerServiceReferencingComponentsEntity> update = revisionManager.updateRevision(claim, user, final RevisionUpdate<ControllerServiceReferencingComponentsEntity> update = revisionManager.updateRevision(claim, user,
new UpdateRevisionTask<ControllerServiceReferencingComponentsEntity>() { () -> {
@Override
public RevisionUpdate<ControllerServiceReferencingComponentsEntity> update() {
final Set<ComponentNode> updated = controllerServiceDAO.updateControllerServiceReferencingComponents(controllerServiceId, scheduledState, controllerServiceState); final Set<ComponentNode> updated = controllerServiceDAO.updateControllerServiceReferencingComponents(controllerServiceId, scheduledState, controllerServiceState);
controllerFacade.save(); controllerFacade.save();
@ -3035,7 +3011,6 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
final ControllerServiceReferencingComponentsEntity entity = createControllerServiceReferencingComponentsEntity(updatedReference, updatedRevisions); final ControllerServiceReferencingComponentsEntity entity = createControllerServiceReferencingComponentsEntity(updatedReference, updatedRevisions);
return new StandardRevisionUpdate<>(entity, null, new HashSet<>(updatedRevisions.values())); return new StandardRevisionUpdate<>(entity, null, new HashSet<>(updatedRevisions.values()));
}
}); });
return update.getComponent(); return update.getComponent();
@ -4783,9 +4758,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
private List<BulletinEntity> pruneAndSortBulletins(final List<BulletinEntity> bulletinEntities, final int maxBulletins) { private List<BulletinEntity> pruneAndSortBulletins(final List<BulletinEntity> bulletinEntities, final int maxBulletins) {
// sort the bulletins // sort the bulletins
Collections.sort(bulletinEntities, new Comparator<BulletinEntity>() { Collections.sort(bulletinEntities, (o1, o2) -> {
@Override
public int compare(BulletinEntity o1, BulletinEntity o2) {
if (o1 == null && o2 == null) { if (o1 == null && o2 == null) {
return 0; return 0;
} }
@ -4797,7 +4770,6 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
} }
return -Long.compare(o1.getId(), o2.getId()); return -Long.compare(o1.getId(), o2.getId());
}
}); });
// prune the response to only include the max number of bulletins // prune the response to only include the max number of bulletins
@ -6347,9 +6319,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
final RevisionClaim revisionClaim = new StandardRevisionClaim(revisions); final RevisionClaim revisionClaim = new StandardRevisionClaim(revisions);
final RevisionUpdate<ProcessGroupDTO> revisionUpdate = revisionManager.updateRevision(revisionClaim, user, new UpdateRevisionTask<ProcessGroupDTO>() { final RevisionUpdate<ProcessGroupDTO> revisionUpdate = revisionManager.updateRevision(revisionClaim, user, () -> {
@Override
public RevisionUpdate<ProcessGroupDTO> update() {
// update the Process Group // update the Process Group
final VersionedExternalFlow externalFlow = createVersionedExternalFlow(proposedFlowSnapshot); final VersionedExternalFlow externalFlow = createVersionedExternalFlow(proposedFlowSnapshot);
processGroupDAO.updateProcessGroupFlow(groupId, externalFlow, versionControlInfo, componentIdSeed, verifyNotModified, updateSettings, processGroupDAO.updateProcessGroupFlow(groupId, externalFlow, versionControlInfo, componentIdSeed, verifyNotModified, updateSettings,
@ -6369,7 +6339,6 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
final Revision updatedRevision = revisionManager.getRevision(groupId).incrementRevision(revision.getClientId()); final Revision updatedRevision = revisionManager.getRevision(groupId).incrementRevision(revision.getClientId());
final FlowModification lastModification = new FlowModification(updatedRevision, user.getIdentity()); final FlowModification lastModification = new FlowModification(updatedRevision, user.getIdentity());
return new StandardRevisionUpdate<>(dto, lastModification, updatedRevisions); return new StandardRevisionUpdate<>(dto, lastModification, updatedRevisions);
}
}); });
final FlowModification lastModification = revisionUpdate.getLastModification(); final FlowModification lastModification = revisionUpdate.getLastModification();

View File

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

View File

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

View File

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

View File

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

View File

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